You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by GitBox <gi...@apache.org> on 2021/12/20 12:36:21 UTC

[GitHub] [incubator-kyuubi] pan3793 opened a new pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

pan3793 opened a new pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592


   ### _Why are the changes needed?_
   
   Introduce row level operation test for data lake format and remove redundant tests
   
   ### _How was this patch tested?_
   - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible
   
   - [ ] Add screenshots for manual tests if appropriate
   
   - [x] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772784571



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/IcebergRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.IcebergSuiteMixin
+
+trait IcebergRowLevelOperationTests extends HiveJDBCTestHelper with IcebergSuiteMixin {
+
+  private def createAndInitTable(tableName: String)(records: Seq[(Int, String)]): Unit = {

Review comment:
       private def createAndInitTable(statement: Statement, tableName: String)




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] ulysses-you commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773551385



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       what's the behavior of iceberg or delta ?




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772784130



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/IcebergRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.IcebergSuiteMixin
+
+trait IcebergRowLevelOperationTests extends HiveJDBCTestHelper with IcebergSuiteMixin {
+
+  private def createAndInitTable(tableName: String)(records: Seq[(Int, String)]): Unit = {

Review comment:
       Do you mean
   ```
   def createAndInitTable(tableName: String)
                         (records: Seq[(Int, String)])
                         (initSql: Option[String] = None): Unit
   ```




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772792151



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       The complexity of mixins brings more overhead of maintenance. I have never experienced using traits for classification, not abstraction




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773693614



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       BTW, I explained the idea in #1287, if we reject to use traits for classification, we need to reconsider the test classes hierarchical




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772330152



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaMetadataTests.scala
##########
@@ -87,162 +87,4 @@ trait DeltaMetadataTests extends HiveJDBCTestHelper with DeltaSuiteMixin {
       assert(!rs3.next())
     }
   }
-
-  test("get type info") {

Review comment:
       It's independent with table format, but engine.




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter commented on pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#issuecomment-999387134


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1592](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9d20839) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/df1d9f3bb2a336054cb3f3e712c707a817612fdc?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (df1d9f3) will **increase** coverage by `0.01%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 9d20839 differs from pull request most recent head 72fd093. Consider uploading reports for the commit 72fd093 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1592      +/-   ##
   ============================================
   + Coverage     59.02%   59.04%   +0.01%     
     Complexity      196      196              
   ============================================
     Files           256      256              
     Lines         12708    12708              
     Branches       1601     1601              
   ============================================
   + Hits           7501     7503       +2     
   + Misses         4570     4568       -2     
     Partials        637      637              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/kyuubi/server/BackendServiceTimeMetric.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvQmFja2VuZFNlcnZpY2VUaW1lTWV0cmljLnNjYWxh) | `83.78% <0.00%> (+5.40%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [df1d9f3...72fd093](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772783645



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/SparkMetadataTests.scala
##########
@@ -287,4 +292,188 @@ trait SparkMetadataTests extends HiveJDBCTestHelper {
       assert(typeInfo.getInt(NUM_PREC_RADIX) === 0)
     }
   }
+
+  test("audit Kyuubi Hive JDBC connection MetaData") {
+    withJdbcStatement() { statement =>
+      val metaData = statement.getConnection.getMetaData
+      Seq(
+        () => metaData.allProceduresAreCallable(),
+        () => metaData.getURL,
+        () => metaData.getUserName,
+        () => metaData.isReadOnly,
+        () => metaData.nullsAreSortedHigh,
+        () => metaData.nullsAreSortedLow,
+        () => metaData.nullsAreSortedAtStart(),
+        () => metaData.nullsAreSortedAtEnd(),
+        () => metaData.usesLocalFiles(),
+        () => metaData.usesLocalFilePerTable(),
+        () => metaData.supportsMixedCaseIdentifiers(),
+        () => metaData.supportsMixedCaseQuotedIdentifiers(),
+        () => metaData.storesUpperCaseIdentifiers(),
+        () => metaData.storesUpperCaseQuotedIdentifiers(),
+        () => metaData.storesLowerCaseIdentifiers(),
+        () => metaData.storesLowerCaseQuotedIdentifiers(),
+        () => metaData.storesMixedCaseIdentifiers(),
+        () => metaData.storesMixedCaseQuotedIdentifiers(),
+        () => metaData.getSQLKeywords,
+        () => metaData.nullPlusNonNullIsNull,
+        () => metaData.supportsConvert,
+        () => metaData.supportsTableCorrelationNames,
+        () => metaData.supportsDifferentTableCorrelationNames,
+        () => metaData.supportsExpressionsInOrderBy(),
+        () => metaData.supportsOrderByUnrelated,
+        () => metaData.supportsGroupByUnrelated,
+        () => metaData.supportsGroupByBeyondSelect,
+        () => metaData.supportsLikeEscapeClause,
+        () => metaData.supportsMultipleTransactions,
+        () => metaData.supportsMinimumSQLGrammar,
+        () => metaData.supportsCoreSQLGrammar,
+        () => metaData.supportsExtendedSQLGrammar,
+        () => metaData.supportsANSI92EntryLevelSQL,
+        () => metaData.supportsANSI92IntermediateSQL,
+        () => metaData.supportsANSI92FullSQL,
+        () => metaData.supportsIntegrityEnhancementFacility,
+        () => metaData.isCatalogAtStart,
+        () => metaData.supportsSubqueriesInComparisons,
+        () => metaData.supportsSubqueriesInExists,
+        () => metaData.supportsSubqueriesInIns,
+        () => metaData.supportsSubqueriesInQuantifieds,
+        // Spark support this, see https://issues.apache.org/jira/browse/SPARK-18455
+        () => metaData.supportsCorrelatedSubqueries,
+        () => metaData.supportsOpenCursorsAcrossCommit,
+        () => metaData.supportsOpenCursorsAcrossRollback,
+        () => metaData.supportsOpenStatementsAcrossCommit,
+        () => metaData.supportsOpenStatementsAcrossRollback,
+        () => metaData.getMaxBinaryLiteralLength,
+        () => metaData.getMaxCharLiteralLength,
+        () => metaData.getMaxColumnsInGroupBy,
+        () => metaData.getMaxColumnsInIndex,
+        () => metaData.getMaxColumnsInOrderBy,
+        () => metaData.getMaxColumnsInSelect,
+        () => metaData.getMaxColumnsInTable,
+        () => metaData.getMaxConnections,
+        () => metaData.getMaxCursorNameLength,
+        () => metaData.getMaxIndexLength,
+        () => metaData.getMaxSchemaNameLength,
+        () => metaData.getMaxProcedureNameLength,
+        () => metaData.getMaxCatalogNameLength,
+        () => metaData.getMaxRowSize,
+        () => metaData.doesMaxRowSizeIncludeBlobs,
+        () => metaData.getMaxStatementLength,
+        () => metaData.getMaxStatements,
+        () => metaData.getMaxTableNameLength,
+        () => metaData.getMaxTablesInSelect,
+        () => metaData.getMaxUserNameLength,
+        () => metaData.supportsTransactionIsolationLevel(1),
+        () => metaData.supportsDataDefinitionAndDataManipulationTransactions,
+        () => metaData.supportsDataManipulationTransactionsOnly,
+        () => metaData.dataDefinitionCausesTransactionCommit,
+        () => metaData.dataDefinitionIgnoredInTransactions,
+        () => metaData.getColumnPrivileges("", "%", "%", "%"),
+        () => metaData.getTablePrivileges("", "%", "%"),
+        () => metaData.getBestRowIdentifier("", "%", "%", 0, true),
+        () => metaData.getVersionColumns("", "%", "%"),
+        () => metaData.getExportedKeys("", "default", ""),
+        () => metaData.supportsResultSetConcurrency(ResultSet.TYPE_FORWARD_ONLY, 2),
+        () => metaData.ownUpdatesAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.ownDeletesAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.ownInsertsAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.othersUpdatesAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.othersDeletesAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.othersInsertsAreVisible(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.updatesAreDetected(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.deletesAreDetected(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.insertsAreDetected(ResultSet.TYPE_FORWARD_ONLY),
+        () => metaData.supportsNamedParameters(),
+        () => metaData.supportsMultipleOpenResults,
+        () => metaData.supportsGetGeneratedKeys,
+        () => metaData.getSuperTypes("", "%", "%"),
+        () => metaData.getSuperTables("", "%", "%"),
+        () => metaData.getAttributes("", "%", "%", "%"),
+        () => metaData.getResultSetHoldability,
+        () => metaData.locatorsUpdateCopy,
+        () => metaData.supportsStatementPooling,
+        () => metaData.getRowIdLifetime,
+        () => metaData.supportsStoredFunctionsUsingCallSyntax,
+        () => metaData.autoCommitFailureClosesAllResultSets,
+        () => metaData.getClientInfoProperties,
+        () => metaData.getFunctionColumns("", "%", "%", "%"),
+        () => metaData.getPseudoColumns("", "%", "%", "%"),
+        () => metaData.generatedKeyAlwaysReturned).foreach { func =>
+        val e = intercept[SQLFeatureNotSupportedException](func())
+        assert(e.getMessage === "Method not supported")
+      }
+
+      import org.apache.kyuubi.KYUUBI_VERSION

Review comment:
       let's fix this too




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773685230



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       Any suggestion for classification expect traits in scalatest? @yaooqinn 




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773687391



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       > what's the behavior of iceberg or delta ?
   
   I don't get your point.




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773685230



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       Any suggestion for classification except traits in scalatest? @yaooqinn 




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#issuecomment-1000019834


   thanks, merged to master


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772784094



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       how about we rename it to DML and bring more cases in?




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn closed pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn closed pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592


   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772786967



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       Row level operation is the spark offical term introduced in https://issues.apache.org/jira/browse/SPARK-35801, I'm planning to add other cases later in `TimeTravelTests` `SecondaryIndexTests` etc to reflect different capbilities in different table format




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772330152



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaMetadataTests.scala
##########
@@ -87,162 +87,4 @@ trait DeltaMetadataTests extends HiveJDBCTestHelper with DeltaSuiteMixin {
       assert(!rs3.next())
     }
   }
-
-  test("get type info") {

Review comment:
       It's independent with table format, but for specific engine.




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r772783001



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/IcebergRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.IcebergSuiteMixin
+
+trait IcebergRowLevelOperationTests extends HiveJDBCTestHelper with IcebergSuiteMixin {
+
+  private def createAndInitTable(tableName: String)(records: Seq[(Int, String)]): Unit = {

Review comment:
       we can have a statement as parameter here




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#issuecomment-999387134


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1592](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9d20839) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/df1d9f3bb2a336054cb3f3e712c707a817612fdc?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (df1d9f3) will **increase** coverage by `0.01%`.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head 9d20839 differs from pull request most recent head 892feb8. Consider uploading reports for the commit 892feb8 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #1592      +/-   ##
   ============================================
   + Coverage     59.02%   59.04%   +0.01%     
     Complexity      196      196              
   ============================================
     Files           256      256              
     Lines         12708    12708              
     Branches       1601     1601              
   ============================================
   + Hits           7501     7503       +2     
   + Misses         4570     4568       -2     
     Partials        637      637              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/kyuubi/server/BackendServiceTimeMetric.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvQmFja2VuZFNlcnZpY2VUaW1lTWV0cmljLnNjYWxh) | `83.78% <0.00%> (+5.40%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [df1d9f3...892feb8](https://codecov.io/gh/apache/incubator-kyuubi/pull/1592?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #1592: [TEST][ICEBERG][DELTA] Introduce row level operation test for data lake format

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #1592:
URL: https://github.com/apache/incubator-kyuubi/pull/1592#discussion_r773685230



##########
File path: kyuubi-common/src/test/scala/org/apache/kyuubi/operation/DeltaRowLevelOperationTests.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.kyuubi.operation
+
+import org.apache.kyuubi.DeltaSuiteMixin
+
+trait DeltaRowLevelOperationTests extends HiveJDBCTestHelper with DeltaSuiteMixin {

Review comment:
       Any suggestion for classification except to traits in scalatest? @yaooqinn 




-- 
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: commits-unsubscribe@kyuubi.apache.org

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