[HUDI-2502] Refactor index in hudi-client module (#3778)
- Refactor Index to reduce Line of Code and re-use across engines.
This commit is contained in:
@@ -32,9 +32,10 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
@@ -57,7 +58,7 @@ import scala.Tuple2;
|
||||
/**
|
||||
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
||||
*/
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload<T>> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
@@ -65,7 +66,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
* TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple
|
||||
* basepath pointing to the table. Until, then just always assume a BloomIndex
|
||||
*/
|
||||
private final transient HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index;
|
||||
private final transient HoodieIndex<T, ?, ?, ?> index;
|
||||
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
private final transient HoodieSparkEngineContext context;
|
||||
@@ -100,7 +101,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build();
|
||||
this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient);
|
||||
this.index = SparkHoodieIndex.createIndex(clientConfig);
|
||||
this.index = SparkHoodieIndexFactory.createIndex(clientConfig);
|
||||
this.sqlContextOpt = Option.empty();
|
||||
}
|
||||
|
||||
@@ -170,7 +171,9 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
* component (without scheme) of the URI underlying file
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.tagLocation(hoodieKeys.map(k -> new HoodieRecord<>(k, null)), context, hoodieTable)
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(hoodieKeys.map(k -> new HoodieRecord<>(k, null))),
|
||||
context, hoodieTable))
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
@@ -196,7 +199,8 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
|
||||
* @return Tagged RDD of Hoodie records
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) throws HoodieIndexException {
|
||||
return index.tagLocation(hoodieRecords, context, hoodieTable);
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
index.tagLocation(HoodieJavaRDD.of(hoodieRecords), context, hoodieTable));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -44,7 +44,7 @@ import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieClusteringException;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.metrics.DistributedRegistry;
|
||||
@@ -115,8 +115,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex createIndex(HoodieWriteConfig writeConfig) {
|
||||
return SparkHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -141,7 +141,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
|
||||
Timer.Context indexTimer = metrics.getIndexCtx();
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = HoodieJavaRDD.getJavaRDD(
|
||||
getIndex().tagLocation(HoodieJavaRDD.of(hoodieRecords), context, table));
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
@@ -91,8 +91,8 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> HoodieData<T> parallelize(List<T> data) {
|
||||
return HoodieJavaRDD.of(javaSparkContext.parallelize(data, data.size()));
|
||||
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
|
||||
return HoodieJavaRDD.of(javaSparkContext.parallelize(data, parallelism));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -19,7 +19,6 @@
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
@@ -34,12 +33,4 @@ public class SparkMemoryUtils {
|
||||
public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL_VALUE));
|
||||
}
|
||||
|
||||
public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE));
|
||||
}
|
||||
|
||||
public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* 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.data;
|
||||
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.data.HoodiePairData;
|
||||
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 org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Implementation of {@link HoodiePairData} using Spark {@link JavaPairRDD}.
|
||||
*
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
*/
|
||||
public class HoodieJavaPairRDD<K, V> extends HoodiePairData<K, V> {
|
||||
|
||||
private final JavaPairRDD<K, V> pairRDDData;
|
||||
|
||||
private HoodieJavaPairRDD(JavaPairRDD<K, V> pairRDDData) {
|
||||
this.pairRDDData = pairRDDData;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param pairRDDData a {@link JavaPairRDD} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return a new instance containing the {@link JavaPairRDD<K, V>} reference.
|
||||
*/
|
||||
public static <K, V> HoodieJavaPairRDD<K, V> of(JavaPairRDD<K, V> pairRDDData) {
|
||||
return new HoodieJavaPairRDD<>(pairRDDData);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param hoodiePairData {@link HoodieJavaPairRDD <K, V>} instance containing the {@link JavaPairRDD} of pairs.
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
* @return the {@link JavaPairRDD} of pairs.
|
||||
*/
|
||||
public static <K, V> JavaPairRDD<K, V> getJavaPairRDD(HoodiePairData<K, V> hoodiePairData) {
|
||||
return ((HoodieJavaPairRDD<K, V>) hoodiePairData).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<K, V> get() {
|
||||
return pairRDDData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String storageLevel) {
|
||||
pairRDDData.persist(StorageLevel.fromString(storageLevel));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unpersist() {
|
||||
pairRDDData.unpersist();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<K> keys() {
|
||||
return HoodieJavaRDD.of(pairRDDData.keys());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<V> values() {
|
||||
return HoodieJavaRDD.of(pairRDDData.values());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long count() {
|
||||
return pairRDDData.count();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<K, Long> countByKey() {
|
||||
return pairRDDData.countByKey();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
|
||||
return HoodieJavaRDD.of(pairRDDData.map(
|
||||
tuple -> func.apply(new ImmutablePair<>(tuple._1, tuple._2))));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
|
||||
return HoodieJavaPairRDD.of(pairRDDData.mapToPair(pair -> {
|
||||
Pair<L, W> newPair = mapToPairFunc.call(new ImmutablePair<>(pair._1, pair._2));
|
||||
return new Tuple2<>(newPair.getLeft(), newPair.getRight());
|
||||
}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {
|
||||
return HoodieJavaPairRDD.of(JavaPairRDD.fromJavaRDD(
|
||||
pairRDDData.leftOuterJoin(HoodieJavaPairRDD.getJavaPairRDD(other))
|
||||
.map(tuple -> new Tuple2<>(tuple._1,
|
||||
new ImmutablePair<>(tuple._2._1, Option.ofNullable(tuple._2._2.orElse(null)))))));
|
||||
}
|
||||
}
|
||||
@@ -21,13 +21,19 @@ package org.apache.hudi.data;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.data.HoodiePairData;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Holds a {@link JavaRDD} of objects.
|
||||
*
|
||||
@@ -76,19 +82,52 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
return rddData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String storageLevel) {
|
||||
rddData.persist(StorageLevel.fromString(storageLevel));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void unpersist() {
|
||||
rddData.unpersist();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return rddData.isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long count() {
|
||||
return rddData.count();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
|
||||
return HoodieJavaRDD.of(rddData.map(func::apply));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
|
||||
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
|
||||
return HoodieJavaRDD.of(rddData.flatMap(func::apply));
|
||||
return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc) {
|
||||
return HoodieJavaPairRDD.of(rddData.mapToPair(input -> {
|
||||
Pair<K, V> pair = mapToPairFunc.call(input);
|
||||
return new Tuple2<>(pair.getLeft(), pair.getRight());
|
||||
}));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieData<T> distinct() {
|
||||
return HoodieJavaRDD.of(rddData.distinct());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -7,13 +7,14 @@
|
||||
* "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;
|
||||
@@ -21,89 +22,52 @@ 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.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
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.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.spark.api.java.JavaRDD;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public abstract class SparkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public abstract class SparkHoodieIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
protected SparkHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
public static SparkHoodieIndex 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;
|
||||
}
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new SparkHoodieHBaseIndex<>(config);
|
||||
case INMEMORY:
|
||||
return new SparkInMemoryHashIndex(config);
|
||||
case BLOOM:
|
||||
return new SparkHoodieBloomIndex<>(config);
|
||||
case GLOBAL_BLOOM:
|
||||
return new SparkHoodieGlobalBloomIndex<>(config);
|
||||
case SIMPLE:
|
||||
return new SparkHoodieSimpleIndex(config);
|
||||
case GLOBAL_SIMPLE:
|
||||
return new SparkHoodieGlobalSimpleIndex(config);
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether index is global or not.
|
||||
* @param config HoodieWriteConfig to use.
|
||||
* @return {@code true} if index is a global one. else {@code false}.
|
||||
*/
|
||||
public static boolean isGlobalIndex(HoodieWriteConfig config) {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return true;
|
||||
case INMEMORY:
|
||||
return true;
|
||||
case BLOOM:
|
||||
return false;
|
||||
case GLOBAL_BLOOM:
|
||||
return true;
|
||||
case SIMPLE:
|
||||
return false;
|
||||
case GLOBAL_SIMPLE:
|
||||
return true;
|
||||
default:
|
||||
return createIndex(config).isGlobal();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
@Deprecated
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.DEPRECATED)
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieJavaRDD.of(tagLocation(HoodieJavaRDD.getJavaRDD(records), context, hoodieTable));
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatuses, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) throws HoodieIndexException {
|
||||
return HoodieJavaRDD.of(updateLocation(HoodieJavaRDD.getJavaRDD(writeStatuses), context, hoodieTable));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,103 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
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.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.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 java.io.IOException;
|
||||
|
||||
/**
|
||||
* 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 (HoodieIndex) instance;
|
||||
}
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new SparkHoodieHBaseIndex<>(config);
|
||||
case INMEMORY:
|
||||
return new HoodieInMemoryHashIndex<>(config);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
case GLOBAL_BLOOM:
|
||||
return new HoodieGlobalBloomIndex<>(config, SparkHoodieBloomIndexHelper.getInstance());
|
||||
case SIMPLE:
|
||||
return new HoodieSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));
|
||||
case GLOBAL_SIMPLE:
|
||||
return new HoodieGlobalSimpleIndex<>(config, getKeyGeneratorForSimpleIndex(config));
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether index is global or not.
|
||||
* @param config HoodieWriteConfig to use.
|
||||
* @return {@code true} if index is a global one. else {@code false}.
|
||||
*/
|
||||
public static boolean isGlobalIndex(HoodieWriteConfig config) {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return true;
|
||||
case INMEMORY:
|
||||
return true;
|
||||
case BLOOM:
|
||||
return false;
|
||||
case GLOBAL_BLOOM:
|
||||
return true;
|
||||
case SIMPLE:
|
||||
return false;
|
||||
case GLOBAL_SIMPLE:
|
||||
return true;
|
||||
default:
|
||||
return createIndex(config).isGlobal();
|
||||
}
|
||||
}
|
||||
|
||||
private static Option<BaseKeyGenerator> getKeyGeneratorForSimpleIndex(HoodieWriteConfig config) {
|
||||
try {
|
||||
return config.populateMetaFields() ? Option.empty()
|
||||
: Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps())));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("KeyGenerator instantiation failed ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,135 +0,0 @@
|
||||
/*
|
||||
* 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.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.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.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by an in-memory Hash map.
|
||||
* <p>
|
||||
* ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkInMemoryHashIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
public SparkInMemoryHashIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
synchronized (SparkInMemoryHashIndex.class) {
|
||||
if (recordLocationMap == null) {
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return writeStatusRDD.map(writeStatus -> {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
// Delete existing index for a deleted record
|
||||
recordLocationMap.remove(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey.
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum, Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.unseal();
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
rec.seal();
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,299 +0,0 @@
|
||||
/*
|
||||
* 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.client.utils.SparkMemoryUtils;
|
||||
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.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.io.HoodieRangeInfoHandle;
|
||||
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.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
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.
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkHoodieBloomIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndex.class);
|
||||
|
||||
public SparkHoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
|
||||
// Step 0: cache the input record RDD
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.persist(SparkMemoryUtils.getBloomIndexInputStorageLevel(config.getProps()));
|
||||
}
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
recordRDD.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, context, hoodieTable);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
keyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
long totalTaggedRecords = keyFilenamePairRDD.count();
|
||||
LOG.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
|
||||
}
|
||||
|
||||
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
||||
// Cost: 4 sec.
|
||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD, recordRDD);
|
||||
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.unpersist(); // unpersist the input Record RDD
|
||||
keyFilenamePairRDD.unpersist();
|
||||
}
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the location for each record key and return the pair<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present.
|
||||
*/
|
||||
private JavaPairRDD<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList =
|
||||
loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo =
|
||||
fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id,
|
||||
// that contains it.
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD);
|
||||
Map<String, Long> comparisonsPerFileGroup =
|
||||
computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context);
|
||||
int inputParallelism = partitionRecordKeyPairRDD.partitions().size();
|
||||
int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
|
||||
+ config.getBloomIndexParallelism() + "}");
|
||||
return findMatchingFilesForRecordKeys(fileComparisonsRDD, joinParallelism, hoodieTable,
|
||||
comparisonsPerFileGroup);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the estimated number of bloom filter comparisons to be performed on each file group.
|
||||
*/
|
||||
private Map<String, Long> computeComparisonsPerFileGroup(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
final JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
final HoodieEngineContext context) {
|
||||
Map<String, Long> fileToComparisons;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
// FIX(vc): Only do sampling here and extrapolate?
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files");
|
||||
fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey();
|
||||
} else {
|
||||
fileToComparisons = new HashMap<>();
|
||||
partitionToFileInfo.forEach((key, value) -> {
|
||||
for (BloomIndexFileInfo fileInfo : value) {
|
||||
// each file needs to be compared against all the records coming into the partition
|
||||
fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(key));
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileToComparisons;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream()
|
||||
.map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId()))
|
||||
.collect(toList());
|
||||
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// also obtain file ranges, if range pruning is enabled
|
||||
context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)");
|
||||
return context.map(partitionPathFileIDList, pf -> {
|
||||
try {
|
||||
HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf);
|
||||
String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
LOG.warn("Unable to find range metadata in file :" + pf);
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()));
|
||||
}
|
||||
}, Math.max(partitionPathFileIDList.size(), 1));
|
||||
} else {
|
||||
return partitionPathFileIDList.stream()
|
||||
.map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is not global, since we depend on the partitionPath to do the lookup.
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* No indexes into log files yet.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bloom filters are stored, into the same data files.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
* <p>
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info.
|
||||
*/
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
IndexFileFilter indexFileFilter =
|
||||
config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream()
|
||||
.map(partitionFileIdPair -> new Tuple2<>(partitionFileIdPair.getRight(),
|
||||
new HoodieKey(recordKey, partitionPath)))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
* <p>
|
||||
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that each RDD
|
||||
* partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey
|
||||
* <p>
|
||||
* Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
int shuffleParallelism,
|
||||
HoodieTable hoodieTable,
|
||||
Map<String, Long> fileGroupToComparisons) {
|
||||
|
||||
if (config.useBloomIndexBucketizedChecking()) {
|
||||
Partitioner partitioner = new BucketizedBloomCheckPartitioner(shuffleParallelism, fileGroupToComparisons,
|
||||
config.getBloomIndexKeysPerBucket());
|
||||
|
||||
fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
||||
.repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2);
|
||||
} else {
|
||||
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, shuffleParallelism);
|
||||
}
|
||||
|
||||
return fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
|
||||
.flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
||||
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
||||
.collect(Collectors.toList()).iterator());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
*/
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyRecordPairRDD =
|
||||
recordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
|
||||
// so we do left outer join.
|
||||
return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values()
|
||||
.map(v1 -> HoodieIndexUtils.getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
@@ -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.HoodieJavaPairRDD;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
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 BaseHoodieBloomIndexHelper {
|
||||
|
||||
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
|
||||
public HoodiePairData<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
HoodieWriteConfig config, HoodieEngineContext context, HoodieTable hoodieTable,
|
||||
HoodiePairData<String, String> partitionRecordKeyPairs,
|
||||
HoodieData<ImmutablePair<String, HoodieKey>> fileComparisonPairs,
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
Map<String, Long> recordsPerPartition) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
HoodieJavaRDD.getJavaRDD(fileComparisonPairs)
|
||||
.map(pair -> new Tuple2<>(pair.getLeft(), pair.getRight()));
|
||||
Map<String, Long> comparisonsPerFileGroup = computeComparisonsPerFileGroup(
|
||||
config, recordsPerPartition, partitionToFileInfo, fileComparisonsRDD, context);
|
||||
int inputParallelism =
|
||||
HoodieJavaPairRDD.getJavaPairRDD(partitionRecordKeyPairs).partitions().size();
|
||||
int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
|
||||
+ config.getBloomIndexParallelism() + "}");
|
||||
|
||||
if (config.useBloomIndexBucketizedChecking()) {
|
||||
Partitioner partitioner = new BucketizedBloomCheckPartitioner(joinParallelism, comparisonsPerFileGroup,
|
||||
config.getBloomIndexKeysPerBucket());
|
||||
|
||||
fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
||||
.repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2);
|
||||
} else {
|
||||
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, joinParallelism);
|
||||
}
|
||||
|
||||
return HoodieJavaPairRDD.of(fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
|
||||
.flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
||||
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
||||
.collect(Collectors.toList()).iterator()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the estimated number of bloom filter comparisons to be performed on each file group.
|
||||
*/
|
||||
private Map<String, Long> computeComparisonsPerFileGroup(
|
||||
final HoodieWriteConfig config,
|
||||
final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
final JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD,
|
||||
final HoodieEngineContext context) {
|
||||
Map<String, Long> fileToComparisons;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
// FIX(vc): Only do sampling here and extrapolate?
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files");
|
||||
fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey();
|
||||
} else {
|
||||
fileToComparisons = new HashMap<>();
|
||||
partitionToFileInfo.forEach((key, value) -> {
|
||||
for (BloomIndexFileInfo fileInfo : value) {
|
||||
// each file needs to be compared against all the records coming into the partition
|
||||
fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(key));
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileToComparisons;
|
||||
}
|
||||
}
|
||||
@@ -1,143 +0,0 @@
|
||||
/*
|
||||
* 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.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.Optional;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This filter will only work with hoodie table since it will only load partitions with .hoodie_partition_metadata
|
||||
* file in it.
|
||||
*/
|
||||
public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends SparkHoodieBloomIndex<T> {
|
||||
|
||||
public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
|
||||
*/
|
||||
@Override
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For tables, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
* <p>
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info. the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will
|
||||
* be ignored since the search scope should be bigger than that
|
||||
*/
|
||||
|
||||
@Override
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
|
||||
IndexFileFilter indexFileFilter =
|
||||
config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFilesAndPartition(partitionPath, recordKey).stream()
|
||||
.map(partitionFileIdPair -> new Tuple2<>(partitionFileIdPair.getRight(),
|
||||
new HoodieKey(recordKey, partitionFileIdPair.getLeft())))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tagging for global index should only consider the record key.
|
||||
*/
|
||||
@Override
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyLocationPairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
|
||||
JavaPairRDD<String, HoodieRecord<T>> incomingRowKeyRecordPairRDD =
|
||||
recordRDD.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||
|
||||
JavaPairRDD<String, Tuple2<HoodieRecordLocation, HoodieKey>> existingRecordKeyToRecordLocationHoodieKeyMap =
|
||||
keyLocationPairRDD.mapToPair(p -> new Tuple2<>(p._1.getRecordKey(), new Tuple2<>(p._2, p._1)));
|
||||
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
|
||||
return incomingRowKeyRecordPairRDD.leftOuterJoin(existingRecordKeyToRecordLocationHoodieKeyMap).values().flatMap(record -> {
|
||||
final HoodieRecord<T> hoodieRecord = record._1;
|
||||
final Optional<Tuple2<HoodieRecordLocation, HoodieKey>> recordLocationHoodieKeyPair = record._2;
|
||||
if (recordLocationHoodieKeyPair.isPresent()) {
|
||||
// Record key matched to file
|
||||
if (config.getBloomIndexUpdatePartitionPath()
|
||||
&& !recordLocationHoodieKeyPair.get()._2.getPartitionPath().equals(hoodieRecord.getPartitionPath())) {
|
||||
// Create an empty record to delete the record in the old partition
|
||||
HoodieRecord<T> deleteRecord = new HoodieRecord(recordLocationHoodieKeyPair.get()._2,
|
||||
new EmptyHoodieRecordPayload());
|
||||
deleteRecord.setCurrentLocation(recordLocationHoodieKeyPair.get()._1());
|
||||
deleteRecord.seal();
|
||||
// Tag the incoming record for inserting to the new partition
|
||||
HoodieRecord<T> insertRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty());
|
||||
return Arrays.asList(deleteRecord, insertRecord).iterator();
|
||||
} else {
|
||||
// Ignore the incoming record's partition, regardless of whether it differs from its old partition or not.
|
||||
// When it differs, the record will still be updated at its old partition.
|
||||
return Collections.singletonList(
|
||||
(HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get()._2, hoodieRecord.getData()),
|
||||
Option.ofNullable(recordLocationHoodieKeyPair.get()._1))).iterator();
|
||||
}
|
||||
} else {
|
||||
return Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.index.hbase;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -35,9 +36,10 @@ import org.apache.hudi.common.util.RateLimiter;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -83,7 +85,8 @@ import scala.Tuple2;
|
||||
/**
|
||||
* Hoodie Index implementation backed by HBase.
|
||||
*/
|
||||
public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload<T>>
|
||||
extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled";
|
||||
@@ -291,10 +294,11 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
|
||||
public HoodieData<HoodieRecord<T>> tagLocation(
|
||||
HoodieData<HoodieRecord<T>> records, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
return HoodieJavaRDD.of(HoodieJavaRDD.getJavaRDD(records)
|
||||
.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true));
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
@@ -395,16 +399,17 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>,
|
||||
JavaRDD<WriteStatus>> hoodieTable) {
|
||||
final Option<Float> desiredQPSFraction = calculateQPSFraction(writeStatusRDD);
|
||||
public HoodieData<WriteStatus> updateLocation(
|
||||
HoodieData<WriteStatus> writeStatus, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
JavaRDD<WriteStatus> writeStatusRDD = HoodieJavaRDD.getJavaRDD(writeStatus);
|
||||
final Option<Float> desiredQPSFraction = calculateQPSFraction(writeStatusRDD);
|
||||
final Map<String, Integer> fileIdPartitionMap = mapFileWithInsertsToUniquePartition(writeStatusRDD);
|
||||
JavaRDD<WriteStatus> partitionedRDD = this.numWriteStatusWithInserts == 0 ? writeStatusRDD :
|
||||
writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w))
|
||||
.partitionBy(new WriteStatusPartitioner(fileIdPartitionMap,
|
||||
this.numWriteStatusWithInserts))
|
||||
.map(w -> w._2());
|
||||
writeStatusRDD.mapToPair(w -> new Tuple2<>(w.getFileId(), w))
|
||||
.partitionBy(new WriteStatusPartitioner(fileIdPartitionMap,
|
||||
this.numWriteStatusWithInserts))
|
||||
.map(w -> w._2());
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
acquireQPSResourcesAndSetBatchSize(desiredQPSFraction, jsc);
|
||||
JavaRDD<WriteStatus> writeStatusJavaRDD = partitionedRDD.mapPartitionsWithIndex(updateLocationFunction(),
|
||||
@@ -414,7 +419,7 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
|
||||
// force trigger update location(hbase puts)
|
||||
writeStatusJavaRDD.count();
|
||||
this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
|
||||
return writeStatusJavaRDD;
|
||||
return HoodieJavaRDD.of(writeStatusJavaRDD);
|
||||
}
|
||||
|
||||
private Option<Float> calculateQPSFraction(JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
|
||||
@@ -1,154 +0,0 @@
|
||||
/*
|
||||
* 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.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions;
|
||||
|
||||
/**
|
||||
* A global 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>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieSimpleIndex<T> {
|
||||
|
||||
public SparkHoodieGlobalSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return tagLocationInternal(recordRDD, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tags records location for incoming records.
|
||||
*
|
||||
* @param inputRecordRDD {@link JavaRDD} of incoming records
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} to use
|
||||
* @return {@link JavaRDD} of records with record locations set
|
||||
*/
|
||||
@Override
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
|
||||
JavaPairRDD<String, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable,
|
||||
config.getGlobalSimpleIndexParallelism());
|
||||
return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch record locations for passed in {@link HoodieKey}s.
|
||||
*
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} of interest
|
||||
* @param parallelism parallelism to use
|
||||
* @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation}
|
||||
*/
|
||||
protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchAllRecordLocations(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism) {
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getAllBaseFilesInTable(context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all files for all partitions as <Partition, filename> pair RDD.
|
||||
*/
|
||||
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final HoodieEngineContext context,
|
||||
final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), metaClient.getBasePath());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tag records with right {@link HoodieRecordLocation}.
|
||||
*
|
||||
* @param incomingRecords incoming {@link HoodieRecord}s
|
||||
* @param existingRecords existing records with {@link HoodieRecordLocation}s
|
||||
* @return {@link JavaRDD} of {@link HoodieRecord}s with tagged {@link HoodieRecordLocation}s
|
||||
*/
|
||||
private JavaRDD<HoodieRecord<T>> getTaggedRecords(JavaPairRDD<String, HoodieRecord<T>> incomingRecords, JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords) {
|
||||
JavaPairRDD<String, Pair<String, HoodieRecordLocation>> existingRecordByRecordKey = existingRecords
|
||||
.mapToPair(entry -> new Tuple2<>(entry._1.getRecordKey(), Pair.of(entry._1.getPartitionPath(), entry._2)));
|
||||
|
||||
return incomingRecords.leftOuterJoin(existingRecordByRecordKey).values()
|
||||
.flatMap(entry -> {
|
||||
HoodieRecord<T> inputRecord = entry._1;
|
||||
Option<Pair<String, HoodieRecordLocation>> partitionPathLocationPair = Option.ofNullable(entry._2.orNull());
|
||||
List<HoodieRecord<T>> taggedRecords;
|
||||
|
||||
if (partitionPathLocationPair.isPresent()) {
|
||||
String partitionPath = partitionPathLocationPair.get().getKey();
|
||||
HoodieRecordLocation location = partitionPathLocationPair.get().getRight();
|
||||
if (config.getGlobalSimpleIndexUpdatePartitionPath() && !(inputRecord.getPartitionPath().equals(partitionPath))) {
|
||||
// Create an empty record to delete the record in the old partition
|
||||
HoodieRecord<T> deleteRecord = new HoodieRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload());
|
||||
deleteRecord.setCurrentLocation(location);
|
||||
deleteRecord.seal();
|
||||
// Tag the incoming record for inserting to the new partition
|
||||
HoodieRecord<T> insertRecord = (HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty());
|
||||
taggedRecords = Arrays.asList(deleteRecord, insertRecord);
|
||||
} else {
|
||||
// Ignore the incoming record's partition, regardless of whether it differs from its old partition or not.
|
||||
// When it differs, the record will still be updated at its old partition.
|
||||
HoodieRecord<T> newRecord = new HoodieRecord<>(new HoodieKey(inputRecord.getRecordKey(), partitionPath), inputRecord.getData());
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(newRecord, Option.ofNullable(location)));
|
||||
}
|
||||
} else {
|
||||
taggedRecords = Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty()));
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -1,165 +0,0 @@
|
||||
/*
|
||||
* 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.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
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.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.io.HoodieKeyLocationFetchHandle;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
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>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkHoodieSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
public SparkHoodieSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
return tagLocationInternal(recordRDD, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tags records location for incoming records.
|
||||
*
|
||||
* @param inputRecordRDD {@link JavaRDD} of incoming records
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} to use
|
||||
* @return {@link JavaRDD} of records with record locations set
|
||||
*/
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecordRDD.persist(SparkMemoryUtils.getSimpleIndexInputStorageLevel(config.getProps()));
|
||||
}
|
||||
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), context, hoodieTable,
|
||||
config.getSimpleIndexParallelism());
|
||||
|
||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = keyedInputRecordRDD.leftOuterJoin(existingLocationsOnTable)
|
||||
.map(entry -> {
|
||||
final HoodieRecord<T> untaggedRecord = entry._2._1;
|
||||
final Option<HoodieRecordLocation> location = Option.ofNullable(entry._2._2.orNull());
|
||||
return HoodieIndexUtils.getTaggedRecord(untaggedRecord, location);
|
||||
});
|
||||
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecordRDD.unpersist();
|
||||
}
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch record locations for passed in {@link HoodieKey}s.
|
||||
*
|
||||
* @param hoodieKeys {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} of interest
|
||||
* @param parallelism parallelism to use
|
||||
* @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation}
|
||||
*/
|
||||
protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(JavaRDD<HoodieKey> hoodieKeys,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism) {
|
||||
List<String> affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect();
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable);
|
||||
return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
int parallelism,
|
||||
List<Pair<String, HoodieBaseFile>> baseFiles) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism));
|
||||
|
||||
try {
|
||||
Option<BaseKeyGenerator> keyGeneratorOpt = config.populateMetaFields() ? Option.empty()
|
||||
: Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps())));
|
||||
return jsc.parallelize(baseFiles, fetchParallelism)
|
||||
.flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile, keyGeneratorOpt)
|
||||
.locations().map(x -> Tuple2.apply(((Pair)x).getLeft(), ((Pair)x).getRight())).iterator());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("KeyGenerator instantiation failed ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -33,7 +33,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
@@ -102,8 +102,8 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
return SparkHoodieIndexFactory.createIndex(config);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -55,6 +55,7 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -182,8 +183,8 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context,
|
||||
table);
|
||||
JavaRDD<WriteStatus> statuses = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table));
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
commitOnAutoCommit(result);
|
||||
|
||||
@@ -38,6 +38,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
@@ -231,7 +232,8 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context, table);
|
||||
JavaRDD<WriteStatus> statuses = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table));
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
return statuses;
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
@@ -95,8 +96,8 @@ public class SparkDeleteHelper<T extends HoodieRecordPayload,R> extends
|
||||
dedupedKeys.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
Instant beginTag = Instant.now();
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords =
|
||||
table.getIndex().tagLocation(dedupedRecords, context, table);
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table));
|
||||
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
|
||||
|
||||
// filter out non existent keys/records
|
||||
|
||||
@@ -19,10 +19,13 @@
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
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.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
@@ -47,9 +50,15 @@ public class SparkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractW
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
protected JavaRDD<HoodieRecord<T>> tag(JavaRDD<HoodieRecord<T>> dedupedRecords, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table) {
|
||||
return HoodieJavaRDD.getJavaRDD(
|
||||
table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(
|
||||
JavaRDD<HoodieRecord<T>> records, HoodieIndex<T, ?, ?, ?> index, int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
return records.mapToPair(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
|
||||
Reference in New Issue
Block a user