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/07/01 12:20:30 UTC

[GitHub] [ignite-3] AMashenkov opened a new pull request, #916: IGNITE-17203 SQL API: Implement scale and precision metadata

AMashenkov opened a new pull request, #916:
URL: https://github.com/apache/ignite-3/pull/916

   https://issues.apache.org/jira/browse/IGNITE-17203


-- 
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


[GitHub] [ignite-3] AMashenkov commented on pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on PR #916:
URL: https://github.com/apache/ignite-3/pull/916#issuecomment-1172297705

   There are number of todo's about unsupported types.
   I'll create a ticket for some that will not be fixed in [1].
   
   [1] https://issues.apache.org/jira/browse/IGNITE-17219


-- 
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


[GitHub] [ignite-3] ptupitsyn commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r912049171


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeSystem.java:
##########
@@ -44,29 +45,80 @@ public int getMaxNumericPrecision() {
         return Short.MAX_VALUE;
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public int getMaxPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIME:
+            case TIME_WITH_LOCAL_TIME_ZONE:
+            case TIMESTAMP:
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return TemporalColumnType.MAX_TIME_PRECISION;
+            default:
+                return super.getMaxPrecision(typeName);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int getDefaultPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIMESTAMP: // DATETIME
+                return TemporalColumnType.DEFAULT_TIME_PRECISION;
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE: // TIMESTAMP
+                return TemporalColumnType.DEFAULT_TIMESTAMP_PRECISION;
+            default:
+                return super.getDefaultPrecision(typeName);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override
     public RelDataType deriveSumType(RelDataTypeFactory typeFactory, RelDataType argumentType) {
         RelDataType sumType;
+        // This method is used to derive types for aggregate functions only.
+        // We allow type widening for aggregates to prevent unwanted number overflow.
+        //
+        // SQL`99 part 2 section 9.3 syntax rule 3:
+        // Shortly, the standard says both, the return type and the argument type, must be of the same kind,
+        // but doesn't restrict them being exactly of the same type.
         if (argumentType instanceof BasicSqlType) {
             switch (argumentType.getSqlTypeName()) {
                 case INTEGER:
                 case TINYINT:
                 case SMALLINT:
-                    sumType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+                    sumType = typeFactory.createTypeWithNullability(
+                            // TODO: can we cache the type?

Review Comment:
   TODOs should have tickets.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
         assert relType != null;
+        assert relType instanceof BasicSqlType
+                || relType instanceof IntervalSqlType : "Not supported yet."; // Implement Class->ColumnType mapping if failed.
 
-        Type javaType = getResultClass(relType);
-
-        if (javaType == byte[].class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.blobOf() :
-                ColumnType.blobOf(relType.getPrecision());
-        } else if (javaType == String.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.string() :
-                ColumnType.stringOf(relType.getPrecision());
-        } else if (javaType == BigInteger.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.numberOf() :
-                ColumnType.numberOf(relType.getPrecision());
-        } else if (javaType == BigDecimal.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.decimalOf() :
-                ColumnType.decimalOf(relType.getPrecision(), relType.getScale());
-        } else {
-            return SchemaConfigurationConverter.columnType((Class<?>) javaType);
+        switch (relType.getSqlTypeName()) {
+            case BOOLEAN:
+                throw new IllegalArgumentException("Type is not supported yet.");

Review Comment:
   Do we need a ticket for this?



-- 
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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r914494953


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeSystem.java:
##########
@@ -44,29 +45,69 @@ public int getMaxNumericPrecision() {
         return Short.MAX_VALUE;
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public int getMaxPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIME:
+            case TIME_WITH_LOCAL_TIME_ZONE:
+            case TIMESTAMP:
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return TemporalColumnType.MAX_TIME_PRECISION;
+            default:
+                return super.getMaxPrecision(typeName);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int getDefaultPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIMESTAMP: // DATETIME
+                return TemporalColumnType.DEFAULT_TIME_PRECISION;

Review Comment:
   Could you please explain why have you decided to use _time precision_ here? In my opinion, we should use _timestamp precision_ for both TIMESTAMP and TIMESTAMP W/ TIMEZONE types



-- 
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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r914706010


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/util/QueryChecker.java:
##########
@@ -377,7 +391,23 @@ public void check() {
             assertThat("Column types don't match", colTypes, equalTo(expectedColumnTypes));
         }
 
-        var res = CursorUtils.getAllFromCursor(cur);
+        if (metadataMatchers != null) {
+            List<ColumnMetadata> columnMetadata = cur.metadata().columns();
+
+            Iterator<ColumnMetadata> valueIterator = columnMetadata.iterator();
+            Iterator<MetadataMatcher> matcherIterator = metadataMatchers.iterator();
+
+            while (matcherIterator.hasNext() && valueIterator.hasNext()) {
+                MetadataMatcher matcher = matcherIterator.next();
+                ColumnMetadata actualElement = valueIterator.next();
+
+                matcher.check(actualElement);
+            }
+
+            assertFalse(matcherIterator.hasNext(), "Column metadata doesn't match");

Review Comment:
   ok



-- 
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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r912903482


##########
modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java:
##########
@@ -460,8 +460,26 @@ public int hashCode() {
      * Column type of variable length.
      */
     public static class TemporalColumnType extends ColumnType {
-        /** Default temporal type precision: microseconds. */
-        public static final int DEFAULT_PRECISION = 6;
+        /**
+         * Default TIME type precision: microseconds.
+         *
+         * SQL99 part 2 section 6.1 syntax rule 30

Review Comment:
   ```suggestion
            * 
            * <p>SQL99 part 2 section 6.1 syntax rule 30
   ```
   
   please fix this here and below



##########
modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java:
##########
@@ -460,8 +460,23 @@ public int hashCode() {
      * Column type of variable length.
      */
     public static class TemporalColumnType extends ColumnType {
-        /** Default temporal type precision: microseconds. */
-        public static final int DEFAULT_PRECISION = 6;
+        /**
+         * Default TIME type precision: microseconds.
+         * SQL99 part 2 section 6.1 syntax rule 30
+         */
+        public static final int DEFAULT_TIMESTAMP_PRECISION = 6;
+
+        /**
+         * Default TIMESTAMP type precision: seconds.
+         * SQL99 part 2 section 6.1 syntax rule 30
+         */
+        public static final int DEFAULT_TIME_PRECISION = 0;
+
+        /**
+         * Max TIME precision.
+         * SQL99 part 2 section 6.1 syntax rule 31

Review Comment:
   did you mean "rule 32"? It's definitely 32 rule in my PDF



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
         assert relType != null;

Review Comment:
   this is covered by the next assertion



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeSystem.java:
##########
@@ -44,29 +45,69 @@ public int getMaxNumericPrecision() {
         return Short.MAX_VALUE;
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public int getMaxPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIME:
+            case TIME_WITH_LOCAL_TIME_ZONE:
+            case TIMESTAMP:
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return TemporalColumnType.MAX_TIME_PRECISION;
+            default:
+                return super.getMaxPrecision(typeName);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int getDefaultPrecision(SqlTypeName typeName) {
+        switch (typeName) {
+            case TIMESTAMP: // DATETIME
+                return TemporalColumnType.DEFAULT_TIME_PRECISION;

Review Comment:
   Could you please explain why have you decided to use _<time precision>_ here? In my opinion, we should use _<timestamp precision>_ for both TIMESTAMP and TIMESTAMP W/ TIMEZONE types



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java:
##########
@@ -107,4 +115,118 @@ public void columnOrder() {
                 .columnNames("DOUBLE_C", "LONG_C", "STRING_C", "INT_C")
                 .check();
     }
+
+    @Test
+    public void metadata() {
+        CLUSTER_NODES.get(0).sql().createSession().execute(null, "CREATE TABLE METADATA_TABLE (" + "ID INT PRIMARY KEY, "

Review Comment:
   ```suggestion
           sql("CREATE TABLE METADATA_TABLE (" + "ID INT PRIMARY KEY, "
   ```



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
         assert relType != null;
+        assert relType instanceof BasicSqlType
+                || relType instanceof IntervalSqlType : "Not supported yet."; // Implement Class->ColumnType mapping if failed.
 
-        Type javaType = getResultClass(relType);
-
-        if (javaType == byte[].class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.blobOf() :
-                ColumnType.blobOf(relType.getPrecision());
-        } else if (javaType == String.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.string() :
-                ColumnType.stringOf(relType.getPrecision());
-        } else if (javaType == BigInteger.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.numberOf() :
-                ColumnType.numberOf(relType.getPrecision());
-        } else if (javaType == BigDecimal.class) {
-            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.decimalOf() :
-                ColumnType.decimalOf(relType.getPrecision(), relType.getScale());
-        } else {
-            return SchemaConfigurationConverter.columnType((Class<?>) javaType);
+        switch (relType.getSqlTypeName()) {
+            case BOOLEAN:
+                throw new IllegalArgumentException("Type is not supported yet.");
+            case TINYINT:
+                return ColumnType.INT8;
+            case SMALLINT:
+                return ColumnType.INT16;
+            case INTEGER:
+                return ColumnType.INT32;
+            case BIGINT:
+                return ColumnType.INT64;
+            case DECIMAL:
+                return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? ColumnType.decimalOf() :

Review Comment:
   I believe `PRECISION_NOT_SPECIFIED` should be valid precision for varying types only, like `VARCHAR` or `VARBINARY`. Probably we could put an assertion here instead, WDYT?



-- 
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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r915484429


##########
modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java:
##########
@@ -460,8 +460,26 @@ public int hashCode() {
      * Column type of variable length.
      */
     public static class TemporalColumnType extends ColumnType {
-        /** Default temporal type precision: microseconds. */
-        public static final int DEFAULT_PRECISION = 6;
+        /**
+         * Default TIME type precision: microseconds.
+         *
+         * <p>SQL99 part 2 section 6.1 syntax rule 30
+         */
+        public static final int DEFAULT_TIMESTAMP_PRECISION = 6;
+
+        /**
+         * Default TIMESTAMP type precision: seconds.

Review Comment:
   ```suggestion
            * Default TIME type precision: seconds.
   ```



-- 
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


[GitHub] [ignite-3] ptupitsyn commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r914732606


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
-        assert relType != null;
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
+        assert relType instanceof BasicSqlType

Review Comment:
   Let's add TODOs with ticket links in all places where "not supported" exception is thrown.



-- 
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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r915484255


##########
modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java:
##########
@@ -460,8 +460,26 @@ public int hashCode() {
      * Column type of variable length.
      */
     public static class TemporalColumnType extends ColumnType {
-        /** Default temporal type precision: microseconds. */
-        public static final int DEFAULT_PRECISION = 6;
+        /**
+         * Default TIME type precision: microseconds.

Review Comment:
   ```suggestion
            * Default TIMESTAMP type precision: microseconds.
   ```



-- 
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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r915486969


##########
modules/api/src/main/java/org/apache/ignite/schema/definition/ColumnType.java:
##########
@@ -203,14 +203,14 @@ public static TemporalColumnType time(int precision) {
     }
 
     /**
-     * Returns timezone-free datetime encoded as (date, time) with the default time precision of 6 (microseconds).
+     * Returns timezone-free datetime encoded as (date, time) with the default time precision of 0 (seconds).
      *
      * @return Native type.
-     * @see TemporalColumnType#DEFAULT_PRECISION
+     * @see TemporalColumnType#DEFAULT_TIME_PRECISION
      * @see #datetime(int)
      */
     public static TemporalColumnType datetime() {
-        return new TemporalColumnType(ColumnTypeSpec.DATETIME, TemporalColumnType.DEFAULT_PRECISION);
+        return new TemporalColumnType(ColumnTypeSpec.DATETIME, TemporalColumnType.DEFAULT_TIME_PRECISION);

Review Comment:
   I believe this here should be TIMESTAMP_PRECISION, since DATETIME represented by a SQL TIMESTAMP type 



-- 
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


[GitHub] [ignite-3] AMashenkov merged pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
AMashenkov merged PR #916:
URL: https://github.com/apache/ignite-3/pull/916


-- 
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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r914705770


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
-        assert relType != null;
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
+        assert relType instanceof BasicSqlType

Review Comment:
   https://issues.apache.org/jira/browse/IGNITE-17219
   https://issues.apache.org/jira/browse/IGNITE-17298



-- 
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


[GitHub] [ignite-3] ptupitsyn commented on a diff in pull request #916: IGNITE-17203 SQL API: Implement scale and precision metadata

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on code in PR #916:
URL: https://github.com/apache/ignite-3/pull/916#discussion_r914672524


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/type/IgniteTypeFactory.java:
##########
@@ -163,30 +161,72 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Gets ColumnType type for given class.
+     * Gets ColumnType type for RelDataType.
      *
      * @param relType Rel type.
      * @return ColumnType type or null.
      */
-    public ColumnType columnType(RelDataType relType) {
-        assert relType != null;
+    public static ColumnType relDataTypeToColumnType(RelDataType relType) {
+        assert relType instanceof BasicSqlType

Review Comment:
   Do we have tickets for unsupported types?



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/util/QueryChecker.java:
##########
@@ -377,7 +391,23 @@ public void check() {
             assertThat("Column types don't match", colTypes, equalTo(expectedColumnTypes));
         }
 
-        var res = CursorUtils.getAllFromCursor(cur);
+        if (metadataMatchers != null) {
+            List<ColumnMetadata> columnMetadata = cur.metadata().columns();
+
+            Iterator<ColumnMetadata> valueIterator = columnMetadata.iterator();
+            Iterator<MetadataMatcher> matcherIterator = metadataMatchers.iterator();
+
+            while (matcherIterator.hasNext() && valueIterator.hasNext()) {
+                MetadataMatcher matcher = matcherIterator.next();
+                ColumnMetadata actualElement = valueIterator.next();
+
+                matcher.check(actualElement);
+            }
+
+            assertFalse(matcherIterator.hasNext(), "Column metadata doesn't match");

Review Comment:
   `valueIterator.hasNext()` may be true, which also indicates a mismatch.
   Let's replace this with `assertEquals(columnMetadata.size(), metadataMatchers.size())`.



-- 
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