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/10/11 07:14:33 UTC

[GitHub] [hudi] yihua opened a new pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

yihua opened a new pull request #3778:
URL: https://github.com/apache/hudi/pull/3778


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute 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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+@Deprecated

Review comment:
       Do we still keep these classes as user-facing?  Internally all the built-in indices directly extend `HoodieIndex`.




-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
##########
@@ -85,7 +85,8 @@
 /**
  * Hoodie Index implementation backed by HBase.
  */
-public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
+public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
+    extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {

Review comment:
       So for all internal indices, I make them directly extend `HoodieIndex`.  The `JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>` for `I, K, O` are anyway not used by the new APIs, and they're not supposed to be used in the new abstraction.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java
##########
@@ -7,72 +7,77 @@
  * "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
+ *   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.
+ * 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.index;
+package org.apache.hudi.index.inmemory;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.stream.Collectors;
 
 /**
  * Hoodie Index implementation backed by an in-memory Hash map.
  * <p>
  * ONLY USE FOR LOCAL TESTING
  */
-@SuppressWarnings("checkstyle:LineLength")
-public class JavaInMemoryHashIndex<T extends HoodieRecordPayload> extends JavaHoodieIndex<T> {
+public class HoodieInMemoryHashIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
 
   private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
 
-  public JavaInMemoryHashIndex(HoodieWriteConfig config) {
+  public HoodieInMemoryHashIndex(HoodieWriteConfig config) {
     super(config);
-    synchronized (JavaInMemoryHashIndex.class) {
+    synchronized (HoodieInMemoryHashIndex.class) {
       if (recordLocationMap == null) {
         recordLocationMap = new ConcurrentHashMap<>();
       }
     }
   }
 
   @Override
-  public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records, HoodieEngineContext context,
-                                           HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
-    List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
-    records.stream().forEach(record -> {
-      if (recordLocationMap.containsKey(record.getKey())) {
-        record.unseal();
-        record.setCurrentLocation(recordLocationMap.get(record.getKey()));
-        record.seal();
+  public HoodieData<HoodieRecord<T>> tagLocation(
+      HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    return records.mapPartitions(hoodieRecordIterator -> {

Review comment:
       The change here is only regarding the transformations.  For Java List, `mapPartitions()` simply falls back to `forEachRemaining()` which is the same as `forEach()`.




-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28ca85cd2f8edecf808560538a5c5f5e4de962 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578) 
   * bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -101,7 +101,7 @@
   private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
 
   protected final transient HoodieMetrics metrics;
-  private final transient HoodieIndex<T, I, K, O> index;
+  private final transient HoodieIndex index;

Review comment:
       Actually, I still keep the definition of `HoodieIndex<T extends HoodieRecordPayload, I, K, O>` for compatibility and add back the typed parameters here.




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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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



[GitHub] [hudi] nsivabalan merged pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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


   


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523






-- 
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] vinothchandar commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java
##########
@@ -53,17 +53,19 @@ protected HoodieIndex(HoodieWriteConfig config) {
    * present).
    */
   @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract I tagLocation(I records, HoodieEngineContext context,
-                                HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException;
+  public abstract HoodieData<HoodieRecord<T>> tagLocation(

Review comment:
       this is a public API change? I think people typically extend the Spark and Flink subclasses. Something to check. 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
##########
@@ -46,6 +46,36 @@
   public HoodieWriteMetadata() {
   }
 
+  /**
+   * Clones the write metadata with transformed write statuses.
+   *
+   * @param transformedWriteStatuses transformed write statuses
+   * @param <T>                      type of transformed write statuses
+   * @return Cloned {@link HoodieWriteMetadata<T>} instance
+   */
+  public <T> HoodieWriteMetadata<T> clone(T transformedWriteStatuses) {

Review comment:
       this may be changes from the other PR - that introduces `HoodieData`?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -101,7 +101,7 @@
   private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
 
   protected final transient HoodieMetrics metrics;
-  private final transient HoodieIndex<T, I, K, O> index;
+  private final transient HoodieIndex index;

Review comment:
       do we need a. `HoodieIndex<?>` 




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bede6e6be4fda354793dc429de79e863f35c20ae Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867) 
   * 2a3dab04e4119ffff022e35debf2753fbabc89b3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906",
       "triggerID" : "953109158",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] nsivabalan commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/inmemory/HoodieInMemoryHashIndex.java
##########
@@ -7,72 +7,77 @@
  * "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
+ *   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.
+ * 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.index;
+package org.apache.hudi.index.inmemory;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.stream.Collectors;
 
 /**
  * Hoodie Index implementation backed by an in-memory Hash map.
  * <p>
  * ONLY USE FOR LOCAL TESTING
  */
-@SuppressWarnings("checkstyle:LineLength")
-public class JavaInMemoryHashIndex<T extends HoodieRecordPayload> extends JavaHoodieIndex<T> {
+public class HoodieInMemoryHashIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
 
   private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
 
-  public JavaInMemoryHashIndex(HoodieWriteConfig config) {
+  public HoodieInMemoryHashIndex(HoodieWriteConfig config) {
     super(config);
-    synchronized (JavaInMemoryHashIndex.class) {
+    synchronized (HoodieInMemoryHashIndex.class) {
       if (recordLocationMap == null) {
         recordLocationMap = new ConcurrentHashMap<>();
       }
     }
   }
 
   @Override
-  public List<HoodieRecord<T>> tagLocation(List<HoodieRecord<T>> records, HoodieEngineContext context,
-                                           HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> hoodieTable) {
-    List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
-    records.stream().forEach(record -> {
-      if (recordLocationMap.containsKey(record.getKey())) {
-        record.unseal();
-        record.setCurrentLocation(recordLocationMap.get(record.getKey()));
-        record.seal();
+  public HoodieData<HoodieRecord<T>> tagLocation(
+      HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    return records.mapPartitions(hoodieRecordIterator -> {

Review comment:
       nit. I see you are changing the logic a bit. in general, try to avoid making changes to logic in refactoring PRs. we may not catch any bugs around it since reviewers mostly focus on abstractions, code-reuse etc. 
   

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.index.bloom;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.MetadataNotFoundException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.io.HoodieRangeInfoHandle;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.mapping;
+import static java.util.stream.Collectors.toList;
+import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
+
+/**
+ * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata.
+ */
+public class HoodieBloomIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {

Review comment:
       I assume this is just rename of HoodieBaseBloomIndex with HoodieBloomIndexHelper in addition. Let me know if there are any other additional changes

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.index.simple;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.io.HoodieKeyLocationFetchHandle;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
+
+/**
+ * A simple index which reads interested fields(record key and partition path) from base files and
+ * joins with incoming records to find the tagged location.
+ *
+ * @param <T> type of {@link HoodieRecordPayload}
+ */
+public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {

Review comment:
       any changes as such compared to SparkHoodieSimpleIndex ?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
##########
@@ -18,54 +18,51 @@
 
 package org.apache.hudi.index;
 
-import org.apache.hudi.ApiMaturityLevel;
-import org.apache.hudi.PublicAPIMethod;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
-import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.HoodieBloomIndex;
+import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper;
 import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
-import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex;
-import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
+import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
+import org.apache.hudi.index.simple.HoodieSimpleIndex;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
 
-import org.apache.spark.api.java.JavaRDD;
+import java.io.IOException;
 
-@SuppressWarnings("checkstyle:LineLength")
-public abstract class SparkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
-  protected SparkHoodieIndex(HoodieWriteConfig config) {
-    super(config);
-  }
-
-  public static SparkHoodieIndex createIndex(HoodieWriteConfig config) {
+/**
+ * A factory to generate Spark {@link HoodieIndex}.
+ */
+public final class SparkHoodieIndexFactory {

Review comment:
       yeah, its been a long due to fix the naming to factory :) thanks.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.index.bloom;
+
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDDData;
+import org.apache.hudi.data.HoodieJavaRDDData;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+/**
+ * Helper for {@link HoodieBloomIndex} containing Spark-specific logic.
+ */
+public class SparkHoodieBloomIndexHelper extends HoodieBloomIndexHelper {
+
+  private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndexHelper.class);
+
+  private static final SparkHoodieBloomIndexHelper SINGLETON_INSTANCE =
+      new SparkHoodieBloomIndexHelper();
+
+  private SparkHoodieBloomIndexHelper() {
+  }
+
+  public static SparkHoodieBloomIndexHelper getInstance() {
+    return SINGLETON_INSTANCE;
+  }
+
+  @Override

Review comment:
       I assume no code changes as such and you have just copied over as is. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.common.data;
+
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
+
+/**
+ * Holds a {@link List} of objects.
+ *
+ * @param <T> type of object.
+ */
+public class HoodieListData<T> extends HoodieData<T> {
+
+  private final List<T> listData;
+
+  private HoodieListData(List<T> listData) {
+    this.listData = listData;
+  }
+
+  /**
+   * @param listData a {@link List} of objects in type T.
+   * @param <T>      type of object.
+   * @return a new instance containing the {@link List<T>} reference.
+   */
+  public static <T> HoodieListData<T> of(List<T> listData) {
+    return new HoodieListData<>(listData);
+  }
+
+  /**
+   * @param hoodieData {@link HoodieListData<T>} instance containing the {@link List} of objects.
+   * @param <T>        type of object.
+   * @return the a {@link List} of objects in type T.
+   */
+  public static <T> List<T> getList(HoodieData<T> hoodieData) {
+    return ((HoodieListData<T>) hoodieData).get();
+  }
+
+  @Override
+  public List<T> get() {
+    return listData;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return listData.isEmpty();
+  }
+
+  @Override
+  public void persist(String cacheConfig) {
+    // No OP
+  }
+
+  @Override
+  public void unpersist() {
+    // No OP
+  }
+
+  @Override
+  public long count() {
+    return listData.size();
+  }
+
+  @Override
+  public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
+    return HoodieListData.of(listData.stream().parallel()
+        .map(throwingMapWrapper(func)).collect(Collectors.toList()));
+  }
+
+  @Override
+  public <O> HoodieData<O> mapPartitions(
+      SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
+    List<O> result = new ArrayList<>();
+    throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add);

Review comment:
       this is effectively a map operation right. there is no map Partitions equivalent I guess

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
##########
@@ -18,54 +18,51 @@
 
 package org.apache.hudi.index;
 
-import org.apache.hudi.ApiMaturityLevel;
-import org.apache.hudi.PublicAPIMethod;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
-import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.HoodieBloomIndex;
+import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper;
 import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
-import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex;
-import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
+import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
+import org.apache.hudi.index.simple.HoodieSimpleIndex;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
 
-import org.apache.spark.api.java.JavaRDD;
+import java.io.IOException;
 
-@SuppressWarnings("checkstyle:LineLength")
-public abstract class SparkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
-  protected SparkHoodieIndex(HoodieWriteConfig config) {
-    super(config);
-  }
-
-  public static SparkHoodieIndex createIndex(HoodieWriteConfig config) {
+/**
+ * A factory to generate Spark {@link HoodieIndex}.
+ */
+public final class SparkHoodieIndexFactory {
+  public static HoodieIndex createIndex(HoodieWriteConfig config) {
     // first use index class config to create index.
     if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
       Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
       if (!(instance instanceof HoodieIndex)) {
         throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
       }
-      return (SparkHoodieIndex) instance;
+      return (HoodieIndex) instance;
     }
     switch (config.getIndexType()) {
       case HBASE:
         return new SparkHoodieHBaseIndex<>(config);
       case INMEMORY:
-        return new SparkInMemoryHashIndex(config);
+        return new HoodieInMemoryHashIndex<>(config);
       case BLOOM:
-        return new SparkHoodieBloomIndex<>(config);
+        return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
       case GLOBAL_BLOOM:
-        return new SparkHoodieGlobalBloomIndex<>(config);
+        return new HoodieGlobalBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
       case SIMPLE:
-        return new SparkHoodieSimpleIndex(config);
+        return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));

Review comment:
       not sure why we have to instantiate keyGen here and pass it in. All we need is a write config which we anyways pass to simpleIndex. may I know why can't we keep the keyGen instantiation within index impl. also, looks out of place having  this in factory required only for some of the indexes.

##########
File path: hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java
##########
@@ -62,6 +67,21 @@ public HoodieJavaEngineContext(Configuration conf, TaskContextSupplier taskConte
     super(new SerializableConfiguration(conf), taskContextSupplier);
   }
 
+  @Override
+  public HoodieAccumulator createNewAccumulator() {

Review comment:
       not required in this patch. I see lot similarities between flink engine context and java engine context. Did you think about adding ListBasedEngineContext and make Flink and Java engine contexts inherit from it. so that we can reuse most of the code. 

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPairData.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.common.data;
+
+import org.apache.hudi.common.function.FunctionWrapper;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
+
+/**
+ * Implementation of {@link HoodiePairData} using Java {@link Map}.
+ * The pairs are organized by the key in the Map and values for the same key
+ * are stored in a list as the value corresponding to the key in the Map.
+ *
+ * @param <K> type of key.
+ * @param <V> type of value.
+ */
+public class HoodieMapPairData<K, V> extends HoodiePairData<K, V> {
+
+  private final Map<K, List<V>> mapPairData;
+
+  private HoodieMapPairData(Map<K, List<V>> mapPairData) {
+    this.mapPairData = mapPairData;
+  }
+
+  /**
+   * @param mapPairData a {@link Map} of pairs.
+   * @param <K>         type of key.
+   * @param <V>         type of value.
+   * @return a new instance containing the {@link Map<K, List<V>>} reference.
+   */
+  public static <K, V> HoodieMapPairData<K, V> of(Map<K, List<V>> mapPairData) {
+    return new HoodieMapPairData<>(mapPairData);
+  }
+
+  /**
+   * @param hoodiePairData {@link HoodieMapPairData<K, V>} instance containing the {@link Map} of pairs.
+   * @param <K>            type of key.
+   * @param <V>            type of value.
+   * @return the {@link Map} of pairs.
+   */
+  public static <K, V> Map<K, List<V>> getMapPair(HoodiePairData<K, V> hoodiePairData) {
+    return ((HoodieMapPairData<K, V>) hoodiePairData).get();
+  }
+
+  @Override
+  public Map<K, List<V>> get() {
+    return mapPairData;
+  }
+
+  @Override
+  public void persist(String cacheConfig) {
+    // No OP
+  }
+
+  @Override
+  public void unpersist() {
+    // No OP
+  }
+
+  @Override
+  public HoodieData<K> keys() {
+    return HoodieListData.of(new ArrayList<>(mapPairData.keySet()));
+  }
+
+  @Override
+  public HoodieData<V> values() {
+    return HoodieListData.of(
+        mapPairData.values().stream().flatMap(List::stream).collect(Collectors.toList()));
+  }
+
+  @Override
+  public long count() {
+    return mapPairData.values().stream().map(
+        list -> (long) list.size()).reduce(Long::sum).orElse(0L);
+  }
+
+  @Override
+  public Map<K, Long> countByKey() {
+    return mapPairData.entrySet().stream().collect(
+        Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size()));
+  }
+
+  @Override
+  public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
+    Function<Pair<K, V>, O> throwableFunc = throwingMapWrapper(func);
+    return HoodieListData.of(
+        streamAllPairs().map(throwableFunc).collect(Collectors.toList()));
+  }
+
+  @Override
+  public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
+    Map<L, List<W>> newMap = new HashMap<>();
+    Function<Pair<K, V>, Pair<L, W>> throwableMapToPairFunc =
+        FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc);
+    streamAllPairs().map(pair -> throwableMapToPairFunc.apply(pair)).forEach(newPair -> {
+      List<W> list = newMap.computeIfAbsent(newPair.getKey(), k -> new ArrayList<>());
+      list.add(newPair.getValue());
+    });
+    return HoodieMapPairData.of(newMap);
+  }
+
+  @Override
+  public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {

Review comment:
       not required in this patch. But can you add unit tests for these classes and transformation methods. these are going to be extensively used across the board. So, would be good to have unit tests around these. 




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac66c7483b666b084556845fe1ca78111a360980 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666) 
   * 6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906",
       "triggerID" : "953109158",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906) 
   * e9a88dd7cb4959dbd23b064d20f9add722ebd29a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] nsivabalan commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexHelper.java
##########
@@ -20,41 +20,22 @@
 package org.apache.hudi.index.bloom;
 
 import org.apache.hudi.common.data.HoodieData;
-import org.apache.hudi.common.data.HoodieList;
 import org.apache.hudi.common.data.HoodiePairData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.util.collection.ImmutablePair;
-import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
 import org.apache.hudi.table.HoodieTable;
 
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static java.util.stream.Collectors.toList;
-
 /**
- * Helper for {@link HoodieBloomIndex} containing engine-specific logic,
- * with default logic supporting Java engines.
+ * Helper for {@link HoodieBloomIndex} containing engine-specific logic.
  */
-public class HoodieBloomIndexHelper implements Serializable {
-
-  private static final HoodieBloomIndexHelper SINGLETON_INSTANCE = new HoodieBloomIndexHelper();
-
-  protected HoodieBloomIndexHelper() {
-  }
-
-  public static HoodieBloomIndexHelper getInstance() {
-    return SINGLETON_INSTANCE;
-  }
-
+public abstract class HoodieBloomIndexHelper implements Serializable {

Review comment:
       should we name this base since this is abstract

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
##########
@@ -85,7 +85,8 @@
 /**
  * Hoodie Index implementation backed by HBase.
  */
-public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
+public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
+    extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {

Review comment:
       don't we need to extend from SparkHoodieIndex? 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+@Deprecated

Review comment:
       why do we need to add deprecated annotation to class? would suffice just adding to the methods that are being deprecated right? 

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
##########
@@ -200,7 +200,7 @@ public static synchronized void resetSpark() {
   protected JavaRDD<WriteStatus> updateLocation(
       HoodieIndex index, JavaRDD<WriteStatus> writeStatus, HoodieTable table) {
     return HoodieJavaRDD.getJavaRDD(
-        index.tagLocation(HoodieJavaRDD.of(writeStatus), context, table));
+        index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table));

Review comment:
       I assume we chased down all usages of older api in source code and tests and moved to the new ones added? 




-- 
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] nsivabalan commented on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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


   Good job on the PR. commendable work! 


-- 
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] nsivabalan commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+@Deprecated

Review comment:
       yes. let's leave it that way. Vinoth mentioned users usually extends from engine specific classes right. 




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845) 
   * 67c6e4a2fe83469fd99a821be21636867bf2ecb0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906",
       "triggerID" : "953109158",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28ca85cd2f8edecf808560538a5c5f5e4de962 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28ca85cd2f8edecf808560538a5c5f5e4de962 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
##########
@@ -200,7 +200,7 @@ public static synchronized void resetSpark() {
   protected JavaRDD<WriteStatus> updateLocation(
       HoodieIndex index, JavaRDD<WriteStatus> writeStatus, HoodieTable table) {
     return HoodieJavaRDD.getJavaRDD(
-        index.tagLocation(HoodieJavaRDD.of(writeStatus), context, table));
+        index.updateLocation(HoodieJavaRDD.of(writeStatus), context, table));

Review comment:
       Correct.




-- 
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] nsivabalan commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
##########
@@ -85,7 +85,8 @@
 /**
  * Hoodie Index implementation backed by HBase.
  */
-public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
+public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
+    extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {

Review comment:
       gotcha.




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906",
       "triggerID" : "953109158",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2922",
       "triggerID" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e9a88dd7cb4959dbd23b064d20f9add722ebd29a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2922) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+@Deprecated

Review comment:
       Right, sg.  In my last change, I removed the `@Deprecated` annotations for engine specific Hoodie index classes.




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906",
       "triggerID" : "953109158",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2922",
       "triggerID" : "e9a88dd7cb4959dbd23b064d20f9add722ebd29a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2906) 
   * e9a88dd7cb4959dbd23b064d20f9add722ebd29a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2922) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java
##########
@@ -0,0 +1,237 @@
+/*
+ * 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.index.bloom;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.MetadataNotFoundException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.io.HoodieRangeInfoHandle;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.mapping;
+import static java.util.stream.Collectors.toList;
+import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
+
+/**
+ * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata.
+ */
+public class HoodieBloomIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {

Review comment:
       Right, there is no logic change inside the bloom index.




-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -101,7 +101,7 @@
   private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
 
   protected final transient HoodieMetrics metrics;
-  private final transient HoodieIndex<T, I, K, O> index;
+  private final transient HoodieIndex index;

Review comment:
       For this particular one, I have to erase the type parameters here due to the incompatible type parameter, `T extends HoodieRecordPayload` instead of `T extends HoodieRecordPayload<T>`.  With wildcard `<?>` the code does not compile.  I'll clean this up in a separate PR tracked here: https://issues.apache.org/jira/browse/HUDI-2597.

##########
File path: hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/common/HoodieJavaEngineContext.java
##########
@@ -62,6 +67,21 @@ public HoodieJavaEngineContext(Configuration conf, TaskContextSupplier taskConte
     super(new SerializableConfiguration(conf), taskContextSupplier);
   }
 
+  @Override
+  public HoodieAccumulator createNewAccumulator() {

Review comment:
       I create a Jira ticket to track the issue: https://issues.apache.org/jira/projects/HUDI/issues/HUDI-2626.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.index.simple;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.io.HoodieKeyLocationFetchHandle;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.List;
+
+import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
+
+/**
+ * A simple index which reads interested fields(record key and partition path) from base files and
+ * joins with incoming records to find the tagged location.
+ *
+ * @param <T> type of {@link HoodieRecordPayload}
+ */
+public class HoodieSimpleIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {

Review comment:
       No logic change.  This class is transformed from `SparkHoodieSimpleIndex`.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
##########
@@ -46,6 +46,36 @@
   public HoodieWriteMetadata() {
   }
 
+  /**
+   * Clones the write metadata with transformed write statuses.
+   *
+   * @param transformedWriteStatuses transformed write statuses
+   * @param <T>                      type of transformed write statuses
+   * @return Cloned {@link HoodieWriteMetadata<T>} instance
+   */
+  public <T> HoodieWriteMetadata<T> clone(T transformedWriteStatuses) {

Review comment:
       Yes.  Rebased now.

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.common.data;
+
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
+
+/**
+ * Holds a {@link List} of objects.
+ *
+ * @param <T> type of object.
+ */
+public class HoodieListData<T> extends HoodieData<T> {
+
+  private final List<T> listData;
+
+  private HoodieListData(List<T> listData) {
+    this.listData = listData;
+  }
+
+  /**
+   * @param listData a {@link List} of objects in type T.
+   * @param <T>      type of object.
+   * @return a new instance containing the {@link List<T>} reference.
+   */
+  public static <T> HoodieListData<T> of(List<T> listData) {
+    return new HoodieListData<>(listData);
+  }
+
+  /**
+   * @param hoodieData {@link HoodieListData<T>} instance containing the {@link List} of objects.
+   * @param <T>        type of object.
+   * @return the a {@link List} of objects in type T.
+   */
+  public static <T> List<T> getList(HoodieData<T> hoodieData) {
+    return ((HoodieListData<T>) hoodieData).get();
+  }
+
+  @Override
+  public List<T> get() {
+    return listData;
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return listData.isEmpty();
+  }
+
+  @Override
+  public void persist(String cacheConfig) {
+    // No OP
+  }
+
+  @Override
+  public void unpersist() {
+    // No OP
+  }
+
+  @Override
+  public long count() {
+    return listData.size();
+  }
+
+  @Override
+  public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
+    return HoodieListData.of(listData.stream().parallel()
+        .map(throwingMapWrapper(func)).collect(Collectors.toList()));
+  }
+
+  @Override
+  public <O> HoodieData<O> mapPartitions(
+      SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
+    List<O> result = new ArrayList<>();
+    throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add);

Review comment:
       Yes.  For `HoodieList` the operation here is to iterate through all elements.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndexFactory.java
##########
@@ -18,54 +18,51 @@
 
 package org.apache.hudi.index;
 
-import org.apache.hudi.ApiMaturityLevel;
-import org.apache.hudi.PublicAPIMethod;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
-import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.HoodieBloomIndex;
+import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndexHelper;
 import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
-import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex;
-import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
+import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
+import org.apache.hudi.index.simple.HoodieSimpleIndex;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
 
-import org.apache.spark.api.java.JavaRDD;
+import java.io.IOException;
 
-@SuppressWarnings("checkstyle:LineLength")
-public abstract class SparkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
-  protected SparkHoodieIndex(HoodieWriteConfig config) {
-    super(config);
-  }
-
-  public static SparkHoodieIndex createIndex(HoodieWriteConfig config) {
+/**
+ * A factory to generate Spark {@link HoodieIndex}.
+ */
+public final class SparkHoodieIndexFactory {
+  public static HoodieIndex createIndex(HoodieWriteConfig config) {
     // first use index class config to create index.
     if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
       Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
       if (!(instance instanceof HoodieIndex)) {
         throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
       }
-      return (SparkHoodieIndex) instance;
+      return (HoodieIndex) instance;
     }
     switch (config.getIndexType()) {
       case HBASE:
         return new SparkHoodieHBaseIndex<>(config);
       case INMEMORY:
-        return new SparkInMemoryHashIndex(config);
+        return new HoodieInMemoryHashIndex<>(config);
       case BLOOM:
-        return new SparkHoodieBloomIndex<>(config);
+        return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
       case GLOBAL_BLOOM:
-        return new SparkHoodieGlobalBloomIndex<>(config);
+        return new HoodieGlobalBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
       case SIMPLE:
-        return new SparkHoodieSimpleIndex(config);
+        return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));

Review comment:
       As we synced offline, here for Spark we're providing a Spark-specific key generator for the virtual key scheme.  For Flink, this is not supported yet, and the empty key generator is passed in on the Flink side.  That's why we need to provide the key generator in the engine HoodieIndex factory for now.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndexHelper.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.index.bloom;
+
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDDData;
+import org.apache.hudi.data.HoodieJavaRDDData;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+/**
+ * Helper for {@link HoodieBloomIndex} containing Spark-specific logic.
+ */
+public class SparkHoodieBloomIndexHelper extends HoodieBloomIndexHelper {
+
+  private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndexHelper.class);
+
+  private static final SparkHoodieBloomIndexHelper SINGLETON_INSTANCE =
+      new SparkHoodieBloomIndexHelper();
+
+  private SparkHoodieBloomIndexHelper() {
+  }
+
+  public static SparkHoodieBloomIndexHelper getInstance() {
+    return SINGLETON_INSTANCE;
+  }
+
+  @Override

Review comment:
       Correct.




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bede6e6be4fda354793dc429de79e863f35c20ae Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexHelper.java
##########
@@ -20,41 +20,22 @@
 package org.apache.hudi.index.bloom;
 
 import org.apache.hudi.common.data.HoodieData;
-import org.apache.hudi.common.data.HoodieList;
 import org.apache.hudi.common.data.HoodiePairData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.util.collection.ImmutablePair;
-import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
 import org.apache.hudi.table.HoodieTable;
 
 import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static java.util.stream.Collectors.toList;
-
 /**
- * Helper for {@link HoodieBloomIndex} containing engine-specific logic,
- * with default logic supporting Java engines.
+ * Helper for {@link HoodieBloomIndex} containing engine-specific logic.
  */
-public class HoodieBloomIndexHelper implements Serializable {
-
-  private static final HoodieBloomIndexHelper SINGLETON_INSTANCE = new HoodieBloomIndexHelper();
-
-  protected HoodieBloomIndexHelper() {
-  }
-
-  public static HoodieBloomIndexHelper getInstance() {
-    return SINGLETON_INSTANCE;
-  }
-
+public abstract class HoodieBloomIndexHelper implements Serializable {

Review comment:
       Yes. Fixed.




-- 
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] yihua commented on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

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


   @hudi-bot run azure


-- 
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] yihua commented on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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


   @hudi-bot run azure


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 67c6e4a2fe83469fd99a821be21636867bf2ecb0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 67c6e4a2fe83469fd99a821be21636867bf2ecb0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847) 
   * bede6e6be4fda354793dc429de79e863f35c20ae Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 67c6e4a2fe83469fd99a821be21636867bf2ecb0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847) 
   * bede6e6be4fda354793dc429de79e863f35c20ae UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPairData.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.common.data;
+
+import org.apache.hudi.common.function.FunctionWrapper;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
+
+/**
+ * Implementation of {@link HoodiePairData} using Java {@link Map}.
+ * The pairs are organized by the key in the Map and values for the same key
+ * are stored in a list as the value corresponding to the key in the Map.
+ *
+ * @param <K> type of key.
+ * @param <V> type of value.
+ */
+public class HoodieMapPairData<K, V> extends HoodiePairData<K, V> {
+
+  private final Map<K, List<V>> mapPairData;
+
+  private HoodieMapPairData(Map<K, List<V>> mapPairData) {
+    this.mapPairData = mapPairData;
+  }
+
+  /**
+   * @param mapPairData a {@link Map} of pairs.
+   * @param <K>         type of key.
+   * @param <V>         type of value.
+   * @return a new instance containing the {@link Map<K, List<V>>} reference.
+   */
+  public static <K, V> HoodieMapPairData<K, V> of(Map<K, List<V>> mapPairData) {
+    return new HoodieMapPairData<>(mapPairData);
+  }
+
+  /**
+   * @param hoodiePairData {@link HoodieMapPairData<K, V>} instance containing the {@link Map} of pairs.
+   * @param <K>            type of key.
+   * @param <V>            type of value.
+   * @return the {@link Map} of pairs.
+   */
+  public static <K, V> Map<K, List<V>> getMapPair(HoodiePairData<K, V> hoodiePairData) {
+    return ((HoodieMapPairData<K, V>) hoodiePairData).get();
+  }
+
+  @Override
+  public Map<K, List<V>> get() {
+    return mapPairData;
+  }
+
+  @Override
+  public void persist(String cacheConfig) {
+    // No OP
+  }
+
+  @Override
+  public void unpersist() {
+    // No OP
+  }
+
+  @Override
+  public HoodieData<K> keys() {
+    return HoodieListData.of(new ArrayList<>(mapPairData.keySet()));
+  }
+
+  @Override
+  public HoodieData<V> values() {
+    return HoodieListData.of(
+        mapPairData.values().stream().flatMap(List::stream).collect(Collectors.toList()));
+  }
+
+  @Override
+  public long count() {
+    return mapPairData.values().stream().map(
+        list -> (long) list.size()).reduce(Long::sum).orElse(0L);
+  }
+
+  @Override
+  public Map<K, Long> countByKey() {
+    return mapPairData.entrySet().stream().collect(
+        Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size()));
+  }
+
+  @Override
+  public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
+    Function<Pair<K, V>, O> throwableFunc = throwingMapWrapper(func);
+    return HoodieListData.of(
+        streamAllPairs().map(throwableFunc).collect(Collectors.toList()));
+  }
+
+  @Override
+  public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
+    Map<L, List<W>> newMap = new HashMap<>();
+    Function<Pair<K, V>, Pair<L, W>> throwableMapToPairFunc =
+        FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc);
+    streamAllPairs().map(pair -> throwableMapToPairFunc.apply(pair)).forEach(newPair -> {
+      List<W> list = newMap.computeIfAbsent(newPair.getKey(), k -> new ArrayList<>());
+      list.add(newPair.getValue());
+    });
+    return HoodieMapPairData.of(newMap);
+  }
+
+  @Override
+  public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {

Review comment:
       I added unit tests for `HoodieMapPair` which has more complex transformations.




-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ac66c7483b666b084556845fe1ca78111a360980 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666) 
   * 6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on a change in pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java
##########
@@ -53,17 +53,19 @@ protected HoodieIndex(HoodieWriteConfig config) {
    * present).
    */
   @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract I tagLocation(I records, HoodieEngineContext context,
-                                HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException;
+  public abstract HoodieData<HoodieRecord<T>> tagLocation(

Review comment:
       Right.  As discussed, I keep two sets of APIs now, the old ones with specific types (`I` and `O`) which are the same as before and going to be deprecated in the future, and the new ones using HoodieData abstraction.  SparkHoodieIndex, FlinkHoodieIndex, and JavaHoodieIndex, which are extended by users, have the new APIs wrapping around the old APIs implemented by the users, which are compatible after the refactoring.




-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28ca85cd2f8edecf808560538a5c5f5e4de962 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578) 
   * bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot commented on pull request #3778: [WIP][HUDI-2502] Refactor index in hudi-client module

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ce28ca85cd2f8edecf808560538a5c5f5e4de962 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2a3dab04e4119ffff022e35debf2753fbabc89b3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871) 
   * 397883108f5b8ec76d533218097dda6cd9f667eb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878",
       "triggerID" : "397883108f5b8ec76d533218097dda6cd9f667eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2a3dab04e4119ffff022e35debf2753fbabc89b3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871) 
   * 397883108f5b8ec76d533218097dda6cd9f667eb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2878) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] yihua commented on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

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


   @nsivabalan @vinothchandar I addressed all the comments.  Could you take another pass?


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2a3dab04e4119ffff022e35debf2753fbabc89b3 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845) 
   * 67c6e4a2fe83469fd99a821be21636867bf2ecb0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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] hudi-bot edited a comment on pull request #3778: [HUDI-2502] Refactor index in hudi-client module

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#issuecomment-939762523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2578",
       "triggerID" : "ce28ca85cd2f8edecf808560538a5c5f5e4de962",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2587",
       "triggerID" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc4493a2b866b0b08d31d0f79f91cbeceb9c0ddf",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2592",
       "triggerID" : "940756599",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2665",
       "triggerID" : "57f7040c53e4c8250f38abf86b6416947d019fa6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ac66c7483b666b084556845fe1ca78111a360980",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2666",
       "triggerID" : "ac66c7483b666b084556845fe1ca78111a360980",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2845",
       "triggerID" : "6a9e2f68473b60eb98678f8ecbfb080c8b04ffb7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2847",
       "triggerID" : "67c6e4a2fe83469fd99a821be21636867bf2ecb0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867",
       "triggerID" : "bede6e6be4fda354793dc429de79e863f35c20ae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871",
       "triggerID" : "2a3dab04e4119ffff022e35debf2753fbabc89b3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bede6e6be4fda354793dc429de79e863f35c20ae Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2867) 
   * 2a3dab04e4119ffff022e35debf2753fbabc89b3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=2871) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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