[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)
- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules - Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc - Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common` - Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies - To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.async;
|
||||
|
||||
import org.apache.hudi.client.AbstractCompactor;
|
||||
import org.apache.hudi.client.AbstractHoodieWriteClient;
|
||||
import org.apache.hudi.client.HoodieSparkCompactor;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
|
||||
public class SparkAsyncCompactService extends AsyncCompactService {
|
||||
|
||||
public SparkAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
|
||||
super(context, client);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
|
||||
return new HoodieSparkCompactor(client);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,208 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
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.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
||||
*/
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* 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 HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
private final transient HoodieSparkEngineContext context;
|
||||
private final transient Configuration hadoopConf;
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie table
|
||||
*/
|
||||
public HoodieReadClient(HoodieSparkEngineContext context, String basePath) {
|
||||
this(context, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
// by default we use HoodieBloomIndex
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param context
|
||||
* @param basePath
|
||||
* @param sqlContext
|
||||
*/
|
||||
public HoodieReadClient(HoodieSparkEngineContext context, String basePath, SQLContext sqlContext) {
|
||||
this(context, basePath);
|
||||
this.sqlContextOpt = Option.of(sqlContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public HoodieReadClient(HoodieSparkEngineContext context, HoodieWriteConfig clientConfig) {
|
||||
this.context = context;
|
||||
this.hadoopConf = context.getHadoopConf().get();
|
||||
final String basePath = clientConfig.getBasePath();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
|
||||
this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient);
|
||||
this.index = SparkHoodieIndex.createIndex(clientConfig);
|
||||
this.sqlContextOpt = Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
|
||||
*
|
||||
* @return SparkConf object to be used to construct the SparkContext by caller
|
||||
*/
|
||||
public static SparkConf addHoodieSupport(SparkConf conf) {
|
||||
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
|
||||
return conf;
|
||||
}
|
||||
|
||||
private void assertSqlContext() {
|
||||
if (!sqlContextOpt.isPresent()) {
|
||||
throw new IllegalStateException("SQLContext must be set, when performing dataframe operations");
|
||||
}
|
||||
}
|
||||
|
||||
private Option<String> convertToDataFilePath(Option<Pair<String, String>> partitionPathFileIDPair) {
|
||||
if (partitionPathFileIDPair.isPresent()) {
|
||||
HoodieBaseFile dataFile = hoodieTable.getBaseFileOnlyView()
|
||||
.getLatestBaseFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get();
|
||||
return Option.of(dataFile.getPath());
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame.
|
||||
*
|
||||
* @return a dataframe
|
||||
*/
|
||||
public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = checkExists(hoodieKeys);
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD =
|
||||
lookupResultRDD.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
|
||||
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
|
||||
|
||||
// record locations might be same for multiple keys, so need a unique list
|
||||
Set<String> uniquePaths = new HashSet<>(paths);
|
||||
Dataset<Row> originalDF = sqlContextOpt.get().read().parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
||||
StructType schema = originalDF.schema();
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD().mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
|
||||
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1());
|
||||
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[FullFilePath]] If the optional
|
||||
* FullFilePath value is not present, then the key is not found. If the FullFilePath value is present, it is the path
|
||||
* 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)
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records.
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = tagLocation(hoodieRecords);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
|
||||
* present). Input RDD should contain no duplicates if needed.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records
|
||||
* @return Tagged RDD of Hoodie records
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) throws HoodieIndexException {
|
||||
return index.tagLocation(hoodieRecords, context, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all pending compactions with instant time for clients to decide what to compact next.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public List<Pair<String, HoodieCompactionPlan>> getPendingCompactions() {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(hadoopConf, hoodieTable.getMetaClient().getBasePath(), true);
|
||||
return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream()
|
||||
.map(
|
||||
instantWorkloadPair -> Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieSparkCompactor<T extends HoodieRecordPayload> extends AbstractCompactor<T,
|
||||
JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieSparkCompactor.class);
|
||||
|
||||
public HoodieSparkCompactor(AbstractHoodieWriteClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> compactionClient) {
|
||||
super(compactionClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void compact(HoodieInstant instant) throws IOException {
|
||||
LOG.info("Compactor executing compaction " + instant);
|
||||
JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
|
||||
long numWriteErrors = res.collect().stream().filter(WriteStatus::hasErrors).count();
|
||||
if (numWriteErrors != 0) {
|
||||
// We treat even a single error in compaction as fatal
|
||||
LOG.error("Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
||||
throw new HoodieException(
|
||||
"Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors);
|
||||
}
|
||||
// Commit compaction
|
||||
compactionClient.commitCompaction(instant.getTimestamp(), res, Option.empty());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Result of a write operation.
|
||||
*/
|
||||
public class HoodieWriteResult implements Serializable {
|
||||
|
||||
private JavaRDD<WriteStatus> writeStatuses;
|
||||
private Map<String, List<String>> partitionToReplaceFileIds;
|
||||
|
||||
public HoodieWriteResult(JavaRDD<WriteStatus> writeStatuses) {
|
||||
this(writeStatuses, Collections.emptyMap());
|
||||
}
|
||||
|
||||
public HoodieWriteResult(JavaRDD<WriteStatus> writeStatuses, Map<String, List<String>> partitionToReplaceFileIds) {
|
||||
this.writeStatuses = writeStatuses;
|
||||
this.partitionToReplaceFileIds = partitionToReplaceFileIds;
|
||||
}
|
||||
|
||||
public JavaRDD<WriteStatus> getWriteStatuses() {
|
||||
return this.writeStatuses;
|
||||
}
|
||||
|
||||
public void setWriteStatuses(final JavaRDD<WriteStatus> writeStatuses) {
|
||||
this.writeStatuses = writeStatuses;
|
||||
}
|
||||
|
||||
public Map<String, List<String>> getPartitionToReplaceFileIds() {
|
||||
return this.partitionToReplaceFileIds;
|
||||
}
|
||||
|
||||
public void setPartitionToReplaceFileIds(final Map<String, List<String>> partitionToReplaceFileIds) {
|
||||
this.partitionToReplaceFileIds = partitionToReplaceFileIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieWriteResult{"
|
||||
+ "writeStatuses=" + writeStatuses
|
||||
+ ", partitionToReplaceFileIds=" + partitionToReplaceFileIds
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,319 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
|
||||
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
AbstractHoodieWriteClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class);
|
||||
|
||||
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
|
||||
super(context, clientConfig);
|
||||
}
|
||||
|
||||
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
|
||||
super(context, writeConfig, rollbackPending);
|
||||
}
|
||||
|
||||
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(context, writeConfig, rollbackPending, timelineService);
|
||||
}
|
||||
|
||||
/**
|
||||
* Register hudi classes for Kryo serialization.
|
||||
*
|
||||
* @param conf instance of SparkConf
|
||||
* @return SparkConf
|
||||
*/
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Complete changes performed at the given instantTime marker with specified action.
|
||||
*/
|
||||
@Override
|
||||
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata,
|
||||
String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
|
||||
List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
|
||||
return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createTable(HoodieWriteConfig config,
|
||||
Configuration hadoopConf) {
|
||||
return HoodieSparkTable.create(config, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
// 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);
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to run bootstrap to hudi.
|
||||
*/
|
||||
@Override
|
||||
public void bootstrap(Option<Map<String, String>> extraMetadata) {
|
||||
if (rollbackPending) {
|
||||
rollBackInflightBootstrap();
|
||||
}
|
||||
getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
|
||||
table.validateUpsertSchema();
|
||||
setOperationType(WriteOperationType.UPSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsert(context, instantTime, records);
|
||||
if (result.getIndexLookupDuration().isPresent()) {
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
|
||||
}
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
|
||||
table.validateUpsertSchema();
|
||||
setOperationType(WriteOperationType.UPSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsertPrepped(context,instantTime, preppedRecords);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.INSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insert(context,instantTime, records);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.INSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insertPrepped(context,instantTime, preppedRecords);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table.
|
||||
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public HoodieWriteResult insertOverwrite(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
|
||||
HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.INSERT_OVERWRITE);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records);
|
||||
return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
|
||||
return bulkInsert(records, instantTime, Option.empty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, String instantTime, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.BULK_INSERT);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> bulkInsertPartitioner) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
|
||||
table.validateInsertSchema();
|
||||
setOperationType(WriteOperationType.BULK_INSERT_PREPPED);
|
||||
this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, String instantTime) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
|
||||
setOperationType(WriteOperationType.DELETE);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.delete(context,instantTime, keys);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JavaRDD<WriteStatus> postWrite(HoodieWriteMetadata<JavaRDD<WriteStatus>> result,
|
||||
String instantTime,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
if (result.getIndexLookupDuration().isPresent()) {
|
||||
metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis());
|
||||
}
|
||||
if (result.isCommitted()) {
|
||||
// Perform post commit operations.
|
||||
if (result.getFinalizeDuration().isPresent()) {
|
||||
metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(),
|
||||
result.getWriteStats().get().size());
|
||||
}
|
||||
|
||||
postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty());
|
||||
|
||||
emitCommitMetrics(instantTime, result.getCommitMetadata().get(), hoodieTable.getMetaClient().getCommitActionType());
|
||||
}
|
||||
return result.getWriteStatuses();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
|
||||
HoodieCommitMetadata metadata = SparkCompactHelpers.newInstance().createCompactionMetadata(
|
||||
table, compactionInstantTime, writeStatuses, config.getSchema());
|
||||
extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
|
||||
completeCompaction(metadata, writeStatuses, table, compactionInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String compactionCommitTime) {
|
||||
List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
|
||||
finalizeWrite(table, compactionCommitTime, writeStats);
|
||||
LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
|
||||
SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
|
||||
|
||||
if (compactionTimer != null) {
|
||||
long durationInMs = metrics.getDurationInMs(compactionTimer.stop());
|
||||
try {
|
||||
metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(),
|
||||
durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION);
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction "
|
||||
+ config.getBasePath() + " at time " + compactionCommitTime, e);
|
||||
}
|
||||
}
|
||||
LOG.info("Compacted successfully on commit " + compactionCommitTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
|
||||
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
|
||||
rollbackInflightCompaction(inflightInstant, table);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
compactionTimer = metrics.getCompactionCtx();
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = table.compact(context, compactionInstantTime);
|
||||
JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses();
|
||||
if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
|
||||
completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
|
||||
}
|
||||
return statuses;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
new SparkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
|
||||
return getTableAndInitCtx(metaClient, operationType);
|
||||
}
|
||||
|
||||
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
|
||||
if (operationType == WriteOperationType.DELETE) {
|
||||
setWriteSchemaForDeletes(metaClient);
|
||||
}
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieSparkTable<T> table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
|
||||
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
|
||||
writeTimer = metrics.getCommitCtx();
|
||||
} else {
|
||||
writeTimer = metrics.getDeltaCommitCtx();
|
||||
}
|
||||
return table;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,84 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.client.common.EngineProperty;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.spark.SparkEnv;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.util.Utils;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* Spark task context supplier.
|
||||
*/
|
||||
public class SparkTaskContextSupplier extends TaskContextSupplier implements Serializable {
|
||||
|
||||
@Override
|
||||
public Supplier<Integer> getPartitionIdSupplier() {
|
||||
return TaskContext::getPartitionId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Integer> getStageIdSupplier() {
|
||||
return () -> TaskContext.get().stageId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supplier<Long> getAttemptIdSupplier() {
|
||||
return () -> TaskContext.get().taskAttemptId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<String> getProperty(EngineProperty prop) {
|
||||
if (prop == EngineProperty.TOTAL_MEMORY_AVAILABLE) {
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/SparkContext.scala#L471} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
|
||||
final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
|
||||
if (SparkEnv.get() != null) {
|
||||
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
||||
return Option.ofNullable(String.valueOf(Utils.memoryStringToMb(SparkEnv.get().conf()
|
||||
.get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L));
|
||||
}
|
||||
return Option.empty();
|
||||
} else if (prop == EngineProperty.MEMORY_FRACTION_IN_USE) {
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
|
||||
final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
|
||||
if (SparkEnv.get() != null) {
|
||||
// 0.6 is the default value used by Spark,
|
||||
// look at {@link
|
||||
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||
return Option.ofNullable(SparkEnv.get().conf()
|
||||
.get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
throw new HoodieException("Unknown engine property :" + prop);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.client.bootstrap;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
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.exception.HoodieException;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.spark.sql.avro.SchemaConverters;
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
|
||||
import org.apache.spark.sql.internal.SQLConf;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class HoodieSparkBootstrapSchemaProvider extends HoodieBootstrapSchemaProvider {
|
||||
public HoodieSparkBootstrapSchemaProvider(HoodieWriteConfig writeConfig) {
|
||||
super(writeConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> {
|
||||
try {
|
||||
Path filePath = FileStatusUtils.toPath(fs.getPath());
|
||||
return ParquetUtils.readSchema(context.getHadoopConf().get(), filePath);
|
||||
} catch (Exception ex) {
|
||||
return null;
|
||||
}
|
||||
}).filter(Objects::nonNull).findAny()
|
||||
.orElseThrow(() -> new HoodieException("Could not determine schema from the data files."));
|
||||
|
||||
|
||||
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()),
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()));
|
||||
StructType sparkSchema = converter.convert(parquetSchema);
|
||||
String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
|
||||
String structName = tableName + "_record";
|
||||
String recordNamespace = "hoodie." + tableName;
|
||||
|
||||
return SchemaConverters.toAvroType(sparkSchema, false, structName, recordNamespace);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,112 @@
|
||||
/*
|
||||
* 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.client.common;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.client.common.function.SerializableConsumer;
|
||||
import org.apache.hudi.client.common.function.SerializableFunction;
|
||||
import org.apache.hudi.client.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* A Spark engine implementation of HoodieEngineContext.
|
||||
*/
|
||||
public class HoodieSparkEngineContext extends HoodieEngineContext {
|
||||
|
||||
private final JavaSparkContext javaSparkContext;
|
||||
private SQLContext sqlContext;
|
||||
|
||||
public HoodieSparkEngineContext(JavaSparkContext jsc) {
|
||||
super(new SerializableConfiguration(jsc.hadoopConfiguration()), new SparkTaskContextSupplier());
|
||||
this.javaSparkContext = jsc;
|
||||
this.sqlContext = SQLContext.getOrCreate(jsc.sc());
|
||||
}
|
||||
|
||||
public void setSqlContext(SQLContext sqlContext) {
|
||||
this.sqlContext = sqlContext;
|
||||
}
|
||||
|
||||
public JavaSparkContext getJavaSparkContext() {
|
||||
return javaSparkContext;
|
||||
}
|
||||
|
||||
public SQLContext getSqlContext() {
|
||||
return sqlContext;
|
||||
}
|
||||
|
||||
public static JavaSparkContext getSparkContext(HoodieEngineContext context) {
|
||||
return ((HoodieSparkEngineContext) context).getJavaSparkContext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) {
|
||||
return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism) {
|
||||
return javaSparkContext.parallelize(data, parallelism).flatMap(x -> func.apply(x).iterator()).collect();
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I> void foreach(List<I> data, SerializableConsumer<I> consumer, int parallelism) {
|
||||
javaSparkContext.parallelize(data, parallelism).foreach(consumer::accept);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism) {
|
||||
if (Objects.nonNull(parallelism)) {
|
||||
return javaSparkContext.parallelize(data, parallelism).mapToPair(func::call).collectAsMap();
|
||||
} else {
|
||||
return javaSparkContext.parallelize(data).mapToPair(func::call).collectAsMap();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setProperty(EngineProperty key, String value) {
|
||||
if (key == EngineProperty.COMPACTION_POOL_NAME) {
|
||||
javaSparkContext.setLocalProperty("spark.scheduler.pool", value);
|
||||
} else {
|
||||
throw new HoodieException("Unknown engine property :" + key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<String> getProperty(EngineProperty key) {
|
||||
if (key == EngineProperty.EMBEDDED_SERVER_HOST) {
|
||||
return Option.ofNullable(javaSparkContext.getConf().get("spark.driver.host", null));
|
||||
}
|
||||
throw new HoodieException("Unknown engine property :" + key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setJobStatus(String activeModule, String activityDescription) {
|
||||
javaSparkContext.setJobGroup(activeModule, activityDescription);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,243 @@
|
||||
/*
|
||||
* 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.client.model;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.util.ArrayData;
|
||||
import org.apache.spark.sql.catalyst.util.MapData;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
import org.apache.spark.sql.types.Decimal;
|
||||
import org.apache.spark.unsafe.types.CalendarInterval;
|
||||
import org.apache.spark.unsafe.types.UTF8String;
|
||||
|
||||
/**
|
||||
* Internal Row implementation for Hoodie Row. It wraps an {@link InternalRow} and keeps meta columns locally. But the {@link InternalRow}
|
||||
* does include the meta columns as well just that {@link HoodieInternalRow} will intercept queries for meta columns and serve from its
|
||||
* copy rather than fetching from {@link InternalRow}.
|
||||
*/
|
||||
public class HoodieInternalRow extends InternalRow {
|
||||
|
||||
private String commitTime;
|
||||
private String commitSeqNumber;
|
||||
private String recordKey;
|
||||
private String partitionPath;
|
||||
private String fileName;
|
||||
private InternalRow row;
|
||||
|
||||
public HoodieInternalRow(String commitTime, String commitSeqNumber, String recordKey, String partitionPath,
|
||||
String fileName, InternalRow row) {
|
||||
this.commitTime = commitTime;
|
||||
this.commitSeqNumber = commitSeqNumber;
|
||||
this.recordKey = recordKey;
|
||||
this.partitionPath = partitionPath;
|
||||
this.fileName = fileName;
|
||||
this.row = row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numFields() {
|
||||
return row.numFields();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNullAt(int i) {
|
||||
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
switch (i) {
|
||||
case 0: {
|
||||
this.commitTime = null;
|
||||
break;
|
||||
}
|
||||
case 1: {
|
||||
this.commitSeqNumber = null;
|
||||
break;
|
||||
}
|
||||
case 2: {
|
||||
this.recordKey = null;
|
||||
break;
|
||||
}
|
||||
case 3: {
|
||||
this.partitionPath = null;
|
||||
break;
|
||||
}
|
||||
case 4: {
|
||||
this.fileName = null;
|
||||
break;
|
||||
}
|
||||
default: throw new IllegalArgumentException("Not expected");
|
||||
}
|
||||
} else {
|
||||
row.setNullAt(i);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void update(int i, Object value) {
|
||||
if (i < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
switch (i) {
|
||||
case 0: {
|
||||
this.commitTime = value.toString();
|
||||
break;
|
||||
}
|
||||
case 1: {
|
||||
this.commitSeqNumber = value.toString();
|
||||
break;
|
||||
}
|
||||
case 2: {
|
||||
this.recordKey = value.toString();
|
||||
break;
|
||||
}
|
||||
case 3: {
|
||||
this.partitionPath = value.toString();
|
||||
break;
|
||||
}
|
||||
case 4: {
|
||||
this.fileName = value.toString();
|
||||
break;
|
||||
}
|
||||
default: throw new IllegalArgumentException("Not expected");
|
||||
}
|
||||
} else {
|
||||
row.update(i, value);
|
||||
}
|
||||
}
|
||||
|
||||
private String getMetaColumnVal(int ordinal) {
|
||||
switch (ordinal) {
|
||||
case 0: {
|
||||
return commitTime;
|
||||
}
|
||||
case 1: {
|
||||
return commitSeqNumber;
|
||||
}
|
||||
case 2: {
|
||||
return recordKey;
|
||||
}
|
||||
case 3: {
|
||||
return partitionPath;
|
||||
}
|
||||
case 4: {
|
||||
return fileName;
|
||||
}
|
||||
default: throw new IllegalArgumentException("Not expected");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNullAt(int ordinal) {
|
||||
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
return null == getMetaColumnVal(ordinal);
|
||||
}
|
||||
return row.isNullAt(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean getBoolean(int ordinal) {
|
||||
return row.getBoolean(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte getByte(int ordinal) {
|
||||
return row.getByte(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public short getShort(int ordinal) {
|
||||
return row.getShort(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int ordinal) {
|
||||
return row.getInt(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int ordinal) {
|
||||
return row.getLong(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int ordinal) {
|
||||
return row.getFloat(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int ordinal) {
|
||||
return row.getDouble(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Decimal getDecimal(int ordinal, int precision, int scale) {
|
||||
return row.getDecimal(ordinal, precision, scale);
|
||||
}
|
||||
|
||||
@Override
|
||||
public UTF8String getUTF8String(int ordinal) {
|
||||
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
|
||||
}
|
||||
return row.getUTF8String(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getString(int ordinal) {
|
||||
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
return new String(getMetaColumnVal(ordinal).getBytes());
|
||||
}
|
||||
return row.getString(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getBinary(int ordinal) {
|
||||
return row.getBinary(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CalendarInterval getInterval(int ordinal) {
|
||||
return row.getInterval(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalRow getStruct(int ordinal, int numFields) {
|
||||
return row.getStruct(ordinal, numFields);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayData getArray(int ordinal) {
|
||||
return row.getArray(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MapData getMap(int ordinal) {
|
||||
return row.getMap(ordinal);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(int ordinal, DataType dataType) {
|
||||
if (ordinal < HoodieRecord.HOODIE_META_COLUMNS.size()) {
|
||||
return UTF8String.fromBytes(getMetaColumnVal(ordinal).getBytes());
|
||||
}
|
||||
return row.get(ordinal, dataType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InternalRow copy() {
|
||||
return new HoodieInternalRow(commitTime, commitSeqNumber, recordKey, partitionPath, fileName, row.copy());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* 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.client.utils;
|
||||
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL;
|
||||
|
||||
/**
|
||||
* Spark config utils.
|
||||
*/
|
||||
public class SparkMemoryUtils {
|
||||
public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
|
||||
return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecutor<I, O, E> {
|
||||
|
||||
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
|
||||
// TaskContext properties.
|
||||
final TaskContext sparkThreadTaskContext;
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
|
||||
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
|
||||
}
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, BoundedInMemoryQueueProducer<I> producer,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
|
||||
super(hoodieConfig.getWriteBufferLimitBytes(), producer, Option.of(consumer), bufferedIteratorTransform);
|
||||
this.sparkThreadTaskContext = TaskContext.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preExecute() {
|
||||
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties.
|
||||
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,78 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.TaskContextSupplier;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.WriteHandleFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
public class SparkLazyInsertIterable<T extends HoodieRecordPayload> extends HoodieLazyInsertIterable<T> {
|
||||
|
||||
public SparkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
|
||||
boolean areRecordsSorted,
|
||||
HoodieWriteConfig config,
|
||||
String instantTime,
|
||||
HoodieTable hoodieTable,
|
||||
String idPrefix,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier);
|
||||
}
|
||||
|
||||
public SparkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
|
||||
boolean areRecordsSorted,
|
||||
HoodieWriteConfig config,
|
||||
String instantTime,
|
||||
HoodieTable hoodieTable,
|
||||
String idPrefix,
|
||||
TaskContextSupplier taskContextSupplier,
|
||||
WriteHandleFactory writeHandleFactory) {
|
||||
super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> computeNext() {
|
||||
// Executor service used for launching writer thread.
|
||||
BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor =
|
||||
null;
|
||||
try {
|
||||
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
|
||||
bufferedIteratorExecutor =
|
||||
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema));
|
||||
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
|
||||
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
if (null != bufferedIteratorExecutor) {
|
||||
bufferedIteratorExecutor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
/**
|
||||
* A factory to generate built-in partitioner to repartition input records into at least
|
||||
* expected number of output spark partitions for bulk insert operation.
|
||||
*/
|
||||
public abstract class BulkInsertInternalPartitionerFactory {
|
||||
|
||||
public static BulkInsertPartitioner get(BulkInsertSortMode sortMode) {
|
||||
switch (sortMode) {
|
||||
case NONE:
|
||||
return new NonSortPartitioner();
|
||||
case GLOBAL_SORT:
|
||||
return new GlobalSortPartitioner();
|
||||
case PARTITION_SORT:
|
||||
return new RDDPartitionSortPartitioner();
|
||||
default:
|
||||
throw new HoodieException("The bulk insert sort mode \"" + sortMode.name() + "\" is not supported.");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.execution.SparkLazyInsertIterable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Map function that handles a stream of HoodieRecords.
|
||||
*/
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String instantTime;
|
||||
private boolean areRecordsSorted;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTable hoodieTable;
|
||||
private List<String> fileIDPrefixes;
|
||||
|
||||
public BulkInsertMapFunction(String instantTime, boolean areRecordsSorted,
|
||||
HoodieWriteConfig config, HoodieTable hoodieTable,
|
||||
List<String> fileIDPrefixes) {
|
||||
this.instantTime = instantTime;
|
||||
this.areRecordsSorted = areRecordsSorted;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileIDPrefixes = fileIDPrefixes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new SparkLazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable,
|
||||
fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier());
|
||||
}
|
||||
}
|
||||
@@ -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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
/**
|
||||
* A built-in partitioner that does global sorting for the input records across partitions
|
||||
* after repartition for bulk insert operation, corresponding to the
|
||||
* {@code BulkInsertSortMode.GLOBAL_SORT} mode.
|
||||
*
|
||||
* @param <T> HoodieRecordPayload type
|
||||
*/
|
||||
public class GlobalSortPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions) {
|
||||
// Now, sort the records and line them up nicely for loading.
|
||||
return records.sortBy(record -> {
|
||||
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
|
||||
// the records split evenly across RDD partitions, such that small partitions fit
|
||||
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
|
||||
return new StringBuilder()
|
||||
.append(record.getPartitionPath())
|
||||
.append("+")
|
||||
.append(record.getRecordKey())
|
||||
.toString();
|
||||
}, true, outputSparkPartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
/**
|
||||
* A built-in partitioner that only does coalesce for input records for bulk insert operation,
|
||||
* corresponding to the {@code BulkInsertSortMode.NONE} mode.
|
||||
*
|
||||
* @param <T> HoodieRecordPayload type
|
||||
*/
|
||||
public class NonSortPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions) {
|
||||
return records.coalesce(outputSparkPartitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* A built-in partitioner that does local sorting for each RDD partition
|
||||
* after coalesce for bulk insert operation, corresponding to the
|
||||
* {@code BulkInsertSortMode.PARTITION_SORT} mode.
|
||||
*
|
||||
* @param <T> HoodieRecordPayload type
|
||||
*/
|
||||
public class RDDPartitionSortPartitioner<T extends HoodieRecordPayload>
|
||||
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions) {
|
||||
return records.coalesce(outputSparkPartitions)
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(
|
||||
new StringBuilder()
|
||||
.append(record.getPartitionPath())
|
||||
.append("+")
|
||||
.append(record.getRecordKey())
|
||||
.toString(), record))
|
||||
.mapPartitions(partition -> {
|
||||
// Sort locally in partition
|
||||
List<Tuple2<String, HoodieRecord<T>>> recordList = new ArrayList<>();
|
||||
for (; partition.hasNext(); ) {
|
||||
recordList.add(partition.next());
|
||||
}
|
||||
Collections.sort(recordList, (o1, o2) -> o1._1.compareTo(o2._1));
|
||||
return recordList.stream().map(e -> e._2).iterator();
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean arePartitionRecordsSorted() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
import org.apache.hudi.ApiMaturityLevel;
|
||||
import org.apache.hudi.PublicAPIMethod;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.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.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>> {
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
|
||||
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)
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
|
||||
}
|
||||
@@ -0,0 +1,135 @@
|
||||
/*
|
||||
* 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.client.common.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();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* 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.util.NumericUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* Partitions bloom filter checks by spreading out comparisons across buckets of work.
|
||||
*
|
||||
* Each bucket incurs the following cost
|
||||
*
|
||||
* <pre>
|
||||
* 1) Read bloom filter from file footer
|
||||
* 2) Check keys against bloom filter
|
||||
* 3) [Conditional] If any key had a hit, open file and check
|
||||
* </pre>
|
||||
*
|
||||
* The partitioner performs a two phase bin packing algorithm, to pack enough work into each bucket such that cost of
|
||||
* (1) & (3) is amortized. Also, avoids any skews in the sort based approach, by directly partitioning by the file to be
|
||||
* checked against and ensuring each partition has similar number of buckets. Performance tests show that this approach
|
||||
* could bound the amount of skew to std_dev(numberOfBucketsPerPartition) * cost of (3), lower than sort partitioning.
|
||||
*
|
||||
* Approach has two goals :
|
||||
*
|
||||
* <pre>
|
||||
* 1) Pack as many buckets from same file group into same partition, to amortize cost of (1) and (2) further
|
||||
* 2) Spread buckets across partitions evenly to achieve skew reduction
|
||||
* </pre>
|
||||
*/
|
||||
public class BucketizedBloomCheckPartitioner extends Partitioner {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BucketizedBloomCheckPartitioner.class);
|
||||
|
||||
private int partitions;
|
||||
|
||||
/**
|
||||
* Stores the final mapping of a file group to a list of partitions for its keys.
|
||||
*/
|
||||
private Map<String, List<Integer>> fileGroupToPartitions;
|
||||
|
||||
/**
|
||||
* Create a partitioner that computes a plan based on provided workload characteristics.
|
||||
*
|
||||
* @param targetPartitions maximum number of partitions to target
|
||||
* @param fileGroupToComparisons number of expected comparisons per file group
|
||||
* @param keysPerBucket maximum number of keys to pack in a single bucket
|
||||
*/
|
||||
public BucketizedBloomCheckPartitioner(int targetPartitions, Map<String, Long> fileGroupToComparisons,
|
||||
int keysPerBucket) {
|
||||
this.fileGroupToPartitions = new HashMap<>();
|
||||
|
||||
Map<String, Integer> bucketsPerFileGroup = new HashMap<>();
|
||||
// Compute the buckets needed per file group, using simple uniform distribution
|
||||
fileGroupToComparisons.forEach((f, c) -> bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket)));
|
||||
int totalBuckets = bucketsPerFileGroup.values().stream().mapToInt(i -> i).sum();
|
||||
// If totalBuckets > targetPartitions, no need to have extra partitions
|
||||
this.partitions = Math.min(targetPartitions, totalBuckets);
|
||||
|
||||
// PHASE 1 : start filling upto minimum number of buckets into partitions, taking all but one bucket from each file
|
||||
// This tries to first optimize for goal 1 above, with knowledge that each partition needs a certain minimum number
|
||||
// of buckets and assigns buckets in the same order as file groups. If we were to simply round robin, then buckets
|
||||
// for a file group is more or less guaranteed to be placed on different partitions all the time.
|
||||
int minBucketsPerPartition = Math.max((int) Math.floor((1.0 * totalBuckets) / partitions), 1);
|
||||
LOG.info(String.format("TotalBuckets %d, min_buckets/partition %d", totalBuckets, minBucketsPerPartition));
|
||||
int[] bucketsFilled = new int[partitions];
|
||||
Map<String, AtomicInteger> bucketsFilledPerFileGroup = new HashMap<>();
|
||||
int partitionIndex = 0;
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
for (int b = 0; b < Math.max(1, e.getValue() - 1); b++) {
|
||||
// keep filled counts upto date
|
||||
bucketsFilled[partitionIndex]++;
|
||||
AtomicInteger cnt = bucketsFilledPerFileGroup.getOrDefault(e.getKey(), new AtomicInteger(0));
|
||||
cnt.incrementAndGet();
|
||||
bucketsFilledPerFileGroup.put(e.getKey(), cnt);
|
||||
|
||||
// mark this partition against the file group
|
||||
List<Integer> partitionList = this.fileGroupToPartitions.getOrDefault(e.getKey(), new ArrayList<>());
|
||||
partitionList.add(partitionIndex);
|
||||
this.fileGroupToPartitions.put(e.getKey(), partitionList);
|
||||
|
||||
// switch to new partition if needed
|
||||
if (bucketsFilled[partitionIndex] >= minBucketsPerPartition) {
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// PHASE 2 : for remaining unassigned buckets, round robin over partitions once. Since we withheld 1 bucket from
|
||||
// each file group uniformly, this remaining is also an uniform mix across file groups. We just round robin to
|
||||
// optimize for goal 2.
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
int remaining = e.getValue() - bucketsFilledPerFileGroup.get(e.getKey()).intValue();
|
||||
for (int r = 0; r < remaining; r++) {
|
||||
// mark this partition against the file group
|
||||
this.fileGroupToPartitions.get(e.getKey()).add(partitionIndex);
|
||||
bucketsFilled[partitionIndex]++;
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Partitions assigned per file groups :" + fileGroupToPartitions);
|
||||
StringBuilder str = new StringBuilder();
|
||||
for (int i = 0; i < bucketsFilled.length; i++) {
|
||||
str.append("p" + i + " : " + bucketsFilled[i] + ",");
|
||||
}
|
||||
LOG.debug("Num buckets assigned per file group :" + str);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
final Pair<String, String> parts = (Pair<String, String>) key;
|
||||
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", parts.getRight());
|
||||
final List<Integer> candidatePartitions = fileGroupToPartitions.get(parts.getLeft());
|
||||
final int idx = (int) Math.floorMod((int) hashOfKey, candidatePartitions.size());
|
||||
assert idx >= 0;
|
||||
return candidatePartitions.get(idx);
|
||||
}
|
||||
|
||||
Map<String, List<Integer>> getFileGroupToPartitions() {
|
||||
return fileGroupToPartitions;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* 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.utils.LazyIterableIterator;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files.
|
||||
*/
|
||||
public class HoodieBloomIndexCheckFunction
|
||||
implements Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
|
||||
|
||||
private final HoodieTable hoodieTable;
|
||||
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<KeyLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, HoodieKey>> fileParitionRecordKeyTripletItr) {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<KeyLookupResult>> {
|
||||
|
||||
private HoodieKeyLookupHandle keyLookupHandle;
|
||||
|
||||
LazyKeyCheckIterator(Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
||||
super(filePartitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
|
||||
|
||||
List<HoodieKeyLookupHandle.KeyLookupResult> ret = new ArrayList<>();
|
||||
try {
|
||||
// process one file in each go.
|
||||
while (inputItr.hasNext()) {
|
||||
Tuple2<String, HoodieKey> currentTuple = inputItr.next();
|
||||
String fileId = currentTuple._1;
|
||||
String partitionPath = currentTuple._2.getPartitionPath();
|
||||
String recordKey = currentTuple._2.getRecordKey();
|
||||
Pair<String, String> partitionPathFilePair = Pair.of(partitionPath, fileId);
|
||||
|
||||
// lazily init state
|
||||
if (keyLookupHandle == null) {
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
// if continue on current file
|
||||
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
} else {
|
||||
// do the actual checking of file & break out
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// handle case, where we ran out of input, close pending work, update return val
|
||||
if (!inputItr.hasNext()) {
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,298 @@
|
||||
/*
|
||||
* 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.client.common.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.
|
||||
Map<String, Long> comparisonsPerFileGroup =
|
||||
computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD);
|
||||
int inputParallelism = partitionRecordKeyPairRDD.partitions().size();
|
||||
int joinParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
LOG.info("InputParallelism: ${" + inputParallelism + "}, IndexParallelism: ${"
|
||||
+ config.getBloomIndexParallelism() + "}");
|
||||
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, 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,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
|
||||
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?
|
||||
fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD)
|
||||
.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(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int shuffleParallelism, HoodieTable hoodieTable,
|
||||
Map<String, Long> fileGroupToComparisons) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD);
|
||||
|
||||
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,150 @@
|
||||
/*
|
||||
* 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.common.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.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.Optional;
|
||||
|
||||
import java.io.IOException;
|
||||
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();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,495 @@
|
||||
/*
|
||||
* 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.hbase;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
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.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by HBase.
|
||||
*/
|
||||
public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
|
||||
|
||||
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";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME =
|
||||
"spark.dynamicAllocation.maxExecutors";
|
||||
|
||||
private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||
private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||
private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||
private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||
private static final int SLEEP_TIME_MILLISECONDS = 100;
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkHoodieHBaseIndex.class);
|
||||
private static Connection hbaseConnection = null;
|
||||
private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null;
|
||||
private float qpsFraction;
|
||||
private int maxQpsPerRegionServer;
|
||||
/**
|
||||
* multiPutBatchSize will be computed and re-set in updateLocation if
|
||||
* {@link HoodieHBaseIndexConfig#HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP} is set to true.
|
||||
*/
|
||||
private Integer multiPutBatchSize;
|
||||
private Integer numRegionServersForTable;
|
||||
private final String tableName;
|
||||
private HBasePutBatchSizeCalculator putBatchSizeCalculator;
|
||||
|
||||
public SparkHoodieHBaseIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
this.tableName = config.getHbaseTableName();
|
||||
addShutDownHook();
|
||||
init(config);
|
||||
}
|
||||
|
||||
private void init(HoodieWriteConfig config) {
|
||||
this.multiPutBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
this.qpsFraction = config.getHbaseIndexQPSFraction();
|
||||
this.maxQpsPerRegionServer = config.getHbaseIndexMaxQPSPerRegionServer();
|
||||
this.putBatchSizeCalculator = new HBasePutBatchSizeCalculator();
|
||||
this.hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
}
|
||||
|
||||
public HBaseIndexQPSResourceAllocator createQPSResourceAllocator(HoodieWriteConfig config) {
|
||||
try {
|
||||
LOG.info("createQPSResourceAllocator :" + config.getHBaseQPSResourceAllocatorClass());
|
||||
return (HBaseIndexQPSResourceAllocator) ReflectionUtils
|
||||
.loadClass(config.getHBaseQPSResourceAllocatorClass(), config);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("error while instantiating HBaseIndexQPSResourceAllocator", e);
|
||||
}
|
||||
return new DefaultHBaseQPSResourceAllocator(config);
|
||||
}
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getHbaseZkQuorum();
|
||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||
String zkZnodeParent = config.getHBaseZkZnodeParent();
|
||||
if (zkZnodeParent != null) {
|
||||
hbaseConfig.set("zookeeper.znode.parent", zkZnodeParent);
|
||||
}
|
||||
String port = String.valueOf(config.getHbaseZkPort());
|
||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||
try {
|
||||
return ConnectionFactory.createConnection(hbaseConfig);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieDependentSystemUnavailableException(HoodieDependentSystemUnavailableException.HBASE,
|
||||
quorum + ":" + port);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Since we are sharing the HBaseConnection across tasks in a JVM, make sure the HBaseConnection is closed when JVM
|
||||
* exits.
|
||||
*/
|
||||
private void addShutDownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
|
||||
try {
|
||||
hbaseConnection.close();
|
||||
} catch (Exception e) {
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that any resources used for indexing are released here.
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
|
||||
}
|
||||
|
||||
private Get generateStatement(String key) throws IOException {
|
||||
return new Get(Bytes.toBytes(key)).setMaxVersions(1).addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN).addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
|
||||
}
|
||||
|
||||
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
// Check if the last commit ts for this row is 1) present in the timeline or
|
||||
// 2) is less than the first commit ts in the timeline
|
||||
return !commitTimeline.empty()
|
||||
&& commitTimeline.containsOrBeforeTimelineStarts(commitTs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> locationTagFunction(
|
||||
HoodieTableMetaClient metaClient) {
|
||||
|
||||
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>) (partitionNum,
|
||||
hoodieRecordIterator) -> {
|
||||
|
||||
int multiGetBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
|
||||
// Grab the global HBase connection
|
||||
synchronized (SparkHoodieHBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName))) {
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
statements.add(generateStatement(rec.getRecordKey()));
|
||||
currentBatchOfRecords.add(rec);
|
||||
// iterator till we reach batch size
|
||||
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
|
||||
// get results for batch from Hbase
|
||||
Result[] results = doGet(hTable, statements);
|
||||
// clear statements to be GC'd
|
||||
statements.clear();
|
||||
for (Result result : results) {
|
||||
// first, attempt to grab location from HBase
|
||||
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
|
||||
if (result.getRow() != null) {
|
||||
String keyFromResult = Bytes.toString(result.getRow());
|
||||
String commitTs = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
String partitionPath = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
if (checkIfValidCommit(metaClient, commitTs)) {
|
||||
currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath),
|
||||
currentRecord.getData());
|
||||
currentRecord.unseal();
|
||||
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
currentRecord.seal();
|
||||
taggedRecords.add(currentRecord);
|
||||
// the key from Result and the key being processed should be same
|
||||
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
|
||||
} else { // if commit is invalid, treat this as a new taggedRecord
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
} else {
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private Result[] doGet(HTable hTable, List<Get> keys) throws IOException {
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
return hTable.get(keys);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
synchronized (SparkHoodieHBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
try (BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) {
|
||||
while (statusIterator.hasNext()) {
|
||||
WriteStatus writeStatus = statusIterator.next();
|
||||
List<Mutation> mutations = new ArrayList<>();
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
Option<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if (loc.isPresent()) {
|
||||
if (rec.getCurrentLocation() != null) {
|
||||
// This is an update, no need to update index
|
||||
continue;
|
||||
}
|
||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN, Bytes.toBytes(loc.get().getInstantTime()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN, Bytes.toBytes(loc.get().getFileId()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN, Bytes.toBytes(rec.getPartitionPath()));
|
||||
mutations.add(put);
|
||||
} else {
|
||||
// Delete existing index for a deleted record
|
||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||
mutations.add(delete);
|
||||
}
|
||||
}
|
||||
if (mutations.size() < multiPutBatchSize) {
|
||||
continue;
|
||||
}
|
||||
doMutations(mutator, mutations);
|
||||
}
|
||||
// process remaining puts and deletes, if any
|
||||
doMutations(mutator, mutations);
|
||||
} catch (Exception e) {
|
||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||
LOG.error(we);
|
||||
writeStatus.setGlobalError(we);
|
||||
}
|
||||
writeStatusList.add(writeStatus);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Failed to Update Index locations because of exception with HBase Client", e);
|
||||
}
|
||||
return writeStatusList.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to facilitate performing mutations (including puts and deletes) in Hbase.
|
||||
*/
|
||||
private void doMutations(BufferedMutator mutator, List<Mutation> mutations) throws IOException {
|
||||
if (mutations.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
mutator.mutate(mutations);
|
||||
mutator.flush();
|
||||
mutations.clear();
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
}
|
||||
|
||||
private static void sleepForTime(int sleepTimeMs) {
|
||||
try {
|
||||
Thread.sleep(sleepTimeMs);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Sleep interrupted during throttling", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
|
||||
final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, context);
|
||||
LOG.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize);
|
||||
JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
|
||||
// caching the index updated status RDD
|
||||
writeStatusJavaRDD = writeStatusJavaRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
return writeStatusJavaRDD;
|
||||
}
|
||||
|
||||
private void setPutBatchSize(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final HoodieEngineContext context) {
|
||||
if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
SparkConf conf = jsc.getConf();
|
||||
int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
|
||||
if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) {
|
||||
maxExecutors =
|
||||
Math.max(maxExecutors, conf.getInt(DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1));
|
||||
}
|
||||
|
||||
/*
|
||||
* Each writeStatus represents status information from a write done in one of the IOHandles. If a writeStatus has
|
||||
* any insert, it implies that the corresponding task contacts HBase for doing puts, since we only do puts for
|
||||
* inserts from HBaseIndex.
|
||||
*/
|
||||
final Tuple2<Long, Integer> numPutsParallelismTuple = getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final long numPuts = numPutsParallelismTuple._1;
|
||||
final int hbasePutsParallelism = numPutsParallelismTuple._2;
|
||||
this.numRegionServersForTable = getNumRegionServersAliveForTable();
|
||||
final float desiredQPSFraction =
|
||||
hBaseIndexQPSResourceAllocator.calculateQPSFractionForPutsTime(numPuts, this.numRegionServersForTable);
|
||||
LOG.info("Desired QPSFraction :" + desiredQPSFraction);
|
||||
LOG.info("Number HBase puts :" + numPuts);
|
||||
LOG.info("Hbase Puts Parallelism :" + hbasePutsParallelism);
|
||||
final float availableQpsFraction =
|
||||
hBaseIndexQPSResourceAllocator.acquireQPSResources(desiredQPSFraction, numPuts);
|
||||
LOG.info("Allocated QPS Fraction :" + availableQpsFraction);
|
||||
multiPutBatchSize = putBatchSizeCalculator.getBatchSize(numRegionServersForTable, maxQpsPerRegionServer,
|
||||
hbasePutsParallelism, maxExecutors, SLEEP_TIME_MILLISECONDS, availableQpsFraction);
|
||||
LOG.info("multiPutBatchSize :" + multiPutBatchSize);
|
||||
}
|
||||
}
|
||||
|
||||
public Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
final JavaPairRDD<Long, Integer> insertOnlyWriteStatusRDD = writeStatusRDD
|
||||
.filter(w -> w.getStat().getNumInserts() > 0).mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1));
|
||||
return insertOnlyWriteStatusRDD.fold(new Tuple2<>(0L, 0), (w, c) -> new Tuple2<>(w._1 + c._1, w._2 + c._2));
|
||||
}
|
||||
|
||||
public static class HBasePutBatchSizeCalculator implements Serializable {
|
||||
|
||||
private static final int MILLI_SECONDS_IN_A_SECOND = 1000;
|
||||
private static final Logger LOG = LogManager.getLogger(HBasePutBatchSizeCalculator.class);
|
||||
|
||||
/**
|
||||
* Calculate putBatch size so that sum of requests across multiple jobs in a second does not exceed
|
||||
* maxQpsPerRegionServer for each Region Server. Multiplying qpsFraction to reduce the aggregate load on common RS
|
||||
* across topics. Assumption here is that all tables have regions across all RS, which is not necessarily true for
|
||||
* smaller tables. So, they end up getting a smaller share of QPS than they deserve, but it might be ok.
|
||||
* <p>
|
||||
* Example: int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f)
|
||||
* </p>
|
||||
* <p>
|
||||
* Expected batchSize is 8 because in that case, total request sent to a Region Server in one second is:
|
||||
*
|
||||
* 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) =>
|
||||
* 16000. We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 requests which
|
||||
* happens to be 10% of 16667 (maxQPSPerRegionServer), as expected.
|
||||
* </p>
|
||||
* <p>
|
||||
* Assumptions made here
|
||||
* <li>In a batch, writes get evenly distributed to each RS for that table. Since we do writes only in the case of
|
||||
* inserts and not updates, for this assumption to fail, inserts would have to be skewed towards few RS, likelihood
|
||||
* of which is less if Hbase table is pre-split and rowKeys are UUIDs (random strings). If this assumption fails,
|
||||
* then it is possible for some RS to receive more than maxQpsPerRegionServer QPS, but for simplicity, we are going
|
||||
* ahead with this model, since this is meant to be a lightweight distributed throttling mechanism without
|
||||
* maintaining a global context. So if this assumption breaks, we are hoping the HBase Master relocates hot-spot
|
||||
* regions to new Region Servers.
|
||||
*
|
||||
* </li>
|
||||
* <li>For Region Server stability, throttling at a second level granularity is fine. Although, within a second, the
|
||||
* sum of queries might be within maxQpsPerRegionServer, there could be peaks at some sub second intervals. So, the
|
||||
* assumption is that these peaks are tolerated by the Region Server (which at max can be maxQpsPerRegionServer).
|
||||
* </li>
|
||||
* </p>
|
||||
*/
|
||||
public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer, int numTasksDuringPut,
|
||||
int maxExecutors, int sleepTimeMs, float qpsFraction) {
|
||||
int maxReqPerSec = (int) (qpsFraction * numRegionServersForTable * maxQpsPerRegionServer);
|
||||
int maxParallelPuts = Math.max(1, Math.min(numTasksDuringPut, maxExecutors));
|
||||
int maxReqsSentPerTaskPerSec = MILLI_SECONDS_IN_A_SECOND / sleepTimeMs;
|
||||
int multiPutBatchSize = Math.max(1, maxReqPerSec / (maxParallelPuts * maxReqsSentPerTaskPerSec));
|
||||
LOG.info("HbaseIndexThrottling: qpsFraction :" + qpsFraction);
|
||||
LOG.info("HbaseIndexThrottling: numRSAlive :" + numRegionServersForTable);
|
||||
LOG.info("HbaseIndexThrottling: maxReqPerSec :" + maxReqPerSec);
|
||||
LOG.info("HbaseIndexThrottling: numTasks :" + numTasksDuringPut);
|
||||
LOG.info("HbaseIndexThrottling: maxExecutors :" + maxExecutors);
|
||||
LOG.info("HbaseIndexThrottling: maxParallelPuts :" + maxParallelPuts);
|
||||
LOG.info("HbaseIndexThrottling: maxReqsSentPerTaskPerSec :" + maxReqsSentPerTaskPerSec);
|
||||
LOG.info("HbaseIndexThrottling: numRegionServersForTable :" + numRegionServersForTable);
|
||||
LOG.info("HbaseIndexThrottling: multiPutBatchSize :" + multiPutBatchSize);
|
||||
return multiPutBatchSize;
|
||||
}
|
||||
}
|
||||
|
||||
private Integer getNumRegionServersAliveForTable() {
|
||||
// This is being called in the driver, so there is only one connection
|
||||
// from the driver, so ok to use a local connection variable.
|
||||
if (numRegionServersForTable == null) {
|
||||
try (Connection conn = getHBaseConnection()) {
|
||||
RegionLocator regionLocator = conn.getRegionLocator(TableName.valueOf(tableName));
|
||||
numRegionServersForTable = Math
|
||||
.toIntExact(regionLocator.getAllRegionLocations().stream().map(HRegionLocation::getServerName).distinct().count());
|
||||
return numRegionServersForTable;
|
||||
} catch (IOException e) {
|
||||
LOG.error(e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return numRegionServersForTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
// Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()}
|
||||
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;
|
||||
}
|
||||
|
||||
public void setHbaseConnection(Connection hbaseConnection) {
|
||||
SparkHoodieHBaseIndex.hbaseConnection = hbaseConnection;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,160 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.simple;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.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.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 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>
|
||||
*/
|
||||
@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();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
|
||||
// Obtain the latest data files from all the partitions.
|
||||
return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, 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> 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;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,152 @@
|
||||
/*
|
||||
* 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.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieIndexUtils;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
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>
|
||||
*/
|
||||
@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));
|
||||
return jsc.parallelize(baseFiles, fetchParallelism)
|
||||
.flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,204 @@
|
||||
/*
|
||||
* 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.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.client.model.HoodieInternalRow;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieInternalRowFileWriter;
|
||||
import org.apache.hudi.io.storage.HoodieInternalRowFileWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* Create handle with InternalRow for datasource implemention of bulk insert.
|
||||
*/
|
||||
public class HoodieRowCreateHandle implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieRowCreateHandle.class);
|
||||
private static final AtomicLong SEQGEN = new AtomicLong(1);
|
||||
|
||||
private final String instantTime;
|
||||
private final int taskPartitionId;
|
||||
private final long taskId;
|
||||
private final long taskEpochId;
|
||||
private final HoodieTable table;
|
||||
private final HoodieWriteConfig writeConfig;
|
||||
private final HoodieInternalRowFileWriter fileWriter;
|
||||
private final String partitionPath;
|
||||
private final Path path;
|
||||
private final String fileId;
|
||||
private final FileSystem fs;
|
||||
private final HoodieInternalWriteStatus writeStatus;
|
||||
private final HoodieTimer currTimer;
|
||||
|
||||
public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId,
|
||||
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
|
||||
StructType structType) {
|
||||
this.partitionPath = partitionPath;
|
||||
this.table = table;
|
||||
this.writeConfig = writeConfig;
|
||||
this.instantTime = instantTime;
|
||||
this.taskPartitionId = taskPartitionId;
|
||||
this.taskId = taskId;
|
||||
this.taskEpochId = taskEpochId;
|
||||
this.fileId = fileId;
|
||||
this.currTimer = new HoodieTimer();
|
||||
this.currTimer.startTimer();
|
||||
this.fs = table.getMetaClient().getFs();
|
||||
this.path = makeNewPath(partitionPath);
|
||||
this.writeStatus = new HoodieInternalWriteStatus(!table.getIndex().isImplicitWithStorage(),
|
||||
writeConfig.getWriteStatusFailureFraction());
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.setFileId(fileId);
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata =
|
||||
new HoodiePartitionMetadata(
|
||||
fs,
|
||||
instantTime,
|
||||
new Path(writeConfig.getBasePath()),
|
||||
FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(taskPartitionId);
|
||||
createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, getWriteToken(), this.fileId, table.getBaseFileExtension()));
|
||||
this.fileWriter = createNewFileWriter(path, table, writeConfig, structType);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to initialize file writer for path " + path, e);
|
||||
}
|
||||
LOG.info("New handle created for partition :" + partitionPath + " with fileId " + fileId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes an {@link InternalRow} to the underlying HoodieInternalRowFileWriter. Before writing, value for meta columns are computed as required
|
||||
* and wrapped in {@link HoodieInternalRow}. {@link HoodieInternalRow} is what gets written to HoodieInternalRowFileWriter.
|
||||
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void write(InternalRow record) throws IOException {
|
||||
try {
|
||||
String partitionPath = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
|
||||
HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
|
||||
String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement());
|
||||
String recordKey = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
|
||||
HoodieRecord.RECORD_KEY_METADATA_FIELD)).toString();
|
||||
HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(),
|
||||
record);
|
||||
try {
|
||||
fileWriter.writeRow(recordKey, internalRow);
|
||||
writeStatus.markSuccess(recordKey);
|
||||
} catch (Throwable t) {
|
||||
writeStatus.markFailure(recordKey, t);
|
||||
}
|
||||
} catch (Throwable ge) {
|
||||
writeStatus.setGlobalError(ge);
|
||||
throw ge;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @returns {@code true} if this handle can take in more writes. else {@code false}.
|
||||
*/
|
||||
public boolean canWrite() {
|
||||
return fileWriter.canWrite();
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the {@link HoodieRowCreateHandle} and returns an instance of {@link HoodieInternalWriteStatus} containing the stats and
|
||||
* status of the writes to this handle.
|
||||
* @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle.
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieInternalWriteStatus close() throws IOException {
|
||||
fileWriter.close();
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setPartitionPath(partitionPath);
|
||||
stat.setNumWrites(writeStatus.getTotalRecords());
|
||||
stat.setNumDeletes(0);
|
||||
stat.setNumInserts(writeStatus.getTotalRecords());
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(fileId);
|
||||
stat.setPath(new Path(writeConfig.getBasePath()), path);
|
||||
long fileSizeInBytes = FSUtils.getFileSize(table.getMetaClient().getFs(), path);
|
||||
stat.setTotalWriteBytes(fileSizeInBytes);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getFailedRowsSize());
|
||||
HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats();
|
||||
runtimeStats.setTotalCreateTime(currTimer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
writeStatus.setStat(stat);
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return path.getName();
|
||||
}
|
||||
|
||||
private Path makeNewPath(String partitionPath) {
|
||||
Path path = FSUtils.getPartitionPath(writeConfig.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
|
||||
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, getWriteToken(), fileId,
|
||||
tableConfig.getBaseFileFormat().getFileExtension()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an empty marker file corresponding to storage writer path.
|
||||
*
|
||||
* @param partitionPath Partition path
|
||||
*/
|
||||
private void createMarkerFile(String partitionPath, String dataFileName) {
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, instantTime);
|
||||
markerFiles.create(partitionPath, dataFileName, IOType.CREATE);
|
||||
}
|
||||
|
||||
private String getWriteToken() {
|
||||
return taskPartitionId + "-" + taskId + "-" + taskEpochId;
|
||||
}
|
||||
|
||||
private HoodieInternalRowFileWriter createNewFileWriter(
|
||||
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
|
||||
throws IOException {
|
||||
return HoodieInternalRowFileWriterFactory.getInternalRowFileWriter(
|
||||
path, hoodieTable, config, schema);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Abstraction to assist in writing {@link InternalRow}s to be used in datasource implementation.
|
||||
*/
|
||||
public interface HoodieInternalRowFileWriter {
|
||||
|
||||
/**
|
||||
* @returns {@code true} if this RowFileWriter can take in more writes. else {@code false}.
|
||||
*/
|
||||
boolean canWrite();
|
||||
|
||||
/**
|
||||
* Writes an {@link InternalRow} to the HoodieInternalRowFileWriter.
|
||||
*
|
||||
* @throws IOException on any exception while writing.
|
||||
*/
|
||||
void writeRow(String key, InternalRow row) throws IOException;
|
||||
|
||||
/**
|
||||
* Closes the {@link HoodieInternalRowFileWriter} and may not take in any more writes.
|
||||
*/
|
||||
void close() throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Factory to assist in instantiating a new {@link HoodieInternalRowFileWriter}.
|
||||
*/
|
||||
public class HoodieInternalRowFileWriterFactory {
|
||||
|
||||
/**
|
||||
* Factory method to assist in instantiating an instance of {@link HoodieInternalRowFileWriter}.
|
||||
* @param path path of the RowFileWriter.
|
||||
* @param hoodieTable instance of {@link HoodieTable} in use.
|
||||
* @param config instance of {@link HoodieWriteConfig} to use.
|
||||
* @param schema schema of the dataset in use.
|
||||
* @return the instantiated {@link HoodieInternalRowFileWriter}.
|
||||
* @throws IOException if format is not supported or if any exception during instantiating the RowFileWriter.
|
||||
*
|
||||
*/
|
||||
public static HoodieInternalRowFileWriter getInternalRowFileWriter(
|
||||
Path path, HoodieTable hoodieTable, HoodieWriteConfig config, StructType schema)
|
||||
throws IOException {
|
||||
final String extension = FSUtils.getFileExtension(path.getName());
|
||||
if (PARQUET.getFileExtension().equals(extension)) {
|
||||
return newParquetInternalRowFileWriter(path, config, schema, hoodieTable);
|
||||
}
|
||||
throw new UnsupportedOperationException(extension + " format not supported yet.");
|
||||
}
|
||||
|
||||
private static HoodieInternalRowFileWriter newParquetInternalRowFileWriter(
|
||||
Path path, HoodieWriteConfig writeConfig, StructType structType, HoodieTable table)
|
||||
throws IOException {
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(
|
||||
writeConfig.getBloomFilterNumEntries(),
|
||||
writeConfig.getBloomFilterFPP(),
|
||||
writeConfig.getDynamicBloomFilterMaxNumEntries(),
|
||||
writeConfig.getBloomFilterType());
|
||||
HoodieRowParquetWriteSupport writeSupport =
|
||||
new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, filter);
|
||||
return new HoodieInternalRowParquetWriter(
|
||||
path, new HoodieRowParquetConfig(
|
||||
writeSupport,
|
||||
writeConfig.getParquetCompressionCodec(),
|
||||
writeConfig.getParquetBlockSize(),
|
||||
writeConfig.getParquetPageSize(),
|
||||
writeConfig.getParquetMaxFileSize(),
|
||||
writeSupport.getHadoopConf(),
|
||||
writeConfig.getParquetCompressionRatio()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Parquet's impl of {@link HoodieInternalRowFileWriter} to write {@link InternalRow}s.
|
||||
*/
|
||||
public class HoodieInternalRowParquetWriter extends ParquetWriter<InternalRow>
|
||||
implements HoodieInternalRowFileWriter {
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieRowParquetWriteSupport writeSupport;
|
||||
|
||||
public HoodieInternalRowParquetWriter(Path file, HoodieRowParquetConfig parquetConfig)
|
||||
throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED,
|
||||
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file,
|
||||
parquetConfig.getHadoopConf()));
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize()
|
||||
+ Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeRow(String key, InternalRow row) throws IOException {
|
||||
super.write(row);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
/**
|
||||
* ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
|
||||
*/
|
||||
public class HoodieRowParquetConfig extends HoodieBaseParquetConfig<HoodieRowParquetWriteSupport> {
|
||||
|
||||
public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
|
||||
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
|
||||
double compressionRatio) {
|
||||
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,89 @@
|
||||
/*
|
||||
* 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.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
|
||||
import org.apache.parquet.hadoop.api.WriteSupport;
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
|
||||
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
|
||||
|
||||
/**
|
||||
* Hoodie Write Support for directly writing Row to Parquet.
|
||||
*/
|
||||
public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private BloomFilter bloomFilter;
|
||||
private String minRecordKey;
|
||||
private String maxRecordKey;
|
||||
|
||||
public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, BloomFilter bloomFilter) {
|
||||
super();
|
||||
Configuration hadoopConf = new Configuration(conf);
|
||||
hadoopConf.set("spark.sql.parquet.writeLegacyFormat", "false");
|
||||
hadoopConf.set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MILLIS");
|
||||
this.hadoopConf = hadoopConf;
|
||||
setSchema(structType, hadoopConf);
|
||||
this.bloomFilter = bloomFilter;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteSupport.FinalizedWriteContext finalizeWrite() {
|
||||
HashMap<String, String> extraMetaData = new HashMap<>();
|
||||
if (bloomFilter != null) {
|
||||
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
|
||||
if (minRecordKey != null && maxRecordKey != null) {
|
||||
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
|
||||
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
|
||||
}
|
||||
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
|
||||
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
|
||||
}
|
||||
}
|
||||
return new WriteSupport.FinalizedWriteContext(extraMetaData);
|
||||
}
|
||||
|
||||
public void add(String recordKey) {
|
||||
this.bloomFilter.add(recordKey);
|
||||
if (minRecordKey != null) {
|
||||
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
|
||||
} else {
|
||||
minRecordKey = recordKey;
|
||||
}
|
||||
|
||||
if (maxRecordKey != null) {
|
||||
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
|
||||
} else {
|
||||
maxRecordKey = recordKey;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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.keygen;
|
||||
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
/**
|
||||
* Spark key generator interface.
|
||||
*/
|
||||
public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface {
|
||||
|
||||
String getRecordKey(Row row);
|
||||
|
||||
String getPartitionPath(Row row);
|
||||
}
|
||||
@@ -0,0 +1,215 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.clean.SparkCleanActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkMergeHelper;
|
||||
import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
|
||||
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with
|
||||
* zero read amplification.
|
||||
* <p>
|
||||
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it
|
||||
* <p>
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*/
|
||||
public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieSparkTable<T> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class);
|
||||
|
||||
public HoodieSparkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
|
||||
return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> insert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
|
||||
return new SparkInsertCommitActionExecutor<>((HoodieSparkEngineContext)context, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
||||
Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
|
||||
return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, config,
|
||||
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> delete(HoodieEngineContext context, String instantTime, JavaRDD<HoodieKey> keys) {
|
||||
return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
|
||||
return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config,
|
||||
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
|
||||
return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieBootstrapWriteMetadata<JavaRDD<WriteStatus>> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata) {
|
||||
return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
|
||||
new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, instantTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime,
|
||||
String fileId) throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
SparkMergeHelper.newInstance().runMerge(this, upsertHandle);
|
||||
}
|
||||
|
||||
// TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||
+ upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
|
||||
if (requireSortedRecords()) {
|
||||
return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
|
||||
dataFileToBeMerged, taskContextSupplier);
|
||||
} else {
|
||||
return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
|
||||
dataFileToBeMerged,taskContextSupplier);
|
||||
}
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier);
|
||||
createHandle.write();
|
||||
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) {
|
||||
return new SparkCleanActionExecutor((HoodieSparkEngineContext)context, config, this, cleanInstantTime).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) {
|
||||
return new SparkCopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) {
|
||||
return new SavepointActionExecutor(context, config, this, instantToSavepoint, user, comment).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
|
||||
return new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,163 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkInsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkInsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
|
||||
import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor;
|
||||
import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification.
|
||||
*
|
||||
* <p>
|
||||
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
|
||||
* smallest existing file, to expand it
|
||||
* </p>
|
||||
* <p>
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the
|
||||
* base file.
|
||||
* </p>
|
||||
* <p>
|
||||
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit
|
||||
* action
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieSparkCopyOnWriteTable<T> {
|
||||
|
||||
HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
|
||||
return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> insert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
|
||||
return new SparkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records,
|
||||
Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
|
||||
return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config,
|
||||
this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> delete(HoodieEngineContext context, String instantTime, JavaRDD<HoodieKey> keys) {
|
||||
return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsertPrepped(HoodieEngineContext context, String instantTime,
|
||||
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
|
||||
return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config,
|
||||
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
|
||||
BaseScheduleCompactionActionExecutor scheduleCompactionExecutor = new SparkScheduleCompactionActionExecutor(
|
||||
context, config, this, instantTime, extraMetadata);
|
||||
return scheduleCompactionExecutor.execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
|
||||
SparkRunCompactionActionExecutor compactionExecutor = new SparkRunCompactionActionExecutor((HoodieSparkEngineContext) context, config, this, compactionInstantTime);
|
||||
return compactionExecutor.execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieBootstrapWriteMetadata<JavaRDD<WriteStatus>> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata) {
|
||||
return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
|
||||
new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRollbackMetadata rollback(HoodieEngineContext context,
|
||||
String rollbackInstantTime,
|
||||
HoodieInstant commitInstant,
|
||||
boolean deleteInstants) {
|
||||
return new SparkMergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
|
||||
return new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalizeWrite(HoodieEngineContext context, String instantTs, List<HoodieWriteStat> stats)
|
||||
throws HoodieIOException {
|
||||
// delegate to base class for MOR tables
|
||||
super.finalizeWrite(context, instantTs, stats);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
extends HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieSparkTable<T> create(HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
|
||||
context.getHadoopConf().get(),
|
||||
config.getBasePath(),
|
||||
true,
|
||||
config.getConsistencyGuardConfig(),
|
||||
Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))
|
||||
);
|
||||
return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieSparkTable<T> create(HoodieWriteConfig config,
|
||||
HoodieSparkEngineContext context,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieSparkCopyOnWriteTable<>(config, context, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieSparkMergeOnReadTable<>(config, context, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config) {
|
||||
return SparkHoodieIndex.createIndex(config);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,404 @@
|
||||
/*
|
||||
* 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.table.action.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapRecordPayload;
|
||||
import org.apache.hudi.client.bootstrap.BootstrapWriteStatus;
|
||||
import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
|
||||
import org.apache.hudi.client.bootstrap.HoodieSparkBootstrapSchemaProvider;
|
||||
import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.BootstrapFileMapping;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetReaderIterator;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
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.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
|
||||
import org.apache.hudi.io.HoodieBootstrapHandle;
|
||||
import org.apache.hudi.keygen.KeyGeneratorInterface;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.BaseCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.format.converter.ParquetMetadataConverter;
|
||||
import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieBootstrapWriteMetadata> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class);
|
||||
protected String bootstrapSchema = null;
|
||||
private transient FileSystem bootstrapSourceFileSystem;
|
||||
|
||||
public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, new HoodieWriteConfig.Builder().withProps(config.getProps())
|
||||
.withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class)
|
||||
.withBulkInsertParallelism(config.getBootstrapParallelism())
|
||||
.build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP,
|
||||
extraMetadata);
|
||||
bootstrapSourceFileSystem = FSUtils.getFs(config.getBootstrapSourceBasePath(), hadoopConf);
|
||||
}
|
||||
|
||||
private void validate() {
|
||||
ValidationUtils.checkArgument(config.getBootstrapSourceBasePath() != null,
|
||||
"Ensure Bootstrap Source Path is set");
|
||||
ValidationUtils.checkArgument(config.getBootstrapModeSelectorClass() != null,
|
||||
"Ensure Bootstrap Partition Selector is set");
|
||||
ValidationUtils.checkArgument(config.getBootstrapKeyGeneratorClass() != null,
|
||||
"Ensure bootstrap key generator class is set");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieBootstrapWriteMetadata execute() {
|
||||
validate();
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
Option<HoodieInstant> completetedInstant =
|
||||
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
|
||||
ValidationUtils.checkArgument(!completetedInstant.isPresent(),
|
||||
"Active Timeline is expected to be empty for bootstrap to be performed. "
|
||||
+ "If you want to re-bootstrap, please rollback bootstrap first !!");
|
||||
Map<BootstrapMode, List<Pair<String, List<HoodieFileStatus>>>> partitionSelections = listAndProcessSourcePartitions();
|
||||
|
||||
// First run metadata bootstrap which will auto commit
|
||||
Option<HoodieWriteMetadata> metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY));
|
||||
// if there are full bootstrap to be performed, perform that too
|
||||
Option<HoodieWriteMetadata> fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD));
|
||||
return new HoodieBootstrapWriteMetadata(metadataResult, fullBootstrapResult);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getSchemaToStoreInCommit() {
|
||||
return bootstrapSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform Metadata Bootstrap.
|
||||
* @param partitionFilesList List of partitions and files within that partitions
|
||||
*/
|
||||
protected Option<HoodieWriteMetadata> metadataBootstrap(List<Pair<String, List<HoodieFileStatus>>> partitionFilesList) {
|
||||
if (null == partitionFilesList || partitionFilesList.isEmpty()) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(),
|
||||
HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS));
|
||||
|
||||
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
|
||||
metaClient.getCommitActionType(), HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS), Option.empty());
|
||||
|
||||
JavaRDD<BootstrapWriteStatus> bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList);
|
||||
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result);
|
||||
return Option.of(result);
|
||||
}
|
||||
|
||||
private void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
|
||||
// cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
|
||||
// RDD actions that are performed after updating the index.
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context,
|
||||
table);
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
commitOnAutoCommit(result);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(JavaRDD<HoodieRecord<T>> inputRecords) {
|
||||
// NO_OP
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
|
||||
// Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index
|
||||
// is all done in a single job DAG.
|
||||
Map<String, List<Pair<BootstrapFileMapping, HoodieWriteStat>>> bootstrapSourceAndStats =
|
||||
result.getWriteStatuses().collect().stream()
|
||||
.map(w -> {
|
||||
BootstrapWriteStatus ws = (BootstrapWriteStatus) w;
|
||||
return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat());
|
||||
}).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath()));
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
try (BootstrapIndex.IndexWriter indexWriter = BootstrapIndex.getBootstrapIndex(metaClient)
|
||||
.createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) {
|
||||
LOG.info("Starting to write bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table "
|
||||
+ config.getBasePath());
|
||||
indexWriter.begin();
|
||||
bootstrapSourceAndStats.forEach((key, value) -> indexWriter.appendNextPartition(key,
|
||||
value.stream().map(Pair::getKey).collect(Collectors.toList())));
|
||||
indexWriter.finish();
|
||||
LOG.info("Finished writing bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table "
|
||||
+ config.getBasePath());
|
||||
}
|
||||
|
||||
commit(extraMetadata, result, bootstrapSourceAndStats.values().stream()
|
||||
.flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList()));
|
||||
LOG.info("Committing metadata bootstrap !!");
|
||||
}
|
||||
|
||||
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result, List<HoodieWriteStat> stats) {
|
||||
String actionType = table.getMetaClient().getCommitActionType();
|
||||
LOG.info("Committing " + instantTime + ", action Type " + actionType);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
|
||||
result.setCommitted(true);
|
||||
stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat));
|
||||
result.setWriteStats(stats);
|
||||
|
||||
// Finalize write
|
||||
finalizeWrite(instantTime, stats, result);
|
||||
|
||||
// add in extra metadata
|
||||
if (extraMetadata.isPresent()) {
|
||||
extraMetadata.get().forEach(metadata::addMetadata);
|
||||
}
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
|
||||
metadata.setOperationType(operationType);
|
||||
|
||||
try {
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
LOG.info("Committed " + instantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
|
||||
e);
|
||||
}
|
||||
result.setCommitMetadata(Option.of(metadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform Full Bootstrap.
|
||||
* @param partitionFilesList List of partitions and files within that partitions
|
||||
*/
|
||||
protected Option<HoodieWriteMetadata> fullBootstrap(List<Pair<String, List<HoodieFileStatus>>> partitionFilesList) {
|
||||
if (null == partitionFilesList || partitionFilesList.isEmpty()) {
|
||||
return Option.empty();
|
||||
}
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
FullRecordBootstrapDataProvider inputProvider =
|
||||
(FullRecordBootstrapDataProvider) ReflectionUtils.loadClass(config.getFullBootstrapInputProvider(),
|
||||
properties, context);
|
||||
JavaRDD<HoodieRecord> inputRecordsRDD =
|
||||
(JavaRDD<HoodieRecord>) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(),
|
||||
partitionFilesList);
|
||||
// Start Full Bootstrap
|
||||
final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(),
|
||||
HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS);
|
||||
table.getActiveTimeline().createNewInstant(requested);
|
||||
|
||||
// Setup correct schema and run bulk insert.
|
||||
return Option.of(getBulkInsertActionExecutor(inputRecordsRDD).execute());
|
||||
}
|
||||
|
||||
protected BaseSparkCommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||
return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps())
|
||||
.withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
|
||||
inputRecordsRDD, extraMetadata);
|
||||
}
|
||||
|
||||
private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, String partitionPath,
|
||||
HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) {
|
||||
|
||||
Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath());
|
||||
HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS,
|
||||
table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier());
|
||||
Schema avroSchema = null;
|
||||
try {
|
||||
ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath,
|
||||
ParquetMetadataConverter.NO_FILTER);
|
||||
MessageType parquetSchema = readFooter.getFileMetaData().getSchema();
|
||||
avroSchema = new AvroSchemaConverter().convert(parquetSchema);
|
||||
Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema,
|
||||
keyGenerator.getRecordKeyFieldNames());
|
||||
LOG.info("Schema to be used for reading record Keys :" + recordKeySchema);
|
||||
AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema);
|
||||
AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema);
|
||||
|
||||
BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void> wrapper = null;
|
||||
try (ParquetReader<IndexedRecord> reader =
|
||||
AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build()) {
|
||||
wrapper = new SparkBoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config,
|
||||
new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
|
||||
String recKey = keyGenerator.getKey(inp).getRecordKey();
|
||||
GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA);
|
||||
gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey);
|
||||
BootstrapRecordPayload payload = new BootstrapRecordPayload(gr);
|
||||
HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload);
|
||||
return rec;
|
||||
});
|
||||
wrapper.execute();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
bootstrapHandle.close();
|
||||
if (null != wrapper) {
|
||||
wrapper.shutdownNow();
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
BootstrapWriteStatus writeStatus = (BootstrapWriteStatus)bootstrapHandle.getWriteStatus();
|
||||
BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping(
|
||||
config.getBootstrapSourceBasePath(), srcPartitionPath, partitionPath,
|
||||
srcFileStatus, writeStatus.getFileId());
|
||||
writeStatus.setBootstrapSourceFileMapping(bootstrapFileMapping);
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return Bootstrap Mode selections for partitions listed and figure out bootstrap Schema.
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private Map<BootstrapMode, List<Pair<String, List<HoodieFileStatus>>>> listAndProcessSourcePartitions() throws IOException {
|
||||
List<Pair<String, List<HoodieFileStatus>>> folders = BootstrapUtils.getAllLeafFoldersWithFiles(
|
||||
table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), context);
|
||||
|
||||
LOG.info("Fetching Bootstrap Schema !!");
|
||||
HoodieBootstrapSchemaProvider sourceSchemaProvider = new HoodieSparkBootstrapSchemaProvider(config);
|
||||
bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(context, folders).toString();
|
||||
LOG.info("Bootstrap Schema :" + bootstrapSchema);
|
||||
|
||||
BootstrapModeSelector selector =
|
||||
(BootstrapModeSelector) ReflectionUtils.loadClass(config.getBootstrapModeSelectorClass(), config);
|
||||
|
||||
Map<BootstrapMode, List<String>> result = selector.select(folders);
|
||||
Map<String, List<HoodieFileStatus>> partitionToFiles = folders.stream().collect(
|
||||
Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Ensure all partitions are accounted for
|
||||
ValidationUtils.checkArgument(partitionToFiles.keySet().equals(
|
||||
result.values().stream().flatMap(Collection::stream).collect(Collectors.toSet())));
|
||||
|
||||
return result.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue().stream()
|
||||
.map(p -> Pair.of(p, partitionToFiles.get(p))).collect(Collectors.toList())))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
private JavaRDD<BootstrapWriteStatus> runMetadataBootstrap(List<Pair<String, List<HoodieFileStatus>>> partitions) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
if (null == partitions || partitions.isEmpty()) {
|
||||
return jsc.emptyRDD();
|
||||
}
|
||||
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
KeyGeneratorInterface keyGenerator = (KeyGeneratorInterface) ReflectionUtils.loadClass(config.getBootstrapKeyGeneratorClass(),
|
||||
properties);
|
||||
BootstrapPartitionPathTranslator translator = (BootstrapPartitionPathTranslator) ReflectionUtils.loadClass(
|
||||
config.getBootstrapPartitionPathTranslatorClass(), properties);
|
||||
|
||||
List<Pair<String, Pair<String, HoodieFileStatus>>> bootstrapPaths = partitions.stream()
|
||||
.flatMap(p -> {
|
||||
String translatedPartitionPath = translator.getBootstrapTranslatedPath(p.getKey());
|
||||
return p.getValue().stream().map(f -> Pair.of(p.getKey(), Pair.of(translatedPartitionPath, f)));
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return jsc.parallelize(bootstrapPaths, config.getBootstrapParallelism())
|
||||
.map(partitionFsPair -> handleMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(),
|
||||
partitionFsPair.getRight().getRight(), keyGenerator));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr) {
|
||||
throw new UnsupportedOperationException("Should not called in bootstrap code path");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
throw new UnsupportedOperationException("Should not called in bootstrap code path");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.table.action.bootstrap;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkBootstrapDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends SparkBootstrapCommitActionExecutor<T> {
|
||||
|
||||
public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BaseSparkCommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||
return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps())
|
||||
.withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
|
||||
inputRecordsRDD, extraMetadata);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,134 @@
|
||||
/*
|
||||
* 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.table.action.clean;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.CleanFileInfo;
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFlatMapFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCleanActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCleanActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkCleanActionExecutor.class);
|
||||
|
||||
public SparkCleanActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>
|
||||
deleteFilesFunc(HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>) iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, CleanFileInfo> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
|
||||
String deletePathStr = deletePath.toString();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
if (isBootstrapBasePathFile) {
|
||||
// For Bootstrap Base file deletions, store the full file path.
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
|
||||
partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
|
||||
} else {
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
|
||||
partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
|
||||
}
|
||||
}
|
||||
return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
int cleanerParallelism = Math.min(
|
||||
(int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()),
|
||||
config.getCleanerParallelism());
|
||||
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions");
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
|
||||
.flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(),
|
||||
new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))))
|
||||
.collect(Collectors.toList()), cleanerParallelism)
|
||||
.mapPartitionsToPair(deleteFilesFunc(table))
|
||||
.reduceByKey(PartitionCleanStat::merge).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
|
||||
? partitionCleanStatsMap.get(partitionPath)
|
||||
: new PartitionCleanStat(partitionPath);
|
||||
HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(Option.ofNullable(
|
||||
actionInstant != null
|
||||
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
|
||||
actionInstant.getAction(), actionInstant.getTimestamp())
|
||||
: null))
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns())
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles())
|
||||
.withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
|
||||
.withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
|
||||
.withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
|
||||
.build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,342 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.utils.SparkMemoryUtils;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
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.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.execution.SparkLazyInsertIterable;
|
||||
import org.apache.hudi.io.CreateHandleFactory;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
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 scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieWriteMetadata> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class);
|
||||
|
||||
public BaseSparkCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
WriteOperationType operationType) {
|
||||
super(context, config, table, instantTime, operationType, Option.empty());
|
||||
}
|
||||
|
||||
public BaseSparkCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
WriteOperationType operationType,
|
||||
Option extraMetadata) {
|
||||
super(context, config, table, instantTime, operationType, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = new HoodieWriteMetadata<>();
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
// TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling
|
||||
if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) {
|
||||
inputRecordsRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
} else {
|
||||
LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
|
||||
}
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
if (isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(buildProfile(inputRecordsRDD));
|
||||
LOG.info("Workload profile :" + profile);
|
||||
saveWorkloadProfileMetadataToInflight(profile, instantTime);
|
||||
}
|
||||
|
||||
// partition using the insert partitioner
|
||||
final Partitioner partitioner = getPartitioner(profile);
|
||||
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(inputRecordsRDD, partitioner);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||
if (WriteOperationType.isChangingRecords(operationType)) {
|
||||
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
} else {
|
||||
return handleInsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true).flatMap(List::iterator);
|
||||
|
||||
updateIndexAndCommitIfNeeded(writeStatusRDD, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
private Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
|
||||
WorkloadStat globalStat = new WorkloadStat();
|
||||
|
||||
// group the records by partitionPath + currentLocation combination, count the number of
|
||||
// records in each partition
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecordsRDD
|
||||
.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
|
||||
.countByKey();
|
||||
|
||||
// count the number of both inserts and updates in each partition, update the counts to workLoadStats
|
||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
|
||||
String partitionPath = e.getKey()._1();
|
||||
Long count = e.getValue();
|
||||
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isPresent()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
return Pair.of(partitionPathStatMap, globalStat);
|
||||
}
|
||||
|
||||
protected Partitioner getPartitioner(WorkloadProfile profile) {
|
||||
if (WriteOperationType.isChangingRecords(operationType)) {
|
||||
return getUpsertPartitioner(profile);
|
||||
} else {
|
||||
return getInsertPartitioner(profile);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
|
||||
JavaPairRDD<Tuple2, HoodieRecord<T>> mappedRDD = dedupedRecords.mapToPair(
|
||||
record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record));
|
||||
|
||||
JavaPairRDD<Tuple2, HoodieRecord<T>> partitionedRDD;
|
||||
if (table.requireSortedRecords()) {
|
||||
// Partition and sort within each partition as a single step. This is faster than partitioning first and then
|
||||
// applying a sort.
|
||||
Comparator<Tuple2> comparator = (Comparator<Tuple2> & Serializable)(t1, t2) -> {
|
||||
HoodieKey key1 = (HoodieKey) t1._1;
|
||||
HoodieKey key2 = (HoodieKey) t2._1;
|
||||
return key1.getRecordKey().compareTo(key2.getRecordKey());
|
||||
};
|
||||
|
||||
partitionedRDD = mappedRDD.repartitionAndSortWithinPartitions(partitioner, comparator);
|
||||
} else {
|
||||
// Partition only
|
||||
partitionedRDD = mappedRDD.partitionBy(partitioner);
|
||||
}
|
||||
|
||||
return partitionedRDD.map(Tuple2::_2);
|
||||
}
|
||||
|
||||
protected void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata result) {
|
||||
// cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
|
||||
// RDD actions that are performed after updating the index.
|
||||
writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
Instant indexStartTime = Instant.now();
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context, table);
|
||||
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
|
||||
result.setWriteStatuses(statuses);
|
||||
result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses));
|
||||
commitOnAutoCommit(result);
|
||||
}
|
||||
|
||||
protected String getCommitActionType() {
|
||||
return table.getMetaClient().getCommitActionType();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
|
||||
commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect());
|
||||
}
|
||||
|
||||
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result, List<HoodieWriteStat> writeStats) {
|
||||
String actionType = getCommitActionType();
|
||||
LOG.info("Committing " + instantTime + ", action Type " + actionType);
|
||||
result.setCommitted(true);
|
||||
result.setWriteStats(writeStats);
|
||||
// Finalize write
|
||||
finalizeWrite(instantTime, writeStats, result);
|
||||
|
||||
try {
|
||||
LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType());
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
|
||||
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
|
||||
|
||||
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
LOG.info("Committed " + instantTime);
|
||||
result.setCommitMetadata(Option.of(metadata));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(binfo.fileIdPrefix, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
LOG.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
LOG.info("Empty partition with fileId => " + fileId);
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, fileId);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords,
|
||||
HoodieBaseFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
SparkMergeHelper.newInstance().runMerge(table, upsertHandle);
|
||||
}
|
||||
|
||||
// TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
|
||||
+ upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
if (table.requireSortedRecords()) {
|
||||
return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
} else {
|
||||
return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords,
|
||||
HoodieBaseFile dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords,
|
||||
partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws Exception {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
LOG.info("Empty partition");
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new SparkLazyInsertIterable(recordItr, true, config, instantTime, table, idPfx,
|
||||
taskContextSupplier, new CreateHandleFactory<>());
|
||||
}
|
||||
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile, context, table, config);
|
||||
}
|
||||
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkBulkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||
|
||||
public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||
this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
|
||||
}
|
||||
|
||||
public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
try {
|
||||
return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config,
|
||||
this, true, bulkInsertPartitioner);
|
||||
} catch (HoodieInsertException ie) {
|
||||
throw ie;
|
||||
} catch (Throwable e) {
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,102 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory;
|
||||
import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* A spark implementation of {@link AbstractBulkInsertHelper}.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkBulkInsertHelper<T extends HoodieRecordPayload, R> extends AbstractBulkInsertHelper<T, JavaRDD<HoodieRecord<T>>,
|
||||
JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
|
||||
|
||||
private SparkBulkInsertHelper() {
|
||||
}
|
||||
|
||||
private static class BulkInsertHelperHolder {
|
||||
private static final SparkBulkInsertHelper SPARK_BULK_INSERT_HELPER = new SparkBulkInsertHelper();
|
||||
}
|
||||
|
||||
public static SparkBulkInsertHelper newInstance() {
|
||||
return BulkInsertHelperHolder.SPARK_BULK_INSERT_HELPER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(JavaRDD<HoodieRecord<T>> inputRecords,
|
||||
String instantTime,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
HoodieWriteConfig config,
|
||||
BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> executor,
|
||||
boolean performDedupe,
|
||||
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
|
||||
HoodieWriteMetadata result = new HoodieWriteMetadata();
|
||||
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords = inputRecords;
|
||||
|
||||
if (performDedupe) {
|
||||
dedupedRecords = (JavaRDD<HoodieRecord<T>>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
|
||||
config.getBulkInsertShuffleParallelism(), table);
|
||||
}
|
||||
|
||||
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
||||
final int parallelism = config.getBulkInsertShuffleParallelism();
|
||||
BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent()
|
||||
? userDefinedBulkInsertPartitioner.get()
|
||||
: BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode());
|
||||
repartitionedRecords = (JavaRDD<HoodieRecord<T>>) partitioner.repartitionRecords(dedupedRecords, parallelism);
|
||||
|
||||
// generate new file ID prefixes for each output partition
|
||||
final List<String> fileIDPrefixes =
|
||||
IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
|
||||
|
||||
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED,
|
||||
table.getMetaClient().getCommitActionType(), instantTime), Option.empty(),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(instantTime,
|
||||
partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true)
|
||||
.flatMap(List::iterator);
|
||||
|
||||
((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatusRDD, result);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,63 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkBulkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
||||
private final Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner;
|
||||
|
||||
public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
||||
Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
|
||||
super(context, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
||||
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
||||
this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
try {
|
||||
return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config,
|
||||
this, false, userDefinedBulkInsertPartitioner);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkDeleteCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieKey> keys;
|
||||
|
||||
public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieKey> keys) {
|
||||
super(context, config, table, instantTime, WriteOperationType.DELETE);
|
||||
this.keys = keys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,123 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.time.Instant;
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
* A spark implementation of {@link AbstractDeleteHelper}.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkDeleteHelper<T extends HoodieRecordPayload,R> extends
|
||||
AbstractDeleteHelper<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
|
||||
private SparkDeleteHelper() {
|
||||
}
|
||||
|
||||
private static class DeleteHelperHolder {
|
||||
private static final SparkDeleteHelper SPARK_DELETE_HELPER = new SparkDeleteHelper();
|
||||
}
|
||||
|
||||
public static SparkDeleteHelper newInstance() {
|
||||
return DeleteHelperHolder.SPARK_DELETE_HELPER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieKey> deduplicateKeys(JavaRDD<HoodieKey> keys, HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, int parallelism) {
|
||||
boolean isIndexingGlobal = table.getIndex().isGlobal();
|
||||
if (isIndexingGlobal) {
|
||||
return keys.keyBy(HoodieKey::getRecordKey)
|
||||
.reduceByKey((key1, key2) -> key1, parallelism)
|
||||
.values();
|
||||
} else {
|
||||
return keys.distinct(parallelism);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(String instantTime,
|
||||
JavaRDD<HoodieKey> keys,
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> deleteExecutor) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
|
||||
try {
|
||||
HoodieWriteMetadata result = null;
|
||||
JavaRDD<HoodieKey> dedupedKeys = keys;
|
||||
final int parallelism = config.getDeleteShuffleParallelism();
|
||||
if (config.shouldCombineBeforeDelete()) {
|
||||
// De-dupe/merge if needed
|
||||
dedupedKeys = deduplicateKeys(keys, table, parallelism);
|
||||
} else if (!keys.partitions().isEmpty()) {
|
||||
dedupedKeys = keys.repartition(parallelism);
|
||||
}
|
||||
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
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);
|
||||
Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
|
||||
|
||||
// filter out non existent keys/records
|
||||
JavaRDD<HoodieRecord<T>> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown);
|
||||
if (!taggedValidRecords.isEmpty()) {
|
||||
result = deleteExecutor.execute(taggedValidRecords);
|
||||
result.setIndexLookupDuration(tagLocationDuration);
|
||||
} else {
|
||||
// if entire set of keys are non existent
|
||||
deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime);
|
||||
result = new HoodieWriteMetadata();
|
||||
result.setWriteStatuses(jsc.emptyRDD());
|
||||
deleteExecutor.commitOnAutoCommit(result);
|
||||
}
|
||||
return result;
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
|
||||
public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
|
||||
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class SparkInsertOverwriteCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkInsertOverwriteCommitActionExecutor.class);
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
|
||||
public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata execute() {
|
||||
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
|
||||
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Partitioner getPartitioner(WorkloadProfile profile) {
|
||||
return new SparkInsertOverwritePartitioner(profile, context, table, config);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCommitActionType() {
|
||||
return HoodieTimeline.REPLACE_COMMIT_ACTION;
|
||||
}
|
||||
|
||||
protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
|
||||
return writeStatuses.map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath ->
|
||||
new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
|
||||
}
|
||||
|
||||
private List<String> getAllExistingFileIds(String partitionPath) {
|
||||
// because new commit is not complete. it is safe to mark all existing file Ids as old files
|
||||
return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
|
||||
*/
|
||||
public class SparkInsertOverwritePartitioner extends UpsertPartitioner {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkInsertOverwritePartitioner.class);
|
||||
|
||||
public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
super(profile, context, table, config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path.
|
||||
*/
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
// for overwrite, we ignore all existing files. So dont consider any file to be smallFiles
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return super.execute(preppedRecords);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
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.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.BinaryDecoder;
|
||||
import org.apache.avro.io.BinaryEncoder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
public class SparkMergeHelper<T extends HoodieRecordPayload> extends AbstractMergeHelper<T, JavaRDD<HoodieRecord<T>>,
|
||||
JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static class MergeHelperHolder {
|
||||
private static final SparkMergeHelper SPARK_MERGE_HELPER = new SparkMergeHelper();
|
||||
}
|
||||
|
||||
public static SparkMergeHelper newInstance() {
|
||||
return SparkMergeHelper.MergeHelperHolder.SPARK_MERGE_HELPER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void runMerge(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
HoodieMergeHandle<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> upsertHandle) throws IOException {
|
||||
final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
|
||||
Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
|
||||
HoodieMergeHandle<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> mergeHandle = upsertHandle;
|
||||
HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
|
||||
|
||||
final GenericDatumWriter<GenericRecord> gWriter;
|
||||
final GenericDatumReader<GenericRecord> gReader;
|
||||
Schema readSchema;
|
||||
if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
|
||||
gWriter = new GenericDatumWriter<>(readSchema);
|
||||
gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields());
|
||||
} else {
|
||||
gReader = null;
|
||||
gWriter = null;
|
||||
readSchema = mergeHandle.getWriterSchemaWithMetafields();
|
||||
}
|
||||
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<T, GenericRecord>getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
|
||||
try {
|
||||
final Iterator<GenericRecord> readerIterator;
|
||||
if (baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
|
||||
} else {
|
||||
readerIterator = reader.getRecordIterator(readSchema);
|
||||
}
|
||||
|
||||
ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
|
||||
ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
|
||||
wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator,
|
||||
new UpdateHandler(mergeHandle), record -> {
|
||||
if (!externalSchemaTransformation) {
|
||||
return record;
|
||||
}
|
||||
return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record);
|
||||
});
|
||||
wrapper.execute();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
if (reader != null) {
|
||||
reader.close();
|
||||
}
|
||||
mergeHandle.close();
|
||||
if (null != wrapper) {
|
||||
wrapper.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkUpsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
|
||||
public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
|
||||
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkUpsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return super.execute(preppedRecords);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
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.index.HoodieIndex;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* A spark implementation of {@link AbstractWriteHelper}.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class SparkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractWriteHelper<T, JavaRDD<HoodieRecord<T>>,
|
||||
JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
|
||||
private SparkWriteHelper() {
|
||||
}
|
||||
|
||||
private static class WriteHelperHolder {
|
||||
private static final SparkWriteHelper SPARK_WRITE_HELPER = new SparkWriteHelper();
|
||||
}
|
||||
|
||||
public static SparkWriteHelper newInstance() {
|
||||
return WriteHelperHolder.SPARK_WRITE_HELPER;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index,
|
||||
int parallelism) {
|
||||
boolean isIndexingGlobal = index.isGlobal();
|
||||
return records.mapToPair(record -> {
|
||||
HoodieKey hoodieKey = record.getKey();
|
||||
// If index used is global, then records are expected to differ in their partitionPath
|
||||
Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
|
||||
return new Tuple2<>(key, record);
|
||||
}).reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect
|
||||
// everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism).map(Tuple2::_2);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,328 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
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.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition).
|
||||
*/
|
||||
public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends Partitioner {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(UpsertPartitioner.class);
|
||||
|
||||
/**
|
||||
* List of all small files to be corrected.
|
||||
*/
|
||||
protected List<SmallFile> smallFiles = new ArrayList<>();
|
||||
/**
|
||||
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into.
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining inserts, upserts etc.
|
||||
*/
|
||||
private WorkloadProfile profile;
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
private HashMap<String, Integer> updateLocationToBucket;
|
||||
/**
|
||||
* Helps us pack inserts into 1 or more buckets depending on number of incoming records.
|
||||
*/
|
||||
private HashMap<String, List<InsertBucketCumulativeWeightPair>> partitionPathToInsertBucketInfos;
|
||||
/**
|
||||
* Remembers what type each bucket is for later.
|
||||
*/
|
||||
private HashMap<Integer, BucketInfo> bucketInfoMap;
|
||||
|
||||
protected final HoodieTable table;
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBucketInfos = new HashMap<>();
|
||||
bucketInfoMap = new HashMap<>();
|
||||
this.profile = profile;
|
||||
this.table = table;
|
||||
this.config = config;
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile, context);
|
||||
|
||||
LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n"
|
||||
+ "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n"
|
||||
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
|
||||
}
|
||||
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
Set<Entry<String, WorkloadStat>> partitionStatEntries = profile.getPartitionPathStatMap().entrySet();
|
||||
for (Map.Entry<String, WorkloadStat> partitionStat : partitionStatEntries) {
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry :
|
||||
partitionStat.getValue().getUpdateLocationToCount().entrySet()) {
|
||||
addUpdateBucket(partitionStat.getKey(), updateLocEntry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int addUpdateBucket(String partitionPath, String fileIdHint) {
|
||||
int bucket = totalBuckets;
|
||||
updateLocationToBucket.put(fileIdHint, bucket);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.UPDATE;
|
||||
bucketInfo.fileIdPrefix = fileIdHint;
|
||||
bucketInfo.partitionPath = partitionPath;
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
return bucket;
|
||||
}
|
||||
|
||||
private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) {
|
||||
// for new inserts, compute buckets depending on how many records we have for each partition
|
||||
Set<String> partitionPaths = profile.getPartitionPaths();
|
||||
long averageRecordSize =
|
||||
averageBytesPerRecord(table.getMetaClient().getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
|
||||
config);
|
||||
LOG.info("AvgRecordSize => " + averageRecordSize);
|
||||
|
||||
Map<String, List<SmallFile>> partitionSmallFilesMap =
|
||||
getSmallFilesForPartitions(new ArrayList<String>(partitionPaths), context);
|
||||
|
||||
for (String partitionPath : partitionPaths) {
|
||||
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
|
||||
if (pStat.getNumInserts() > 0) {
|
||||
|
||||
List<SmallFile> smallFiles = partitionSmallFilesMap.get(partitionPath);
|
||||
this.smallFiles.addAll(smallFiles);
|
||||
|
||||
LOG.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles);
|
||||
|
||||
long totalUnassignedInserts = pStat.getNumInserts();
|
||||
List<Integer> bucketNumbers = new ArrayList<>();
|
||||
List<Long> recordsPerBucket = new ArrayList<>();
|
||||
|
||||
// first try packing this into one of the smallFiles
|
||||
for (SmallFile smallFile : smallFiles) {
|
||||
long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
|
||||
totalUnassignedInserts);
|
||||
if (recordsToAppend > 0 && totalUnassignedInserts > 0) {
|
||||
// create a new bucket or re-use an existing bucket
|
||||
int bucket;
|
||||
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
|
||||
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
|
||||
LOG.info("Assigning " + recordsToAppend + " inserts to existing update bucket " + bucket);
|
||||
} else {
|
||||
bucket = addUpdateBucket(partitionPath, smallFile.location.getFileId());
|
||||
LOG.info("Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
|
||||
}
|
||||
bucketNumbers.add(bucket);
|
||||
recordsPerBucket.add(recordsToAppend);
|
||||
totalUnassignedInserts -= recordsToAppend;
|
||||
}
|
||||
}
|
||||
|
||||
// if we have anything more, create new insert buckets, like normal
|
||||
if (totalUnassignedInserts > 0) {
|
||||
long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize();
|
||||
if (config.shouldAutoTuneInsertSplits()) {
|
||||
insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize;
|
||||
}
|
||||
|
||||
int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket);
|
||||
LOG.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts
|
||||
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket);
|
||||
for (int b = 0; b < insertBuckets; b++) {
|
||||
bucketNumbers.add(totalBuckets);
|
||||
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.INSERT;
|
||||
bucketInfo.partitionPath = partitionPath;
|
||||
bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx();
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
}
|
||||
}
|
||||
|
||||
// Go over all such buckets, and assign weights as per amount of incoming inserts.
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = new ArrayList<>();
|
||||
double curentCumulativeWeight = 0;
|
||||
for (int i = 0; i < bucketNumbers.size(); i++) {
|
||||
InsertBucket bkt = new InsertBucket();
|
||||
bkt.bucketNumber = bucketNumbers.get(i);
|
||||
bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts();
|
||||
curentCumulativeWeight += bkt.weight;
|
||||
insertBuckets.add(new InsertBucketCumulativeWeightPair(bkt, curentCumulativeWeight));
|
||||
}
|
||||
LOG.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
|
||||
partitionPathToInsertBucketInfos.put(partitionPath, insertBuckets);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, List<SmallFile>> getSmallFilesForPartitions(List<String> partitionPaths, HoodieEngineContext context) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
Map<String, List<SmallFile>> partitionSmallFilesMap = new HashMap<>();
|
||||
if (partitionPaths != null && partitionPaths.size() > 0) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions");
|
||||
JavaRDD<String> partitionPathRdds = jsc.parallelize(partitionPaths, partitionPaths.size());
|
||||
partitionSmallFilesMap = partitionPathRdds.mapToPair((PairFunction<String, String, List<SmallFile>>)
|
||||
partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath))).collectAsMap();
|
||||
}
|
||||
|
||||
return partitionSmallFilesMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path.
|
||||
*/
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = table.getMetaClient().getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieBaseFile> allFiles = table.getBaseFileOnlyView()
|
||||
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
|
||||
|
||||
for (HoodieBaseFile file : allFiles) {
|
||||
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
|
||||
String filename = file.getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = file.getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public BucketInfo getBucketInfo(int bucketNumber) {
|
||||
return bucketInfoMap.get(bucketNumber);
|
||||
}
|
||||
|
||||
public List<InsertBucketCumulativeWeightPair> getInsertBuckets(String partitionPath) {
|
||||
return partitionPathToInsertBucketInfos.get(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return totalBuckets;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation =
|
||||
(Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
|
||||
if (keyLocation._2().isPresent()) {
|
||||
HoodieRecordLocation location = keyLocation._2().get();
|
||||
return updateLocationToBucket.get(location.getFileId());
|
||||
} else {
|
||||
String partitionPath = keyLocation._1().getPartitionPath();
|
||||
List<InsertBucketCumulativeWeightPair> targetBuckets = partitionPathToInsertBucketInfos.get(partitionPath);
|
||||
// pick the target bucket to use based on the weights.
|
||||
final long totalInserts = Math.max(1, profile.getWorkloadStat(partitionPath).getNumInserts());
|
||||
final long hashOfKey = NumericUtils.getMessageDigestHash("MD5", keyLocation._1().getRecordKey());
|
||||
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
|
||||
|
||||
int index = Collections.binarySearch(targetBuckets, new InsertBucketCumulativeWeightPair(new InsertBucket(), r));
|
||||
|
||||
if (index >= 0) {
|
||||
return targetBuckets.get(index).getKey().bucketNumber;
|
||||
}
|
||||
|
||||
if ((-1 * index - 1) < targetBuckets.size()) {
|
||||
return targetBuckets.get((-1 * index - 1)).getKey().bucketNumber;
|
||||
}
|
||||
|
||||
// return first one, by default
|
||||
return targetBuckets.get(0).getKey().bucketNumber;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains the average record size based on records written during previous commits. Used for estimating how many
|
||||
* records pack into one file.
|
||||
*/
|
||||
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig) {
|
||||
long avgSize = hoodieWriteConfig.getCopyOnWriteRecordSizeEstimate();
|
||||
long fileSizeThreshold = (long) (hoodieWriteConfig.getRecordSizeEstimationThreshold() * hoodieWriteConfig.getParquetSmallFileLimit());
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
// Go over the reverse ordered commits to get a more recent estimate of average record size.
|
||||
Iterator<HoodieInstant> instants = commitTimeline.getReverseOrderedInstants().iterator();
|
||||
while (instants.hasNext()) {
|
||||
HoodieInstant instant = instants.next();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
|
||||
long totalBytesWritten = commitMetadata.fetchTotalBytesWritten();
|
||||
long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten();
|
||||
if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) {
|
||||
avgSize = (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// make this fail safe.
|
||||
LOG.error("Error trying to compute average bytes/record ", t);
|
||||
}
|
||||
return avgSize;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,243 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.IOUtils;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Compacts a hoodie table with merge on read storage. Computes all possible compactions,
|
||||
* passes it through a CompactionFilter and executes all the compactions and writes a new version of base files and make
|
||||
* a normal commit
|
||||
*
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload> implements HoodieCompactor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieSparkMergeOnReadTableCompactor.class);
|
||||
// Accumulator to keep track of total log files for a table
|
||||
private AccumulatorV2<Long, Long> totalLogFiles;
|
||||
// Accumulator to keep track of total log file slices for a table
|
||||
private AccumulatorV2<Long, Long> totalFileSlices;
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
return jsc.emptyRDD();
|
||||
}
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieSparkCopyOnWriteTable table = new HoodieSparkCopyOnWriteTable(config, context, metaClient);
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Compactor compacting " + operations + " files");
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices");
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime)).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String instantTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
|
||||
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
LOG.info("Compacting base " + operation.getDataFileName() + " with delta files " + operation.getDeltaFileNames()
|
||||
+ " for commit " + instantTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
String maxInstantTime = metaClient
|
||||
.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
|
||||
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
|
||||
LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
|
||||
|
||||
List<String> logFiles = operation.getDeltaFileNames().stream().map(
|
||||
p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString())
|
||||
.collect(toList());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), logFiles,
|
||||
readerSchema, maxInstantTime, maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
|
||||
config.getSpillableMapBasePath());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
Option<HoodieBaseFile> oldDataFileOpt =
|
||||
operation.getBaseFile(metaClient.getBasePath(), operation.getPartitionPath());
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
// If the dataFile is present, perform updates else perform inserts into a new base file.
|
||||
if (oldDataFileOpt.isPresent()) {
|
||||
result = hoodieCopyOnWriteTable.handleUpdate(instantTime, operation.getPartitionPath(),
|
||||
operation.getFileId(), scanner.getRecords(),
|
||||
oldDataFileOpt.get());
|
||||
} else {
|
||||
result = hoodieCopyOnWriteTable.handleInsert(instantTime, operation.getPartitionPath(), operation.getFileId(),
|
||||
scanner.getRecords());
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> {
|
||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
s.getStat()
|
||||
.setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
|
||||
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
|
||||
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
|
||||
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||
s.getStat().setRuntimeStats(runtimeStats);
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
|
||||
HoodieWriteConfig config, String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions)
|
||||
throws IOException {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
totalLogFiles = new LongAccumulator();
|
||||
totalFileSlices = new LongAccumulator();
|
||||
jsc.sc().register(totalLogFiles);
|
||||
jsc.sc().register(totalFileSlices);
|
||||
|
||||
ValidationUtils.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"Can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
|
||||
+ hoodieTable.getMetaClient().getTableType().name());
|
||||
|
||||
// TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
LOG.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
// filter the partition paths if needed to reduce list status
|
||||
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
|
||||
|
||||
if (partitionPaths.isEmpty()) {
|
||||
// In case no partitions could be picked, return no compaction plan
|
||||
return null;
|
||||
}
|
||||
|
||||
SliceView fileSystemView = hoodieTable.getSliceView();
|
||||
LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact");
|
||||
|
||||
List<HoodieCompactionOperation> operations = context.flatMap(partitionPaths, partitionPath -> {
|
||||
return fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(s -> {
|
||||
List<HoodieLogFile> logFiles =
|
||||
s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
|
||||
// for spark Map operations and collecting them finally in Avro generated classes for storing
|
||||
// into meta files.
|
||||
Option<HoodieBaseFile> dataFile = s.getBaseFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFileNames().isEmpty());
|
||||
}, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
|
||||
|
||||
LOG.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
LOG.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
LOG.info("Total number of log files " + totalLogFiles.value());
|
||||
LOG.info("Total number of file slices " + totalFileSlices.value());
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
|
||||
ValidationUtils.checkArgument(
|
||||
compactionPlan.getOperations().stream().noneMatch(
|
||||
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
|
||||
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
|
||||
+ "Please fix your strategy implementation. FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
|
||||
+ ", Selected workload :" + compactionPlan);
|
||||
if (compactionPlan.getOperations().isEmpty()) {
|
||||
LOG.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
}
|
||||
return compactionPlan;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A spark implementation of {@link AbstractCompactHelpers}.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public class SparkCompactHelpers<T extends HoodieRecordPayload> extends
|
||||
AbstractCompactHelpers<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private SparkCompactHelpers() {
|
||||
}
|
||||
|
||||
private static class CompactHelperHolder {
|
||||
private static final SparkCompactHelpers SPARK_COMPACT_HELPERS = new SparkCompactHelpers();
|
||||
}
|
||||
|
||||
public static SparkCompactHelpers newInstance() {
|
||||
return CompactHelperHolder.SPARK_COMPACT_HELPERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCommitMetadata createCompactionMetadata(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String compactionInstantTime,
|
||||
JavaRDD<WriteStatus> writeStatuses,
|
||||
String schema) throws IOException {
|
||||
byte[] planBytes = table.getActiveTimeline().readCompactionPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get();
|
||||
HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(planBytes);
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(WriteStatus::getStat).collect();
|
||||
org.apache.hudi.common.model.HoodieCommitMetadata metadata = new org.apache.hudi.common.model.HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY, schema);
|
||||
if (compactionPlan.getExtraMetadata() != null) {
|
||||
compactionPlan.getExtraMetadata().forEach(metadata::addMetadata);
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,95 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
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.model.HoodieCommitMetadata;
|
||||
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.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.BaseActionExecutor;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkRunCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieWriteMetadata<JavaRDD<WriteStatus>>> {
|
||||
|
||||
public SparkRunCompactionActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime) {
|
||||
super(context, config, table, instantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(instantTime);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
if (!pendingCompactionTimeline.containsInstant(instant)) {
|
||||
throw new IllegalStateException(
|
||||
"No Compaction request available at " + instantTime + " to run compaction");
|
||||
}
|
||||
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>();
|
||||
try {
|
||||
HoodieActiveTimeline timeline = table.getActiveTimeline();
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.getCompactionPlan(table.getMetaClient(), instantTime);
|
||||
// Mark instant as compaction inflight
|
||||
timeline.transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
|
||||
JavaRDD<WriteStatus> statuses = compactor.compact(context, compactionPlan, table, config, instantTime);
|
||||
|
||||
statuses.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
|
||||
List<HoodieWriteStat> updateStatusMap = statuses.map(WriteStatus::getStat).collect();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema());
|
||||
|
||||
compactionMetadata.setWriteStatuses(statuses);
|
||||
compactionMetadata.setCommitted(false);
|
||||
compactionMetadata.setCommitMetadata(Option.of(metadata));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
|
||||
return compactionMetadata;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,88 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkScheduleCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseScheduleCompactionActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkScheduleCompactionActionExecutor.class);
|
||||
|
||||
public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieCompactionPlan scheduleCompaction() {
|
||||
LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String lastCompactionTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
lastCompactionTs = lastCompaction.get().getTimestamp();
|
||||
}
|
||||
|
||||
int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
|
||||
.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
|
||||
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
||||
LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
|
||||
+ " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
|
||||
+ config.getInlineCompactDeltaCommitMax());
|
||||
return new HoodieCompactionPlan();
|
||||
}
|
||||
|
||||
LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
|
||||
HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
|
||||
try {
|
||||
return compactor.generateCompactionPlan(context, table, config, instantTime,
|
||||
((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations()
|
||||
.map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
|
||||
.collect(Collectors.toSet()));
|
||||
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,102 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.execution.SparkLazyInsertIterable;
|
||||
import org.apache.hudi.io.AppendHandleFactory;
|
||||
import org.apache.hudi.io.HoodieAppendHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
public abstract class AbstractSparkDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends BaseSparkCommitActionExecutor<T> {
|
||||
private static final Logger LOG = LogManager.getLogger(AbstractSparkDeltaCommitActionExecutor.class);
|
||||
|
||||
// UpsertPartitioner for MergeOnRead table type
|
||||
private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner;
|
||||
|
||||
public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, WriteOperationType operationType) {
|
||||
this(context, config, table, instantTime, operationType, Option.empty());
|
||||
}
|
||||
|
||||
public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, WriteOperationType operationType,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, operationType, extraMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config);
|
||||
return mergeOnReadUpsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
LOG.info("Merging updates for commit " + instantTime + " for file " + fileId);
|
||||
|
||||
if (!table.getIndex().canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId);
|
||||
return super.handleUpdate(partitionPath, fileId, recordItr);
|
||||
} else {
|
||||
HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table,
|
||||
partitionPath, fileId, recordItr, taskContextSupplier);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())).iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws Exception {
|
||||
// If canIndexLogFiles, write inserts to log files else write inserts to base files
|
||||
if (table.getIndex().canIndexLogFiles()) {
|
||||
return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table,
|
||||
idPfx, taskContextSupplier, new AppendHandleFactory<>());
|
||||
} else {
|
||||
return super.handleInsert(idPfx, recordItr);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkBulkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||
|
||||
public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||
this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
|
||||
}
|
||||
|
||||
public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
|
||||
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
try {
|
||||
return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config,
|
||||
this, true, bulkInsertPartitioner);
|
||||
} catch (HoodieInsertException ie) {
|
||||
throw ie;
|
||||
} catch (Throwable e) {
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkBulkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
|
||||
private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
|
||||
|
||||
public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
|
||||
Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
|
||||
super(context, config, table, instantTime, WriteOperationType.BULK_INSERT);
|
||||
this.preppedInputRecordRdd = preppedInputRecordRdd;
|
||||
this.bulkInsertPartitioner = bulkInsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
try {
|
||||
return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config,
|
||||
this, false, bulkInsertPartitioner);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkDeleteHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkDeleteDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieKey> keys;
|
||||
|
||||
public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieKey> keys) {
|
||||
super(context, config, table, instantTime, WriteOperationType.DELETE);
|
||||
this.keys = keys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkWriteHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
|
||||
public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
|
||||
config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return super.execute(preppedRecords);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.commit.SparkWriteHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkUpsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> inputRecordsRDD;
|
||||
|
||||
public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT);
|
||||
this.inputRecordsRDD = inputRecordsRDD;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata execute() {
|
||||
return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
|
||||
config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
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.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
import org.apache.hudi.table.action.commit.UpsertPartitioner;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet files to larger ones
|
||||
* without the need for an index in the logFile.
|
||||
*/
|
||||
public class SparkUpsertDeltaCommitPartitioner<T extends HoodieRecordPayload<T>> extends UpsertPartitioner<T> {
|
||||
|
||||
SparkUpsertDeltaCommitPartitioner(WorkloadProfile profile, HoodieSparkEngineContext context, HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
super(profile, context, table, config);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
||||
|
||||
// Find out all eligible small file slices
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
List<FileSlice> allSmallFileSlices = new ArrayList<>();
|
||||
// If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to
|
||||
// it. Doing this overtime for a partition, we ensure that we handle small file issues
|
||||
if (!table.getIndex().canIndexLogFiles()) {
|
||||
// TODO : choose last N small files since there can be multiple small files written to a single partition
|
||||
// by different spark partitions in a single batch
|
||||
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(table.getSliceView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
|
||||
.filter(
|
||||
fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getBaseFile().get().getFileSize() < config
|
||||
.getParquetSmallFileLimit())
|
||||
.min((FileSlice left, FileSlice right) ->
|
||||
left.getBaseFile().get().getFileSize() < right.getBaseFile().get().getFileSize() ? -1 : 1));
|
||||
if (smallFileSlice.isPresent()) {
|
||||
allSmallFileSlices.add(smallFileSlice.get());
|
||||
}
|
||||
} else {
|
||||
// If we can index log files, we can add more inserts to log files for fileIds including those under
|
||||
// pending compaction.
|
||||
List<FileSlice> allFileSlices =
|
||||
table.getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : allFileSlices) {
|
||||
if (isSmallFile(fileSlice)) {
|
||||
allSmallFileSlices.add(fileSlice);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Create SmallFiles from the eligible file slices
|
||||
for (FileSlice smallFileSlice : allSmallFileSlices) {
|
||||
SmallFile sf = new SmallFile();
|
||||
if (smallFileSlice.getBaseFile().isPresent()) {
|
||||
// TODO : Move logic of file name, file id, base commit time handling inside file slice
|
||||
String filename = smallFileSlice.getBaseFile().get().getFileName();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
} else {
|
||||
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
|
||||
FSUtils.getFileIdFromLogPath(logFile.getPath()));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public List<String> getSmallFileIds() {
|
||||
return (List<String>) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private long getTotalFileSize(FileSlice fileSlice) {
|
||||
if (!fileSlice.getBaseFile().isPresent()) {
|
||||
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
} else {
|
||||
return fileSlice.getBaseFile().get().getFileSize()
|
||||
+ convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSmallFile(FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(fileSlice);
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
|
||||
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(size -> size > 0).reduce(Long::sum).orElse(0L);
|
||||
// Here we assume that if there is no base parquet file, all log files contain only inserts.
|
||||
// We can then just get the parquet equivalent size of these log files, compare that with
|
||||
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
|
||||
return (long) (totalSizeOfLogFiles * config.getLogFileToParquetCompressionRatio());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.table.action.deltacommit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
public class SparkUpsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
|
||||
extends AbstractSparkDeltaCommitActionExecutor<T> {
|
||||
|
||||
private final JavaRDD<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
|
||||
return super.execute(preppedRecords);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.table.action.restore;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCopyOnWriteRestoreActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
public SparkCopyOnWriteRestoreActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
String restoreInstantTime) {
|
||||
super(context, config, table, instantTime, restoreInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
SparkCopyOnWriteRollbackActionExecutor rollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(
|
||||
(HoodieSparkEngineContext) context,
|
||||
config,
|
||||
table,
|
||||
HoodieActiveTimeline.createNewInstantTime(),
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
if (!instantToRollback.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
&& !instantToRollback.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
|
||||
throw new HoodieRollbackException("Unsupported action in rollback instant:" + instantToRollback);
|
||||
}
|
||||
return rollbackActionExecutor.execute();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.action.restore;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
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.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
public SparkMergeOnReadRestoreActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
String restoreInstantTime) {
|
||||
super(context, config, table, instantTime, restoreInstantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
SparkMergeOnReadRollbackActionExecutor rollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor(
|
||||
context,
|
||||
config,
|
||||
table,
|
||||
HoodieActiveTimeline.createNewInstantTime(),
|
||||
instantToRollback,
|
||||
true,
|
||||
true,
|
||||
false);
|
||||
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
case HoodieTimeline.REPLACE_COMMIT_ACTION:
|
||||
// TODO : Get file status and create a rollback stat and file
|
||||
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
||||
// delete these files when it does not see a corresponding instant file under .hoodie
|
||||
return rollbackActionExecutor.execute();
|
||||
default:
|
||||
throw new IllegalArgumentException("invalid action name " + instantToRollback.getAction());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,239 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Performs Rollback of Hoodie Tables.
|
||||
*/
|
||||
public class ListingBasedRollbackHelper implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(ListingBasedRollbackHelper.class);
|
||||
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public ListingBasedRollbackHelper(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
this.metaClient = metaClient;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs all rollback actions that we have collected in parallel.
|
||||
*/
|
||||
public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
|
||||
int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
|
||||
JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, true);
|
||||
return partitionPathRollbackStatsPairRDD.reduceByKey(RollbackUtils::mergeRollbackStat).map(Tuple2::_2).collect();
|
||||
}
|
||||
|
||||
/**
|
||||
* Collect all file info that needs to be rollbacked.
|
||||
*/
|
||||
public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
|
||||
int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
|
||||
JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, false);
|
||||
return partitionPathRollbackStatsPairRDD.map(Tuple2::_2).collect();
|
||||
}
|
||||
|
||||
/**
|
||||
* May be delete interested files and collect stats or collect stats only.
|
||||
*
|
||||
* @param context instance of {@link HoodieEngineContext} to use.
|
||||
* @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
|
||||
* @param rollbackRequests List of {@link ListingBasedRollbackRequest} to be operated on.
|
||||
* @param sparkPartitions number of spark partitions to use for parallelism.
|
||||
* @param doDelete {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
|
||||
* @return stats collected with or w/o actual deletions.
|
||||
*/
|
||||
JavaPairRDD<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests,
|
||||
int sparkPartitions, boolean doDelete) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
return jsc.parallelize(rollbackRequests, sparkPartitions).mapToPair(rollbackRequest -> {
|
||||
switch (rollbackRequest.getType()) {
|
||||
case DELETE_DATA_FILES_ONLY: {
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
|
||||
rollbackRequest.getPartitionPath(), doDelete);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withDeletedFileResults(filesToDeletedStatus).build());
|
||||
}
|
||||
case DELETE_DATA_AND_LOG_FILES: {
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withDeletedFileResults(filesToDeletedStatus).build());
|
||||
}
|
||||
case APPEND_ROLLBACK_BLOCK: {
|
||||
Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(metaClient.getBasePath(), rollbackRequest.getPartitionPath()))
|
||||
.withFileId(rollbackRequest.getFileId().get())
|
||||
.overBaseCommit(rollbackRequest.getLatestBaseInstant().get()).withFs(metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
|
||||
// generate metadata
|
||||
if (doDelete) {
|
||||
Map<HeaderMetadataType, String> header = generateHeader(instantToRollback.getTimestamp());
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
}
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException("Failed to rollback for instant " + instantToRollback, io);
|
||||
} finally {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Error appending rollback block..", io);
|
||||
}
|
||||
}
|
||||
|
||||
// This step is intentionally done after writer is closed. Guarantees that
|
||||
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
|
||||
// cloud-storage : HUDI-168
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = Collections.singletonMap(
|
||||
metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()),
|
||||
1L
|
||||
);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build());
|
||||
}
|
||||
default:
|
||||
throw new IllegalStateException("Unknown Rollback action " + rollbackRequest);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
|
||||
*/
|
||||
private Map<FileStatus, Boolean> deleteBaseAndLogFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
|
||||
String commit, String partitionPath, boolean doDelete) throws IOException {
|
||||
LOG.info("Cleaning path " + partitionPath);
|
||||
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
|
||||
SerializablePathFilter filter = (path) -> {
|
||||
if (path.toString().endsWith(basefileExtension)) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
} else if (FSUtils.isLogFile(path)) {
|
||||
// Since the baseCommitTime is the only commit for new log files, it's okay here
|
||||
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
final Map<FileStatus, Boolean> results = new HashMap<>();
|
||||
FileSystem fs = metaClient.getFs();
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
if (doDelete) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
LOG.info("Delete file " + file.getPath() + "\t" + success);
|
||||
} else {
|
||||
results.put(file, true);
|
||||
}
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out base files under a partition path during rollback of a set of commits.
|
||||
*/
|
||||
private Map<FileStatus, Boolean> deleteBaseFiles(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
|
||||
String commit, String partitionPath, boolean doDelete) throws IOException {
|
||||
final Map<FileStatus, Boolean> results = new HashMap<>();
|
||||
LOG.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = metaClient.getFs();
|
||||
String basefileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
|
||||
PathFilter filter = (path) -> {
|
||||
if (path.toString().contains(basefileExtension)) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
if (doDelete) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
LOG.info("Delete file " + file.getPath() + "\t" + success);
|
||||
} else {
|
||||
results.put(file, true);
|
||||
}
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
private Map<HeaderMetadataType, String> generateHeader(String commit) {
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = new HashMap<>(3);
|
||||
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
|
||||
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
|
||||
return header;
|
||||
}
|
||||
|
||||
public interface SerializablePathFilter extends PathFilter, Serializable {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseCopyOnWriteRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
HoodieInstant commitInstant,
|
||||
boolean deleteInstants) {
|
||||
super(context, config, table, instantTime, commitInstant, deleteInstants);
|
||||
}
|
||||
|
||||
public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
HoodieInstant commitInstant,
|
||||
boolean deleteInstants,
|
||||
boolean skipTimelinePublish,
|
||||
boolean useMarkerBasedStrategy) {
|
||||
super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
|
||||
if (useMarkerBasedStrategy) {
|
||||
return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
|
||||
} else {
|
||||
return this::executeRollbackUsingFileListing;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
|
||||
List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
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.model.IOType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> extends AbstractMarkerBasedRollbackStrategy<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public SparkMarkerBasedRollbackStrategy(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {
|
||||
super(table, context, config, instantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
try {
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
|
||||
List<String> markerFilePaths = markerFiles.allMarkerFilePaths();
|
||||
int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
|
||||
return jsc.parallelize(markerFilePaths, parallelism)
|
||||
.map(markerFilePath -> {
|
||||
String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
|
||||
IOType type = IOType.valueOf(typeStr);
|
||||
switch (type) {
|
||||
case MERGE:
|
||||
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
||||
case APPEND:
|
||||
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
|
||||
case CREATE:
|
||||
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
|
||||
default:
|
||||
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
|
||||
}
|
||||
})
|
||||
.mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat))
|
||||
.reduceByKey(RollbackUtils::mergeRollbackStat)
|
||||
.map(Tuple2::_2).collect();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,82 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
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.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class SparkMergeOnReadRollbackActionExecutor<T extends HoodieRecordPayload> extends
|
||||
BaseMergeOnReadRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
HoodieInstant commitInstant,
|
||||
boolean deleteInstants) {
|
||||
super(context, config, table, instantTime, commitInstant, deleteInstants);
|
||||
}
|
||||
|
||||
public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String instantTime,
|
||||
HoodieInstant commitInstant,
|
||||
boolean deleteInstants,
|
||||
boolean skipTimelinePublish,
|
||||
boolean useMarkerBasedStrategy) {
|
||||
super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
|
||||
if (useMarkerBasedStrategy) {
|
||||
return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
|
||||
} else {
|
||||
return this::executeRollbackUsingFileListing;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant resolvedInstant) {
|
||||
List<ListingBasedRollbackRequest> rollbackRequests;
|
||||
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
|
||||
try {
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, context);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error generating rollback requests by file listing.", e);
|
||||
}
|
||||
return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, resolvedInstant, rollbackRequests);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table.upgrade;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Downgrade handle to assist in downgrading hoodie table from version 1 to 0.
|
||||
*/
|
||||
public class OneToZeroDowngradeHandler implements DowngradeHandler {
|
||||
|
||||
@Override
|
||||
public void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
|
||||
// fetch pending commit info
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
|
||||
for (HoodieInstant commitInstant : commits) {
|
||||
// delete existing marker files
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
||||
markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.table.upgrade;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpgradeDowngradeException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class SparkUpgradeDowngrade extends AbstractUpgradeDowngrade {
|
||||
|
||||
public SparkUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) {
|
||||
super(metaClient, config, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(HoodieTableMetaClient metaClient,
|
||||
HoodieTableVersion toVersion,
|
||||
HoodieWriteConfig config,
|
||||
HoodieEngineContext context,
|
||||
String instantTime) {
|
||||
try {
|
||||
new SparkUpgradeDowngrade(metaClient, config, context).run(toVersion, instantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
|
||||
if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) {
|
||||
new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime);
|
||||
} else {
|
||||
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
|
||||
if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) {
|
||||
new OneToZeroDowngradeHandler().downgrade(config, context, instantTime);
|
||||
} else {
|
||||
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,136 @@
|
||||
/*
|
||||
* 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.table.upgrade;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
|
||||
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.table.action.rollback.RollbackUtils;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Upgrade handle to assist in upgrading hoodie table from version 0 to 1.
|
||||
*/
|
||||
public class ZeroToOneUpgradeHandler implements UpgradeHandler {
|
||||
|
||||
@Override
|
||||
public void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
|
||||
// fetch pending commit info
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (commits.size() > 0 && instantTime != null) {
|
||||
// ignore the latest inflight commit since a new commit would have been started and we need to fix any pending commits from previous launch
|
||||
commits.remove(instantTime);
|
||||
}
|
||||
for (String commit : commits) {
|
||||
// for every pending commit, delete old marker files and re-create marker files in new format
|
||||
recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Recreate marker files in new format.
|
||||
* Step1: Delete existing marker files
|
||||
* Step2: Collect all rollback file info.
|
||||
* Step3: recreate marker files for all interested files.
|
||||
*
|
||||
* @param commitInstantTime instant of interest for which marker files need to be recreated.
|
||||
* @param table instance of {@link HoodieSparkTable} to use
|
||||
* @param context instance of {@link HoodieEngineContext} to use
|
||||
* @throws HoodieRollbackException on any exception during upgrade.
|
||||
*/
|
||||
private static void recreateMarkerFiles(final String commitInstantTime,
|
||||
HoodieSparkTable table,
|
||||
HoodieEngineContext context,
|
||||
int parallelism) throws HoodieRollbackException {
|
||||
try {
|
||||
// fetch hoodie instant
|
||||
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
|
||||
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
|
||||
.findFirst());
|
||||
if (commitInstantOpt.isPresent()) {
|
||||
// delete existing marker files
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
|
||||
markerFiles.quietDeleteMarkerDir(context, parallelism);
|
||||
|
||||
// generate rollback stats
|
||||
List<ListingBasedRollbackRequest> rollbackRequests;
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.COPY_ON_WRITE) {
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
|
||||
table.getConfig().shouldAssumeDatePartitioning());
|
||||
} else {
|
||||
rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
|
||||
}
|
||||
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
|
||||
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
|
||||
|
||||
// recreate marker files adhering to marker based rollback
|
||||
for (HoodieRollbackStat rollbackStat : rollbackStats) {
|
||||
for (String path : rollbackStat.getSuccessDeleteFiles()) {
|
||||
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
|
||||
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
|
||||
markerFiles.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
|
||||
}
|
||||
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
|
||||
markerFiles.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieRollbackException("Exception thrown while upgrading Hoodie Table from version 0 to 1", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Curates file name for marker from existing log file path.
|
||||
* log file format : partitionpath/.fileid_baseInstant.log.writetoken
|
||||
* marker file format : partitionpath/fileId_writetoken_baseinstant.basefileExtn.marker.APPEND
|
||||
*
|
||||
* @param logFilePath log file path for which marker file name needs to be generated.
|
||||
* @return the marker file name thus curated.
|
||||
*/
|
||||
private static String getFileNameForMarkerFromLogFile(String logFilePath, HoodieTable table) {
|
||||
Path logPath = new Path(table.getMetaClient().getBasePath(), logFilePath);
|
||||
String fileId = FSUtils.getFileIdFromLogPath(logPath);
|
||||
String baseInstant = FSUtils.getBaseCommitTimeFromLogPath(logPath);
|
||||
String writeToken = FSUtils.getWriteTokenFromLogPath(logPath);
|
||||
|
||||
return FSUtils.makeDataFileName(baseInstant, writeToken, fileId, table.getBaseFileFormat().getFileExtension());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,23 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=INFO, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
@@ -0,0 +1,320 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test Cases for rollback of snapshots and commits.
|
||||
*/
|
||||
public class TestClientRollback extends HoodieClientTestBase {
|
||||
|
||||
/**
|
||||
* Test case for rollback-savepoint interaction.
|
||||
*/
|
||||
@Test
|
||||
public void testSavepointAndRollback() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build();
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint("hoodie-unit-test", "test");
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view1 = table.getBaseFileOnlyView();
|
||||
|
||||
List<HoodieBaseFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 003 should be present");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 002 should be present");
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view2 = table.getBaseFileOnlyView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 004 should be present");
|
||||
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
assertThrows(HoodieRollbackException.class, () -> {
|
||||
client.restoreToSavepoint("001");
|
||||
}, "Rolling back to non-existent savepoint should not be allowed");
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.restoreToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view3 = table.getBaseFileOnlyView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"))).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 002 be available");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003"))).collect(Collectors.toList());
|
||||
assertEquals(0, dataFiles.size(), "The data files for commit 003 should be rolled back");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
|
||||
assertEquals(0, dataFiles.size(), "The data files for commit 004 should be rolled back");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Cases for effects of rollbacking completed/inflight commits.
|
||||
*/
|
||||
@Test
|
||||
public void testRollbackCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
final String commitTime1 = "20160501010101";
|
||||
final String commitTime2 = "20160502020601";
|
||||
final String commitTime3 = "20160506030611";
|
||||
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id11");
|
||||
put(p2, "id12");
|
||||
put(p3, "id13");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id21");
|
||||
put(p2, "id22");
|
||||
put(p3, "id23");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id31");
|
||||
put(p2, "id32");
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
|
||||
|
||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||
assertThrows(HoodieRollbackException.class, () -> {
|
||||
client.rollback(commitTime1);
|
||||
}, "Should have thrown an exception ");
|
||||
|
||||
// Rollback commit3
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// simulate partial failure, where .inflight was not deleted, but data files were.
|
||||
testTable.addInflightCommit(commitTime3);
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// Rollback commit2
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(testTable.commitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
|
||||
// .inflight commit and a bunch of data files around.
|
||||
testTable.addInflightCommit(commitTime2).withBaseFilesInPartitions(partitionAndFileId2);
|
||||
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(testTable.commitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// Let's rollback commit1, Check results
|
||||
client.rollback(commitTime1);
|
||||
assertFalse(testTable.commitExists(commitTime1));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime1));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test auto-rollback of commits which are in flight.
|
||||
*/
|
||||
@Test
|
||||
public void testAutoRollbackInflightCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
final String commitTime1 = "20160501010101";
|
||||
final String commitTime2 = "20160502020601";
|
||||
final String commitTime3 = "20160506030611";
|
||||
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id11");
|
||||
put(p2, "id12");
|
||||
put(p3, "id13");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id21");
|
||||
put(p2, "id22");
|
||||
put(p3, "id23");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id31");
|
||||
put(p2, "id32");
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addInflightCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
// Turn auto rollback off
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
final String commitTime4 = "20160506030621";
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
|
||||
client.startCommitWithTime(commitTime4);
|
||||
// Check results, nothing changed
|
||||
assertTrue(testTable.commitExists(commitTime1));
|
||||
assertTrue(testTable.inflightCommitExists(commitTime2));
|
||||
assertTrue(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
}
|
||||
|
||||
// Turn auto rollback on
|
||||
final String commitTime5 = "20160506030631";
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, true)) {
|
||||
client.startCommitWithTime(commitTime5);
|
||||
assertTrue(testTable.commitExists(commitTime1));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,353 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.CompactionTestUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.action.compact.OperationResult;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.client.CompactionAdminClient.getRenamingActionsToAlignWithCompactionOperation;
|
||||
import static org.apache.hudi.client.CompactionAdminClient.renameLogFile;
|
||||
import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestCompactionAdminClient extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestCompactionAdminClient.class);
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private CompactionAdminClient client;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
metaClient = HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
|
||||
client = new CompactionAdminClient(context, basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionFileId() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
Map<String, CompactionOperation> instantsWithOp =
|
||||
Stream.of("001", "003", "005", "007").map(instant -> {
|
||||
try {
|
||||
return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
}).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream()
|
||||
.map(op -> Pair.of(instantWithPlan.getLeft(), CompactionOperation.convertFromAvroRecordInstance(op)))
|
||||
.findFirst().get()).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "000", "001", instantsWithOp.get("001"), 2);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "002", "003", instantsWithOp.get("003"), 2);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "004", "005", instantsWithOp.get("005"), 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "006", "007", instantsWithOp.get("007"), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRepairCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
private void validateRepair(String ingestionInstant, String compactionInstant, int numEntriesPerInstant,
|
||||
int expNumRepairs) throws Exception {
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRepairs, true);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
List<ValidationOpResult> result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
if (expNumRepairs > 0) {
|
||||
assertTrue(result.stream().anyMatch(r -> !r.isSuccess()), "Expect some failures in validation");
|
||||
}
|
||||
// Now repair
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> undoFiles =
|
||||
result.stream().flatMap(r -> getRenamingActionsToAlignWithCompactionOperation(metaClient,
|
||||
compactionInstant, r.getOperation(), Option.empty()).stream()).map(rn -> {
|
||||
try {
|
||||
renameLogFile(metaClient, rn.getKey(), rn.getValue());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return rn;
|
||||
}).collect(Collectors.toList());
|
||||
Map<String, String> renameFilesFromUndo = undoFiles.stream()
|
||||
.collect(Collectors.toMap(p -> p.getRight().getPath().toString(), x -> x.getLeft().getPath().toString()));
|
||||
Map<String, String> expRenameFiles = renameFiles.stream()
|
||||
.collect(Collectors.toMap(p -> p.getLeft().getPath().toString(), x -> x.getRight().getPath().toString()));
|
||||
if (expNumRepairs > 0) {
|
||||
assertFalse(renameFiles.isEmpty(), "Rename Files must be non-empty");
|
||||
} else {
|
||||
assertTrue(renameFiles.isEmpty(), "Rename Files must be empty");
|
||||
}
|
||||
expRenameFiles.forEach((key, value) -> LOG.info("Key :" + key + " renamed to " + value + " rolled back to "
|
||||
+ renameFilesFromUndo.get(key)));
|
||||
|
||||
assertEquals(expRenameFiles, renameFilesFromUndo, "Undo must completely rollback renames");
|
||||
// Now expect validation to succeed
|
||||
result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
assertTrue(result.stream().allMatch(OperationResult::isSuccess), "Expect no failures in validation");
|
||||
assertEquals(expNumRepairs, undoFiles.size(), "Expected Num Repairs");
|
||||
}
|
||||
|
||||
/**
|
||||
* Enssure compaction plan is valid.
|
||||
*
|
||||
* @param compactionInstant Compaction Instant
|
||||
*/
|
||||
private void ensureValidCompactionPlan(String compactionInstant) throws Exception {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
// Ensure compaction-plan is good to begin with
|
||||
List<ValidationOpResult> validationResults = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
assertFalse(validationResults.stream().anyMatch(v -> !v.isSuccess()),
|
||||
"Some validations failed");
|
||||
}
|
||||
|
||||
private void validateRenameFiles(List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles, String ingestionInstant,
|
||||
String compactionInstant, HoodieTableFileSystemView fsView) {
|
||||
// Ensure new names of log-files are on expected lines
|
||||
Set<HoodieLogFile> uniqNewLogFiles = new HashSet<>();
|
||||
Set<HoodieLogFile> uniqOldLogFiles = new HashSet<>();
|
||||
|
||||
renameFiles.forEach(lfPair -> {
|
||||
assertFalse(uniqOldLogFiles.contains(lfPair.getKey()), "Old Log File Names do not collide");
|
||||
assertFalse(uniqNewLogFiles.contains(lfPair.getValue()), "New Log File Names do not collide");
|
||||
uniqOldLogFiles.add(lfPair.getKey());
|
||||
uniqNewLogFiles.add(lfPair.getValue());
|
||||
});
|
||||
|
||||
renameFiles.forEach(lfPair -> {
|
||||
HoodieLogFile oldLogFile = lfPair.getLeft();
|
||||
HoodieLogFile newLogFile = lfPair.getValue();
|
||||
assertEquals(ingestionInstant, newLogFile.getBaseCommitTime(), "Base Commit time is expected");
|
||||
assertEquals(compactionInstant, oldLogFile.getBaseCommitTime(), "Base Commit time is expected");
|
||||
assertEquals(oldLogFile.getFileId(), newLogFile.getFileId(), "File Id is expected");
|
||||
HoodieLogFile lastLogFileBeforeCompaction =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], ingestionInstant)
|
||||
.filter(fs -> fs.getFileId().equals(oldLogFile.getFileId())).map(fs -> fs.getLogFiles().findFirst().get())
|
||||
.findFirst().get();
|
||||
assertEquals(lastLogFileBeforeCompaction.getLogVersion() + oldLogFile.getLogVersion(),
|
||||
newLogFile.getLogVersion(), "Log Version expected");
|
||||
assertTrue(newLogFile.getLogVersion() > lastLogFileBeforeCompaction.getLogVersion(),
|
||||
"Log version does not collide");
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames) throws Exception {
|
||||
return validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRenames,
|
||||
false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames,
|
||||
boolean skipUnSchedule) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1, Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).flatMap(FileSlice::getLogFiles)
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(expLogFilesToBeRenamed, gotLogFilesToBeRenamed,
|
||||
"Log files belonging to file-slices created because of compaction request must be renamed");
|
||||
|
||||
if (skipUnSchedule) {
|
||||
// Do the renaming only but do not touch the compaction plan - Needed for repair tests
|
||||
renameFiles.forEach(lfPair -> {
|
||||
try {
|
||||
renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
}
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
|
||||
client.unscheduleCompactionPlan(compactionInstant, false, 1, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.forEach(fs -> {
|
||||
assertFalse(fs.getBaseFile().isPresent(), "No Data file must be present");
|
||||
assertEquals(0, fs.getLogFiles().count(), "No Log Files");
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
assertEquals(fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming,
|
||||
"Each File Id has same number of log-files");
|
||||
assertEquals(numEntriesPerInstant, fileIdToCountsAfterRenaming.size(), "Not Empty");
|
||||
assertEquals(expNumRenames, renameFiles.size(), "Expected number of renames");
|
||||
return renameFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private void validateUnScheduleFileId(CompactionAdminClient client, String ingestionInstant, String compactionInstant,
|
||||
CompactionOperation op, int expNumRenames) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles = client
|
||||
.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).flatMap(FileSlice::getLogFiles)
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(expLogFilesToBeRenamed, gotLogFilesToBeRenamed,
|
||||
"Log files belonging to file-slices created because of compaction request must be renamed");
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
client.unscheduleCompactionFileId(op.getFileGroupId(), false, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.forEach(fs -> {
|
||||
assertFalse(fs.getBaseFile().isPresent(), "No Data file must be present");
|
||||
assertEquals(0, fs.getLogFiles().count(), "No Log Files");
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
assertEquals(fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming,
|
||||
"Each File Id has same number of log-files");
|
||||
assertEquals(1, fileIdToCountsAfterRenaming.size(), "Not Empty");
|
||||
assertEquals(expNumRenames, renameFiles.size(), "Expected number of renames");
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieInternalWriteStatus}.
|
||||
*/
|
||||
public class TestHoodieInternalWriteStatus {
|
||||
|
||||
@Test
|
||||
public void testFailureFraction() {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String partitionPath = UUID.randomUUID().toString();
|
||||
status.setFileId(fileId);
|
||||
status.setPartitionPath(partitionPath);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markFailure(UUID.randomUUID().toString(), t);
|
||||
}
|
||||
// verification
|
||||
assertEquals(fileId, status.getFileId());
|
||||
assertEquals(partitionPath, status.getPartitionPath());
|
||||
assertTrue(status.getFailedRecordKeys().size() > 0);
|
||||
assertTrue(status.getFailedRecordKeys().size() < 150); // 150 instead of 100, to prevent flaky test
|
||||
assertTrue(status.hasErrors());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessRecordTracking() {
|
||||
boolean[] vals = {true, false};
|
||||
for (boolean trackSuccess : vals) {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(trackSuccess, 1.0);
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
status.setFileId(fileId);
|
||||
String partitionPath = UUID.randomUUID().toString();
|
||||
status.setPartitionPath(partitionPath);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markSuccess(UUID.randomUUID().toString());
|
||||
status.markFailure(UUID.randomUUID().toString(), t);
|
||||
}
|
||||
// verification
|
||||
assertEquals(fileId, status.getFileId());
|
||||
assertEquals(partitionPath, status.getPartitionPath());
|
||||
assertEquals(1000, status.getFailedRecordKeys().size());
|
||||
assertTrue(status.hasErrors());
|
||||
if (trackSuccess) {
|
||||
assertEquals(1000, status.getSuccessRecordKeys().size());
|
||||
} else {
|
||||
assertTrue(status.getSuccessRecordKeys().isEmpty());
|
||||
}
|
||||
assertEquals(2000, status.getTotalRecords());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobalError() {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
status.setGlobalError(t);
|
||||
assertEquals(t, status.getGlobalError());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,235 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.AnalysisException;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
/**
|
||||
* Test-cases for covering HoodieReadClient APIs
|
||||
*/
|
||||
public class TestHoodieReadClient extends HoodieClientTestBase {
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insert.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsert() throws Exception {
|
||||
testReadFilterExist(getConfig(), SparkRDDWriteClient::insert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insertPrepped.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfig(), SparkRDDWriteClient::insertPreppedRecords);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsert.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsert() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsertPrepped.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(),
|
||||
(writeClient, recordRDD, instantTime) -> {
|
||||
return writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty());
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadROViewFailsWithoutSqlContext() {
|
||||
HoodieReadClient readClient = new HoodieReadClient(context, getConfig());
|
||||
JavaRDD<HoodieKey> recordsRDD = jsc.parallelize(new ArrayList<>(), 1);
|
||||
assertThrows(IllegalStateException.class, () -> {
|
||||
readClient.readROView(recordsRDD, 1);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to write new records using one of HoodieWriteClient's write API and use ReadClient to test filterExists()
|
||||
* API works correctly.
|
||||
*
|
||||
* @param config Hoodie Write Config
|
||||
* @param writeFn Write Function for writing records
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testReadFilterExist(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
|
||||
// Should not find any files
|
||||
assertEquals(100, filteredRDD.collect().size());
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieReadClient anotherReadClient = getHoodieReadClient(config.getBasePath());
|
||||
filteredRDD = anotherReadClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
assertEquals(25, result.size());
|
||||
|
||||
// check path exists for written keys
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToPathPair =
|
||||
anotherReadClient.checkExists(recordsRDD.map(HoodieRecord::getKey));
|
||||
JavaRDD<HoodieKey> keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
assertEquals(75, keysWithPaths.count());
|
||||
|
||||
// verify rows match inserted records
|
||||
Dataset<Row> rows = anotherReadClient.readROView(keysWithPaths, 1);
|
||||
assertEquals(75, rows.count());
|
||||
|
||||
JavaRDD<HoodieKey> keysWithoutPaths = keyToPathPair.filter(keyPath -> !keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
|
||||
assertThrows(AnalysisException.class, () -> {
|
||||
anotherReadClient.readROView(keysWithoutPaths, 1);
|
||||
});
|
||||
|
||||
// Actual tests of getPendingCompactions method are in TestAsyncCompaction
|
||||
// This is just testing empty list
|
||||
assertEquals(0, anotherReadClient.getPendingCompactions().size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insert().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsert() throws Exception {
|
||||
testTagLocation(getConfig(), SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insertPrepped().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsertPrepped() throws Exception {
|
||||
testTagLocation(getConfig(), SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords,
|
||||
true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulk-insert().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsert() throws Exception {
|
||||
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert,
|
||||
SparkRDDWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulkInsertPrepped().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsertPrepped() throws Exception {
|
||||
testTagLocation(
|
||||
getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, instantTime) -> writeClient
|
||||
.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
|
||||
SparkRDDWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to test tagLocation after using different HoodieWriteClient write APIS.
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param insertFn Hoodie Write Client first Insert API
|
||||
* @param updateFn Hoodie Write Client upsert API
|
||||
* @param isPrepped isPrepped flag.
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testTagLocation(HoodieWriteConfig hoodieWriteConfig,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> updateFn, boolean isPrepped)
|
||||
throws Exception {
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) {
|
||||
// Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result = insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime,
|
||||
numRecords, insertFn, isPrepped, true, numRecords);
|
||||
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
|
||||
// since they have been modified in the DAG
|
||||
JavaRDD<HoodieRecord> recordRDD =
|
||||
jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList()));
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, updateFn, isPrepped, true,
|
||||
numRecords, 200, 2);
|
||||
recordRDD =
|
||||
jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList()));
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,122 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestMultiFS extends HoodieClientTestHarness {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestMultiFS.class);
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
protected String tableName = "hoodie_rt";
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initDFS();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable(tableName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
// Initialize table and filesystem
|
||||
HoodieTableMetaClient.initTableType(hadoopConf, dfsBasePath, HoodieTableType.valueOf(tableType),
|
||||
tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
// Create write client to write some records in
|
||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||
|
||||
try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||
SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
LOG.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals(readRecords.count(), records.size(), "Should contain 100 records");
|
||||
|
||||
// Write to local
|
||||
HoodieTableMetaClient.initTableType(hadoopConf, tablePath, HoodieTableType.valueOf(tableType),
|
||||
tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
LOG.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
LOG.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
|
||||
LOG.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords =
|
||||
HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals(localReadRecords.count(), localRecords.size(), "Should contain 100 records");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,480 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_1;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.EXTRA_TYPE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.FARE_NESTED_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.MAP_TYPE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TIP_NESTED_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_PREFIX;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_SUFFIX;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestTableSchemaEvolution extends HoodieClientTestBase {
|
||||
|
||||
private final String initCommitTime = "000";
|
||||
private HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE;
|
||||
private HoodieTestDataGenerator dataGenEvolved = new HoodieTestDataGenerator();
|
||||
private HoodieTestDataGenerator dataGenDevolved = new HoodieTestDataGenerator();
|
||||
|
||||
public static final String EXTRA_FIELD_SCHEMA =
|
||||
"{\"name\": \"new_field\", \"type\": \"boolean\", \"default\": false},";
|
||||
|
||||
// TRIP_EXAMPLE_SCHEMA with a new_field added
|
||||
public static final String TRIP_EXAMPLE_SCHEMA_EVOLVED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
|
||||
+ FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
|
||||
// TRIP_EXAMPLE_SCHEMA with tip field removed
|
||||
public static final String TRIP_EXAMPLE_SCHEMA_DEVOLVED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
|
||||
+ FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
|
||||
@Test
|
||||
public void testSchemaCompatibilityBasic() throws Exception {
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA),
|
||||
"Same schema is compatible");
|
||||
|
||||
String reorderedSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + TIP_NESTED_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ MAP_TYPE_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema),
|
||||
"Reordered fields are compatible");
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA),
|
||||
"Reordered fields are compatible");
|
||||
|
||||
String renamedSchema = TRIP_EXAMPLE_SCHEMA.replace("tip_history", "tip_future");
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema),
|
||||
"Renamed fields are not compatible");
|
||||
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_DEVOLVED),
|
||||
"Deleted single field is not compatible");
|
||||
String deletedMultipleFieldSchema = TRIP_SCHEMA_PREFIX + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, deletedMultipleFieldSchema),
|
||||
"Deleted multiple fields are not compatible");
|
||||
|
||||
String renamedRecordSchema = TRIP_EXAMPLE_SCHEMA.replace("triprec", "triprec_renamed");
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema),
|
||||
"Renamed record name is not compatible");
|
||||
|
||||
String swappedFieldSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA.replace("city_to_state", "fare")
|
||||
+ FARE_NESTED_SCHEMA.replace("fare", "city_to_state") + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema),
|
||||
"Swapped fields are not compatible");
|
||||
|
||||
String typeChangeSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ TIP_NESTED_SCHEMA.replace("string", "boolean") + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchema),
|
||||
"Field type change is not compatible");
|
||||
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED),
|
||||
"Added field with default is compatible (Evolved Schema)");
|
||||
|
||||
String multipleAddedFieldSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + EXTRA_FIELD_SCHEMA.replace("new_field", "new_new_field")
|
||||
+ TRIP_SCHEMA_SUFFIX;
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema),
|
||||
"Multiple added fields with defauls are compatible");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMORTable() throws Exception {
|
||||
tableType = HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
// Create the table
|
||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
|
||||
HoodieTableType.MERGE_ON_READ, metaClient.getTableConfig().getTableName(),
|
||||
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Initial inserts with TRIP_EXAMPLE_SCHEMA
|
||||
int numRecords = 10;
|
||||
insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
|
||||
numRecords, SparkRDDWriteClient::insert, false, false, numRecords);
|
||||
checkLatestDeltaCommit("001");
|
||||
|
||||
// Compact once so we can incrementally read later
|
||||
assertTrue(client.scheduleCompactionAtInstant("002", Option.empty()));
|
||||
client.compact("002");
|
||||
|
||||
// Updates with same schema is allowed
|
||||
final int numUpdateRecords = 5;
|
||||
updateBatch(hoodieWriteConfig, client, "003", "002", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
checkLatestDeltaCommit("003");
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Delete with same schema is allowed
|
||||
final int numDeleteRecords = 2;
|
||||
numRecords -= numDeleteRecords;
|
||||
deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords,
|
||||
SparkRDDWriteClient::delete, false, false, 0, 0);
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with evolved schema is not allowed
|
||||
HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false);
|
||||
final List<HoodieRecord> failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
try {
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a devolved schema and insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
|
||||
writeBatch(client, "005", "004", Option.empty(), "003", numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
|
||||
fail("Insert with devolved scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
client.rollback("005");
|
||||
}
|
||||
|
||||
// Update with devolved schema is also not allowed
|
||||
try {
|
||||
updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
fail("Update with devolved scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
client.rollback("005");
|
||||
}
|
||||
|
||||
// Insert with an evolved scheme is allowed
|
||||
HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false);
|
||||
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a evolved schemaand insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
|
||||
final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "005", "004", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
|
||||
|
||||
// new commit
|
||||
checkLatestDeltaCommit("005");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Updates with evolved schema is allowed
|
||||
final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("006", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "006", "005", Option.empty(), initCommitTime,
|
||||
numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0);
|
||||
// new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Now even the original schema cannot be used for updates as it is devolved in relation to the
|
||||
// current schema of the dataset.
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
try {
|
||||
updateBatch(hoodieWriteConfig, client, "007", "006", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
fail("Update with original scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
client.rollback("007");
|
||||
}
|
||||
|
||||
// Now even the original schema cannot be used for inserts as it is devolved in relation to the
|
||||
// current schema of the dataset.
|
||||
try {
|
||||
// We are not using insertBatch directly here because insertion of these
|
||||
// records will fail and we dont want to keep these records within HoodieTestDataGenerator as we
|
||||
// will be testing updates later.
|
||||
failedRecords.clear();
|
||||
failedRecords.addAll(dataGen.generateInserts("007", numRecords));
|
||||
writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with original scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
client.rollback("007");
|
||||
|
||||
// Remove the inserts from the in-memory state of HoodieTestDataGenerator
|
||||
// as these records were never inserted in the dataset. This is required so
|
||||
// that future calls to updateBatch or deleteBatch do not generate updates
|
||||
// or deletes for records which do not even exist.
|
||||
for (HoodieRecord record : failedRecords) {
|
||||
assertTrue(dataGen.deleteExistingKeyIfPresent(record.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
// Rollback to the original schema
|
||||
client.restoreToInstant("004");
|
||||
checkLatestDeltaCommit("004");
|
||||
|
||||
// Updates with original schema are now allowed
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
updateBatch(hoodieWriteConfig, client, "008", "004", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
// new commit
|
||||
checkLatestDeltaCommit("008");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Insert with original schema is allowed now
|
||||
insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert,
|
||||
false, false, 0, 0, 0);
|
||||
checkLatestDeltaCommit("009");
|
||||
checkReadRecords("000", 3 * numRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyOnWriteTable() throws Exception {
|
||||
// Create the table
|
||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
|
||||
HoodieTableType.COPY_ON_WRITE, metaClient.getTableConfig().getTableName(),
|
||||
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Initial inserts with TRIP_EXAMPLE_SCHEMA
|
||||
int numRecords = 10;
|
||||
insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
|
||||
numRecords, SparkRDDWriteClient::insert, false, true, numRecords);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Updates with same schema is allowed
|
||||
final int numUpdateRecords = 5;
|
||||
updateBatch(hoodieWriteConfig, client, "002", "001", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, numRecords, 2);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Delete with same schema is allowed
|
||||
final int numDeleteRecords = 2;
|
||||
numRecords -= numDeleteRecords;
|
||||
deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords,
|
||||
SparkRDDWriteClient::delete, false, true, 0, numRecords);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with devolved schema is not allowed
|
||||
HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false);
|
||||
final List<HoodieRecord> failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
try {
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a devolved schema.
|
||||
writeBatch(client, "004", "003", Option.empty(), "003", numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with devolved scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
client.rollback("004");
|
||||
}
|
||||
|
||||
// Update with devolved schema is not allowed
|
||||
try {
|
||||
updateBatch(hoodieDevolvedWriteConfig, client, "004", "003", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, 2 * numRecords, 5);
|
||||
fail("Update with devolved scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
client.rollback("004");
|
||||
}
|
||||
|
||||
// Insert with evolved scheme is allowed
|
||||
HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false);
|
||||
final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a evolved schema.
|
||||
writeBatch(client, "004", "003", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4);
|
||||
// new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("004"));
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Updates with evolved schema is allowed
|
||||
final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("005", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "005", "004", Option.empty(), initCommitTime,
|
||||
numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Now even the original schema cannot be used for updates as it is devolved
|
||||
// in relation to the current schema of the dataset.
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
try {
|
||||
updateBatch(hoodieWriteConfig, client, "006", "005", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, numRecords, 2);
|
||||
fail("Update with original scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("005"));
|
||||
client.rollback("006");
|
||||
}
|
||||
|
||||
// Now even the original schema cannot be used for inserts as it is devolved
|
||||
// in relation to the current schema of the dataset.
|
||||
try {
|
||||
// We are not using insertBatch directly here because insertion of these
|
||||
// records will fail and we dont want to keep these records within
|
||||
// HoodieTestDataGenerator.
|
||||
failedRecords.clear();
|
||||
failedRecords.addAll(dataGen.generateInserts("006", numRecords));
|
||||
writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with original scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("005"));
|
||||
client.rollback("006");
|
||||
|
||||
// Remove the inserts from the in-memory state of HoodieTestDataGenerator
|
||||
// as these records were never inserted in the dataset. This is required so
|
||||
// that future calls to updateBatch or deleteBatch do not generate updates
|
||||
// or deletes for records which do not even exist.
|
||||
for (HoodieRecord record : failedRecords) {
|
||||
assertTrue(dataGen.deleteExistingKeyIfPresent(record.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
// Revert to the older commit and ensure that the original schema can now
|
||||
// be used for inserts and inserts.
|
||||
client.restoreToInstant("003");
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with original schema is allowed now
|
||||
insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert,
|
||||
false, true, numRecords, 2 * numRecords, 1);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Update with original schema is allowed now
|
||||
updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, 2 * numRecords, 5);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
}
|
||||
|
||||
private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException {
|
||||
if (tableType == HoodieTableType.COPY_ON_WRITE) {
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline();
|
||||
assertEquals(numExpectedRecords, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, instantTime));
|
||||
} else {
|
||||
// TODO: This code fails to read records under the following conditions:
|
||||
// 1. No parquet files yet (i.e. no compaction done yet)
|
||||
// 2. Log file but no base file with the same FileID
|
||||
/*
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataAndLogFilesInPath(metaClient.getFs(), basePath);
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitsTimeline();
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline, allFiles);
|
||||
List<String> dataFiles = fsView.getLatestBaseFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
String absTableName = "hoodie." + metaClient.getTableConfig().getTableName();
|
||||
conf.set(absTableName + ".consume.mode", "INCREMENTAL");
|
||||
conf.set(absTableName + ".consume.start.timestamp", instantTime);
|
||||
conf.set(absTableName + ".consume.max.commits", "-1");
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath, conf);
|
||||
assertEquals(recordsRead.size(), numExpectedRecords);
|
||||
*/
|
||||
}
|
||||
}
|
||||
|
||||
private void checkLatestDeltaCommit(String instantTime) {
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertTrue(timeline.lastInstant().get().getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION));
|
||||
assertTrue(timeline.lastInstant().get().getTimestamp().equals(instantTime));
|
||||
}
|
||||
|
||||
private List<HoodieRecord> generateInsertsWithSchema(String commitTime, int numRecords, String schemaStr) {
|
||||
HoodieTestDataGenerator gen = schemaStr.equals(TRIP_EXAMPLE_SCHEMA_EVOLVED) ? dataGenEvolved : dataGenDevolved;
|
||||
List<HoodieRecord> records = gen.generateInserts(commitTime, numRecords);
|
||||
return convertToSchema(records, schemaStr);
|
||||
}
|
||||
|
||||
private List<HoodieRecord> generateUpdatesWithSchema(String commitTime, int numRecords, String schemaStr) {
|
||||
HoodieTestDataGenerator gen = schemaStr.equals(TRIP_EXAMPLE_SCHEMA_EVOLVED) ? dataGenEvolved : dataGenDevolved;
|
||||
List<HoodieRecord> records = gen.generateUniqueUpdates(commitTime, numRecords);
|
||||
return convertToSchema(records, schemaStr);
|
||||
}
|
||||
|
||||
private List<HoodieRecord> convertToSchema(List<HoodieRecord> records, String schemaStr) {
|
||||
Schema newSchema = new Schema.Parser().parse(schemaStr);
|
||||
return records.stream().map(r -> {
|
||||
HoodieKey key = r.getKey();
|
||||
GenericRecord payload;
|
||||
try {
|
||||
payload = (GenericRecord)r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get();
|
||||
GenericRecord newPayload = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(payload, newSchema);
|
||||
return new HoodieRecord(key, new RawTripTestPayload(newPayload.toString(), key.getRecordKey(), key.getPartitionPath(), schemaStr));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Conversion to new schema failed");
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getWriteConfig(String schema) {
|
||||
return getConfigBuilder(schema)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withAvroSchemaValidate(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return tableType;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,148 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
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.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||
initSparkContexts("TestUpdateSchemaEvolution");
|
||||
initFileSystem();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
final HoodieSparkTable table = HoodieSparkTable.create(config, context);
|
||||
|
||||
final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
List<HoodieRecord> insertRecords = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
Map<String, HoodieRecord> insertRecordMap = insertRecords.stream()
|
||||
.collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity()));
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecordMap, supplier);
|
||||
createHandle.write();
|
||||
return createHandle.close();
|
||||
}).collect();
|
||||
|
||||
final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
|
||||
FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile);
|
||||
|
||||
// Now try an update with an evolved schema
|
||||
// Evolved schema does not have guarantee on preserving the original field ordering
|
||||
final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
|
||||
final WriteStatus insertResult = statuses.get(0);
|
||||
String fileId = insertResult.getFileId();
|
||||
|
||||
final HoodieSparkTable table2 = HoodieSparkTable.create(config, context);
|
||||
assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
// New content with values for the newly added field
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
|
||||
List<HoodieRecord> updateRecords = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
record1.unseal();
|
||||
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
|
||||
record1.seal();
|
||||
updateRecords.add(record1);
|
||||
|
||||
assertDoesNotThrow(() -> {
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
||||
updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
|
||||
Configuration conf = new Configuration();
|
||||
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchemaWithMetafields());
|
||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
||||
new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
|
||||
for (GenericRecord rec : oldRecords) {
|
||||
mergeHandle.write(rec);
|
||||
}
|
||||
mergeHandle.close();
|
||||
}, "UpdateFunction could not read records written with exampleSchema.txt using the "
|
||||
+ "exampleEvolvedSchema.txt");
|
||||
|
||||
return 1;
|
||||
}).collect().size());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String name) {
|
||||
Schema schema = getSchemaFromResource(getClass(), name);
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class TestWriteStatus {
|
||||
@Test
|
||||
public void testFailureFraction() {
|
||||
WriteStatus status = new WriteStatus(true, 0.1);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markFailure(mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertTrue(status.getFailedRecords().size() > 0);
|
||||
assertTrue(status.getFailedRecords().size() < 150); // 150 instead of 100, to prevent flaky test
|
||||
assertTrue(status.hasErrors());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessRecordTracking() {
|
||||
WriteStatus status = new WriteStatus(false, 1.0);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markSuccess(mock(HoodieRecord.class), null);
|
||||
status.markFailure(mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertEquals(1000, status.getFailedRecords().size());
|
||||
assertTrue(status.hasErrors());
|
||||
assertTrue(status.getWrittenRecords().isEmpty());
|
||||
assertEquals(2000, status.getTotalRecords());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.selector.BootstrapRegexModeSelector;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestBootstrapRegexModeSelector {
|
||||
|
||||
private HoodieWriteConfig getConfig(String regex, BootstrapMode selectedMode) {
|
||||
return HoodieWriteConfig.newBuilder().withPath("")
|
||||
.withBootstrapConfig(HoodieBootstrapConfig.newBuilder()
|
||||
.withBootstrapModeSelectorRegex(regex)
|
||||
.withBootstrapModeForRegexMatch(selectedMode).build())
|
||||
.forTable("test-trip-table").build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testModeSelector() {
|
||||
List<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||
String regex = "2020/05/1[0-9]";
|
||||
BootstrapRegexModeSelector regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex,
|
||||
BootstrapMode.FULL_RECORD));
|
||||
|
||||
Map<BootstrapMode, List<String>> result = regexModeSelector.select(input);
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/01"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/02"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/10"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/11"));
|
||||
assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size());
|
||||
assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size());
|
||||
|
||||
regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex,
|
||||
BootstrapMode.METADATA_ONLY));
|
||||
result = regexModeSelector.select(input);
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/01"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/02"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/10"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/11"));
|
||||
assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size());
|
||||
assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.UniformBootstrapModeSelector;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestUniformBootstrapModeSelector {
|
||||
|
||||
@Test
|
||||
public void testFullBootstrapModeSelector() {
|
||||
|
||||
FullRecordBootstrapModeSelector modeSelector = new FullRecordBootstrapModeSelector(
|
||||
HoodieWriteConfig.newBuilder().withPath("").build());
|
||||
testModeSelector(modeSelector, BootstrapMode.FULL_RECORD);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataOnlyBootstrapModeSelector() {
|
||||
MetadataOnlyBootstrapModeSelector modeSelector = new MetadataOnlyBootstrapModeSelector(
|
||||
HoodieWriteConfig.newBuilder().withPath("").build());
|
||||
testModeSelector(modeSelector, BootstrapMode.METADATA_ONLY);
|
||||
}
|
||||
|
||||
private void testModeSelector(UniformBootstrapModeSelector modeSelector, BootstrapMode mode) {
|
||||
List<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||
Map<BootstrapMode, List<String>> result = modeSelector.select(input);
|
||||
assertTrue(result.get(mode).contains("2020/05/01"));
|
||||
assertTrue(result.get(mode).contains("2020/05/02"));
|
||||
assertTrue(result.get(mode).contains("2020/05/10"));
|
||||
assertTrue(result.get(mode).contains("2020/05/11"));
|
||||
assertEquals(4, result.get(mode).size());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,239 @@
|
||||
/*
|
||||
* 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.client.model;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieInternalRow}.
|
||||
*/
|
||||
public class TestHoodieInternalRow {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
private static final int INTEGER_INDEX = 5;
|
||||
private static final int STRING_INDEX = 6;
|
||||
private static final int BOOLEAN_INDEX = 7;
|
||||
private static final int SHORT_INDEX = 8;
|
||||
private static final int BYTE_INDEX = 9;
|
||||
private static final int LONG_INDEX = 10;
|
||||
private static final int FLOAT_INDEX = 11;
|
||||
private static final int DOUBLE_INDEX = 12;
|
||||
private static final int DECIMAL_INDEX = 13;
|
||||
private static final int BINARY_INDEX = 14;
|
||||
private static final int STRUCT_INDEX = 15;
|
||||
// to do array and map
|
||||
private static final int ARRAY_INDEX = 16;
|
||||
private static final int MAP_INDEX = 17;
|
||||
|
||||
private List<Integer> nullIndices;
|
||||
|
||||
public TestHoodieInternalRow() {
|
||||
this.nullIndices = new ArrayList<>();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGet() {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdate() {
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
hoodieInternalRow.update(0, "commitTime_updated");
|
||||
hoodieInternalRow.update(1, "commitSeqNo_updated");
|
||||
hoodieInternalRow.update(2, "recordKey_updated");
|
||||
hoodieInternalRow.update(3, "partitionPath_updated");
|
||||
hoodieInternalRow.update(4, "fileName_updated");
|
||||
|
||||
values = getRandomValue(true);
|
||||
hoodieInternalRow.update(INTEGER_INDEX, values[INTEGER_INDEX]);
|
||||
hoodieInternalRow.update(BOOLEAN_INDEX, values[BOOLEAN_INDEX]);
|
||||
hoodieInternalRow.update(SHORT_INDEX, values[SHORT_INDEX]);
|
||||
hoodieInternalRow.update(BYTE_INDEX, values[BYTE_INDEX]);
|
||||
hoodieInternalRow.update(LONG_INDEX, values[LONG_INDEX]);
|
||||
hoodieInternalRow.update(FLOAT_INDEX, values[FLOAT_INDEX]);
|
||||
hoodieInternalRow.update(DOUBLE_INDEX, values[DOUBLE_INDEX]);
|
||||
//hoodieInternalRow.update(decimalIndex, values[decimalIndex]);
|
||||
hoodieInternalRow.update(BINARY_INDEX, values[BINARY_INDEX]);
|
||||
hoodieInternalRow.update(STRUCT_INDEX, values[STRUCT_INDEX]);
|
||||
hoodieInternalRow.update(STRING_INDEX, values[STRING_INDEX].toString());
|
||||
|
||||
assertValues(hoodieInternalRow, "commitTime_updated", "commitSeqNo_updated", "recordKey_updated", "partitionPath_updated",
|
||||
"fileName_updated", values, nullIndices);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsNullCheck() {
|
||||
|
||||
for (int i = 0; i < 16; i++) {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
hoodieInternalRow.setNullAt(i);
|
||||
nullIndices.clear();
|
||||
nullIndices.add(i);
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
|
||||
// try setting multiple values as null
|
||||
// run it for 5 rounds
|
||||
for (int i = 0; i < 5; i++) {
|
||||
int numNullValues = 1 + RANDOM.nextInt(4);
|
||||
List<Integer> nullsSoFar = new ArrayList<>();
|
||||
while (nullsSoFar.size() < numNullValues) {
|
||||
int randomIndex = RANDOM.nextInt(16);
|
||||
if (!nullsSoFar.contains(randomIndex)) {
|
||||
nullsSoFar.add(randomIndex);
|
||||
}
|
||||
}
|
||||
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
nullIndices.clear();
|
||||
|
||||
for (Integer index : nullsSoFar) {
|
||||
hoodieInternalRow.setNullAt(index);
|
||||
nullIndices.add(index);
|
||||
}
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches a random Object[] of values for testing.
|
||||
*
|
||||
* @param withStructType true if structType need to be added as one of the elements in the Object[]
|
||||
* @return the random Object[] thus generated
|
||||
*/
|
||||
private Object[] getRandomValue(boolean withStructType) {
|
||||
Object[] values = new Object[16];
|
||||
values[INTEGER_INDEX] = RANDOM.nextInt();
|
||||
values[STRING_INDEX] = UUID.randomUUID().toString();
|
||||
values[BOOLEAN_INDEX] = RANDOM.nextBoolean();
|
||||
values[SHORT_INDEX] = (short) RANDOM.nextInt(2);
|
||||
byte[] bytes = new byte[1];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BYTE_INDEX] = bytes[0];
|
||||
values[LONG_INDEX] = RANDOM.nextLong();
|
||||
values[FLOAT_INDEX] = RANDOM.nextFloat();
|
||||
values[DOUBLE_INDEX] = RANDOM.nextDouble();
|
||||
// TODO fix decimal type.
|
||||
values[DECIMAL_INDEX] = RANDOM.nextFloat();
|
||||
bytes = new byte[20];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BINARY_INDEX] = bytes;
|
||||
if (withStructType) {
|
||||
Object[] structField = getRandomValue(false);
|
||||
values[STRUCT_INDEX] = new GenericInternalRow(structField);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
private void assertValues(HoodieInternalRow hoodieInternalRow, String commitTime, String commitSeqNo, String recordKey, String partitionPath, String filename, Object[] values,
|
||||
List<Integer> nullIndexes) {
|
||||
for (Integer index : nullIndexes) {
|
||||
assertTrue(hoodieInternalRow.isNullAt(index));
|
||||
}
|
||||
for (int i = 0; i < 16; i++) {
|
||||
if (!nullIndexes.contains(i)) {
|
||||
assertFalse(hoodieInternalRow.isNullAt(i));
|
||||
}
|
||||
}
|
||||
if (!nullIndexes.contains(0)) {
|
||||
assertEquals(commitTime, hoodieInternalRow.get(0, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(1)) {
|
||||
assertEquals(commitSeqNo, hoodieInternalRow.get(1, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(2)) {
|
||||
assertEquals(recordKey, hoodieInternalRow.get(2, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(3)) {
|
||||
assertEquals(partitionPath, hoodieInternalRow.get(3, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(4)) {
|
||||
assertEquals(filename, hoodieInternalRow.get(4, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(INTEGER_INDEX)) {
|
||||
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.getInt(INTEGER_INDEX));
|
||||
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.get(INTEGER_INDEX, DataTypes.IntegerType));
|
||||
}
|
||||
if (!nullIndexes.contains(STRING_INDEX)) {
|
||||
assertEquals(values[STRING_INDEX].toString(), hoodieInternalRow.get(STRING_INDEX, DataTypes.StringType));
|
||||
}
|
||||
if (!nullIndexes.contains(BOOLEAN_INDEX)) {
|
||||
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.getBoolean(BOOLEAN_INDEX));
|
||||
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.get(BOOLEAN_INDEX, DataTypes.BooleanType));
|
||||
}
|
||||
if (!nullIndexes.contains(SHORT_INDEX)) {
|
||||
assertEquals(values[SHORT_INDEX], hoodieInternalRow.getShort(SHORT_INDEX));
|
||||
assertEquals(values[SHORT_INDEX], hoodieInternalRow.get(SHORT_INDEX, DataTypes.ShortType));
|
||||
}
|
||||
if (!nullIndexes.contains(BYTE_INDEX)) {
|
||||
assertEquals(values[BYTE_INDEX], hoodieInternalRow.getByte(BYTE_INDEX));
|
||||
assertEquals(values[BYTE_INDEX], hoodieInternalRow.get(BYTE_INDEX, DataTypes.ByteType));
|
||||
}
|
||||
if (!nullIndexes.contains(LONG_INDEX)) {
|
||||
assertEquals(values[LONG_INDEX], hoodieInternalRow.getLong(LONG_INDEX));
|
||||
assertEquals(values[LONG_INDEX], hoodieInternalRow.get(LONG_INDEX, DataTypes.LongType));
|
||||
}
|
||||
if (!nullIndexes.contains(FLOAT_INDEX)) {
|
||||
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.getFloat(FLOAT_INDEX));
|
||||
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.get(FLOAT_INDEX, DataTypes.FloatType));
|
||||
}
|
||||
if (!nullIndexes.contains(DOUBLE_INDEX)) {
|
||||
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.getDouble(DOUBLE_INDEX));
|
||||
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.get(DOUBLE_INDEX, DataTypes.DoubleType));
|
||||
}
|
||||
if (!nullIndexes.contains(BINARY_INDEX)) {
|
||||
assertEquals(values[BINARY_INDEX], hoodieInternalRow.getBinary(BINARY_INDEX));
|
||||
assertEquals(values[BINARY_INDEX], hoodieInternalRow.get(BINARY_INDEX, DataTypes.BinaryType));
|
||||
}
|
||||
if (!nullIndexes.contains(STRUCT_INDEX)) {
|
||||
assertEquals(values[STRUCT_INDEX], hoodieInternalRow.getStruct(STRUCT_INDEX, 18));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,332 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SizeEstimator;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
initExecutorServiceWithFixedThreadPool(2);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
// Test to ensure that we are reading all records from queue iterator in the same order
|
||||
// without any exceptions.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testRecordReading() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
// Produce
|
||||
Future<Boolean> resFuture = executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
|
||||
queue.close();
|
||||
return true;
|
||||
});
|
||||
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
|
||||
int recordsRead = 0;
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieRecord originalRecord = originalRecordIterator.next();
|
||||
final Option<IndexedRecord> originalInsertValue =
|
||||
originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
|
||||
// Ensure that record ordering is guaranteed.
|
||||
assertEquals(originalRecord, payload.record);
|
||||
// cached insert value matches the expected insert value.
|
||||
assertEquals(originalInsertValue,
|
||||
payload.record.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
recordsRead++;
|
||||
}
|
||||
assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
|
||||
// all the records should be read successfully.
|
||||
assertEquals(numRecords, recordsRead);
|
||||
// should not throw any exceptions.
|
||||
resFuture.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure that we are reading all records from queue iterator when we have multiple producers.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testCompositeProducerRecordReading() throws Exception {
|
||||
final int numRecords = 1000;
|
||||
final int numProducers = 40;
|
||||
final List<List<HoodieRecord>> recs = new ArrayList<>();
|
||||
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Record Key to <Producer Index, Rec Index within a producer>
|
||||
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
List<HoodieRecord> pRecs = dataGen.generateInserts(instantTime, numRecords);
|
||||
int j = 0;
|
||||
for (HoodieRecord r : pRecs) {
|
||||
assertFalse(keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
|
||||
keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j));
|
||||
j++;
|
||||
}
|
||||
recs.add(pRecs);
|
||||
}
|
||||
|
||||
List<BoundedInMemoryQueueProducer<HoodieRecord>> producers = new ArrayList<>();
|
||||
for (int i = 0; i < recs.size(); i++) {
|
||||
final List<HoodieRecord> r = recs.get(i);
|
||||
// Alternate between pull and push based iterators
|
||||
if (i % 2 == 0) {
|
||||
producers.add(new IteratorBasedQueueProducer<>(r.iterator()));
|
||||
} else {
|
||||
producers.add(new FunctionBasedQueueProducer<>((buf) -> {
|
||||
Iterator<HoodieRecord> itr = r.iterator();
|
||||
while (itr.hasNext()) {
|
||||
try {
|
||||
buf.insertRecord(itr.next());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
||||
final List<Future<Boolean>> futureList = producers.stream().map(producer -> {
|
||||
return executorService.submit(() -> {
|
||||
producer.produce(queue);
|
||||
return true;
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
// Close queue
|
||||
Future<Boolean> closeFuture = executorService.submit(() -> {
|
||||
try {
|
||||
for (Future f : futureList) {
|
||||
f.get();
|
||||
}
|
||||
queue.close();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return true;
|
||||
});
|
||||
|
||||
// Used to ensure that consumer sees the records generated by a single producer in FIFO order
|
||||
Map<Integer, Integer> lastSeenMap =
|
||||
IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> -1));
|
||||
Map<Integer, Integer> countMap =
|
||||
IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0));
|
||||
|
||||
// Read recs and ensure we have covered all producer recs.
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
|
||||
final HoodieRecord rec = payload.record;
|
||||
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
|
||||
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
|
||||
countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
|
||||
lastSeenMap.put(producerPos._1(), lastSeenPos + 1);
|
||||
// Ensure we are seeing the next record generated
|
||||
assertEquals(lastSeenPos + 1, producerPos._2().intValue());
|
||||
}
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
// Ensure we have seen all the records for each producers
|
||||
assertEquals(Integer.valueOf(numRecords), countMap.get(i));
|
||||
}
|
||||
|
||||
// Ensure Close future is done
|
||||
closeFuture.get();
|
||||
}
|
||||
|
||||
// Test to ensure that record queueing is throttled when we hit memory limit.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testMemoryLimitForBuffering() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
// maximum number of records to keep in memory.
|
||||
final int recordLimit = 5;
|
||||
final SizeEstimator<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
|
||||
final long objSize = sizeEstimator.sizeEstimate(payload);
|
||||
final long memoryLimitInBytes = recordLimit * objSize;
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
|
||||
return true;
|
||||
});
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
assertEquals(0, queue.rateLimiter.availablePermits());
|
||||
assertEquals(recordLimit, queue.currentRateLimit);
|
||||
assertEquals(recordLimit, queue.size());
|
||||
assertEquals(recordLimit - 1, queue.samplingRecordCounter.get());
|
||||
|
||||
// try to read 2 records.
|
||||
assertEquals(hoodieRecords.get(0), queue.iterator().next().record);
|
||||
assertEquals(hoodieRecords.get(1), queue.iterator().next().record);
|
||||
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
// No change is expected in rate limit or number of queued records. We only expect
|
||||
// queueing thread to read
|
||||
// 2 more records into the queue.
|
||||
assertEquals(0, queue.rateLimiter.availablePermits());
|
||||
assertEquals(recordLimit, queue.currentRateLimit);
|
||||
assertEquals(recordLimit, queue.size());
|
||||
assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get());
|
||||
}
|
||||
|
||||
// Test to ensure that exception in either queueing thread or BufferedIterator-reader thread
|
||||
// is propagated to
|
||||
// another thread.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testException() throws Exception {
|
||||
final int numRecords = 256;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
|
||||
final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue));
|
||||
final long memoryLimitInBytes = 4 * objSize;
|
||||
|
||||
// first let us throw exception from queueIterator reader and test that queueing thread
|
||||
// stops and throws
|
||||
// correct exception back.
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue1 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
Future<Boolean> resFuture = executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1);
|
||||
return true;
|
||||
});
|
||||
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue1.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
// notify queueing thread of an exception and ensure that it exits.
|
||||
final Exception e = new Exception("Failing it :)");
|
||||
queue1.markAsFailed(e);
|
||||
final Throwable thrown1 = assertThrows(ExecutionException.class, resFuture::get,
|
||||
"exception is expected");
|
||||
assertEquals(HoodieException.class, thrown1.getCause().getClass());
|
||||
assertEquals(e, thrown1.getCause().getCause());
|
||||
|
||||
// second let us raise an exception while doing record queueing. this exception should get
|
||||
// propagated to
|
||||
// queue iterator reader.
|
||||
final RuntimeException expectedException = new RuntimeException("failing record reading");
|
||||
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
|
||||
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
|
||||
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue2 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
Future<Boolean> res = executorService.submit(() -> {
|
||||
try {
|
||||
new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2);
|
||||
} catch (Exception ex) {
|
||||
queue2.markAsFailed(ex);
|
||||
throw ex;
|
||||
}
|
||||
return true;
|
||||
});
|
||||
|
||||
final Throwable thrown2 = assertThrows(Exception.class, () -> {
|
||||
queue2.iterator().hasNext();
|
||||
}, "exception is expected");
|
||||
assertEquals(expectedException, thrown2.getCause());
|
||||
// queueing thread should also have exited. make sure that it is not running.
|
||||
final Throwable thrown3 = assertThrows(ExecutionException.class, res::get,
|
||||
"exception is expected");
|
||||
assertEquals(expectedException, thrown3.getCause());
|
||||
}
|
||||
|
||||
private boolean isQueueFull(Semaphore rateLimiter) {
|
||||
return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,100 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecutor() {
|
||||
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, 100);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
|
||||
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
|
||||
BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
|
||||
new BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
|
||||
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) {
|
||||
count++;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Integer getResult() {
|
||||
return count;
|
||||
}
|
||||
};
|
||||
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
try {
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
int result = executor.execute();
|
||||
// It should buffer and write 100 records
|
||||
assertEquals(100, result);
|
||||
// There should be no remaining records in the buffer
|
||||
assertFalse(executor.isRemaining());
|
||||
} finally {
|
||||
if (executor != null) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
|
||||
public static JavaRDD<HoodieRecord> generateTestRecordsForBulkInsert(JavaSparkContext jsc) {
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
// RDD partition 1
|
||||
List<HoodieRecord> records1 = dataGenerator.generateInserts("0", 100);
|
||||
// RDD partition 2
|
||||
List<HoodieRecord> records2 = dataGenerator.generateInserts("0", 150);
|
||||
return jsc.parallelize(records1, 1).union(jsc.parallelize(records2, 1));
|
||||
}
|
||||
|
||||
public static Map<String, Long> generateExpectedPartitionNumRecords(JavaRDD<HoodieRecord> records) {
|
||||
return records.map(record -> record.getPartitionPath()).countByValue();
|
||||
}
|
||||
|
||||
private static JavaRDD<HoodieRecord> generateTripleTestRecordsForBulkInsert(JavaSparkContext jsc)
|
||||
throws Exception {
|
||||
return generateTestRecordsForBulkInsert(jsc).union(generateTestRecordsForBulkInsert(jsc))
|
||||
.union(generateTestRecordsForBulkInsert(jsc));
|
||||
}
|
||||
|
||||
private static Stream<Arguments> configParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{BulkInsertSortMode.GLOBAL_SORT, true, true},
|
||||
{BulkInsertSortMode.PARTITION_SORT, false, true},
|
||||
{BulkInsertSortMode.NONE, false, false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
private void verifyRecordAscendingOrder(List<HoodieRecord> records) {
|
||||
List<HoodieRecord> expectedRecords = new ArrayList<>(records);
|
||||
Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())));
|
||||
assertEquals(expectedRecords, records);
|
||||
}
|
||||
|
||||
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
|
||||
JavaRDD<HoodieRecord> records,
|
||||
boolean isGloballySorted, boolean isLocallySorted,
|
||||
Map<String, Long> expectedPartitionNumRecords) {
|
||||
int numPartitions = 2;
|
||||
JavaRDD<HoodieRecord> actualRecords = (JavaRDD<HoodieRecord>) partitioner.repartitionRecords(records, numPartitions);
|
||||
assertEquals(numPartitions, actualRecords.getNumPartitions());
|
||||
List<HoodieRecord> collectedActualRecords = actualRecords.collect();
|
||||
if (isGloballySorted) {
|
||||
// Verify global order
|
||||
verifyRecordAscendingOrder(collectedActualRecords);
|
||||
} else if (isLocallySorted) {
|
||||
// Verify local order
|
||||
actualRecords.mapPartitions(partition -> {
|
||||
List<HoodieRecord> partitionRecords = new ArrayList<>();
|
||||
partition.forEachRemaining(partitionRecords::add);
|
||||
verifyRecordAscendingOrder(partitionRecords);
|
||||
return Collections.emptyList().iterator();
|
||||
}).collect();
|
||||
}
|
||||
|
||||
// Verify number of records per partition path
|
||||
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||
for (HoodieRecord record : collectedActualRecords) {
|
||||
String partitionPath = record.getPartitionPath();
|
||||
actualPartitionNumRecords.put(partitionPath,
|
||||
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + 1);
|
||||
}
|
||||
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "[{index}] {0}")
|
||||
@MethodSource("configParams")
|
||||
public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode,
|
||||
boolean isGloballySorted, boolean isLocallySorted)
|
||||
throws Exception {
|
||||
JavaRDD<HoodieRecord> records1 = generateTestRecordsForBulkInsert(jsc);
|
||||
JavaRDD<HoodieRecord> records2 = generateTripleTestRecordsForBulkInsert(jsc);
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1));
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,433 @@
|
||||
/*
|
||||
* 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.fs.ConsistencyGuardConfig;
|
||||
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.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.Assertions;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.txt", true);
|
||||
private final Random random = new Random();
|
||||
private IndexType indexType;
|
||||
private HoodieIndex index;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
private void setUp(IndexType indexType) throws Exception {
|
||||
this.indexType = indexType;
|
||||
initResources();
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.build()).withAutoCommit(false).build();
|
||||
writeClient = getHoodieWriteClient(config);
|
||||
this.index = writeClient.getIndex();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testSimpleTagLocationAndUpdate(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = "001";
|
||||
int totalRecords = 10 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert totalRecords records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, index should not tag them since it was a failed
|
||||
// commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, index should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size());
|
||||
assertEquals(totalRecords, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testTagLocationAndDuplicateUpdate(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = "001";
|
||||
int totalRecords = 10 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
assertEquals(totalRecords, javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size());
|
||||
assertEquals(totalRecords, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
int totalRecords = 20 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords);
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0);
|
||||
List<String> taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect();
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
// both lists should match
|
||||
assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds));
|
||||
// Rollback the last commit
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0);
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "SIMPLE",})
|
||||
public void testTagLocationAndFetchRecordLocations(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String p1 = "2016/01/31";
|
||||
String p2 = "2015/01/31";
|
||||
String rowKey1 = UUID.randomUUID().toString();
|
||||
String rowKey2 = UUID.randomUUID().toString();
|
||||
String rowKey3 = UUID.randomUUID().toString();
|
||||
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// place same row key under a different partition.
|
||||
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
String fileId1 = testTable.addCommit("001").withInserts(p1, record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts(p1, record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts(p2, record4);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getRecordKey().equals(rowKey1)) {
|
||||
if (record.getPartitionPath().equals(p2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
} else {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
}
|
||||
} else if (record.getRecordKey().equals(rowKey2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId2);
|
||||
} else if (record.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(recordRDD.map(HoodieRecord::getKey), hoodieTable);
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> entry : recordLocations.collect()) {
|
||||
if (entry._1.getRecordKey().equals(rowKey1)) {
|
||||
assertTrue(entry._2.isPresent(), "Row1 should have been present ");
|
||||
if (entry._1.getPartitionPath().equals(p2)) {
|
||||
assertTrue(entry._2.isPresent(), "Row1 should have been present ");
|
||||
assertEquals(entry._2.get().getRight(), fileId3);
|
||||
} else {
|
||||
assertEquals(entry._2.get().getRight(), fileId1);
|
||||
}
|
||||
} else if (entry._1.getRecordKey().equals(rowKey2)) {
|
||||
assertTrue(entry._2.isPresent(), "Row2 should have been present ");
|
||||
assertEquals(entry._2.get().getRight(), fileId2);
|
||||
} else if (entry._1.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(entry._2.isPresent(), "Row3 should have been absent ");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"GLOBAL_SIMPLE"})
|
||||
public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.withGlobalSimpleIndexUpdatePartitionPath(true)
|
||||
.withBloomIndexUpdatePartitionPath(true)
|
||||
.build()).build();
|
||||
writeClient = getHoodieWriteClient(config);
|
||||
index = writeClient.getIndex();
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
final String p1 = "2016/01/31";
|
||||
final String p2 = "2016/02/28";
|
||||
|
||||
// Create the original partition, and put a record, along with the meta file
|
||||
// "2016/01/31": 1 file (1_0_20160131101010.parquet)
|
||||
// this record will be saved in table and will be tagged to an empty record
|
||||
RawTripTestPayload originalPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord originalRecord =
|
||||
new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()),
|
||||
originalPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord but different time so different partition
|
||||
Because GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should
|
||||
- tag the original partition of the originalRecord to an empty record for deletion, and
|
||||
- tag the new partition of the incomingRecord
|
||||
*/
|
||||
RawTripTestPayload incomingPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord incomingRecord =
|
||||
new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()),
|
||||
incomingPayload);
|
||||
/*
|
||||
This record has the same record key as originalRecord and the same partition
|
||||
Though GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should just tag the original partition
|
||||
*/
|
||||
RawTripTestPayload incomingPayloadSamePartition =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}");
|
||||
HoodieRecord incomingRecordSamePartition =
|
||||
new HoodieRecord(
|
||||
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
// We have some records to be tagged (two different partitions)
|
||||
testTable.addCommit("1000").withInserts(p1, originalRecord);
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getPartitionPath()) {
|
||||
case p1:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertTrue(record.getData() instanceof EmptyHoodieRecordPayload);
|
||||
break;
|
||||
case p2:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(incomingPayload.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
break;
|
||||
default:
|
||||
fail(String.format("Should not get partition path: %s", record.getPartitionPath()));
|
||||
}
|
||||
}
|
||||
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(p1, record.getPartitionPath());
|
||||
assertEquals(incomingPayloadSamePartition.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
private JavaPairRDD<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
|
||||
JavaRDD<HoodieRecord> recordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(
|
||||
keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable);
|
||||
return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,178 @@
|
||||
/*
|
||||
* 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.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
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.SparkHoodieSimpleIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.nio.file.Path;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class TestHoodieIndexConfigs {
|
||||
|
||||
private String basePath;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp(@TempDir Path tempDir) {
|
||||
basePath = tempDir.toString();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE"})
|
||||
public void testCreateIndex(IndexType indexType) throws Exception {
|
||||
HoodieWriteConfig config;
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
switch (indexType) {
|
||||
case INMEMORY:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex);
|
||||
break;
|
||||
case BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex);
|
||||
break;
|
||||
case GLOBAL_BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex);
|
||||
break;
|
||||
case SIMPLE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex);
|
||||
break;
|
||||
case HBASE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build())
|
||||
.build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex);
|
||||
break;
|
||||
default:
|
||||
// no -op. just for checkstyle errors
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateDummyIndex() {
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateIndexWithException() {
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown1 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config1);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex"));
|
||||
|
||||
final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown2 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config2);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown2.getMessage().contains("Unable to instantiate class"));
|
||||
}
|
||||
|
||||
public static class DummyHoodieIndex<T extends HoodieRecordPayload<T>> extends SparkHoodieIndex<T> {
|
||||
|
||||
public DummyHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static class IndexWithConstructor {
|
||||
|
||||
public IndexWithConstructor(HoodieWriteConfig config) {
|
||||
}
|
||||
}
|
||||
|
||||
public static class IndexWithoutConstructor {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
/*
|
||||
* 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.util.collection.Pair;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestBucketizedBloomCheckPartitioner {
|
||||
|
||||
@Test
|
||||
public void testAssignmentCorrectness() {
|
||||
Map<String, Long> fileToComparisons = new HashMap<String, Long>() {
|
||||
{
|
||||
put("f1", 40L);
|
||||
put("f2", 35L);
|
||||
put("f3", 20L);
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(4, fileToComparisons, 10);
|
||||
Map<String, List<Integer>> assignments = p.getFileGroupToPartitions();
|
||||
assertEquals(4, assignments.get("f1").size(), "f1 should have 4 buckets");
|
||||
assertEquals(4, assignments.get("f2").size(), "f2 should have 4 buckets");
|
||||
assertEquals(2, assignments.get("f3").size(), "f3 should have 2 buckets");
|
||||
assertArrayEquals(new Integer[] {0, 0, 1, 3}, assignments.get("f1").toArray(), "f1 spread across 3 partitions");
|
||||
assertArrayEquals(new Integer[] {1, 2, 2, 0}, assignments.get("f2").toArray(), "f2 spread across 3 partitions");
|
||||
assertArrayEquals(new Integer[] {3, 1}, assignments.get("f3").toArray(), "f3 spread across 2 partitions");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUniformPacking() {
|
||||
// evenly distribute 10 buckets/file across 100 partitions
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner partitioner = new BucketizedBloomCheckPartitioner(100, comparisons1, 10);
|
||||
Map<String, List<Integer>> assignments = partitioner.getFileGroupToPartitions();
|
||||
assignments.forEach((key, value) -> assertEquals(10, value.size()));
|
||||
Map<Integer, Long> partitionToNumBuckets =
|
||||
assignments.entrySet().stream().flatMap(e -> e.getValue().stream().map(p -> Pair.of(p, e.getKey())))
|
||||
.collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting()));
|
||||
partitionToNumBuckets.forEach((key, value) -> assertEquals(1L, value.longValue()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumPartitions() {
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(10000, comparisons1, 10);
|
||||
assertEquals(100, p.numPartitions(), "num partitions must equal total buckets");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetPartitions() {
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 100000).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(1000, comparisons1, 10);
|
||||
|
||||
IntStream.range(0, 100000).forEach(f -> {
|
||||
int partition = p.getPartition(Pair.of("f" + f, "value"));
|
||||
assertTrue(0 <= partition && partition <= 1000, "partition is out of range: " + partition);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,458 @@
|
||||
/*
|
||||
* 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.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
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.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.txt", true);
|
||||
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
|
||||
|
||||
public static Stream<Arguments> configParams() {
|
||||
Object[][] data =
|
||||
new Object[][] {{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initFileSystem();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
|
||||
.bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
|
||||
.bloomIndexKeysPerBucket(2).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
|
||||
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
|
||||
.withInserts("2015/03/12", "3", record1)
|
||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||
|
||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
assertEquals(4, filesList.size());
|
||||
|
||||
if (rangePruning) {
|
||||
// these files will not have the key ranges
|
||||
assertNull(filesList.get(0)._2().getMaxRecordKey());
|
||||
assertNull(filesList.get(0)._2().getMinRecordKey());
|
||||
assertFalse(filesList.get(1)._2().hasKeyRanges());
|
||||
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
|
||||
assertNotNull(filesList.get(2)._2().getMinRecordKey());
|
||||
assertTrue(filesList.get(3)._2().hasKeyRanges());
|
||||
|
||||
// no longer sorted, but should have same files.
|
||||
|
||||
List<Tuple2<String, BloomIndexFileInfo>> expected =
|
||||
Arrays.asList(new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
|
||||
assertEquals(expected, filesList);
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22",
|
||||
Arrays.asList(new BloomIndexFileInfo("f1"), new BloomIndexFileInfo("f2", "000", "000"),
|
||||
new BloomIndexFileInfo("f3", "001", "003"), new BloomIndexFileInfo("f4", "002", "007"),
|
||||
new BloomIndexFileInfo("f5", "009", "010")));
|
||||
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(t -> t._1, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("003")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("004")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("005")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckUUIDsAgainstOneFile() throws Exception {
|
||||
final String partition = "2016/01/31";
|
||||
// Create some records to use
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// We write record1, record2 to a parquet file, but the bloom filter contains (record1,
|
||||
// record2, record3).
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record3.getRecordKey());
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
|
||||
String fileId = testTable.addCommit("000").withInserts(partition, record1, record2);
|
||||
String filename = testTable.getBaseFileNameById(fileId);
|
||||
|
||||
// The bloom filter contains 3 records
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record3.getRecordKey()));
|
||||
assertFalse(filter.mightContain(record4.getRecordKey()));
|
||||
|
||||
// Compare with file
|
||||
List<String> uuids =
|
||||
Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey());
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
|
||||
List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
|
||||
new Path(Paths.get(basePath, partition, filename).toString()));
|
||||
assertEquals(results.size(), 2);
|
||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
||||
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|
||||
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
|
||||
// TODO(vc): Need more coverage on actual filenames
|
||||
// assertTrue(results.get(0)._2().equals(filename));
|
||||
// assertTrue(results.get(1)._2().equals(filename));
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testTagLocationWithEmptyRDD(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
|
||||
assertDoesNotThrow(() -> {
|
||||
bloomIndex.tagLocation(recordRDD, context, table);
|
||||
}, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String rowKey1 = UUID.randomUUID().toString();
|
||||
String rowKey2 = UUID.randomUUID().toString();
|
||||
String rowKey3 = UUID.randomUUID().toString();
|
||||
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// place same row key under a different partition.
|
||||
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String fileId1 = testTable.addCommit("001").withInserts("2016/01/31", record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts("2016/01/31", record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4);
|
||||
|
||||
// We do the tag again
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient));
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getRecordKey().equals(rowKey1)) {
|
||||
if (record.getPartitionPath().equals("2015/01/31")) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
} else {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
}
|
||||
} else if (record.getRecordKey().equals(rowKey2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId2);
|
||||
} else if (record.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// record key same as recordStr2
|
||||
String recordStr4 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
|
||||
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
|
||||
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
|
||||
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
|
||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
|
||||
// Should not find any files
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String fileId1 = testTable.addCommit("001").withInserts("2016/01/31", record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts("2016/01/31", record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
|
||||
// Check results
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
|
||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
assertEquals(fileId1, record._2.get().getRight());
|
||||
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
if (record._1.getPartitionPath().equals("2015/01/31")) {
|
||||
assertEquals(fileId3, record._2.get().getRight());
|
||||
} else {
|
||||
assertEquals(fileId2, record._2.get().getRight());
|
||||
}
|
||||
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertFalse(record._2.isPresent());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testBloomFilterFalseError(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have two hoodie records
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
|
||||
// We write record1 to a parquet file, using a bloom filter having both records
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record2.getRecordKey());
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
|
||||
String fileId = testTable.addCommit("000").withInserts("2016/01/31", record1);
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
|
||||
// We do the tag
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId);
|
||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,345 @@
|
||||
/*
|
||||
* 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.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.txt", true);
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoadInvolvedFiles() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files, along with the meta file
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
|
||||
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
|
||||
.withInserts("2015/03/12", "3", record1)
|
||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||
|
||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
assertEquals(4, filesList.size());
|
||||
|
||||
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
|
||||
// key ranges checks
|
||||
assertNull(filesMap.get("2016/04/01/2").getMaxRecordKey());
|
||||
assertNull(filesMap.get("2016/04/01/2").getMinRecordKey());
|
||||
assertFalse(filesMap.get("2015/03/12/1").hasKeyRanges());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMaxRecordKey());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMinRecordKey());
|
||||
assertTrue(filesMap.get("2015/03/12/3").hasKeyRanges());
|
||||
|
||||
Map<String, BloomIndexFileInfo> expected = new HashMap<>();
|
||||
expected.put("2016/04/01/2", new BloomIndexFileInfo("2"));
|
||||
expected.put("2015/03/12/1", new BloomIndexFileInfo("1"));
|
||||
expected.put("2015/03/12/3", new BloomIndexFileInfo("3", "000", "000"));
|
||||
expected.put("2015/03/12/4", new BloomIndexFileInfo("4", "001", "003"));
|
||||
|
||||
assertEquals(expected, filesMap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExplodeRecordRDDWithFileComparisons() {
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
|
||||
new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003")));
|
||||
|
||||
partitionToFileIndexInfo.put("2017/10/23",
|
||||
Arrays.asList(new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010")));
|
||||
|
||||
// the partition of the key of the incoming records will be ignored
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
|
||||
/*
|
||||
* expecting: f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} f1, HoodieKey { recordKey=003
|
||||
* partitionPath=2017/10/22} f3, HoodieKey { recordKey=003 partitionPath=2017/10/22} f4, HoodieKey { recordKey=002
|
||||
* partitionPath=2017/10/23} f1, HoodieKey { recordKey=002 partitionPath=2017/10/22} f3, HoodieKey { recordKey=002
|
||||
* partitionPath=2017/10/22} f4, HoodieKey { recordKey=005 partitionPath=2017/10/23} f1, HoodieKey { recordKey=005
|
||||
* partitionPath=2017/10/22} f4, HoodieKey { recordKey=004 partitionPath=2017/10/23} f1, HoodieKey { recordKey=004
|
||||
* partitionPath=2017/10/22}
|
||||
*/
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(Tuple2::_1, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("003")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("004")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("005")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocation() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files, along with the meta file
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
|
||||
// this record will be saved in table and will be tagged to the incoming record5
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// this has the same record key as record4 but different time so different partition, but globalbloomIndex should
|
||||
// tag the original partition of the saved record4
|
||||
RawTripTestPayload rowChange5 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record5 =
|
||||
new HoodieRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5);
|
||||
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
String fileId1 = testTable.addCommit("1000").withInserts("2016/04/01", record1);
|
||||
String fileId2 = testTable.addCommit("2000").withInserts("2015/03/12");
|
||||
String fileId3 = testTable.addCommit("3000").withInserts("2015/03/12", record2);
|
||||
String fileId4 = testTable.addCommit("4000").withInserts("2015/03/12", record4);
|
||||
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getRecordKey()) {
|
||||
case "000":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange1.getJsonData());
|
||||
break;
|
||||
case "001":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange2.getJsonData());
|
||||
break;
|
||||
case "002":
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange3.getJsonData());
|
||||
break;
|
||||
case "003":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId4);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange5.getJsonData());
|
||||
break;
|
||||
case "004":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId4);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange4.getJsonData());
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown Key: " + record.getRecordKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
|
||||
.build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
final String p1 = "2016/01/31";
|
||||
final String p2 = "2016/02/28";
|
||||
|
||||
// Create the original partition, and put a record, along with the meta file
|
||||
// "2016/01/31": 1 file (1_0_20160131101010.parquet)
|
||||
// this record will be saved in table and will be tagged to an empty record
|
||||
RawTripTestPayload originalPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord originalRecord =
|
||||
new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()),
|
||||
originalPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord but different time so different partition
|
||||
Because GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should
|
||||
- tag the original partition of the originalRecord to an empty record for deletion, and
|
||||
- tag the new partition of the incomingRecord
|
||||
*/
|
||||
RawTripTestPayload incomingPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord incomingRecord =
|
||||
new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()),
|
||||
incomingPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord and the same partition
|
||||
Though GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should just tag the original partition
|
||||
*/
|
||||
RawTripTestPayload incomingPayloadSamePartition =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}");
|
||||
HoodieRecord incomingRecordSamePartition =
|
||||
new HoodieRecord(
|
||||
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
testTable.addCommit("1000").withInserts(p1, originalRecord);
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getPartitionPath()) {
|
||||
case p1:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertTrue(record.getData() instanceof EmptyHoodieRecordPayload);
|
||||
break;
|
||||
case p2:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(incomingPayload.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
break;
|
||||
default:
|
||||
fail(String.format("Should not get partition path: %s", record.getPartitionPath()));
|
||||
}
|
||||
}
|
||||
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(p1, record.getPartitionPath());
|
||||
assertEquals(incomingPayloadSamePartition.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
}
|
||||
|
||||
// convert list to map to avoid sorting order dependencies
|
||||
private static Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
|
||||
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
|
||||
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
|
||||
}
|
||||
return filesMap;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,176 @@
|
||||
/*
|
||||
* 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.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Tests {@link KeyRangeLookupTree}.
|
||||
*/
|
||||
public class TestKeyRangeLookupTree {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
private KeyRangeLookupTree keyRangeLookupTree;
|
||||
private Map<String, HashSet<String>> expectedMatches;
|
||||
|
||||
public TestKeyRangeLookupTree() {
|
||||
keyRangeLookupTree = new KeyRangeLookupTree();
|
||||
expectedMatches = new HashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for single node in the tree for different inputs.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpOneEntry() {
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(300), Long.toString(450), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
testRangeOfInputs(290, 305);
|
||||
testRangeOfInputs(390, 400);
|
||||
testRangeOfInputs(445, 455);
|
||||
testRangeOfInputs(600, 605);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for many entries in the tree with same start value and different end values.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpManyEntriesWithSameStartValue() {
|
||||
String startKey = Long.toString(120);
|
||||
long endKey = 250;
|
||||
KeyRangeNode toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
endKey += 1 + RANDOM.nextInt(100);
|
||||
toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
}
|
||||
testRangeOfInputs(110, endKey + 5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for many duplicte entries in the tree.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpManyDulicateEntries() {
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
}
|
||||
testRangeOfInputs(1050, 1100);
|
||||
testRangeOfInputs(1500, 1600);
|
||||
testRangeOfInputs(1990, 2100);
|
||||
}
|
||||
|
||||
// Tests helpers
|
||||
|
||||
/**
|
||||
* Tests for curated entries in look up tree.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUp() {
|
||||
|
||||
// testing with hand curated inputs
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(500), Long.toString(600), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(950), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(550), Long.toString(775), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(725), Long.toString(850), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(825), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(990), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(800), Long.toString(820), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(200), Long.toString(550), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(520), Long.toString(600), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
testRangeOfInputs(110, 999);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to test the look up tree for different range of input keys.
|
||||
*
|
||||
* @param start starting value of the look up key
|
||||
* @param end end value of the look up tree
|
||||
*/
|
||||
private void testRangeOfInputs(long start, long end) {
|
||||
for (long i = start; i <= end; i++) {
|
||||
String iStr = Long.toString(i);
|
||||
if (!expectedMatches.containsKey(iStr)) {
|
||||
assertEquals(Collections.EMPTY_SET, keyRangeLookupTree.getMatchingIndexFiles(iStr));
|
||||
} else {
|
||||
assertEquals(expectedMatches.get(iStr), keyRangeLookupTree.getMatchingIndexFiles(iStr));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the expected matches for a given {@link KeyRangeNode}.
|
||||
*
|
||||
* @param toInsert the {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
private void updateExpectedMatchesToTest(KeyRangeNode toInsert) {
|
||||
long startKey = Long.parseLong(toInsert.getMinRecordKey());
|
||||
long endKey = Long.parseLong(toInsert.getMaxRecordKey());
|
||||
for (long i = startKey; i <= endKey; i++) {
|
||||
String iStr = Long.toString(i);
|
||||
if (!expectedMatches.containsKey(iStr)) {
|
||||
expectedMatches.put(iStr, new HashSet<>());
|
||||
}
|
||||
expectedMatches.get(iStr).add(toInsert.getFileNameList().get(0));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,481 @@
|
||||
/*
|
||||
* 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.hbase;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.FunctionalTestHarness;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.MethodOrderer;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestMethodOrder;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.atMost;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown across tests,
|
||||
* (see one problem here : https://issues.apache.org/jira/browse/HBASE-15835). Hence, the need to use
|
||||
* {@link MethodOrderer.Alphanumeric} to make sure the tests run in order. Please alter the order of tests running carefully.
|
||||
*/
|
||||
@TestMethodOrder(MethodOrderer.Alphanumeric.class)
|
||||
@Tag("functional")
|
||||
public class TestHBaseIndex extends FunctionalTestHarness {
|
||||
|
||||
private static final String TABLE_NAME = "test_table";
|
||||
private static HBaseTestingUtility utility;
|
||||
private static Configuration hbaseConfig;
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private HoodieTestDataGenerator dataGen;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
@AfterAll
|
||||
public static void clean() throws Exception {
|
||||
if (utility != null) {
|
||||
utility.deleteTable(TABLE_NAME);
|
||||
utility.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeAll
|
||||
public static void init() throws Exception {
|
||||
// Initialize HbaseMiniCluster
|
||||
hbaseConfig = HBaseConfiguration.create();
|
||||
hbaseConfig.set("zookeeper.znode.parent", "/hudi-hbase-test");
|
||||
|
||||
utility = new HBaseTestingUtility(hbaseConfig);
|
||||
utility.startMiniCluster();
|
||||
hbaseConfig = utility.getConnection().getConfiguration();
|
||||
utility.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes("_s"));
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
hadoopConf = jsc().hadoopConfiguration();
|
||||
hadoopConf.addResource(utility.getConfiguration());
|
||||
metaClient = getHoodieMetaClient(hadoopConf, basePath());
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleTagLocationAndUpdateCOW() throws Exception {
|
||||
testSimpleTagLocationAndUpdate(HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
@Test void testSimpleTagLocationAndUpdateMOR() throws Exception {
|
||||
testSimpleTagLocationAndUpdate(HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
|
||||
public void testSimpleTagLocationAndUpdate(HoodieTableType tableType) throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath(), tableType);
|
||||
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationAndDuplicateUpdate() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
final String newCommitTime = writeClient.startCommit();
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
assertEquals(0, records2.stream().filter(record -> record.getCurrentLocation().getFileId() == null).count());
|
||||
List<String> taggedFileIds = records2.stream().map(record -> record.getCurrentLocation().getFileId()).distinct().collect(Collectors.toList());
|
||||
|
||||
// both lists should match
|
||||
assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds));
|
||||
// Rollback the last commit
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTotalGetsBatching() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
|
||||
// Mock hbaseConnection and related entities
|
||||
Connection hbaseConnection = mock(Connection.class);
|
||||
HTable table = mock(HTable.class);
|
||||
when(hbaseConnection.getTable(TableName.valueOf(TABLE_NAME))).thenReturn(table);
|
||||
when(table.get((List<Get>) any())).thenReturn(new Result[0]);
|
||||
|
||||
// only for test, set the hbaseConnection to mocked object
|
||||
index.setHbaseConnection(hbaseConnection);
|
||||
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
// start a commit and generate test data
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Insert 250 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
|
||||
// 3 batches should be executed given batchSize = 100 and parallelism = 1
|
||||
verify(table, times(3)).get((List<Get>) any());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTotalPutsBatching() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
// start a commit and generate test data
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
|
||||
// Mock hbaseConnection and related entities
|
||||
Connection hbaseConnection = mock(Connection.class);
|
||||
HTable table = mock(HTable.class);
|
||||
when(hbaseConnection.getTable(TableName.valueOf(TABLE_NAME))).thenReturn(table);
|
||||
when(table.get((List<Get>) any())).thenReturn(new Result[0]);
|
||||
|
||||
// only for test, set the hbaseConnection to mocked object
|
||||
index.setHbaseConnection(hbaseConnection);
|
||||
|
||||
// Get all the files generated
|
||||
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
|
||||
|
||||
index.updateLocation(writeStatues, context(), hoodieTable);
|
||||
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
|
||||
// so each fileId ideally gets updates
|
||||
verify(table, atMost(numberOfDataFileIds)).put((List<Put>) any());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsHBasePutAccessParallelism() {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
final JavaRDD<WriteStatus> writeStatusRDD = jsc().parallelize(
|
||||
Arrays.asList(getSampleWriteStatus(1, 2), getSampleWriteStatus(0, 3), getSampleWriteStatus(10, 0)), 10);
|
||||
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
|
||||
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
|
||||
assertEquals(10, writeStatusRDD.getNumPartitions());
|
||||
assertEquals(2, hbasePutAccessParallelism);
|
||||
assertEquals(11, hbaseNumPuts);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsHBasePutAccessParallelismWithNoInserts() {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
final JavaRDD<WriteStatus> writeStatusRDD =
|
||||
jsc().parallelize(Arrays.asList(getSampleWriteStatus(0, 2), getSampleWriteStatus(0, 1)), 10);
|
||||
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
|
||||
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
|
||||
assertEquals(10, writeStatusRDD.getNumPartitions());
|
||||
assertEquals(0, hbasePutAccessParallelism);
|
||||
assertEquals(0, hbaseNumPuts);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSmallBatchSize() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig(2);
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
|
||||
// commit
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelete() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
|
||||
// Delete all records. This has to be done directly as deleting index entries
|
||||
// is not implemented via HoodieWriteClient
|
||||
JavaRDD<WriteStatus> deleteWriteStatues = writeStatues.map(w -> {
|
||||
WriteStatus newWriteStatus = new WriteStatus(true, 1.0);
|
||||
w.getWrittenRecords().forEach(r -> newWriteStatus.markSuccess(new HoodieRecord(r.getKey(), null), Option.empty()));
|
||||
assertEquals(w.getTotalRecords(), newWriteStatus.getTotalRecords());
|
||||
newWriteStatus.setStat(new HoodieWriteStat());
|
||||
return newWriteStatus;
|
||||
});
|
||||
JavaRDD<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable);
|
||||
assertEquals(deleteStatus.count(), deleteWriteStatues.count());
|
||||
assertNoWriteErrors(deleteStatus.collect());
|
||||
|
||||
// Ensure no records can be tagged
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
private WriteStatus getSampleWriteStatus(final int numInserts, final int numUpdateWrites) {
|
||||
final WriteStatus writeStatus = new WriteStatus(false, 0.1);
|
||||
HoodieWriteStat hoodieWriteStat = new HoodieWriteStat();
|
||||
hoodieWriteStat.setNumInserts(numInserts);
|
||||
hoodieWriteStat.setNumUpdateWrites(numUpdateWrites);
|
||||
writeStatus.setStat(hoodieWriteStat);
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder(100).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(int hbaseIndexBatchSize) {
|
||||
return getConfigBuilder(hbaseIndexBatchSize).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(1, 1).withDeleteParallelism(1)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder()
|
||||
.hbaseZkPort(Integer.parseInt(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
|
||||
.hbaseIndexPutBatchSizeAutoCompute(true)
|
||||
.hbaseZkZnodeParent(hbaseConfig.get("zookeeper.znode.parent", ""))
|
||||
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(TABLE_NAME)
|
||||
.hbaseIndexGetBatchSize(hbaseIndexBatchSize).build())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.hbase;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestHBaseIndexUsage {
|
||||
|
||||
@Test
|
||||
public void testFeatureSupport() {
|
||||
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
assertTrue(index.canIndexLogFiles());
|
||||
}
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user