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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]