You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/03 02:07:22 UTC

[GitHub] [hudi] nsivabalan opened a new pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

nsivabalan opened a new pull request #2400:
URL: https://github.com/apache/hudi/pull/2400


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571431282



##########
File path: docker/demo/config/test-suite/cow-clustering-example.yaml
##########
@@ -0,0 +1,76 @@
+# 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.
+dag_name: cow-clustering-example.yaml
+dag_rounds: 3
+dag_intermittent_delay_mins: 0
+dag_content:
+  first_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 1000
+    type: InsertNode
+    deps: none
+  second_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 10000
+    deps: first_insert
+    type: InsertNode
+  third_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 300
+    deps: second_insert
+    type: InsertNode
+  first_delete:
+    config:
+      num_partitions_delete: 1
+      num_records_delete: 9000
+    type: DeleteNode
+    deps: third_insert
+  first_hive_sync:
+    config:
+      queue_name: "adhoc"
+      engine: "mr"
+    type: HiveSyncNode
+    deps: first_delete
+  first_validate:
+    config:
+      validate_hive: true
+    type: ValidateDatasetNode
+    deps: first_hive_sync
+  first_cluster:
+    config:
+      execute_itr_count: 2

Review comment:
       nope. execute this node only during iteration count 2. 




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

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



[GitHub] [hudi] n3nash commented on pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

Posted by GitBox <gi...@apache.org>.
n3nash commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-754414853


   @nsivabalan Please work with @satishkotha before landing this since he is also looking to add similar tests around clustering


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

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



[GitHub] [hudi] nsivabalan commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-774571369


   @vinothchandar : yes, I synced up with nishith yesterday on some of the pending comments. we should land it sooner. 


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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567568366



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -707,7 +707,7 @@ public abstract void commitCompaction(String compactionInstantTime, O writeStatu
    */
   protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses,
                                              HoodieTable<T, I, K, O> table, String compactionCommitTime);
-  
+

Review comment:
       Revert this change ?




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

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



[GitHub] [hudi] codecov-io edited a comment on pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036






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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569713



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;
+
+        int maxCommitsRetailed = executionContext.getHoodieTestSuiteWriter().getWriteConfig() != null ?
+            executionContext.getHoodieTestSuiteWriter().getWriteConfig().getCleanerCommitsRetained() + 1 : 11;
+        FileSystem fs = new Path(hudiPath)

Review comment:
       Instead, can we get fileSystem from FsUtils please ?




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567567637



##########
File path: docker/demo/config/test-suite/complex-dag-mor.yaml
##########
@@ -13,107 +13,72 @@
 # 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.
-dag_name: complex-dag-mor.yaml
+dag_name: cow-long-running-example.yaml

Review comment:
       @nsivabalan This dag is is being named as cow-long-running-example but it is only running 1 round, is this intended ?




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

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



[GitHub] [hudi] n3nash commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-773542370


   @nsivabalan Is this ready ?


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

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



[GitHub] [hudi] n3nash commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-773542370


   @nsivabalan Is this ready ?


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

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



[GitHub] [hudi] codecov-io edited a comment on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=h1) Report
   > Merging [#2400](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=desc) (729c34c) into [master](https://codecov.io/gh/apache/hudi/commit/a38612b10f6ae04644519270f9b5eb631a77c148?el=desc) (a38612b) will **increase** coverage by `18.75%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2400/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2400       +/-   ##
   =============================================
   + Coverage     50.69%   69.44%   +18.75%     
   + Complexity     3059      357     -2702     
   =============================================
     Files           419       53      -366     
     Lines         18810     1931    -16879     
     Branches       1924      230     -1694     
   =============================================
   - Hits           9535     1341     -8194     
   + Misses         8498      458     -8040     
   + Partials        777      132      -645     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.44% <0.00%> (-0.04%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `70.35% <0.00%> (-0.51%)` | `51.00 <0.00> (ø)` | |
   | [.../org/apache/hudi/io/storage/HoodieHFileReader.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaW8vc3RvcmFnZS9Ib29kaWVIRmlsZVJlYWRlci5qYXZh) | | | |
   | [.../apache/hudi/hive/MultiPartKeysValueExtractor.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTXVsdGlQYXJ0S2V5c1ZhbHVlRXh0cmFjdG9yLmphdmE=) | | | |
   | [...src/main/java/org/apache/hudi/QuickstartUtils.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvUXVpY2tzdGFydFV0aWxzLmphdmE=) | | | |
   | [...org/apache/hudi/cli/commands/RollbacksCommand.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL2NvbW1hbmRzL1JvbGxiYWNrc0NvbW1hbmQuamF2YQ==) | | | |
   | [...he/hudi/hadoop/SafeParquetRecordReaderWrapper.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL1NhZmVQYXJxdWV0UmVjb3JkUmVhZGVyV3JhcHBlci5qYXZh) | | | |
   | [...e/hudi/common/table/log/block/HoodieDataBlock.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9ibG9jay9Ib29kaWVEYXRhQmxvY2suamF2YQ==) | | | |
   | [.../apache/hudi/common/fs/TimedFSDataInputStream.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL1RpbWVkRlNEYXRhSW5wdXRTdHJlYW0uamF2YQ==) | | | |
   | [...util/jvm/OpenJ9MemoryLayoutSpecification32bit.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvanZtL09wZW5KOU1lbW9yeUxheW91dFNwZWNpZmljYXRpb24zMmJpdC5qYXZh) | | | |
   | [...org/apache/hudi/common/util/collection/Triple.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvY29sbGVjdGlvbi9UcmlwbGUuamF2YQ==) | | | |
   | ... and [358 more](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree-more) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571431179



##########
File path: docker/demo/config/test-suite/complex-dag-mor.yaml
##########
@@ -13,107 +13,72 @@
 # 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.
-dag_name: complex-dag-mor.yaml
+dag_name: cow-long-running-example.yaml
 dag_rounds: 1
-dag_intermittent_delay_mins: 10
+dag_intermittent_delay_mins: 1
 dag_content:
   first_insert:
     config:
-      record_size: 70000
+      record_size: 100

Review comment:
       anyways, w/o complex schema, we are not generating records of size 7000 bytes. So, thought will keep it to some sane value. 




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571432278



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ClusteringNode.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+/**
+ * Triggers inline clustering. Works only with writeClient. Also, add this as last node and end with validation if possible. As of now, after clustering, further inserts/upserts may not work since we

Review comment:
       I have created a follow up [task](https://issues.apache.org/jira/browse/HUDI-1590) for satish to look into async clustering. I went through clustering test examples and added support for inline. haven't much time to get async clustering working. 




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571432701



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;
+
+        int maxCommitsRetailed = executionContext.getHoodieTestSuiteWriter().getWriteConfig() != null ?

Review comment:
       have fixed this. I instantiate writeClientConfig in HoodieTestSuiteWriter for both code paths(deltastreamer, writeClient) and use it 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.

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569285



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java
##########
@@ -40,10 +40,11 @@ public CompactNode(Config config) {
    * if it has one.
    *
    * @param executionContext Execution context to run this compaction
+   * @param itrCount
    * @throws Exception  will be thrown if any error occurred.
    */
   @Override
-  public void execute(ExecutionContext executionContext) throws Exception {
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {

Review comment:
       I'd like to use the config to drive the itrCount and not have this signature change please..




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r554496508



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       @nsivabalan This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants. But I think this is already fixed with this PR -> https://github.com/apache/hudi/pull/2400




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

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



[GitHub] [hudi] vinothchandar commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-774540359


   @nsivabalan can we create a JIRA for this. prioritize this higher and land it. We want to get the integ-suite into good shape asap. 


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r552072132



##########
File path: docker/demo/config/test-suite/complex-dag-cow.yaml
##########
@@ -14,41 +14,47 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 dag_name: cow-long-running-example.yaml
-dag_rounds: 2
+dag_rounds: 1

Review comment:
       ignore reviewing these two files for now:
   complex-dag-cow.yaml
   and complex-dag-mor.yaml
   I am yet to fix these. 

##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java
##########
@@ -111,17 +111,19 @@ public void execute(ExecutionContext context) throws Exception {
       throw new AssertionError("Hudi contents does not match contents input data. ");
     }
 
-    String database = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY());
-    String tableName = context.getWriterContext().getProps().getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
-    log.warn("Validating hive table with db : " + database + " and table : " + tableName);
-    Dataset<Row> cowDf = session.sql("SELECT * FROM " + database + "." + tableName);
-    Dataset<Row> trimmedCowDf = cowDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD)
-        .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD);
-    intersectionDf = inputSnapshotDf.intersect(trimmedDf);
-    // the intersected df should be same as inputDf. if not, there is some mismatch.
-    if (inputSnapshotDf.except(intersectionDf).count() != 0) {
-      log.error("Data set validation failed for COW hive table. Total count in hudi " + trimmedCowDf.count() + ", input df count " + inputSnapshotDf.count());
-      throw new AssertionError("Hudi hive table contents does not match contents input data. ");
+    if (config.isValidateHive()) {

Review comment:
       no changes except adding this if condition

##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java
##########
@@ -41,21 +46,17 @@
   private GenericRecord lastRecord;
   // Partition path field name
   private Set<String> partitionPathFieldNames;
-  private String firstPartitionPathField;
 
   public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, String schema) {
     this(maxEntriesToProduce, GenericRecordFullPayloadGenerator.DEFAULT_PAYLOAD_SIZE, schema, null, 0);
   }
 
   public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, int minPayloadSize, String schemaStr,
-      List<String> partitionPathFieldNames, int numPartitions) {
+      List<String> partitionPathFieldNames, int partitionIndex) {

Review comment:
       all changes in this file are part of reverting e33a8f733c4a9a94479c166ad13ae9d53142cd3f

##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java
##########
@@ -127,9 +127,13 @@ public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, Sp
     return ws;
   }
 
+  public int getBatchId() {
+    return this.batchId;
+  }
+
   public JavaRDD<GenericRecord> generateInserts(Config operation) {
     int numPartitions = operation.getNumInsertPartitions();
-    long recordsPerPartition = operation.getNumRecordsInsert();
+    long recordsPerPartition = operation.getNumRecordsInsert() / numPartitions;

Review comment:
       these are part of reverting e33a8f733c4a9a94479c166ad13ae9d53142cd3f

##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java
##########
@@ -140,7 +144,7 @@ public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, Sp
     JavaRDD<GenericRecord> inputBatch = jsc.parallelize(partitionIndexes, numPartitions)
         .mapPartitionsWithIndex((index, p) -> {
           return new LazyRecordGeneratorIterator(new FlexibleSchemaRecordGenerationIterator(recordsPerPartition,
-              minPayloadSize, schemaStr, partitionPathFieldNames, numPartitions));
+            minPayloadSize, schemaStr, partitionPathFieldNames, (Integer)index));

Review comment:
       these are part of reverting e33a8f733c4a9a94479c166ad13ae9d53142cd3f

##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java
##########
@@ -46,9 +46,9 @@
  */
 public class GenericRecordFullPayloadGenerator implements Serializable {
 
-  private static final Logger LOG = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class);
+  private static Logger LOG = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class);

Review comment:
       all changes in this file are part of reverting e33a8f733c4a9a94479c166ad13ae9d53142cd3f

##########
File path: hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java
##########
@@ -134,38 +133,4 @@ public void testComplexPayloadWithLargeMinSize() throws Exception {
     assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize + 0.1 * minPayloadSize);
   }
 
-  @Test

Review comment:
       these are part of reverting e33a8f733c4a9a94479c166ad13ae9d53142cd3f

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       @n3nash @satishkotha : after clustering, when we call readFromSource in deltaStreamer, execution was going into lines 314 to 316. hence have added this condition to return empty checkpoint. Can you confirm this looks 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.

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



[GitHub] [hudi] yihua commented on a change in pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r762399404



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       `This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants.`
   
   @n3nash @nsivabalan This is not correct.  The commit timeline does have the clustering instants (`replacecommit`) based on the changes in #2048.  So this breaks the logic of getting the last checkpoint, even if we have the walk-back logic in #4034, which is skipped.




-- 
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@hudi.apache.org

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567568943



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CleanNode.java
##########
@@ -32,7 +32,7 @@ public CleanNode(Config config) {
   }
 
   @Override
-  public void execute(ExecutionContext executionContext) throws Exception {
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {

Review comment:
       Can we just use "REPEAT_COUNT" ?




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571432394



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java
##########
@@ -88,6 +88,10 @@ public Configuration getConfiguration() {
     private static String REINIT_CONTEXT = "reinitialize_context";
     private static String START_PARTITION = "start_partition";
     private static String DELETE_INPUT_DATA = "delete_input_data";
+    private static String VALIDATE_HIVE = "validate_hive";
+    private static String EXECUTE_ITR_COUNT = "execute_itr_count";

Review comment:
       may be "itr_count_to_execute" to be explicit. 




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

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



[GitHub] [hudi] yihua commented on a change in pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r762399404



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       `This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants.`
   
   @n3nash @nsivabalan This is not correct.  The commit time does have the clustering instants (`replacecommit`) based on the changes in #2048.  So this breaks the logic of getting the last checkpoint, even if we have the walk-back logic in #4034, which is skipped.




-- 
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@hudi.apache.org

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r554496508



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       @nsivabalan This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants. 




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r575384984



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ClusteringNode.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+/**
+ * Triggers inline clustering. Works only with writeClient. Also, add this as last node and end with validation if possible. As of now, after clustering, further inserts/upserts may not work since we

Review comment:
       Thanks, Satish has a PR ready as well.




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571431931



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java
##########
@@ -88,6 +88,10 @@ public Configuration getConfiguration() {
     private static String REINIT_CONTEXT = "reinitialize_context";
     private static String START_PARTITION = "start_partition";
     private static String DELETE_INPUT_DATA = "delete_input_data";
+    private static String VALIDATE_HIVE = "validate_hive";
+    private static String EXECUTE_ITR_COUNT = "execute_itr_count";

Review comment:
       @n3nash : this refers to the iteration count/number where this node needs to be executed. Basically "execute at iteration count" config. I couldn't come up w/ a better name. If you have good suggestions, let me know. 




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567568804



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java
##########
@@ -88,6 +88,10 @@ public Configuration getConfiguration() {
     private static String REINIT_CONTEXT = "reinitialize_context";
     private static String START_PARTITION = "start_partition";
     private static String DELETE_INPUT_DATA = "delete_input_data";
+    private static String VALIDATE_HIVE = "validate_hive";
+    private static String EXECUTE_ITR_COUNT = "execute_itr_count";

Review comment:
       Can we just rename this to REPEAT_COUNT and use it that way (if that's the intention) ?




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567568178



##########
File path: docker/demo/config/test-suite/cow-clustering-example.yaml
##########
@@ -0,0 +1,76 @@
+# 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.
+dag_name: cow-clustering-example.yaml
+dag_rounds: 3
+dag_intermittent_delay_mins: 0
+dag_content:
+  first_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 1000
+    type: InsertNode
+    deps: none
+  second_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 10000
+    deps: first_insert
+    type: InsertNode
+  third_insert:
+    config:
+      record_size: 1000
+      num_partitions_insert: 1
+      repeat_count: 1
+      num_records_insert: 300
+    deps: second_insert
+    type: InsertNode
+  first_delete:
+    config:
+      num_partitions_delete: 1
+      num_records_delete: 9000
+    type: DeleteNode
+    deps: third_insert
+  first_hive_sync:
+    config:
+      queue_name: "adhoc"
+      engine: "mr"
+    type: HiveSyncNode
+    deps: first_delete
+  first_validate:
+    config:
+      validate_hive: true
+    type: ValidateDatasetNode
+    deps: first_hive_sync
+  first_cluster:
+    config:
+      execute_itr_count: 2

Review comment:
       @nsivabalan Is this supposed to mean run this twice ? 




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

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



[GitHub] [hudi] codecov-io edited a comment on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=h1) Report
   > Merging [#2400](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=desc) (ab40bd6) into [master](https://codecov.io/gh/apache/hudi/commit/698694a1571cdcc9848fc79aa34c8cbbf9662bc4?el=desc) (698694a) will **decrease** coverage by `40.20%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2400/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2400       +/-   ##
   =============================================
   - Coverage     50.23%   10.03%   -40.21%     
   + Complexity     2985       48     -2937     
   =============================================
     Files           410       52      -358     
     Lines         18398     1854    -16544     
     Branches       1884      224     -1660     
   =============================================
   - Hits           9242      186     -9056     
   + Misses         8398     1655     -6743     
   + Partials        758       13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.03% <0.00%> (-59.63%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `0.00% <0.00%> (-70.76%)` | `0.00 <0.00> (-49.00)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree-more) | |
   


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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569480



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;

Review comment:
       s/hudiPath/basePath




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

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



[GitHub] [hudi] nsivabalan commented on pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-778473048


   @n3nash : https://issues.apache.org/jira/browse/HUDI-1616
   @vinothchandar : sure. I will sync up with nishith on this and will take it up. 


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

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



[GitHub] [hudi] codecov-io commented on pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=h1) Report
   > Merging [#2400](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=desc) (4f4c081) into [master](https://codecov.io/gh/apache/hudi/commit/da51aa64fcaf8cd3099ef9c085c207283999306f?el=desc) (da51aa6) will **decrease** coverage by `42.17%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2400/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2400       +/-   ##
   =============================================
   - Coverage     52.21%   10.04%   -42.18%     
   + Complexity     2662       48     -2614     
   =============================================
     Files           335       52      -283     
     Lines         14983     1851    -13132     
     Branches       1506      223     -1283     
   =============================================
   - Hits           7824      186     -7638     
   + Misses         6535     1652     -4883     
   + Partials        624       13      -611     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <0.00%> (-59.61%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `0.00% <ø> (-70.76%)` | `0.00 <0.00> (-49.00)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `0.00% <0.00%> (-68.98%)` | `0.00 <0.00> (-18.00)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | ... and [314 more](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree-more) | |
   


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

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



[GitHub] [hudi] codecov-io edited a comment on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=h1) Report
   > Merging [#2400](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=desc) (729c34c) into [master](https://codecov.io/gh/apache/hudi/commit/a38612b10f6ae04644519270f9b5eb631a77c148?el=desc) (a38612b) will **decrease** coverage by `41.00%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2400/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2400       +/-   ##
   ============================================
   - Coverage     50.69%   9.68%   -41.01%     
   + Complexity     3059      48     -3011     
   ============================================
     Files           419      53      -366     
     Lines         18810    1931    -16879     
     Branches       1924     230     -1694     
   ============================================
   - Hits           9535     187     -9348     
   + Misses         8498    1731     -6767     
   + Partials        777      13      -764     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <0.00%> (-59.80%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `0.00% <0.00%> (-70.87%)` | `0.00 <0.00> (-51.00)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | ... and [397 more](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree-more) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571432157



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CleanNode.java
##########
@@ -32,7 +32,7 @@ public CleanNode(Config config) {
   }
 
   @Override
-  public void execute(ExecutionContext executionContext) throws Exception {
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {

Review comment:
       as synced up offline, this refers to current iteration count. Have renamed this arg to "curItrCount".




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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r571432328



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java
##########
@@ -40,10 +40,11 @@ public CompactNode(Config config) {
    * if it has one.
    *
    * @param executionContext Execution context to run this compaction
+   * @param itrCount
    * @throws Exception  will be thrown if any error occurred.
    */
   @Override
-  public void execute(ExecutionContext executionContext) throws Exception {
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {

Review comment:
       yeah, this refers to current iteration count. Don't think we can get away with 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.

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



[GitHub] [hudi] yihua commented on a change in pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
yihua commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r762399404



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       `This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants.`
   
   @n3nash @nsivabalan This is not correct.  The commit time does have the clustering instants (r`eplacecommit`) based on the changes in #2048.  So this breaks the logic of getting the last checkpoint, even if we have the walk-back logic in #4034, which is skipped.




-- 
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@hudi.apache.org

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r554496508



##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       @nsivabalan This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants. 

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
##########
@@ -307,7 +307,10 @@ public void refreshTimeline() throws IOException {
           if (!commitMetadata.getMetadata(CHECKPOINT_KEY).isEmpty()) {
             resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
           }
-        }  else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        } else if (commitMetadata.getOperationType() == WriteOperationType.CLUSTER) {
+          // incase of CLUSTER commit, no checkpoint will be available in metadata.

Review comment:
       @nsivabalan This should be fine since the commit timeline to read the CHECKPOINT from does not include the clustering instants. But I think this is already fixed with this PR -> https://github.com/apache/hudi/pull/2400




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

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



[GitHub] [hudi] codecov-io edited a comment on pull request #2400: [WIP] Some fixes to test suite framework. Adding clustering node

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-753557036


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=h1) Report
   > Merging [#2400](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=desc) (4f4c081) into [master](https://codecov.io/gh/apache/hudi/commit/da51aa64fcaf8cd3099ef9c085c207283999306f?el=desc) (da51aa6) will **decrease** coverage by `42.17%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2400/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2400       +/-   ##
   =============================================
   - Coverage     52.21%   10.04%   -42.18%     
   + Complexity     2662       48     -2614     
   =============================================
     Files           335       52      -283     
     Lines         14983     1851    -13132     
     Branches       1506      223     -1283     
   =============================================
   - Hits           7824      186     -7638     
   + Misses         6535     1652     -4883     
   + Partials        624       13      -611     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <0.00%> (-59.61%)` | `0.00 <0.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2400?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `0.00% <ø> (-70.76%)` | `0.00 <0.00> (-49.00)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `0.00% <0.00%> (-68.98%)` | `0.00 <0.00> (-18.00)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | ... and [314 more](https://codecov.io/gh/apache/hudi/pull/2400/diff?src=pr&el=tree-more) | |
   


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

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



[GitHub] [hudi] nsivabalan commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567760070



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;
+
+        int maxCommitsRetailed = executionContext.getHoodieTestSuiteWriter().getWriteConfig() != null ?
+            executionContext.getHoodieTestSuiteWriter().getWriteConfig().getCleanerCommitsRetained() + 1 : 11;
+        FileSystem fs = new Path(hudiPath)
+            .getFileSystem(executionContext.getHoodieTestSuiteWriter().getConfiguration());
+        Map<String, Integer> fileIdCount = new HashMap<>();
+        int maxVal = 0;
+        RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(hudiPath), true);
+        while (itr.hasNext()) {

Review comment:
       sure. 




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569928



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;
+
+        int maxCommitsRetailed = executionContext.getHoodieTestSuiteWriter().getWriteConfig() != null ?
+            executionContext.getHoodieTestSuiteWriter().getWriteConfig().getCleanerCommitsRetained() + 1 : 11;
+        FileSystem fs = new Path(hudiPath)
+            .getFileSystem(executionContext.getHoodieTestSuiteWriter().getConfiguration());
+        Map<String, Integer> fileIdCount = new HashMap<>();
+        int maxVal = 0;
+        RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(hudiPath), true);
+        while (itr.hasNext()) {

Review comment:
       Let's chat about all of this logic 1-1. This code is very difficult to read..may be there is a better way to achieve what you are trying




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569596



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateAsyncOperations.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc.
+ */
+public class ValidateAsyncOperations extends DagNode<Option<String>> {
+
+  public ValidateAsyncOperations(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext, int itrCount) throws Exception {
+    if (config.getExecuteIterationCount() == itrCount) {
+      try {
+        log.warn("Executing ValidateHoodieAsyncOperations node {} with target base path {} ", this.getName(),
+            executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+        String hudiPath = executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath;
+
+        int maxCommitsRetailed = executionContext.getHoodieTestSuiteWriter().getWriteConfig() != null ?

Review comment:
       s/maxCommitsRetailed/maxCommitsRetained. 
   
   Also, not sure what `11` means..




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

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



[GitHub] [hudi] vinothchandar commented on pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-778344811


   @n3nash @nsivabalan I would like for some of this to run on every commit. at least 1 test for each COW and MOR. Can one of you be able to add that to the CI?


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

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



[GitHub] [hudi] n3nash merged pull request #2400: [HUDI-1594] Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash merged pull request #2400:
URL: https://github.com/apache/hudi/pull/2400


   


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

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



[GitHub] [hudi] nsivabalan commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-754812328


   @n3nash : Patch is ready for review. 
   @satishkotha : I have added clustering node. Do check it out. 


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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567567678



##########
File path: docker/demo/config/test-suite/complex-dag-mor.yaml
##########
@@ -13,107 +13,72 @@
 # 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.
-dag_name: complex-dag-mor.yaml
+dag_name: cow-long-running-example.yaml
 dag_rounds: 1
-dag_intermittent_delay_mins: 10
+dag_intermittent_delay_mins: 1
 dag_content:
   first_insert:
     config:
-      record_size: 70000
+      record_size: 100

Review comment:
       Record size of 100 bytes ?




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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567568684



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
##########
@@ -178,6 +189,19 @@ private boolean allowWriteClientAccess(DagNode dagNode) {
     }
   }
 
+  public void inlineClustering() {
+    if (!cfg.useDeltaStreamer) {
+      Option<String> clusteringInstantOpt = writeClient.scheduleClustering(Option.empty());
+      clusteringInstantOpt.ifPresent(clusteringInstant -> {
+        // inline cluster should auto commit as the user is never given control
+        log.warn("Clustering instant :: " + clusteringInstant);
+        writeClient.cluster(clusteringInstant, true);
+      });
+    } else{
+      throw new IllegalArgumentException("Clustering cannot be triggered with deltastreamer");

Review comment:
       @nsivabalan Can you add a TODO 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.

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



[GitHub] [hudi] nsivabalan commented on pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#issuecomment-774571205


   @n3nash : Addressed all your comments.


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

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



[GitHub] [hudi] n3nash commented on a change in pull request #2400: Some fixes and enhancements to test suite framework

Posted by GitBox <gi...@apache.org>.
n3nash commented on a change in pull request #2400:
URL: https://github.com/apache/hudi/pull/2400#discussion_r567569175



##########
File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ClusteringNode.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+/**
+ * Triggers inline clustering. Works only with writeClient. Also, add this as last node and end with validation if possible. As of now, after clustering, further inserts/upserts may not work since we

Review comment:
       Is this because the checkpoint is not passed ? There is some code that passes the checkpoint even if write client is used. 




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

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