You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/06/03 12:54:19 UTC

[GitHub] [ignite-3] korlov42 commented on a diff in pull request #832: IGNITE-16962: SQL API: Implement query metadata

korlov42 commented on code in PR #832:
URL: https://github.com/apache/ignite-3/pull/832#discussion_r888871488


##########
modules/api/src/main/java/org/apache/ignite/sql/ColumnMetadata.java:
##########
@@ -40,16 +42,71 @@ public interface ColumnMetadata {
 
     /**
      * Returns SQL column type.
-     * TODO: IGNITE-16962 replace return type regarding the SQL type system.
      *
      * @return Value type.
      */
-    Object type();
+    SqlColumnType type();
+
+    /**
+     * Returns SQL column precision.
+     *
+     * @return Value precision.
+     */
+    int precision();
+
+    /**
+     * Returns SQL column scale.
+     *
+     * @return Value scale.
+     */
+    int scale();
 
     /**
      * Returns row column nullability flag.
      *
      * @return {@code true} if column is nullable, {@code false} otherwise.
      */
     boolean nullable();
+
+    /**
+     * Return column origin.
+     *
+     * @return Column origin or {@code null} if not applicable.
+     */
+    @Nullable ColumnOrigin origin();
+
+    /**
+     * Represent column origin.
+     *
+     * <p>Example: "SELECT SUM(price), category as cat, subcategory AS subcategory FROM Goods WHERE [condition] GROUP_BY cat, subcategory".
+     *
+     * <p>Column origins:
+     * <ul>
+     * <li>SUM(price): null</li>
+     * <li>cat: {"PUBLIC", "Goods", "category"}</li>
+     * <li>subcategory: {"PUBLIC", "Goods", "subcategory"}</li>
+     * </ul>
+     */
+    interface ColumnOrigin {
+        /**
+         * Return the column's table's schema.
+         *
+         * @return Schema name or "" if not applicable

Review Comment:
   could you please give an example in which cases schema name will be an empty string?



##########
modules/api/src/main/java/org/apache/ignite/sql/ColumnMetadata.java:
##########
@@ -40,16 +42,71 @@ public interface ColumnMetadata {
 
     /**
      * Returns SQL column type.
-     * TODO: IGNITE-16962 replace return type regarding the SQL type system.
      *
      * @return Value type.
      */
-    Object type();
+    SqlColumnType type();
+
+    /**
+     * Returns SQL column precision.
+     *
+     * @return Value precision.
+     */
+    int precision();
+
+    /**
+     * Returns SQL column scale.
+     *
+     * @return Value scale.
+     */
+    int scale();
 
     /**
      * Returns row column nullability flag.
      *
      * @return {@code true} if column is nullable, {@code false} otherwise.
      */
     boolean nullable();
+
+    /**
+     * Return column origin.
+     *
+     * @return Column origin or {@code null} if not applicable.
+     */
+    @Nullable ColumnOrigin origin();
+
+    /**
+     * Represent column origin.
+     *
+     * <p>Example: "SELECT SUM(price), category as cat, subcategory AS subcategory FROM Goods WHERE [condition] GROUP_BY cat, subcategory".

Review Comment:
   ```suggestion
        * <p>Example:
        * <pre>
        *     SELECT SUM(price), category as cat, subcategory AS subcategory 
        *       FROM Goods
        *      WHERE [condition]
        *      GROUP BY cat, subcategory
        * </pre>
   ```



##########
modules/api/src/main/java/org/apache/ignite/sql/ColumnMetadata.java:
##########
@@ -40,16 +42,71 @@ public interface ColumnMetadata {
 
     /**
      * Returns SQL column type.
-     * TODO: IGNITE-16962 replace return type regarding the SQL type system.
      *
      * @return Value type.
      */
-    Object type();
+    SqlColumnType type();
+
+    /**
+     * Returns SQL column precision.
+     *
+     * @return Value precision.

Review Comment:
   it is worth mentioning in the javadoc the value that will be returned for precision/scale in case those characteristics are not applicable



##########
modules/api/src/main/java/org/apache/ignite/sql/async/AsyncResultSet.java:
##########
@@ -50,13 +49,12 @@
  */
 public interface AsyncResultSet {
     /**
-     * Returns metadata for the results if the result contains rows ({@link #hasRowSet()} returns {@code true}), or {@code null} if
-     * inapplicable.
+     * Returns metadata for the results.
      *
      * @return ResultSet metadata.
      * @see ResultSet#metadata()
      */
-    @Nullable ResultSetMetadata metadata();

Review Comment:
   I believe, metadata should remain nullable for the public result set, and here is why: in the internal API any execution produces a result represented by a list of lists. This is true for DML and DDL statements as well. In order to be able to consume the result, there is a metadata which describes the type of the rows returned. On the other hand, in public API, results of DDL and DML statements are unwrapped to a particular fields in the ResultSet (affectedRows, wasApplied), hence metadata is only needed to process a result set returned by a query



##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -317,22 +318,22 @@ public CompletableFuture<JdbcMetaColumnsResult> queryMetadataAsync(JdbcQueryMeta
      * @param fldMeta field metadata contains info about column.
      * @return JdbcColumnMeta object.
      */
-    private JdbcColumnMeta createColumnMetadata(ResultFieldMetadata fldMeta) {
-        List<String> origin = fldMeta.origin();
+    private JdbcColumnMeta createColumnMetadata(ColumnMetadata fldMeta) {
+        ColumnOrigin origin = fldMeta.origin();
 
-        String schemaName = origin == null ? null : origin.get(0);
-        String tblName = origin == null ? null : origin.get(1);
-        String colName = origin == null ? null : origin.get(2);
+        String schemaName = origin == null ? null : origin.schemaName();
+        String tblName = origin == null ? null : origin.tableName();
+        String colName = origin == null ? null : origin.columnName();

Review Comment:
   probably, it would be better to check the origin only once in more traditional if-then-else way
   



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/DdlPlan.java:
##########
@@ -45,7 +45,7 @@ public Type type() {
     /** {@inheritDoc} */
     @Override
     public ResultSetMetadata metadata() {
-        return List::of;
+        return ResultSetMetadataImpl.NO_METADATA;

Review Comment:
   let's introduce a constant for DDL meta with a single column of type Bool and name "Applied"



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/FragmentPlan.java:
##########
@@ -53,7 +53,7 @@ public Type type() {
     /** {@inheritDoc} */
     @Override
     public ResultSetMetadata metadata() {
-        return List::of;
+        return ResultSetMetadataImpl.NO_METADATA;

Review Comment:
   I wonder if anyone asking fragment about its meta. Probably it would be better to throw IllegaleState here, WDYT? 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/api/ResultSetMetadataImpl.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.api;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.internal.tostring.IgniteToStringExclude;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.sql.ColumnMetadata;
+import org.apache.ignite.sql.ResultSetMetadata;
+
+/**
+ * Results set metadata holder.
+ */
+public class ResultSetMetadataImpl implements ResultSetMetadata {
+    /** Empty metadata holder. */
+    public static final ResultSetMetadata NO_METADATA = new ResultSetMetadataImpl(List.of());
+
+    /** Column`s metadata ordered list. */
+    private final List<ColumnMetadata> columns;
+
+    /** Column`s metadata map. */
+    @IgniteToStringExclude
+    private final Map<String, Integer> columnsIndices;
+
+    /**
+     * Constructor.
+     *
+     * @param columns Columns metadata.
+     */
+    public ResultSetMetadataImpl(List<ColumnMetadata> columns) {
+        this.columns = Collections.unmodifiableList(columns);
+
+        columnsIndices = new HashMap<>(columns.size());
+
+        for (int i = 0; i < columns.size(); i++) {
+            ColumnMetadata column = columns.get(i);
+
+            columnsIndices.put(column.name(), i);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public List<ColumnMetadata> columns() {
+        return columns;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int indexOf(String columnName) {
+        Integer idx = columnsIndices.get(columnName);
+
+        return idx == null ? -1 : idx;

Review Comment:
   why don't you use `columnsIndices.getOrDefault(columnName, -1)` here? 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/api/ResultSetMetadataImpl.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.api;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.internal.tostring.IgniteToStringExclude;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.sql.ColumnMetadata;
+import org.apache.ignite.sql.ResultSetMetadata;
+
+/**
+ * Results set metadata holder.
+ */
+public class ResultSetMetadataImpl implements ResultSetMetadata {
+    /** Empty metadata holder. */
+    public static final ResultSetMetadata NO_METADATA = new ResultSetMetadataImpl(List.of());
+
+    /** Column`s metadata ordered list. */
+    private final List<ColumnMetadata> columns;
+
+    /** Column`s metadata map. */
+    @IgniteToStringExclude
+    private final Map<String, Integer> columnsIndices;
+
+    /**
+     * Constructor.
+     *
+     * @param columns Columns metadata.
+     */
+    public ResultSetMetadataImpl(List<ColumnMetadata> columns) {
+        this.columns = Collections.unmodifiableList(columns);
+
+        columnsIndices = new HashMap<>(columns.size());

Review Comment:
   well, default load factor is .75, so you will get one more allocation here.
   
   BTW, should we consider to switch to Object2IntMap ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org