[HUDI-407] Adding Simple Index to Hoodie. (#1402)
This index finds the location by joining incoming records with records from base files.
This commit is contained in:
committed by
GitHub
parent
3c9da2e5f0
commit
29edf4b3b8
@@ -99,4 +99,8 @@ public class SparkConfigUtils {
|
||||
String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
return getMaxMemoryAllowedForMerge(fraction);
|
||||
}
|
||||
|
||||
public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -62,6 +62,12 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BloomFilterTypeCode.SIMPLE.name();
|
||||
public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "hoodie.bloom.index.filter.dynamic.max.entries";
|
||||
public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "100000";
|
||||
public static final String SIMPLE_INDEX_USE_CACHING_PROP = "hoodie.simple.index.use.caching";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_USE_CACHING = "true";
|
||||
public static final String SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.simple.index.parallelism";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_PARALLELISM = "0";
|
||||
public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.global.simple.index.parallelism";
|
||||
public static final String DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM = "0";
|
||||
|
||||
// 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
|
||||
// 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
|
||||
@@ -80,6 +86,8 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level";
|
||||
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
public static final String SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "hoodie.simple.index.input.storage.level";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
|
||||
/**
|
||||
* Only applies if index type is GLOBAL_BLOOM.
|
||||
@@ -92,6 +100,9 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
public static final String BLOOM_INDEX_UPDATE_PARTITION_PATH = "hoodie.bloom.index.update.partition.path";
|
||||
public static final String DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
|
||||
public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = "hoodie.simple.index.update.partition.path";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
|
||||
private HoodieIndexConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
@@ -201,6 +212,31 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder simpleIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexInputStorageLevel(String level) {
|
||||
props.setProperty(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieIndexConfig build() {
|
||||
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
|
||||
@@ -228,6 +264,16 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE);
|
||||
setDefaultOnCondition(props, !props.contains(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES),
|
||||
HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_PARALLELISM_PROP), SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_USE_CACHING_PROP), SIMPLE_INDEX_USE_CACHING_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_INPUT_STORAGE_LEVEL), SIMPLE_INDEX_INPUT_STORAGE_LEVEL,
|
||||
DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP), GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_UPDATE_PARTITION_PATH),
|
||||
SIMPLE_INDEX_UPDATE_PARTITION_PATH, DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH);
|
||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||
return config;
|
||||
|
||||
@@ -441,6 +441,22 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH));
|
||||
}
|
||||
|
||||
public int getSimpleIndexParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_PARALLELISM_PROP));
|
||||
}
|
||||
|
||||
public boolean getSimpleIndexUseCaching() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_USE_CACHING_PROP));
|
||||
}
|
||||
|
||||
public int getGlobalSimpleIndexParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP));
|
||||
}
|
||||
|
||||
public boolean getGlobalSimpleIndexUpdatePartitionPath() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH));
|
||||
}
|
||||
|
||||
/**
|
||||
* storage properties.
|
||||
*/
|
||||
|
||||
@@ -32,6 +32,8 @@ 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.hbase.HBaseIndex;
|
||||
import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
|
||||
import org.apache.hudi.index.simple.HoodieSimpleIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
@@ -70,6 +72,10 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
return new HoodieBloomIndex<>(config);
|
||||
case GLOBAL_BLOOM:
|
||||
return new HoodieGlobalBloomIndex<>(config);
|
||||
case SIMPLE:
|
||||
return new HoodieSimpleIndex<>(config);
|
||||
case GLOBAL_SIMPLE:
|
||||
return new HoodieGlobalSimpleIndex<>(config);
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
@@ -87,7 +93,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* present).
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
@@ -95,7 +101,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at instantTime.
|
||||
@@ -128,9 +134,10 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
/**
|
||||
* Each index type should implement it's own logic to release any resources acquired during the process.
|
||||
*/
|
||||
public void close() {}
|
||||
public void close() {
|
||||
}
|
||||
|
||||
public enum IndexType {
|
||||
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM
|
||||
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,90 @@
|
||||
/*
|
||||
* 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.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Hoodie Index Utilities.
|
||||
*/
|
||||
public class HoodieIndexUtils {
|
||||
|
||||
/**
|
||||
* Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions.
|
||||
*
|
||||
* @param partitions list of partitions of interest
|
||||
* @param jsc instance of {@link JavaSparkContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} of interest
|
||||
* @return the list of Pairs of partition path and fileId
|
||||
*/
|
||||
public static List<Pair<String, HoodieBaseFile>> getLatestBaseFilesForAllPartitions(final List<String> partitions,
|
||||
final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMap(partitionPath -> {
|
||||
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, HoodieBaseFile>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles = hoodieTable.getBaseFileOnlyView()
|
||||
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> Pair.of(partitionPath, f))
|
||||
.collect(toList());
|
||||
}
|
||||
return filteredFiles.iterator();
|
||||
})
|
||||
.collect();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get tagged record for the passed in {@link HoodieRecord}.
|
||||
*
|
||||
* @param inputRecord instance of {@link HoodieRecord} for which tagging is requested
|
||||
* @param location {@link HoodieRecordLocation} for the passed in {@link HoodieRecord}
|
||||
* @return the tagged {@link HoodieRecord}
|
||||
*/
|
||||
public static HoodieRecord getTaggedRecord(HoodieRecord inputRecord, Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord record = inputRecord;
|
||||
if (location.isPresent()) {
|
||||
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD
|
||||
// will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2
|
||||
// separate filenames that the record is found in. This will result in setting
|
||||
// currentLocation 2 times and it will fail the second time. So creating a new in memory
|
||||
// copy of the hoodie record.
|
||||
record = new HoodieRecord<>(inputRecord);
|
||||
record.unseal();
|
||||
record.setCurrentLocation(location.get());
|
||||
record.seal();
|
||||
}
|
||||
return record;
|
||||
}
|
||||
}
|
||||
@@ -24,12 +24,12 @@ 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.timeline.HoodieInstant;
|
||||
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.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.io.HoodieRangeInfoHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
@@ -52,6 +52,7 @@ 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.
|
||||
@@ -192,18 +193,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
final HoodieTable hoodieTable) {
|
||||
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> partitionPathFileIDList =
|
||||
jsc.parallelize(partitions, Math.max(partitions.size(), 1)).flatMap(partitionPath -> {
|
||||
Option<HoodieInstant> latestCommitTime =
|
||||
hoodieTable.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, String>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles = hoodieTable.getBaseFileOnlyView()
|
||||
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> Pair.of(partitionPath, f.getFileId())).collect(toList());
|
||||
}
|
||||
return filteredFiles.iterator();
|
||||
}).collect();
|
||||
List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, jsc, 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
|
||||
@@ -312,21 +304,6 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
.collect(Collectors.toList()).iterator());
|
||||
}
|
||||
|
||||
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord, Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord<T> record = inputRecord;
|
||||
if (location.isPresent()) {
|
||||
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD
|
||||
// will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2
|
||||
// separate filenames that the record is found in. This will result in setting
|
||||
// currentLocation 2 times and it will fail the second time. So creating a new in memory
|
||||
// copy of the hoodie record.
|
||||
record = new HoodieRecord<>(inputRecord);
|
||||
record.unseal();
|
||||
record.setCurrentLocation(location.get());
|
||||
record.seal();
|
||||
}
|
||||
return record;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
@@ -338,7 +315,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
// 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 -> getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
|
||||
.map(v1 -> HoodieIndexUtils.getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
@@ -125,17 +126,17 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
|
||||
HoodieRecord<T> emptyRecord = new HoodieRecord(recordLocationHoodieKeyPair.get()._2,
|
||||
new EmptyHoodieRecordPayload());
|
||||
// Tag the incoming record for inserting to the new partition
|
||||
HoodieRecord<T> taggedRecord = getTaggedRecord(hoodieRecord, Option.empty());
|
||||
HoodieRecord<T> taggedRecord = HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty());
|
||||
return Arrays.asList(emptyRecord, taggedRecord).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(
|
||||
getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get()._2, hoodieRecord.getData()),
|
||||
(HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(new HoodieRecord<>(recordLocationHoodieKeyPair.get()._2, hoodieRecord.getData()),
|
||||
Option.ofNullable(recordLocationHoodieKeyPair.get()._1))).iterator();
|
||||
}
|
||||
} else {
|
||||
return Collections.singletonList(getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
return Collections.singletonList((HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(hoodieRecord, Option.empty())).iterator();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@@ -0,0 +1,169 @@
|
||||
/*
|
||||
* 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.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.exception.HoodieIOException;
|
||||
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.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
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>
|
||||
*/
|
||||
public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends HoodieSimpleIndex<T> {
|
||||
|
||||
public HoodieGlobalSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return tagLocationInternal(recordRDD, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tags records location for incoming records.
|
||||
*
|
||||
* @param inputRecordRDD {@link JavaRDD} of incoming records
|
||||
* @param jsc instance of {@link JavaSparkContext} 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, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
|
||||
JavaPairRDD<String, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable = fetchAllRecordLocations(jsc, hoodieTable,
|
||||
config.getGlobalSimpleIndexParallelism());
|
||||
return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch record locations for passed in {@link HoodieKey}s.
|
||||
*
|
||||
* @param jsc instance of {@link JavaSparkContext} 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(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable,
|
||||
int parallelism) {
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getAllBaseFilesInTable(jsc, hoodieTable);
|
||||
return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all files for all partitions as <Partition, filename> pair RDD.
|
||||
*/
|
||||
protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final JavaSparkContext jsc, final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
return getLatestBaseFilesForAllPartitions(allPartitionPaths, jsc, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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> emptyRecord = new HoodieRecord(new HoodieKey(inputRecord.getRecordKey(), partitionPath), new EmptyHoodieRecordPayload());
|
||||
// Tag the incoming record for inserting to the new partition
|
||||
HoodieRecord<T> taggedRecord = (HoodieRecord<T>) HoodieIndexUtils.getTaggedRecord(inputRecord, Option.empty());
|
||||
taggedRecords = Arrays.asList(emptyRecord, taggedRecord);
|
||||
} 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();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not.
|
||||
* found.
|
||||
*
|
||||
* @param hoodieKeys keys to lookup
|
||||
* @param jsc spark context
|
||||
* @param hoodieTable hoodie table object
|
||||
*/
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getGlobalSimpleIndexParallelism());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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.SparkConfigUtils;
|
||||
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.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.io.HoodieKeyLocationFetchHandle;
|
||||
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.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>
|
||||
*/
|
||||
public class HoodieSimpleIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
public HoodieSimpleIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> 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, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return tagLocationInternal(recordRDD, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option. Empty if the key is not
|
||||
* found.
|
||||
*
|
||||
* @param hoodieKeys keys to lookup
|
||||
* @param jsc spark context
|
||||
* @param hoodieTable hoodie table object
|
||||
*/
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
|
||||
return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getSimpleIndexParallelism());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tags records location for incoming records.
|
||||
*
|
||||
* @param inputRecordRDD {@link JavaRDD} of incoming records
|
||||
* @param jsc instance of {@link JavaSparkContext} 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, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
if (config.getSimpleIndexUseCaching()) {
|
||||
inputRecordRDD.persist(SparkConfigUtils.getSimpleIndexInputStorageLevel(config.getProps()));
|
||||
}
|
||||
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), jsc, 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 JavaRDD} of HoodieKeys.
|
||||
*
|
||||
* @param lookupKeys {@link JavaRDD} of {@link HoodieKey}s
|
||||
* @param jsc instance of {@link JavaSparkContext} to use
|
||||
* @param hoodieTable instance of {@link HoodieTable} of interest
|
||||
* @param parallelism parallelism to use
|
||||
* @return Hoodiekeys mapped to partitionpath and filenames
|
||||
*/
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocationInternal(JavaRDD<HoodieKey> lookupKeys, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable, int parallelism) {
|
||||
JavaPairRDD<HoodieKey, Option<HoodieRecordLocation>> keyLocationsRDD = lookupKeys.mapToPair(key -> new Tuple2<>(key, Option.empty()));
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords = fetchRecordLocationsForAffectedPartitions(lookupKeys, jsc, hoodieTable, parallelism);
|
||||
|
||||
return keyLocationsRDD.leftOuterJoin(existingRecords)
|
||||
.mapToPair(entry -> {
|
||||
final Option<HoodieRecordLocation> locationOpt = Option.ofNullable(entry._2._2.orNull());
|
||||
final HoodieKey key = entry._1;
|
||||
return locationOpt
|
||||
.map(location -> new Tuple2<>(key, Option.of(Pair.of(key.getPartitionPath(), location.getFileId()))))
|
||||
.orElse(new Tuple2<>(key, Option.empty()));
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch record locations for passed in {@link HoodieKey}s.
|
||||
*
|
||||
* @param hoodieKeys {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched
|
||||
* @param jsc instance of {@link JavaSparkContext} 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, JavaSparkContext jsc, HoodieTable<T> hoodieTable,
|
||||
int parallelism) {
|
||||
List<String> affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect();
|
||||
List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, jsc, hoodieTable);
|
||||
return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles);
|
||||
}
|
||||
|
||||
protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(JavaSparkContext jsc, HoodieTable<T> hoodieTable, int parallelism, List<Pair<String, HoodieBaseFile>> baseFiles) {
|
||||
int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism));
|
||||
return jsc.parallelize(baseFiles, fetchParallelism)
|
||||
.flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* {@link HoodieRecordLocation} fetch handle for all records from {@link HoodieBaseFile} of interest.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
|
||||
|
||||
private final Pair<String, HoodieBaseFile> partitionPathBaseFilePair;
|
||||
|
||||
public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, HoodieBaseFile> partitionPathBaseFilePair) {
|
||||
super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId()));
|
||||
this.partitionPathBaseFilePair = partitionPathBaseFilePair;
|
||||
}
|
||||
|
||||
public Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> locations() {
|
||||
HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight();
|
||||
return ParquetUtils.fetchRecordKeyPartitionPathFromParquet(hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream()
|
||||
.map(entry -> new Tuple2<>(entry,
|
||||
new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))).iterator();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user