diff --git a/docker/demo/sparksql-batch2.commands b/docker/demo/sparksql-batch2.commands index 521a4ebe6dcde..f561c94e12608 100644 --- a/docker/demo/sparksql-batch2.commands +++ b/docker/demo/sparksql-batch2.commands @@ -26,4 +26,8 @@ spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from s spark.sql("select symbol, max(ts) from stock_ticks_mor_rt group by symbol HAVING symbol = 'GOOG'").show(100, false) spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close from stock_ticks_mor_rt where symbol = 'GOOG'").show(100, false) +// Copy COW as parquet dataset for bootstrap +val df = spark.sql("select _hoodie_record_key, volume, ts, symbol, year, month, high, low key, date, close, open, day, dt from stock_ticks_cow") +df.withColumnRenamed("_hoodie_record_key", "_row_key").write.parquet("/user/hive/warehouse/stock_ticks_cow_parquet") + System.exit(0) diff --git a/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapKeyGenerator.java b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapKeyGenerator.java new file mode 100644 index 0000000000000..33da21ba96691 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapKeyGenerator.java @@ -0,0 +1,62 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.client.utils.ClientUtils; +import org.apache.hudi.config.HoodieWriteConfig; + +import org.apache.avro.generic.GenericRecord; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +public class BootstrapKeyGenerator implements Serializable { + + private final HoodieWriteConfig writeConfig; + private final List keyColumns; + private final List topLevelKeyColumns; + + public BootstrapKeyGenerator(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + this.keyColumns = Arrays.asList(writeConfig.getBootstrapRecordKeyColumns().split(",")); + // For nested columns, pick top level column name + this.topLevelKeyColumns = keyColumns.stream().map(k -> { + int idx = k.indexOf('.'); + return idx > 0 ? k.substring(0, idx) : k; + }).collect(Collectors.toList()); + } + + /** + * Returns record key from generic record. The generic record + */ + public String getRecordKey(GenericRecord record) { + return keyColumns.stream().map(key -> ClientUtils.getNestedFieldValAsString(record, key)) + .collect(Collectors.joining("_")); + } + + public List getKeyColumns() { + return keyColumns; + } + + public List getTopLevelKeyColumns() { + return topLevelKeyColumns; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapPartitionSelector.java b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapPartitionSelector.java new file mode 100644 index 0000000000000..2d757b9400dc9 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapPartitionSelector.java @@ -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.bootstrap; + +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieWriteConfig.BootstrapMode; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +public abstract class BootstrapPartitionSelector implements Serializable { + + protected final HoodieWriteConfig writeConfig; + + public BootstrapPartitionSelector(HoodieWriteConfig writeConfig) { + this.writeConfig = writeConfig; + } + + /** + * Classify partitions for the purpose of bootstrapping. + * @param partitions List of partitions with files present in each partitions + * @return a partitions grouped by bootstrap mode + */ + public abstract Map>>> select( + List>> partitions); +} diff --git a/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapWriteStatus.java b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapWriteStatus.java new file mode 100644 index 0000000000000..36a365e3cc1eb --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/bootstrap/BootstrapWriteStatus.java @@ -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.bootstrap; + +import org.apache.hudi.WriteStatus; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.util.collection.Pair; + +import java.io.Serializable; + +/** + * WriteStatus for Bootstrap. + */ +public class BootstrapWriteStatus extends WriteStatus { + + public static final class BootstrapSourceInfo implements Serializable { + private final String bootstrapBasePath; + private final String bootstrapPartitionPath; + private final String fileName; + + public BootstrapSourceInfo(String bootstrapBasePath, String bootstrapPartitionPath, String fileName) { + this.bootstrapBasePath = bootstrapBasePath; + this.bootstrapPartitionPath = bootstrapPartitionPath; + this.fileName = fileName; + } + + public String getBootstrapBasePath() { + return bootstrapBasePath; + } + + public String getBootstrapPartitionPath() { + return bootstrapPartitionPath; + } + + public String getFileName() { + return fileName; + } + } + + private BootstrapSourceInfo bootstrapSourceInfo; + + public BootstrapWriteStatus(Boolean trackSuccessRecords, Double failureFraction) { + super(trackSuccessRecords, failureFraction); + } + + public BootstrapSourceInfo getBootstrapSourceInfo() { + return bootstrapSourceInfo; + } + + public Pair getBootstrapSourceAndWriteStat() { + return Pair.of(getBootstrapSourceInfo(), getStat()); + } + + public void setBootstrapSourceInfo(BootstrapSourceInfo bootstrapSourceInfo) { + this.bootstrapSourceInfo = bootstrapSourceInfo; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/bootstrap/HoodieBootstrapClient.java b/hudi-client/src/main/java/org/apache/hudi/bootstrap/HoodieBootstrapClient.java new file mode 100644 index 0000000000000..aea59aa6383c4 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/bootstrap/HoodieBootstrapClient.java @@ -0,0 +1,295 @@ +/* + * 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.bootstrap; + +import org.apache.hudi.AbstractHoodieClient; +import org.apache.hudi.bootstrap.BootstrapWriteStatus.BootstrapSourceInfo; +import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.common.SerializableConfiguration; +import org.apache.hudi.common.consolidated.CompositeMapFile; +import org.apache.hudi.common.consolidated.CompositeMapFile.DataPayloadWriter; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieExternalFileIdMapping; +import org.apache.hudi.common.model.HoodiePartitionExternalDataFiles; +import org.apache.hudi.common.model.HoodieRollingStat; +import org.apache.hudi.common.model.HoodieRollingStatMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTimeline; +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.util.FSUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.SerializationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieWriteConfig.BootstrapMode; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.metrics.HoodieMetrics; +import org.apache.hudi.table.HoodieTable; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +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.FlatMapFunction; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HoodieBootstrapClient extends AbstractHoodieClient { + + private static Logger logger = LogManager.getLogger(HoodieBootstrapClient.class); + + private final HoodieWriteConfig writeConfig; + private final transient HoodieMetrics metrics; + private transient Timer.Context writeContext = null; + + public HoodieBootstrapClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { + super(jsc, new HoodieWriteConfig.Builder().withProps(clientConfig.getProps()) + .withWriteStatusClass(BootstrapWriteStatus.class).build()); + this.writeConfig = clientConfig; + this.metrics = new HoodieMetrics(config, config.getTableName()); + } + + public HoodieBootstrapClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, + Option timelineServer) { + super(jsc, clientConfig, timelineServer); + this.writeConfig = clientConfig; + this.metrics = new HoodieMetrics(config, config.getTableName()); + } + + public void bootstrap() throws IOException { + writeContext = metrics.getCommitCtx(); + HoodieTableMetaClient metaClient = createMetaClient(false); + HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); + metaClient.getActiveTimeline().createNewInstant( + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, HoodieTimeline.BOOTSTRAP_INSTANT_TS)); + + Map>>> partitionSelections = + listSourcePartitionsAndTagBootstrapMode(metaClient); + table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + HoodieTimeline.COMMIT_ACTION, HoodieTimeline.BOOTSTRAP_INSTANT_TS), Option.empty()); + JavaRDD writeStatuses = + runMetadataBootstrap(table, partitionSelections.get(BootstrapMode.METADATA_ONLY_BOOTSTRAP)); + List> bootstrapSourceAndStats = + writeStatuses.map(BootstrapWriteStatus::getBootstrapSourceAndWriteStat).collect(); + List>> sourceStatsByPartition = new ArrayList<>( + bootstrapSourceAndStats.stream().map(sourceStatPair -> Pair.of(sourceStatPair.getRight().getPartitionPath(), + sourceStatPair)).collect(Collectors.groupingBy(Pair::getKey, + Collectors.mapping(x -> x.getValue(), Collectors.toList()))).values()); + + final CompositeMapFile bootstrapMetadataStorage = new CompositeMapFile("bootstrap", + new SerializableConfiguration(metaClient.getHadoopConf()), metaClient.getConsistencyGuardConfig(), + metaClient.getMetaBootstrapPath(), metaClient.getMetaBootstrapIndexPath()); + List>> partitionToFileOfssets = jsc.parallelize(sourceStatsByPartition, + config.getBootstrapMetadataWriterParallelism()).mapPartitions( + (FlatMapFunction>>, + Pair>>) listIterator -> { + DataPayloadWriter writer = bootstrapMetadataStorage.getDataPayloadWriter( + HoodieTimeline.BOOTSTRAP_INSTANT_TS, config.getBootstrapMetadataWriterMaxFileSizeInBytes()); + try { + while (listIterator.hasNext()) { + List> sourceAndStat = listIterator.next(); + HoodiePartitionExternalDataFiles externalDataFile = generateExternalDataFileMapping(sourceAndStat); + //TODO : See if we need serialization which support evolving schema ? + writer.update(externalDataFile.getHoodiePartitionPath(), + SerializationUtils.serialize(externalDataFile)); + } + return writer.getWrittenFileNameOffsets().iterator(); + } finally { + if (null != writer) { + writer.close(); + } + } + }, true).collect(); + + // Write Index to bootstrap metadata + final CompositeMapFile.IndexWriter indexWriter = bootstrapMetadataStorage.getIndexWriter(null, + HoodieTimeline.BOOTSTRAP_INSTANT_TS); + indexWriter.addIndexEntries(partitionToFileOfssets.stream()); + indexWriter.commit(); + logger.info("Bootstrap Metadata Index written !!"); + + commit(HoodieTimeline.BOOTSTRAP_INSTANT_TS, sourceStatsByPartition.stream() + .flatMap(sourceStat -> sourceStat.stream().map(s -> s.getRight())).collect(Collectors.toList()), + Option.empty(), HoodieTimeline.COMMIT_ACTION); + } + + private HoodiePartitionExternalDataFiles generateExternalDataFileMapping( + List> sourceAndStats) { + + if (null != sourceAndStats && !sourceAndStats.isEmpty()) { + List externalFileIdMappings = + sourceAndStats.stream().map(sourceAndStat -> new HoodieExternalFileIdMapping( + sourceAndStat.getKey().getFileName(), new Path(sourceAndStat.getRight().getPath()).getName())) + .collect(Collectors.toList()); + + BootstrapSourceInfo sourceInfo = sourceAndStats.get(0).getKey(); + String hoodiePartitionPath = sourceAndStats.get(0).getRight().getPartitionPath(); + return new HoodiePartitionExternalDataFiles(sourceInfo.getBootstrapBasePath(), + sourceInfo.getBootstrapPartitionPath(), hoodiePartitionPath, externalFileIdMappings); + } + return null; + } + + private Map>>> listSourcePartitionsAndTagBootstrapMode( + HoodieTableMetaClient metaClient) throws IOException { + List>> folders = + FSUtils.getAllLeafFoldersWithFiles(metaClient.getFs(), + writeConfig.getBootstrapSourceBasePath(), new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().endsWith(".parquet"); + } + }); + + BootstrapPartitionSelector selector = + (BootstrapPartitionSelector) ReflectionUtils.loadClass(writeConfig.getPartitionSelectorClass(), + writeConfig); + return selector.select(folders); + } + + private JavaRDD runMetadataBootstrap(HoodieTable table, List>> partitions) { + if (null == partitions || partitions.isEmpty()) { + return jsc.emptyRDD(); + } + + BootstrapKeyGenerator keyGenerator = new BootstrapKeyGenerator(writeConfig); + + return jsc.parallelize(partitions.stream().flatMap(p -> p.getValue().stream().map(f -> Pair.of(p.getLeft(), f))) + .collect(Collectors.toList()), writeConfig.getBootstrapParallelism()).map(partitionFilePathPair -> { + BootstrapSourceInfo sourceInfo = new BootstrapSourceInfo(writeConfig.getBootstrapSourceBasePath(), + partitionFilePathPair.getLeft(), partitionFilePathPair.getValue()); + return table.handleMetadataBootstrap(sourceInfo, partitionFilePathPair.getLeft(), keyGenerator); + }); + } + + private boolean commit(String commitTime, List stats, + Option> extraMetadata, String actionType) { + + logger.info("Commiting " + commitTime); + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc); + + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + + updateMetadataAndRollingStats(actionType, metadata, stats); + + // Finalize write + finalizeWrite(table, commitTime, stats); + + // add in extra metadata + if (extraMetadata.isPresent()) { + extraMetadata.get().forEach(metadata::addMetadata); + } + + try { + System.out.println("Hoodie Write Stats=" + metadata.toJsonString()); + activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + if (writeContext != null) { + long durationInMs = metrics.getDurationInMs(writeContext.stop()); + metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs, + metadata, actionType); + writeContext = null; + } + logger.info("Committed " + commitTime); + } catch (IOException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + commitTime, + e); + } catch (ParseException e) { + throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + commitTime + + "Instant time is not of valid format", e); + } + return true; + } + + private void updateMetadataAndRollingStats(String actionType, HoodieCommitMetadata metadata, + List writeStats) { + // TODO : make sure we cannot rollback / archive last commit file + try { + // Create a Hoodie table which encapsulated the commits and files visible + HoodieTable table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc); + // 0. All of the rolling stat management is only done by the DELTA commit for MOR and COMMIT for COW other wise + // there may be race conditions + HoodieRollingStatMetadata rollingStatMetadata = new HoodieRollingStatMetadata(actionType); + // 1. Look up the previous compaction/commit and get the HoodieCommitMetadata from there. + // 2. Now, first read the existing rolling stats and merge with the result of current metadata. + + // Need to do this on every commit (delta or commit) to support COW and MOR. + + for (HoodieWriteStat stat : writeStats) { + String partitionPath = stat.getPartitionPath(); + // TODO: why is stat.getPartitionPath() null at times here. + metadata.addWriteStat(partitionPath, stat); + HoodieRollingStat hoodieRollingStat = new HoodieRollingStat(stat.getFileId(), + stat.getNumWrites() - (stat.getNumUpdateWrites() - stat.getNumDeletes()), stat.getNumUpdateWrites(), + stat.getNumDeletes(), stat.getTotalWriteBytes()); + rollingStatMetadata.addRollingStat(partitionPath, hoodieRollingStat); + } + // The last rolling stat should be present in the completed timeline + Option lastInstant = + table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + if (lastInstant.isPresent()) { + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + table.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + Option lastRollingStat = Option + .ofNullable(commitMetadata.getExtraMetadata().get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); + if (lastRollingStat.isPresent()) { + rollingStatMetadata = rollingStatMetadata + .merge(HoodieCommitMetadata.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class)); + } + } + metadata.addMetadata(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, rollingStatMetadata.toJsonString()); + } catch (IOException io) { + throw new HoodieCommitException("Unable to save rolling stats"); + } + } + + private void finalizeWrite(HoodieTable table, String instantTime, List stats) { + try { + final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); + table.finalizeWrite(jsc, instantTime, stats); + if (finalizeCtx != null) { + Option durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop())); + durationInMs.ifPresent(duration -> { + logger.info("Finalize write elapsed time (milliseconds): " + duration); + metrics.updateFinalizeWriteMetrics(duration, stats.size()); + }); + } + } catch (HoodieIOException ioe) { + throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe); + } + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/bootstrap/MetadataOnlyBootstrapSelector.java b/hudi-client/src/main/java/org/apache/hudi/bootstrap/MetadataOnlyBootstrapSelector.java new file mode 100644 index 0000000000000..df4fc0031eaa5 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/bootstrap/MetadataOnlyBootstrapSelector.java @@ -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.bootstrap; + +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.config.HoodieWriteConfig.BootstrapMode; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class MetadataOnlyBootstrapSelector extends BootstrapPartitionSelector { + + public MetadataOnlyBootstrapSelector(HoodieWriteConfig bootstrapConfig) { + super(bootstrapConfig); + } + + @Override + public Map>>> select(List>> partitions) { + return partitions.stream().map(p -> Pair.of(BootstrapMode.METADATA_ONLY_BOOTSTRAP, p)) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(x -> x.getValue(), Collectors.toList()))); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java index 8f499a41f32c6..cd92f0679ebbd 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java @@ -22,9 +22,14 @@ 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.HoodieException; +import org.apache.avro.Schema.Field; +import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaSparkContext; +import java.util.stream.Collectors; + public class ClientUtils { /** @@ -39,4 +44,42 @@ public static HoodieTableMetaClient createMetaClient(JavaSparkContext jsc, Hoodi return new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), loadActiveTimelineOnLoad, config.getConsistencyGuardConfig(), Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))); } + + /** + * Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c + */ + public static String getNestedFieldValAsString(GenericRecord record, String fieldName) { + Object obj = getNestedFieldVal(record, fieldName); + return (obj == null) ? null : obj.toString(); + } + + /** + * Obtain value of the provided field, denoted by dot notation. e.g: a.b.c + */ + public static Object getNestedFieldVal(GenericRecord record, String fieldName) { + String[] parts = fieldName.split("\\."); + GenericRecord valueNode = record; + int i = 0; + for (; i < parts.length; i++) { + String part = parts[i]; + Object val = valueNode.get(part); + if (val == null) { + break; + } + + // return, if last part of name + if (i == parts.length - 1) { + return val; + } else { + // VC: Need a test here + if (!(val instanceof GenericRecord)) { + throw new HoodieException("Cannot find a record at part value :" + part); + } + valueNode = (GenericRecord) val; + } + } + throw new HoodieException( + fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :" + + valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList())); + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 4db8e4d41e610..caaf2df1de54c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -20,6 +20,7 @@ import org.apache.hudi.HoodieWriteClient; import org.apache.hudi.WriteStatus; +import org.apache.hudi.bootstrap.MetadataOnlyBootstrapSelector; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -98,12 +99,29 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { private ConsistencyGuardConfig consistencyGuardConfig; + public enum BootstrapMode { + FULL_BOOTSTRAP, + METADATA_ONLY_BOOTSTRAP + } + + private static final String SOURCE_BASE_PATH_PROP = "hoodie.bootstrap.source.base.path"; + private static final String BOOTSTRAP_PARTITION_SELECTOR = "hoodie.bootstrap.partition.selector"; + // Expect configurations of format col1,col2,col3 .... + private static final String BOOTSTRAP_RECORDKEY_COLUMNS = "hoodie.bootstrap.recordkey.columns"; + private static final String BOOTSTRAP_PARALLELISM = "hoodie.bootstrap.parallelism"; + private static final String DEFAULT_BOOTSTRAP_PARALLELISM = "1500"; + private static final String BOOTSTRAP_METADATA_WRITER_PARALLELISM = "hoodie.bootstrap.metadata.writer.parallelism"; + private static final String DEFAULT_BOOTSTRAP_METADATA_WRITER_PARALLELISM = "100"; + private static final String BOOTSTRAP_METADATA_WRITER_MAX_SIZE_PER_FILE = + "hoodie.bootstrap.metadata.writer.maxSizeInBytes"; + private static final String DEFAULT_METADATA_WRITER_MAX_SIZE_PER_FILE = String.valueOf(1024L * 1024L * 1024L); + // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled // We keep track of original config and rewritten config private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig; private FileSystemViewStorageConfig viewStorageConfig; - private HoodieWriteConfig(Properties props) { + protected HoodieWriteConfig(Properties props) { super(props); Properties newProps = new Properties(); newProps.putAll(props); @@ -543,9 +561,33 @@ public FileSystemViewStorageConfig getClientSpecifiedViewStorageConfig() { return clientSpecifiedViewStorageConfig; } + public String getBootstrapSourceBasePath() { + return props.getProperty(SOURCE_BASE_PATH_PROP); + } + + public String getPartitionSelectorClass() { + return props.getProperty(BOOTSTRAP_PARTITION_SELECTOR); + } + + public String getBootstrapRecordKeyColumns() { + return props.getProperty(BOOTSTRAP_RECORDKEY_COLUMNS); + } + + public int getBootstrapParallelism() { + return Integer.parseInt(props.getProperty(BOOTSTRAP_PARALLELISM)); + } + + public int getBootstrapMetadataWriterParallelism() { + return Integer.parseInt(props.getProperty(BOOTSTRAP_METADATA_WRITER_PARALLELISM)); + } + + public long getBootstrapMetadataWriterMaxFileSizeInBytes() { + return Long.parseLong(props.getProperty(BOOTSTRAP_METADATA_WRITER_MAX_SIZE_PER_FILE)); + } + public static class Builder { - private final Properties props = new Properties(); + protected final Properties props = new Properties(); private boolean isIndexConfigSet = false; private boolean isStorageConfigSet = false; private boolean isCompactionConfigSet = false; @@ -697,7 +739,42 @@ public Builder withEmbeddedTimelineServerEnabled(boolean enabled) { return this; } - public HoodieWriteConfig build() { + public Builder withBootstrapSourceBasePath(String basePath) { + props.setProperty(SOURCE_BASE_PATH_PROP, basePath); + return this; + } + + public Builder withBootstrapPartitionSelector(String partitionSelectorClass) { + props.setProperty(BOOTSTRAP_PARTITION_SELECTOR, partitionSelectorClass); + return this; + } + + public Builder withBootstrapRecordKeyColumns(String recordKeyColumnsCsv) { + props.setProperty(BOOTSTRAP_RECORDKEY_COLUMNS, recordKeyColumnsCsv); + return this; + } + + public Builder withBootstrapParallelism(int parallelism) { + props.setProperty(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism)); + return this; + } + + public Builder withBootstrapMetadataWriterParallelism(int parallelism) { + props.setProperty(BOOTSTRAP_METADATA_WRITER_PARALLELISM, String.valueOf(parallelism)); + return this; + } + + public Builder withBootstrapMetadataWriterMaxFileSizeInBytes(long maxFileSizeInBytes) { + props.setProperty(BOOTSTRAP_METADATA_WRITER_MAX_SIZE_PER_FILE, String.valueOf(maxFileSizeInBytes)); + return this; + } + + public Builder withProperties(Properties properties) { + this.props.putAll(properties); + return this; + } + + protected void setDefaults() { // Check for mandatory properties setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM); setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM, @@ -744,13 +821,24 @@ public HoodieWriteConfig build() { FileSystemViewStorageConfig.newBuilder().fromProperties(props).build()); setDefaultOnCondition(props, !isConsistencyGuardSet, ConsistencyGuardConfig.newBuilder().fromProperties(props).build()); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARALLELISM), BOOTSTRAP_PARALLELISM, + DEFAULT_BOOTSTRAP_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_METADATA_WRITER_PARALLELISM), + BOOTSTRAP_METADATA_WRITER_PARALLELISM, DEFAULT_BOOTSTRAP_METADATA_WRITER_PARALLELISM); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_METADATA_WRITER_MAX_SIZE_PER_FILE), + BOOTSTRAP_METADATA_WRITER_MAX_SIZE_PER_FILE, DEFAULT_METADATA_WRITER_MAX_SIZE_PER_FILE); + setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARTITION_SELECTOR), BOOTSTRAP_PARTITION_SELECTOR, + MetadataOnlyBootstrapSelector.class.getCanonicalName()); setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION); // Ensure Layout Version is good new TimelineLayoutVersion(Integer.parseInt(layoutVersion)); + } + public HoodieWriteConfig build() { + setDefaults(); // Build WriteConfig at the end HoodieWriteConfig config = new HoodieWriteConfig(props); Preconditions.checkArgument(config.getBasePath() != null); diff --git a/hudi-client/src/main/java/org/apache/hudi/func/CopyOnWriteLazyInsertIterable.java b/hudi-client/src/main/java/org/apache/hudi/func/CopyOnWriteLazyInsertIterable.java index 4d526f4850bcc..b66f99f11d508 100644 --- a/hudi-client/src/main/java/org/apache/hudi/func/CopyOnWriteLazyInsertIterable.java +++ b/hudi-client/src/main/java/org/apache/hudi/func/CopyOnWriteLazyInsertIterable.java @@ -135,8 +135,7 @@ protected void consumeOneRecord(HoodieInsertValueGenResult payload final HoodieRecord insertPayload = payload.record; // lazily initialize the handle, for the first time if (handle == null) { - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), - getNextFileId(idPrefix)); + handle = getCreateHandle(insertPayload); } if (handle.canWrite(payload.record)) { @@ -146,12 +145,16 @@ protected void consumeOneRecord(HoodieInsertValueGenResult payload // handle is full. statuses.add(handle.close()); // Need to handle the rejected payload & open new handle - handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(), - getNextFileId(idPrefix)); + handle = getCreateHandle(insertPayload); handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload. } } + protected HoodieCreateHandle getCreateHandle(HoodieRecord record) { + return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, record.getPartitionPath(), + getNextFileId(idPrefix)); + } + @Override protected void finish() { if (handle != null) { diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java new file mode 100644 index 0000000000000..edd785ce70431 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.io; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.util.HoodieAvroUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; + +public class HoodieBootstrapHandle extends HoodieCreateHandle { + + public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId) { + super(config, commitTime, hoodieTable, partitionPath, fileId, + Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA, + HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA))); + } + + @Override + public boolean canWrite(HoodieRecord record) { + return true; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java index 6916bad788bb4..c1aef6d699adf 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java @@ -27,12 +27,14 @@ import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; import org.apache.hudi.common.util.FSUtils; 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.HoodieInsertException; import org.apache.hudi.io.storage.HoodieStorageWriter; import org.apache.hudi.io.storage.HoodieStorageWriterFactory; import org.apache.hudi.table.HoodieTable; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -57,7 +59,12 @@ public class HoodieCreateHandle extends HoodieWri public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String partitionPath, String fileId) { - super(config, commitTime, fileId, hoodieTable); + this(config, commitTime, hoodieTable, partitionPath, fileId, generateOriginalAndHoodieWriteSchema(config)); + } + + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Pair originalAndHoodieSchema) { + super(config, commitTime, fileId, hoodieTable, originalAndHoodieSchema); writeStatus.setFileId(fileId); writeStatus.setPartitionPath(partitionPath); @@ -81,7 +88,14 @@ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTab */ public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, String partitionPath, String fileId, Iterator> recordIterator) { - this(config, commitTime, hoodieTable, partitionPath, fileId); + this(config, commitTime, hoodieTable, partitionPath, fileId, recordIterator, + generateOriginalAndHoodieWriteSchema(config)); + } + + public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable hoodieTable, + String partitionPath, String fileId, Iterator> recordIterator, + Pair originalAndHoodieSchema) { + this(config, commitTime, hoodieTable, partitionPath, fileId, originalAndHoodieSchema); this.recordIterator = recordIterator; this.useWriterSchema = true; } diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java index 7a1939a4747dc..c65fdcff9043c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java +++ b/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -59,8 +60,22 @@ public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String fi super(config, instantTime, hoodieTable); this.fileId = fileId; this.writeToken = makeSparkWriteToken(); - this.originalSchema = new Schema.Parser().parse(config.getSchema()); - this.writerSchema = createHoodieWriteSchema(originalSchema); + Pair originalAndHoodieSchema = generateOriginalAndHoodieWriteSchema(config); + + this.originalSchema = originalAndHoodieSchema.getKey(); + this.writerSchema = originalAndHoodieSchema.getValue(); + this.timer = new HoodieTimer().startTimer(); + this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), + !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); + } + + public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String fileId, + HoodieTable hoodieTable, Pair originalAndHoodieSchema) { + super(config, instantTime, hoodieTable); + this.fileId = fileId; + this.writeToken = makeSparkWriteToken(); + this.originalSchema = originalAndHoodieSchema.getKey(); + this.writerSchema = originalAndHoodieSchema.getValue(); this.timer = new HoodieTimer().startTimer(); this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(), !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction()); @@ -74,8 +89,10 @@ private static String makeSparkWriteToken() { TaskContext.get().taskAttemptId()); } - public static Schema createHoodieWriteSchema(Schema originalSchema) { - return HoodieAvroUtils.addMetadataFields(originalSchema); + protected static Pair generateOriginalAndHoodieWriteSchema(HoodieWriteConfig config) { + Schema originalSchema = new Schema.Parser().parse(config.getSchema()); + Schema hoodieSchema = HoodieAvroUtils.addMetadataFields(originalSchema); + return Pair.of(originalSchema, hoodieSchema); } public Path makeNewPath(String partitionPath) { diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java index 2a6ce86776608..059cad8f1142c 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java @@ -22,6 +22,9 @@ import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.bootstrap.BootstrapKeyGenerator; +import org.apache.hudi.bootstrap.BootstrapWriteStatus; +import org.apache.hudi.bootstrap.BootstrapWriteStatus.BootstrapSourceInfo; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.HoodieRollbackStat; import org.apache.hudi.common.model.HoodieCleaningPolicy; @@ -37,6 +40,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.FSUtils; +import org.apache.hudi.common.util.HoodieAvroUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor; @@ -49,11 +53,14 @@ import org.apache.hudi.func.CopyOnWriteLazyInsertIterable; import org.apache.hudi.func.ParquetReaderIterator; import org.apache.hudi.func.SparkBoundedInMemoryExecutor; +import org.apache.hudi.io.HoodieBootstrapHandle; import org.apache.hudi.io.HoodieCleanHelper; import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import com.google.common.hash.Hashing; +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; @@ -62,7 +69,12 @@ 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.Partitioner; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -157,6 +169,54 @@ public boolean isWorkloadProfileNeeded() { return true; } + @Override + public BootstrapWriteStatus handleMetadataBootstrap(BootstrapSourceInfo bootstrapSourceInfo, String partitionPath, + BootstrapKeyGenerator keyGenerator) { + + Path sourceFilePath = new Path(FSUtils.getPartitionPath(bootstrapSourceInfo.getBootstrapBasePath(), + bootstrapSourceInfo.getBootstrapPartitionPath()), bootstrapSourceInfo.getFileName()); + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.BOOTSTRAP_INSTANT_TS, + this, partitionPath, FSUtils.createNewFileIdPfx()); + try { + ParquetMetadata readFooter = ParquetFileReader.readFooter(getHadoopConf(), sourceFilePath, + ParquetMetadataConverter.NO_FILTER); + MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); + Schema avroSchema = new AvroSchemaConverter().convert(parquetSchema); + Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, + keyGenerator.getTopLevelKeyColumns()); + LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); + AvroReadSupport.setAvroReadSchema(getHadoopConf(), recordKeySchema); + AvroReadSupport.setRequestedProjection(getHadoopConf(), recordKeySchema); + + BoundedInMemoryExecutor wrapper = null; + try (ParquetReader reader = + AvroParquetReader.builder(sourceFilePath).withConf(getHadoopConf()).build()) { + wrapper = new SparkBoundedInMemoryExecutor(config, + new ParquetReaderIterator(reader), new BootstrapHandler(bootstrapHandle), inp -> { + String recKey = keyGenerator.getRecordKey(inp); + 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) { + e.printStackTrace(); + } + BootstrapWriteStatus writeStatus = (BootstrapWriteStatus)bootstrapHandle.getWriteStatus(); + writeStatus.setBootstrapSourceInfo(bootstrapSourceInfo); + return writeStatus; + } + @Override public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String commitTime) { throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table"); @@ -442,6 +502,35 @@ protected Void getResult() { } } + /** + * Consumer that dequeues records from queue and sends to Merge Handle. + */ + private static class BootstrapHandler extends BoundedInMemoryQueueConsumer { + + private final HoodieBootstrapHandle bootstrapHandle; + + private BootstrapHandler(HoodieBootstrapHandle bootstrapHandle) { + this.bootstrapHandle = bootstrapHandle; + } + + @Override + protected void consumeOneRecord(HoodieRecord record) { + try { + bootstrapHandle.write(record, record.getData().getInsertValue(bootstrapHandle.getWriterSchema())); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + @Override + protected void finish() {} + + @Override + protected Void getResult() { + return null; + } + } + private static class PartitionCleanStat implements Serializable { private final String partitionPath; @@ -778,4 +867,29 @@ protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, int d } return avgSize; } + + private static class BootstrapRecordPayload implements HoodieRecordPayload { + + private final GenericRecord record; + + public BootstrapRecordPayload(GenericRecord record) { + this.record = record; + } + + @Override + public BootstrapRecordPayload preCombine(BootstrapRecordPayload another) { + return this; + } + + @Override + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + throws IOException { + return Option.ofNullable(record); + } + + @Override + public Option getInsertValue(Schema schema) throws IOException { + return Option.ofNullable(record); + } + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java index 63811c1e4d04d..1f59dda3f7c51 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -22,6 +22,9 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieSavepointMetadata; +import org.apache.hudi.bootstrap.BootstrapKeyGenerator; +import org.apache.hudi.bootstrap.BootstrapWriteStatus; +import org.apache.hudi.bootstrap.BootstrapWriteStatus.BootstrapSourceInfo; import org.apache.hudi.client.utils.ClientUtils; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.HoodieRollbackStat; @@ -255,9 +258,19 @@ public abstract Iterator> handleUpsertPartition(String commitT public abstract Iterator> handleInsertPartition(String commitTime, Integer partition, Iterator> recordIterator, Partitioner partitioner); + /** + * Perform metadata bootstrap for a given external source file. + * @param bootstrapSourceInfo External non hoodie file information + * @param partitionPath Partition Path + * @param keyGenerator Record Key Generator + * @return BootstrapWriteStatus + */ + public abstract BootstrapWriteStatus handleMetadataBootstrap(BootstrapSourceInfo bootstrapSourceInfo, + String partitionPath, BootstrapKeyGenerator keyGenerator); + /** * Schedule compaction for the instant time. - * + * * @param jsc Spark Context * @param instantTime Instant Time for scheduling compaction * @return diff --git a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java index bb716d650aee8..83313768672b4 100644 --- a/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/src/test/java/org/apache/hudi/TestHoodieClientOnCopyOnWriteStorage.java @@ -18,6 +18,8 @@ package org.apache.hudi; +import org.apache.hudi.bootstrap.HoodieBootstrapClient; +import org.apache.hudi.bootstrap.MetadataOnlyBootstrapSelector; import org.apache.hudi.common.HoodieClientTestUtils; import org.apache.hudi.common.HoodieTestDataGenerator; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -26,6 +28,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRollingStat; import org.apache.hudi.common.model.HoodieRollingStatMetadata; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTestUtils; import org.apache.hudi.common.model.TimelineLayoutVersion; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -155,6 +158,21 @@ private void testAutoCommit(Function3, HoodieWriteClient, J assertTrue("After explicit commit, commit file should be created", HoodieTestUtils.doesCommitExist(basePath, newCommitTime)); } + + String newBasePath = basePath + "_2"; + HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), newBasePath, HoodieTableType.COPY_ON_WRITE, + "dummy2", "archived"); + HoodieWriteConfig config = getConfigBuilder().withPath(newBasePath) + .withBootstrapPartitionSelector(MetadataOnlyBootstrapSelector.class.getCanonicalName()) + .withBootstrapSourceBasePath(cfg.getBasePath()) + .withBootstrapParallelism(2) + .withBootstrapMetadataWriterParallelism(1) + .withBootstrapRecordKeyColumns("_row_key") + .build(); + HoodieBootstrapClient bootstrapClient = new HoodieBootstrapClient(jsc, config); + bootstrapClient.bootstrap(); + assertTrue("After bootstrap, Bootstrap commit file should be created", + HoodieTestUtils.doesCommitExist(newBasePath, HoodieTimeline.BOOTSTRAP_INSTANT_TS)); } /** diff --git a/hudi-common/src/main/avro/HoodieBootstrapMetadata.avsc b/hudi-common/src/main/avro/HoodieBootstrapMetadata.avsc new file mode 100644 index 0000000000000..bb2e0a1db4033 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieBootstrapMetadata.avsc @@ -0,0 +1,54 @@ +/* + * 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. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieBootstrapMetadata", + "fields":[ + { + "name":"sourceBasePath", + "type":["null","string"], + "default": null + }, + { + "name":"partitionBootstrapInfo", + "type": ["null", { + "type":"map", + "values":{ + "type":"array", + "items":{ + "name":"HoodieBootstrapFileInfo", + "type":"record", + "fields":[ + { + "name":"sourceFilePath", + "type":["null","string"], + "default": null + }, + { + "name":"fileId", + "type":["null","string"], + "default": null + } + ] + } + } + }] + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/consolidated/CompositeMapFile.java b/hudi-common/src/main/java/org/apache/hudi/common/consolidated/CompositeMapFile.java new file mode 100644 index 0000000000000..e904daf9f4c78 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/consolidated/CompositeMapFile.java @@ -0,0 +1,362 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.consolidated; + +import org.apache.hudi.common.SerializableConfiguration; +import org.apache.hudi.common.util.ConsistencyGuardConfig; +import org.apache.hudi.common.util.FSUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.SequenceFile.CompressionType; +import org.apache.hadoop.io.Text; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Storage Structure similar to org.apache.hadoop.io.MapFile but in this case, + * there are multiple data files (written in parallel) backed by an indexed storage. + */ +public class CompositeMapFile implements Serializable { + + private static Logger log = LogManager.getLogger(CompositeMapFile.class); + + private final String namespace; + private final SerializableConfiguration conf; + private final ConsistencyGuardConfig consistencyGuardConfig; + private final String rootMetadataDir; + private final String rootMetadataIndexDir; + + private transient FileSystem fileSystem; + + public CompositeMapFile(String namespace, SerializableConfiguration conf, + ConsistencyGuardConfig consistencyGuardConfig, String rootDir, String rootIndexDir) { + this.namespace = namespace; + this.rootMetadataDir = rootDir; + this.rootMetadataIndexDir = rootIndexDir; + this.consistencyGuardConfig = consistencyGuardConfig; + this.conf = new SerializableConfiguration(conf); + this.fileSystem = getFileSystem(); + } + + private FileSystem getFileSystem() { + if (null == fileSystem) { + fileSystem = FSUtils.getFs(rootMetadataDir, conf, consistencyGuardConfig); + } + return fileSystem; + } + + public DataPayloadWriter getDataPayloadWriter(String currVersion, long maxFileSizeInBytes) { + return new DataPayloadWriter(maxFileSizeInBytes, currVersion); + } + + public IndexWriter getIndexWriter(String prevVersion, String currVersion) { + return new IndexWriter(prevVersion, currVersion); + } + + public RandomAccessReader getRandomAccessReader(String version) throws IOException { + return new RandomAccessReader(version); + } + + public boolean isIndexExists(String version) { + Path p = getIndexFilePath(version); + try { + return getFileSystem().isFile(p); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + protected Path getNextOutputFilePath(String currVersion, int id) { + return new Path(rootMetadataDir, namespace + "_" + currVersion + "_" + id + ".seq"); + } + + protected Path getIndexFilePath(String version) { + return new Path(rootMetadataIndexDir, "keyidx_" + namespace + "_" + version + ".seq"); + } + + /** + * Random Access Reader. + */ + public class RandomAccessReader { + + private final String version; + + private Map> keyToFileAndOffsetPair = new HashMap<>(); + + public RandomAccessReader(String version) throws IOException { + this.version = version; + keyToFileAndOffsetPair = readExistingKeyToFileOffsetIndex(new SequenceFile.Reader(conf.get(), + SequenceFile.Reader.file(getIndexFilePath(this.version)))); + } + + public int size() { + return keyToFileAndOffsetPair.size(); + } + + public boolean isEmpty() { + return keyToFileAndOffsetPair.isEmpty(); + } + + public boolean containsKey(Object key) { + return keyToFileAndOffsetPair.containsKey(key); + } + + public byte[] get(Object key) { + Pair val = keyToFileAndOffsetPair.get(key); + if (null != val) { + try (SequenceFile.Reader reader = new SequenceFile.Reader(conf.get(), + SequenceFile.Reader.file(new Path(rootMetadataDir, val.getKey())))) { + reader.seek(val.getValue()); + Text keyText = new Text(); + BytesWritable valWritable = new BytesWritable(); + reader.next(keyText, valWritable); + Preconditions.checkArgument(keyText.toString().equals(key), "Expected=" + key + ", Got=" + + keyText.toString()); + return valWritable.getBytes(); + } catch (IOException ioe) { + log.error("Got exception reading data from " + val); + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + return null; + } + } + + /** + * Responsible for writing index to payload. + */ + public class IndexWriter { + + private SequenceFile.Reader prevConsolidatedIdxReader; + private final String currVersion; + private final String prevVersion; + private Map> mergedKeyToFileOffsetIdx; + private FSDataOutputStream outputStream; + private SequenceFile.Writer idxWriter; + + public IndexWriter(String prevVersion, String currVersion) { + this.currVersion = currVersion; + this.prevVersion = prevVersion; + init(); + } + + private void init() { + try { + if (null != prevVersion) { + prevConsolidatedIdxReader = new SequenceFile.Reader(conf.get(), + SequenceFile.Reader.file(getIndexFilePath(prevVersion))); + mergedKeyToFileOffsetIdx = readExistingKeyToFileOffsetIndex(prevConsolidatedIdxReader); + } else { + prevConsolidatedIdxReader = null; + mergedKeyToFileOffsetIdx = new HashMap<>(); + } + log.info("Read " + mergedKeyToFileOffsetIdx.size() + " from previous index"); + outputStream = getFileSystem().create(getIndexFilePath(currVersion), false); + idxWriter = SequenceFile.createWriter(conf.get(), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class), + SequenceFile.Writer.compression(CompressionType.NONE), + SequenceFile.Writer.stream(outputStream)); + } catch (IOException ioe) { + log.error("Got exception starting the writer", ioe); + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + public void addIndexEntries(Stream>> partitionToFileOffsetEntries) + throws IOException { + partitionToFileOffsetEntries.forEach(e -> { + String key = e.getKey(); + String fileName = e.getValue().getKey(); + Long offset = e.getValue().getValue(); + mergedKeyToFileOffsetIdx.put(key, Pair.of(fileName, offset)); + }); + } + + public void rollback() throws IOException { + close(); + } + + public void commit() throws IOException { + try { + for (Entry> entry : mergedKeyToFileOffsetIdx.entrySet()) { + Text key = new Text(entry.getKey()); + Text value = new Text(entry.getValue().getValue() + "," + entry.getValue().getKey()); + idxWriter.append(key, value); + } + } catch (IOException ioe) { + log.error("Got exception trying to save the file offset index"); + throw new HoodieIOException(ioe.getMessage(), ioe); + } + + // Nothing to be done + close(); + } + + private void close() throws IOException { + if (prevConsolidatedIdxReader != null) { + prevConsolidatedIdxReader.close(); + prevConsolidatedIdxReader = null; + } + + if (null != outputStream) { + outputStream.close(); + } + + if (null != idxWriter) { + idxWriter.close(); + } + + mergedKeyToFileOffsetIdx = null; + } + } + + /** + * Responsible for Writing payload. + */ + public class DataPayloadWriter { + + private final long maxFileSizeInBytes; + private final String currVersion; + + private SequenceFile.Writer currWriter; + private FSDataOutputStream currOuputStream; + private Path currOutputFilePath; + private int idGen; + private Map> keyToFileAndOffsetPair = new HashMap<>(); + + public DataPayloadWriter(long maxFileSizeInBytes, String currVersion) { + this.maxFileSizeInBytes = maxFileSizeInBytes; + this.currVersion = currVersion; + init(); + } + + private void init() { + try { + this.idGen = -1; + rollover(); + } catch (IOException ioe) { + log.error("Got exception starting the writer", ioe); + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + + private void rollover() throws IOException { + if (currOuputStream != null) { + currOuputStream.close(); + } + + if (currWriter != null) { + currWriter.close(); + } + + this.currOutputFilePath = getNextOutputFilePathForCurrentVersion(); + this.currOuputStream = getFileSystem().create(currOutputFilePath, false); + this.currWriter = SequenceFile.createWriter(conf.get(), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(BytesWritable.class), + SequenceFile.Writer.compression(CompressionType.RECORD), + SequenceFile.Writer.stream(currOuputStream)); + } + + private Path getNextOutputFilePathForCurrentVersion() { + int currId = ++idGen; + return getNextOutputFilePath(currVersion, currId); + } + + public void update(Stream> entries) throws IOException { + entries.forEach(entry -> { + try { + update(entry.getKey(), entry.getRight()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } + + public void update(String keyStr, byte[] payload) throws IOException { + Text key = new Text(keyStr); + BytesWritable bytesWritable = new BytesWritable(payload); + try { + keyToFileAndOffsetPair.put(keyStr, Pair.of(getNextOutputFilePath(currVersion, idGen).toString(), + currWriter.getLength())); + this.currWriter.append(key, bytesWritable); + if (currWriter.getLength() >= maxFileSizeInBytes) { + rollover(); + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + + public List>> getWrittenFileNameOffsets() { + return keyToFileAndOffsetPair.entrySet().stream().map(entry -> Pair.of(entry.getKey(), entry.getValue())).collect( + Collectors.toList()); + } + + public void close() throws IOException { + if (currOuputStream != null) { + currOuputStream.close(); + currOuputStream = null; + } + + if (currWriter != null) { + currWriter.close(); + currWriter = null; + } + + keyToFileAndOffsetPair = null; + } + } + + private static Map> readExistingKeyToFileOffsetIndex( + SequenceFile.Reader prevConsolidatedIdxReader) throws IOException { + Map> existingKeyToFileOffsetIdx = new HashMap<>(); + if (prevConsolidatedIdxReader != null) { + // Consolidated Metada already existed. We need to merge only the file-offsets + Text key = new Text(); + Text offsetFileCsv = new Text(); + while (prevConsolidatedIdxReader.next(key, offsetFileCsv)) { + String[] offsetFileNames = offsetFileCsv.toString().split(","); + Preconditions.checkArgument(offsetFileNames.length == 2, + "offsetFileCsv not is valid format. offsetFileCsv=" + offsetFileCsv); + Long offset = Long.parseLong(offsetFileNames[0]); + String fileName = offsetFileNames[1]; + existingKeyToFileOffsetIdx.put(key.toString(), Pair.of(fileName, offset)); + } + } + return existingKeyToFileOffsetIdx; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ExternalDataFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ExternalDataFile.java new file mode 100644 index 0000000000000..7eb4a4121aeeb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ExternalDataFile.java @@ -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.common.model; + +import java.io.Serializable; + +/** + * POJO storing (partitionPath, hoodieFileId) -> external data file path. + */ +public class ExternalDataFile implements Serializable { + + private final HoodieFileGroupId fileGroupId; + + private final String externalDataFile; + + public ExternalDataFile(HoodieFileGroupId fileGroupId, String externalDataFile) { + this.fileGroupId = fileGroupId; + this.externalDataFile = externalDataFile; + } + + public HoodieFileGroupId getFileGroupId() { + return fileGroupId; + } + + public String getExternalDataFile() { + return externalDataFile; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java index 7a6521464f919..f473bdd0b5c09 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java @@ -23,6 +23,7 @@ import java.io.Serializable; import java.util.Objects; import java.util.TreeSet; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -52,11 +53,23 @@ public class FileSlice implements Serializable { */ private final TreeSet logFiles; + public FileSlice(FileSlice fileSlice) { + this.baseInstantTime = fileSlice.baseInstantTime; + this.dataFile = fileSlice.dataFile != null ? new HoodieDataFile(fileSlice.dataFile) : null; + this.fileGroupId = fileSlice.fileGroupId; + this.logFiles = new TreeSet<>(fileSlice.logFiles.stream().map(lf -> new HoodieLogFile(lf)) + .collect(Collectors.toList())); + } + public FileSlice(String partitionPath, String baseInstantTime, String fileId) { this(new HoodieFileGroupId(partitionPath, fileId), baseInstantTime); } public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime) { + this(fileGroupId, baseInstantTime, null); + } + + public FileSlice(HoodieFileGroupId fileGroupId, String baseInstantTime, String externalDataFile) { this.fileGroupId = fileGroupId; this.baseInstantTime = baseInstantTime; this.dataFile = null; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java index 4983b74d86acc..052232c68eb30 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieDataFile.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.model; import org.apache.hudi.common.util.FSUtils; +import org.apache.hudi.common.util.Option; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -35,16 +36,38 @@ public class HoodieDataFile implements Serializable { private final String fullPath; private long fileLen; + /** + * In case of index-only bootstrap, this file points to the actual data. + */ + private Option externalDataFile; + + public HoodieDataFile(HoodieDataFile dataFile) { + this.fileStatus = dataFile.fileStatus; + this.fullPath = dataFile.fullPath; + this.fileLen = dataFile.fileLen; + this.externalDataFile = dataFile.externalDataFile; + } + public HoodieDataFile(FileStatus fileStatus) { + this(fileStatus, null); + } + + public HoodieDataFile(FileStatus fileStatus, String externalDataFile) { this.fileStatus = fileStatus; this.fullPath = fileStatus.getPath().toString(); this.fileLen = fileStatus.getLen(); + this.externalDataFile = Option.ofNullable(externalDataFile); } public HoodieDataFile(String filePath) { + this(filePath, null); + } + + public HoodieDataFile(String filePath, String externalDataFile) { this.fileStatus = null; this.fullPath = filePath; this.fileLen = -1; + this.externalDataFile = Option.ofNullable(externalDataFile); } public String getFileId() { @@ -79,6 +102,14 @@ public long getFileLen() { return fileLen; } + public Option getExternalDataFile() { + return externalDataFile; + } + + public void setExternalDataFile(String externalDataFile) { + this.externalDataFile = Option.ofNullable(externalDataFile); + } + @Override public boolean equals(Object o) { if (this == o) { @@ -98,6 +129,7 @@ public int hashCode() { @Override public String toString() { - return "HoodieDataFile{fullPath=" + fullPath + ", fileLen=" + fileLen + '}'; + return "HoodieDataFile{" + "fullPath=" + fullPath + ", fileLen=" + fileLen + + ", ExternalDataFile=" + externalDataFile.orElse(null) + '}'; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieExternalFileIdMapping.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieExternalFileIdMapping.java new file mode 100644 index 0000000000000..7ad9c71d3a9e7 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieExternalFileIdMapping.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.io.Serializable; + +/** + * Hoodie File Name to external file name mapping. + */ +public class HoodieExternalFileIdMapping implements Serializable { + + private final String externalFileName; + private final String hoodieFileName; + + public HoodieExternalFileIdMapping(String externalFileName, String hoodieFileName) { + this.externalFileName = externalFileName; + this.hoodieFileName = hoodieFileName; + } + + public String getExternalFileName() { + return externalFileName; + } + + public String getHoodieFileName() { + return hoodieFileName; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java index 3313586eb9b28..46d6538db6031 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java @@ -21,12 +21,14 @@ import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; import java.io.Serializable; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -58,6 +60,15 @@ public static Comparator getReverseCommitTimeComparator() { */ private final Option lastInstant; + public HoodieFileGroup(HoodieFileGroup fileGroup) { + this.timeline = fileGroup.timeline; + this.fileGroupId = fileGroup.fileGroupId; + this.fileSlices = new TreeMap<>(fileGroup.fileSlices.entrySet().stream() + .map(e -> Pair.of(e.getKey(), new FileSlice(e.getValue()))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue))); + this.lastInstant = fileGroup.lastInstant; + } + public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) { this(new HoodieFileGroupId(partitionPath, id), timeline); } @@ -212,4 +223,5 @@ public Stream getAllRawFileSlices() { public HoodieTimeline getTimeline() { return timeline; } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java index 6369b876b91bf..9deba20d5c13c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLogFile.java @@ -44,6 +44,12 @@ public class HoodieLogFile implements Serializable { private final String pathStr; private long fileLen; + public HoodieLogFile(HoodieLogFile logFile) { + this.fileStatus = logFile.fileStatus; + this.pathStr = logFile.pathStr; + this.fileLen = logFile.fileLen; + } + public HoodieLogFile(FileStatus fileStatus) { this.fileStatus = fileStatus; this.pathStr = fileStatus.getPath().toString(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionExternalDataFiles.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionExternalDataFiles.java new file mode 100644 index 0000000000000..fa1b72fe2ee55 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionExternalDataFiles.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import java.io.Serializable; +import java.util.List; + +/** + * Bootstrap Index information for a single partition. + */ +public class HoodiePartitionExternalDataFiles implements Serializable { + + private final String externalBasePath; + private final String externalPartitionPath; + private final String hoodiePartitionPath; + private final List externalFileIdMappings; + + public HoodiePartitionExternalDataFiles(String externalBasePath, String externalPartitionPath, + String hoodiePartitionPath, + List externalFileIdMappings) { + this.externalBasePath = externalBasePath; + this.externalPartitionPath = externalPartitionPath; + this.hoodiePartitionPath = hoodiePartitionPath; + this.externalFileIdMappings = externalFileIdMappings; + } + + public String getExternalBasePath() { + return externalBasePath; + } + + public String getExternalPartitionPath() { + return externalPartitionPath; + } + + public String getHoodiePartitionPath() { + return hoodiePartitionPath; + } + + public List getExternalFileIdMappings() { + return externalFileIdMappings; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 489b204d4e13a..e36ba7395763e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -70,6 +70,9 @@ public class HoodieTableMetaClient implements Serializable { public static String METAFOLDER_NAME = ".hoodie"; public static String TEMPFOLDER_NAME = METAFOLDER_NAME + File.separator + ".temp"; public static String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + File.separator + ".aux"; + public static String BOOTSTRAP_META_ROOT = METAFOLDER_NAME + File.separator + ".bootstrap"; + public static String BOOTSTRAP_META_IDX = BOOTSTRAP_META_ROOT + File.separator + ".idx"; + public static final String MARKER_EXTN = ".marker"; private String basePath; @@ -165,6 +168,7 @@ public String getMetaPath() { return metaPath; } + /** * @return Temp Folder path */ @@ -189,6 +193,20 @@ public String getMetaAuxiliaryPath() { return basePath + File.separator + AUXILIARYFOLDER_NAME; } + /** + * @return Root Directory containing Bootstrap folder + */ + public String getMetaBootstrapPath() { + return basePath + File.separator + BOOTSTRAP_META_ROOT; + } + + /** + * @return Root Directory containing Bootstrap Index folder + */ + public String getMetaBootstrapIndexPath() { + return basePath + File.separator + BOOTSTRAP_META_IDX; + } + /** * @return path where archived timeline is stored */ @@ -357,6 +375,12 @@ public static HoodieTableMetaClient initDatasetAndGetMetaClient(Configuration ha fs.mkdirs(auxiliaryFolder); } + // This should create both bootstrap data and bootstrap index folders + final Path bootstrapIdxFolder = new Path(basePath, HoodieTableMetaClient.BOOTSTRAP_META_IDX); + if (!fs.exists(bootstrapIdxFolder)) { + fs.mkdirs(bootstrapIdxFolder); + } + HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props); // We should not use fs.getConf as this might be different from the original configuration // used to create the fs in unit tests diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java index c73d19d1dbd67..bdac19d16bd79 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTimeline.java @@ -78,6 +78,8 @@ public interface HoodieTimeline extends Serializable { String INVALID_INSTANT_TS = "0"; + String BOOTSTRAP_INSTANT_TS = "00000000000001"; + /** * Filter this timeline to just include the in-flights. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 81d042b3a4e99..9800062bb9b78 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -227,6 +227,7 @@ public void createNewInstant(HoodieInstant instant) { } public void saveAsComplete(HoodieInstant instant, Option data) { + System.out.println("Marking instant complete " + instant); LOG.info("Marking instant complete " + instant); Preconditions.checkArgument(instant.isInflight(), "Could not mark an already completed instant as complete again " + instant); @@ -399,6 +400,8 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, createFileInMetaPath(fromInstant.getFileName(), data, false); Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName()); Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName()); + LOG.info("Renaming " + fromInstantPath + " to " + toInstantPath); + System.out.println("Renaming " + fromInstantPath + " to " + toInstantPath); boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath); if (!success) { throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath); @@ -410,7 +413,9 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, fromInstant.getFileName()))); // Use Write Once to create Target File createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data); - LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName())); + LOG.info("Created new file for toInstant =" + new Path(metaClient.getMetaPath(), toInstant.getFileName())); + System.out.println("Created new file for toInstant =" + new Path(metaClient.getMetaPath(), toInstant.getFileName())); + } } catch (IOException e) { throw new HoodieIOException("Could not complete " + fromInstant, e); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java index fab7ad1df7adf..cce85a202bd59 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/DataFileDTO.java @@ -35,6 +35,8 @@ public class DataFileDTO { private String fullPath; @JsonProperty("fileLen") private long fileLen; + @JsonProperty("externalDataFile") + private String externalDataFile; public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) { if (null == dto) { @@ -48,6 +50,7 @@ public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) { dataFile = new HoodieDataFile(dto.fullPath); dataFile.setFileLen(dto.fileLen); } + dataFile.setExternalDataFile(dto.externalDataFile); return dataFile; } @@ -60,6 +63,7 @@ public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) { dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus()); dto.fullPath = dataFile.getPath(); dto.fileLen = dataFile.getFileLen(); + dto.externalDataFile = dataFile.getExternalDataFile().orElse(null); return dto; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 9beeea0c55e6f..ed08d46e7dae9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -18,12 +18,16 @@ package org.apache.hudi.common.table.view; +import org.apache.hudi.common.SerializableConfiguration; +import org.apache.hudi.common.consolidated.CompositeMapFile; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.ExternalDataFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieDataFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodiePartitionExternalDataFiles; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTimeline; import org.apache.hudi.common.table.SyncableFileSystemView; @@ -32,6 +36,7 @@ import org.apache.hudi.common.util.FSUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.SerializationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -57,6 +62,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.HoodieTimeline.BOOTSTRAP_INSTANT_TS; + /** * Common thread-safe implementation for multiple TableFileSystemView Implementations. Provides uniform handling of (a) * Loading file-system views from underlying file-system (b) Pending compaction operations and changing file-system @@ -82,6 +89,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV private final ReadLock readLock = globalLock.readLock(); private final WriteLock writeLock = globalLock.writeLock(); + private CompositeMapFile bootstrapMetadataStorage; + private transient CompositeMapFile.RandomAccessReader bootstrapMetadataReader; + private String getPartitionPathFromFilePath(String fullPath) { return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent()); } @@ -92,10 +102,21 @@ private String getPartitionPathFromFilePath(String fullPath) { protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; refreshTimeline(visibleActiveTimeline); - + bootstrapMetadataStorage = new CompositeMapFile("bootstrap", + new SerializableConfiguration(metaClient.getHadoopConf()), metaClient.getConsistencyGuardConfig(), + metaClient.getMetaBootstrapPath(), metaClient.getMetaBootstrapIndexPath()); + if (bootstrapMetadataStorage.isIndexExists(BOOTSTRAP_INSTANT_TS)) { + try { + bootstrapMetadataReader = bootstrapMetadataStorage.getRandomAccessReader(BOOTSTRAP_INSTANT_TS); + } catch (IOException e) { + LOG.error("Unable to open random access reader", e); + throw new HoodieIOException(e.getMessage(), e); + } + } // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); + resetExternalDataFileMapping(Stream.empty()); } /** @@ -119,6 +140,21 @@ protected List addFilesToView(FileStatus[] statuses) { fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).entrySet().forEach(entry -> { String partition = entry.getKey(); if (!isPartitionAvailableInStore(partition)) { + if (null != bootstrapMetadataReader) { + byte[] serializedPartitionDataFiles = bootstrapMetadataReader.get(partition); + if (null != serializedPartitionDataFiles) { + HoodiePartitionExternalDataFiles partitionExternalDataFiles = + SerializationUtils.deserialize(serializedPartitionDataFiles); + LOG.info("Found index bootstrapped files in the partition=" + partition); + Preconditions.checkArgument(partition.equals(partitionExternalDataFiles.getHoodiePartitionPath())); + addExternalDataFileMapping(partitionExternalDataFiles.getExternalFileIdMappings().stream().map(e -> { + return new ExternalDataFile(new HoodieFileGroupId(partitionExternalDataFiles.getHoodiePartitionPath(), + FSUtils.getFileId(e.getHoodieFileName())), + new Path(FSUtils.getPartitionPath(partitionExternalDataFiles.getExternalBasePath(), + partitionExternalDataFiles.getExternalPartitionPath()), e.getExternalFileName()).toString()); + })); + } + } storePartitionView(partition, entry.getValue()); } }); @@ -164,6 +200,7 @@ protected List buildFileGroups(Stream dataFileS if (logFiles.containsKey(pair)) { logFiles.get(pair).forEach(group::addLogFile); } + if (addPendingCompactionFileSlice) { Option> pendingCompaction = getPendingCompactionOperationWithInstant(group.getFileGroupId()); @@ -313,6 +350,41 @@ protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) { return fileSlice; } + protected HoodieFileGroup addExternalDataFileIfPresent(HoodieFileGroup fileGroup) { + boolean hasExternalDataFile = fileGroup.getAllFileSlices() + .anyMatch(fs -> fs.getBaseInstantTime().equals(BOOTSTRAP_INSTANT_TS)); + if (hasExternalDataFile) { + HoodieFileGroup newFileGroup = new HoodieFileGroup(fileGroup); + newFileGroup.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals(BOOTSTRAP_INSTANT_TS)) + .forEach(fs -> fs.setDataFile( + addExternalDataFileIfPresent(fs.getFileGroupId(), fs.getDataFile().get()))); + return newFileGroup; + } + return fileGroup; + } + + protected FileSlice addExternalDataFileIfPresent(FileSlice fileSlice) { + if (fileSlice.getBaseInstantTime().equals(BOOTSTRAP_INSTANT_TS)) { + FileSlice copy = new FileSlice(fileSlice); + copy.getDataFile().ifPresent(dataFile -> { + Option edf = getExternalDataFile(copy.getFileGroupId()); + edf.ifPresent(e -> dataFile.setExternalDataFile(e.getExternalDataFile())); + }); + return copy; + } + return fileSlice; + } + + protected HoodieDataFile addExternalDataFileIfPresent(HoodieFileGroupId fileGroupId, HoodieDataFile dataFile) { + if (dataFile.getCommitTime().equals(BOOTSTRAP_INSTANT_TS)) { + HoodieDataFile copy = new HoodieDataFile(dataFile); + Option edf = getExternalDataFile(fileGroupId); + edf.ifPresent(e -> copy.setExternalDataFile(e.getExternalDataFile())); + return copy; + } + return dataFile; + } + @Override public final Stream> getPendingCompactionOperations() { try { @@ -329,7 +401,8 @@ public final Stream getLatestDataFiles(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestDataFiles(partitionPath); + return fetchLatestDataFiles(partitionPath) + .map(df -> addExternalDataFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -356,7 +429,8 @@ public final Stream getLatestDataFilesBeforeOrOn(String partitio .filter(dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime, HoodieTimeline.LESSER_OR_EQUAL)) .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst())) - .filter(Option::isPresent).map(Option::get); + .filter(Option::isPresent).map(Option::get) + .map(df -> addExternalDataFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -371,7 +445,8 @@ public final Option getDataFileOn(String partitionStr, String in return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllDataFiles() .filter( dataFile -> HoodieTimeline.compareTimestamps(dataFile.getCommitTime(), instantTime, HoodieTimeline.EQUAL)) - .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst().orElse(null)); + .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst().orElse(null)) + .map(df -> addExternalDataFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } finally { readLock.unlock(); } @@ -385,7 +460,8 @@ public final Option getLatestDataFile(String partitionStr, Strin readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestDataFile(partitionPath, fileId); + return fetchLatestDataFile(partitionPath, fileId) + .map(df -> addExternalDataFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } finally { readLock.unlock(); } @@ -396,10 +472,10 @@ public final Stream getLatestDataFilesInRange(List commi try { readLock.lock(); return fetchAllStoredFileGroups().map(fileGroup -> { - return Option.fromJavaOptional( + return Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional( fileGroup.getAllDataFiles().filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime()) - && !isDataFileDueToPendingCompaction(dataFile)).findFirst()); - }).filter(Option::isPresent).map(Option::get); + && !isDataFileDueToPendingCompaction(dataFile)).findFirst())); + }).filter(p -> p.getValue().isPresent()).map(p -> addExternalDataFileIfPresent(p.getKey(), p.getValue().get())); } finally { readLock.unlock(); } @@ -413,7 +489,8 @@ public final Stream getAllDataFiles(String partitionStr) { ensurePartitionLoadedCorrectly(partitionPath); return fetchAllDataFiles(partitionPath) .filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime())) - .filter(df -> !isDataFileDueToPendingCompaction(df)); + .filter(df -> !isDataFileDueToPendingCompaction(df)) + .map(df -> addExternalDataFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); } @@ -425,7 +502,8 @@ public final Stream getLatestFileSlices(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs)); + return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs)) + .map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -440,7 +518,7 @@ public final Option getLatestFileSlice(String partitionStr, String fi String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); Option fs = fetchLatestFileSlice(partitionPath, fileId); - return fs.map(f -> filterDataFileAfterPendingCompaction(f)); + return fs.map(f -> filterDataFileAfterPendingCompaction(f)).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -462,7 +540,7 @@ public final Stream getLatestUnCompactedFileSlices(String partitionSt return fileGroup.getLatestFileSliceBefore(compactionInstantTime); } return Option.of(fileSlice); - }).map(Option::get); + }).map(Option::get).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -477,9 +555,11 @@ public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr ensurePartitionLoadedCorrectly(partitionPath); Stream fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime); if (includeFileSlicesInPendingCompaction) { - return fileSliceStream.map(fs -> filterDataFileAfterPendingCompaction(fs)); + return fileSliceStream.map(fs -> filterDataFileAfterPendingCompaction(fs)) + .map(this::addExternalDataFileIfPresent); } else { - return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId())); + return fileSliceStream.filter(fs -> !isPendingCompactionScheduledForFileId(fs.getFileGroupId())) + .map(this::addExternalDataFileIfPresent); } } finally { readLock.unlock(); @@ -499,7 +579,7 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get())); } return fileSlice; - }).filter(Option::isPresent).map(Option::get); + }).filter(Option::isPresent).map(Option::get).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -509,7 +589,7 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit public final Stream getLatestFileSliceInRange(List commitsToReturn) { try { readLock.lock(); - return fetchLatestFileSliceInRange(commitsToReturn); + return fetchLatestFileSliceInRange(commitsToReturn).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -521,7 +601,7 @@ public final Stream getAllFileSlices(String partitionStr) { readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllFileSlices(partition); + return fetchAllFileSlices(partition).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -543,7 +623,7 @@ public final Stream getAllFileGroups(String partitionStr) { // in other places. String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllStoredFileGroups(partition); + return fetchAllStoredFileGroups(partition).map(this::addExternalDataFileIfPresent); } finally { readLock.unlock(); } @@ -593,6 +673,48 @@ protected abstract Option> getPendingCompactio */ abstract Stream> fetchPendingCompactionOperations(); + /** + * Check if there is an external data file present for this file. + * + * @param fgId File-Group Id + * @return true if there is a pending compaction, false otherwise + */ + protected abstract boolean isExternalDataFilePresentForFileId(HoodieFileGroupId fgId); + + /** + * Resets the external data file stream and overwrite with the new list. + * + * @param externalDataFileStream External Data File Stream + */ + abstract void resetExternalDataFileMapping(Stream externalDataFileStream); + + /** + * Add external data file stream to store. + * + * @param externalDataFileStream External Data File Stream to be added + */ + abstract void addExternalDataFileMapping(Stream externalDataFileStream); + + /** + * Remove external data file stream from store. + * + * @param externalDataFileStream External Data File Stream to be removed + */ + abstract void removeExternalDataFileMapping(Stream externalDataFileStream); + + /** + * Return pending compaction operation for a file-group. + * + * @param fileGroupId File-Group Id + */ + protected abstract Option getExternalDataFile(HoodieFileGroupId fileGroupId); + + /** + * Fetch all external data files. + */ + abstract Stream fetchExternalDataFiles(); + + /** * Checks if partition is pre-loaded and available in store. * @@ -657,16 +779,20 @@ Stream fetchLatestDataFiles(final String partitionPath) { .map(Option::get); } - protected Option getLatestDataFile(HoodieFileGroup fileGroup) { + protected final Option getLatestDataFile(HoodieFileGroup fileGroup) { return Option - .fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst()); + .fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)) + .findFirst().map(df -> addExternalDataFileIfPresent(fileGroup.getFileGroupId(), df))); } /** * Default implementation for fetching latest data-files across all partitions. */ - Stream fetchLatestDataFiles() { - return fetchAllStoredFileGroups().map(this::getLatestDataFile).filter(Option::isPresent).map(Option::get); + final Stream fetchLatestDataFiles() { + return fetchAllStoredFileGroups() + .map(fg -> Pair.of(fg.getFileGroupId(), getLatestDataFile(fg))) + .filter(p -> p.getValue().isPresent()) + .map(p -> addExternalDataFileIfPresent(p.getKey(), p.getValue().get())); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java index b72696a4ed7c2..0df94ab06c57f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java @@ -42,6 +42,8 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem"; public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = "hoodie.filesystem.view.spillable.compaction.mem.fraction"; + public static final String FILESYSTEM_VIEW_EXTERNAL_DATA_FILE_FRACTION = + "hoodie.filesystem.view.spillable.external.data.file.mem.fraction"; private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path"; public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; @@ -54,6 +56,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/"; private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01; + private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05; private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB public static FileSystemViewStorageConfig.Builder newBuilder() { @@ -82,7 +85,7 @@ public Integer getRemoteViewServerPort() { public long getMaxMemoryForFileGroupMap() { long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM)); - return totalMemory - getMaxMemoryForPendingCompaction(); + return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForExternalDataFile(); } public long getMaxMemoryForPendingCompaction() { @@ -93,6 +96,14 @@ public long getMaxMemoryForPendingCompaction() { return reservedForPendingComaction; } + public long getMaxMemoryForExternalDataFile() { + long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM)); + long reservedForExternalDataFile = + new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_EXTERNAL_DATA_FILE_FRACTION))) + .longValue(); + return reservedForExternalDataFile; + } + public String getBaseStoreDir() { return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR); } @@ -162,6 +173,11 @@ public Builder withMemFractionForPendingCompaction(Double memFractionForPendingC return this; } + public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) { + props.setProperty(FILESYSTEM_VIEW_EXTERNAL_DATA_FILE_FRACTION, memFractionForExternalDataFile.toString()); + return this; + } + public Builder withBaseStoreDir(String baseStorePath) { props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath); return this; @@ -190,6 +206,8 @@ public FileSystemViewStorageConfig build() { DEFAULT_MAX_MEMORY_FOR_VIEW.toString()); setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION), FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString()); + setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_EXTERNAL_DATA_FILE_FRACTION), + FILESYSTEM_VIEW_EXTERNAL_DATA_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString()); setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 34dadb0fec3e8..d2d53e5601636 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.ExternalDataFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -58,6 +59,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem */ protected Map> fgIdToPendingCompaction; + /** + * PartitionPath + File-Id to external Data File (Index Only bootstrapped). + */ + protected Map fgIdToExternalDataFile; + /** * Flag to determine if closed. */ @@ -93,6 +99,7 @@ public void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveT protected void resetViewState() { this.fgIdToPendingCompaction = null; this.partitionToFileGroupsMap = null; + this.fgIdToExternalDataFile = null; } protected Map> createPartitionToFileGroups() { @@ -104,6 +111,11 @@ protected Map> createFileId return fileIdToPendingCompaction; } + protected Map createFileIdToExternalDataFileMap( + Map fileGroupIdExternalDataFileMap) { + return fileGroupIdExternalDataFileMap; + } + /** * Create a file system view, as of the given timeline, with the provided file statuses. */ @@ -181,6 +193,48 @@ Stream> fetchPendingCompactionOperations() { } + @Override + protected boolean isExternalDataFilePresentForFileId(HoodieFileGroupId fgId) { + return fgIdToExternalDataFile.containsKey(fgId); + } + + @Override + void resetExternalDataFileMapping(Stream externalDataFileStream) { + // Build fileId to External Data File + this.fgIdToExternalDataFile = createFileIdToExternalDataFileMap(externalDataFileStream + .collect(Collectors.toMap(ExternalDataFile::getFileGroupId, x -> x))); + } + + @Override + void addExternalDataFileMapping(Stream externalDataFileStream) { + externalDataFileStream.forEach(externalDataFile -> { + Preconditions.checkArgument(!fgIdToExternalDataFile.containsKey(externalDataFile.getFileGroupId()), + "Duplicate FileGroupId found in external data file mapping. FgId :" + + externalDataFile.getFileGroupId()); + fgIdToExternalDataFile.put(externalDataFile.getFileGroupId(), externalDataFile); + }); + } + + @Override + void removeExternalDataFileMapping(Stream externalDataFileStream) { + externalDataFileStream.forEach(externalDataFile -> { + Preconditions.checkArgument(fgIdToExternalDataFile.containsKey(externalDataFile.getFileGroupId()), + "Trying to remove a FileGroupId which is not found in external data file mapping. FgId :" + + externalDataFile.getFileGroupId()); + fgIdToExternalDataFile.remove(externalDataFile.getFileGroupId()); + }); + } + + @Override + protected Option getExternalDataFile(HoodieFileGroupId fileGroupId) { + return Option.ofNullable(fgIdToExternalDataFile.get(fileGroupId)); + } + + @Override + Stream fetchExternalDataFiles() { + return fgIdToExternalDataFile.values().stream(); + } + @Override protected Option> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) { return Option.ofNullable(fgIdToPendingCompaction.get(fgId)); @@ -210,6 +264,7 @@ public void close() { super.reset(); partitionToFileGroupsMap = null; fgIdToPendingCompaction = null; + fgIdToExternalDataFile = null; } public boolean isClosed() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index 2502cc154778f..40fa87c5239a5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.ExternalDataFile; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieDataFile; import org.apache.hudi.common.model.HoodieFileGroup; @@ -265,6 +266,62 @@ Stream> fetchPendingCompactionOperations() { .map(Pair::getValue); } + @Override + protected boolean isExternalDataFilePresentForFileId(HoodieFileGroupId fgId) { + return getExternalDataFile(fgId).isPresent(); + } + + @Override + void resetExternalDataFileMapping(Stream externalDataFileStream) { + rocksDB.writeBatch(batch -> { + externalDataFileStream.forEach(externalDataFile -> { + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForExternalDataFile(), + schemaHelper.getKeyForExternalDataFile(externalDataFile.getFileGroupId()), externalDataFile); + }); + LOG.info("Initializing external data file mapping. Count=" + batch.count()); + }); + } + + @Override + void addExternalDataFileMapping(Stream externalDataFileStream) { + rocksDB.writeBatch(batch -> { + externalDataFileStream.forEach(externalDataFile -> { + Preconditions.checkArgument(!isExternalDataFilePresentForFileId(externalDataFile.getFileGroupId()), + "Duplicate FileGroupId found in external data file. FgId :" + externalDataFile.getFileGroupId()); + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForExternalDataFile(), + schemaHelper.getKeyForExternalDataFile(externalDataFile.getFileGroupId()), externalDataFile); + }); + }); + } + + @Override + void removeExternalDataFileMapping(Stream externalDataFileStream) { + rocksDB.writeBatch(batch -> { + externalDataFileStream.forEach(externalDataFile -> { + Preconditions.checkArgument( + getExternalDataFile(externalDataFile.getFileGroupId()) != null, + "Trying to remove a FileGroupId which is not found in external data file mapping. FgId :" + + externalDataFile.getFileGroupId()); + rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForExternalDataFile(), + schemaHelper.getKeyForExternalDataFile(externalDataFile.getFileGroupId())); + }); + }); + } + + @Override + protected Option getExternalDataFile(HoodieFileGroupId fileGroupId) { + String lookupKey = schemaHelper.getKeyForExternalDataFile(fileGroupId); + ExternalDataFile externalDataFile = + rocksDB.get(schemaHelper.getColFamilyForPendingCompaction(), lookupKey); + return Option.ofNullable(externalDataFile); + } + + @Override + Stream fetchExternalDataFiles() { + return rocksDB.prefixSearch(schemaHelper.getColFamilyForExternalDataFile(), "") + .map(Pair::getValue); + } + @Override Stream fetchAllDataFiles(String partitionPath) { return rocksDB.prefixSearch(schemaHelper.getColFamilyForView(), diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index 928e060bb7cb9..2fa8d8344b255 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.table.view; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.ExternalDataFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -46,6 +47,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView { private final long maxMemoryForFileGroupMap; private final long maxMemoryForPendingCompaction; + private final long maxMemoryForExternalDataFile; private final String baseStoreDir; public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, @@ -53,6 +55,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT super(config.isIncrementalTimelineSyncEnabled()); this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap(); this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction(); + this.maxMemoryForExternalDataFile = config.getMaxMemoryForExternalDataFile(); this.baseStoreDir = config.getBaseStoreDir(); init(metaClient, visibleActiveTimeline); } @@ -91,6 +94,21 @@ protected Map> createFileId } } + protected Map createFileIdToExternalDataFileMap( + Map fileGroupIdExternalDataFileMap) { + try { + LOG.info("Creating External Data File Map using external spillable Map. Max Mem=" + maxMemoryForExternalDataFile + + ", BaseDir=" + baseStoreDir); + new File(baseStoreDir).mkdirs(); + Map pendingMap = new ExternalSpillableMap<>( + maxMemoryForExternalDataFile, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>()); + pendingMap.putAll(fileGroupIdExternalDataFileMap); + return pendingMap; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + public Stream getAllFileGroups() { return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream() .flatMap(fg -> ((List) fg).stream()); @@ -99,7 +117,11 @@ public Stream getAllFileGroups() { @Override Stream> fetchPendingCompactionOperations() { return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream(); + } + @Override + Stream fetchExternalDataFiles() { + return ((ExternalSpillableMap) fgIdToExternalDataFile).valueStream(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java index c5261efb4738d..8fe6a4676926b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java @@ -33,6 +33,9 @@ import java.io.IOException; import java.util.List; +/** + * Utility functions for Cleaner. + */ public class CleanerUtils { public static final Integer CLEAN_METADATA_VERSION_1 = CleanV1MigrationHandler.VERSION; public static final Integer CLEAN_METADATA_VERSION_2 = CleanV2MigrationHandler.VERSION; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/FSUtils.java index d9161e5129538..b16c753a6427a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/FSUtils.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.util; +import org.apache.hudi.common.SerializableConfiguration; +import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; @@ -46,13 +48,17 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Map.Entry; +import java.util.OptionalInt; import java.util.UUID; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -198,6 +204,38 @@ public static List getAllFoldersWithPartitionMetaFile(FileSystem fs, Str return partitions; } + public static List>> getAllLeafFoldersWithFiles(FileSystem fs, String basePathStr, + PathFilter filePathFilter) throws IOException { + final Path basePath = new Path(basePathStr); + final Map> levelToPartitions = new HashMap<>(); + final Map> partitionToFiles = new HashMap<>(); + System.out.print("BASE PAth is " + basePathStr); + processFiles(fs, basePathStr, (status) -> { + if (status.isFile() && filePathFilter.accept(status.getPath())) { + System.out.print("Processing File :" + status.getPath()); + String relativePath = FSUtils.getRelativePartitionPath(basePath, status.getPath().getParent()); + List files = partitionToFiles.get(relativePath); + if (null == files) { + Integer level = (int) relativePath.chars().filter(ch -> ch == '/').count(); + List dirs = levelToPartitions.get(level); + if (null == dirs) { + dirs = new ArrayList<>(); + levelToPartitions.put(level, dirs); + } + dirs.add(relativePath); + files = new ArrayList<>(); + partitionToFiles.put(relativePath, files); + } + files.add(status.getPath().getName()); + } + return true; + }, true); + OptionalInt maxLevelOpt = levelToPartitions.keySet().stream().mapToInt(x -> x).max(); + int maxLevel = maxLevelOpt.orElse(-1); + return maxLevel >= 0 ? levelToPartitions.get(maxLevel).stream() + .map(d -> Pair.of(d, partitionToFiles.get(d))).collect(Collectors.toList()) : new ArrayList<>(); + } + public static final List getAllDataFilesForMarkers(FileSystem fs, String basePath, String instantTs, String markerDir) throws IOException { List dataFiles = new LinkedList<>(); @@ -550,4 +588,22 @@ public static boolean isGCSInputStream(FSDataInputStream inputStream) { || inputStream.getWrappedStream().getClass().getCanonicalName() .equals("com.google.cloud.hadoop.fs.gcs.GoogleHadoopFSInputStream"); } + + /** + * Get the FS implementation for this table. + * @param path Path String + * @param hadoopConf Serializable Hadoop Configuration + * @param consistencyGuardConfig Consistency Guard Config + * @return HoodieWrapperFileSystem + */ + public static HoodieWrapperFileSystem getFs(String path, SerializableConfiguration hadoopConf, + ConsistencyGuardConfig consistencyGuardConfig) { + FileSystem fileSystem = FSUtils.getFs(path, hadoopConf.newCopy()); + Preconditions.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem), + "File System not expected to be that of HoodieWrapperFileSystem"); + return new HoodieWrapperFileSystem(fileSystem, + consistencyGuardConfig.isConsistencyCheckEnabled() + ? new FailSafeConsistencyGuard(fileSystem, consistencyGuardConfig) + : new NoOpConsistencyGuard()); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroUtils.java index 40f907ad82c62..cc073d03f6052 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieAvroUtils.java @@ -19,6 +19,8 @@ package org.apache.hudi.common.util; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.SchemaCompatabilityException; @@ -61,7 +63,9 @@ public class HoodieAvroUtils { private static final Schema METADATA_FIELD_SCHEMA = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING))); - private static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + public static final Schema RECORD_KEY_SCHEMA = initRecordKeySchema(); + + public static final Integer NUM_HUDI_METADATA_COLS = 5; /** * Convert a given avro record to bytes. @@ -239,4 +243,27 @@ public static String decompress(byte[] bytes) { throw new HoodieIOException("IOException while decompressing text", e); } } + + /** + * Generate a reader schema off the provided writeSchema, to just project out the provided columns. + */ + public static Schema generateProjectionSchema(Schema originalSchema, List fieldNames) { + Map schemaFieldsMap = originalSchema.getFields().stream() + .map(r -> Pair.of(r.name().toLowerCase(), r)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight)); + List projectedFields = new ArrayList<>(); + for (String fn : fieldNames) { + Schema.Field field = schemaFieldsMap.get(fn.toLowerCase()); + if (field == null) { + throw new HoodieException("Field " + fn + " not found in log schema. Query cannot proceed! " + + "Derived Schema Fields: " + new ArrayList<>(schemaFieldsMap.keySet())); + } else { + projectedFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue())); + } + } + + Schema projectedSchema = Schema.createRecord(originalSchema.getName(), originalSchema.getDoc(), + originalSchema.getNamespace(), originalSchema.isError()); + projectedSchema.setFields(projectedFields); + return projectedSchema; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index 574021b686f20..5c6e55a9bdec5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -45,16 +45,19 @@ public class RocksDBSchemaHelper { private final String colFamilyForView; private final String colFamilyForPendingCompaction; + private final String colFamilyForExternalDataFile; private final String colFamilyForStoredPartitions; public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { + this.colFamilyForExternalDataFile = "hudi_external_datafile_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_"); } public List getAllColumnFamilies() { - return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForStoredPartitions()); + return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForExternalDataFile(), + getColFamilyForStoredPartitions()); } public String getKeyForPartitionLookup(String partition) { @@ -65,6 +68,10 @@ public String getKeyForPendingCompactionLookup(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } + public String getKeyForExternalDataFile(HoodieFileGroupId fgId) { + return getPartitionFileIdBasedLookup(fgId); + } + public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) { return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(), slice.getBaseInstantTime()); @@ -111,6 +118,10 @@ public String getColFamilyForPendingCompaction() { return colFamilyForPendingCompaction; } + public String getColFamilyForExternalDataFile() { + return colFamilyForExternalDataFile; + } + public String getColFamilyForStoredPartitions() { return colFamilyForStoredPartitions; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanMetadataMigrator.java b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanMetadataMigrator.java index 536f148ca8413..71846263a7261 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanMetadataMigrator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanMetadataMigrator.java @@ -24,6 +24,9 @@ import java.util.Arrays; +/** + * Migrator for migrating clean metadata. + */ public class CleanMetadataMigrator extends MetadataMigrator { public CleanMetadataMigrator(HoodieTableMetaClient metaClient) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV1MigrationHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV1MigrationHandler.java index 998d4f04fffc3..133d94879e74b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV1MigrationHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV1MigrationHandler.java @@ -31,6 +31,9 @@ import java.util.Map; import java.util.stream.Collectors; +/** + * V1 Clean Metadata migrator. Keeps absolute file paths for files cleaned. + */ public class CleanV1MigrationHandler extends AbstractMigratorBase { public static final Integer VERSION = 1; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV2MigrationHandler.java b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV2MigrationHandler.java index 84896a031cb52..bfe28f7acd75f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV2MigrationHandler.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/versioning/clean/CleanV2MigrationHandler.java @@ -31,6 +31,9 @@ import java.util.Map; import java.util.stream.Collectors; +/** + * V2 Clean Metadata Migration handler. Uses relative paths. + */ public class CleanV2MigrationHandler extends AbstractMigratorBase { public static final Integer VERSION = 2; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineLayout.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineLayout.java index 386b974a951bd..786d34f6501f4 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineLayout.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineLayout.java @@ -29,6 +29,9 @@ import java.util.List; import java.util.stream.Collectors; +/** + * Timeline Layout Tests. + */ public class TestTimelineLayout { @Test diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/ExternalDataFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/ExternalDataFileSplit.java new file mode 100644 index 0000000000000..64ec85fa72b22 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/ExternalDataFileSplit.java @@ -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.hadoop; + +import org.apache.hadoop.mapred.FileSplit; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +public class ExternalDataFileSplit extends FileSplit { + + private String sourceFileFullPath; + + public ExternalDataFileSplit() { + super(); + } + + public ExternalDataFileSplit(FileSplit baseSplit, String sourceFileFullPath) + throws IOException { + super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); + this.sourceFileFullPath = sourceFileFullPath; + } + + public String getSourceFileFullPath() { + return sourceFileFullPath; + } + + private static void writeString(String str, DataOutput out) throws IOException { + byte[] bytes = str.getBytes(StandardCharsets.UTF_8); + out.writeInt(bytes.length); + out.write(bytes); + } + + private static String readString(DataInput in) throws IOException { + byte[] bytes = new byte[in.readInt()]; + in.readFully(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + writeString(sourceFileFullPath, out); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + sourceFileFullPath = readString(in); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithExternalDataFile.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithExternalDataFile.java new file mode 100644 index 0000000000000..29cb575fa8059 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/FileStatusWithExternalDataFile.java @@ -0,0 +1,39 @@ +/* + * 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.hadoop; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +public class FileStatusWithExternalDataFile extends FileStatus { + + private final String externalDataFilePath; + + public FileStatusWithExternalDataFile(FileStatus fileStatus, String externalDataFilePath) throws IOException { + super(fileStatus); + this.externalDataFilePath = externalDataFilePath; + } + + @Override + public Path getPath() { + return new PathWithExternalDataFile(super.getPath(), externalDataFilePath); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java new file mode 100644 index 0000000000000..5380131036b95 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnProjectionUtils.java @@ -0,0 +1,281 @@ +/* + * 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.hadoop; + +import org.apache.hudi.common.util.collection.Pair; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.io.IOConstants; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Utility funcitons copied from Hive ColumnProjectionUtils.java. + * Needed to copy as we see NoSuchMethod errors when directly using these APIs with/without Spark. Some of these + * methods are not available across hive versions. + */ +public class HoodieColumnProjectionUtils { + public static final Logger LOG = LoggerFactory.getLogger(ColumnProjectionUtils.class); + + public static final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids"; + /** + * the nested column path is the string from the root to the leaf + * e.g. + * c:struct_of (a:string,b:string). + * the column a's path is c.a and b's path is c.b + */ + public static final String READ_NESTED_COLUMN_PATH_CONF_STR = + "hive.io.file.readNestedColumn.paths"; + public static final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns"; + public static final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names"; + private static final String READ_COLUMN_IDS_CONF_STR_DEFAULT = ""; + private static final String READ_COLUMN_NAMES_CONF_STR_DEFAULT = ""; + private static final String READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT = ""; + private static final boolean READ_ALL_COLUMNS_DEFAULT = true; + private static final Joiner CSV_JOINER = Joiner.on(",").skipNulls(); + + /** + * Sets the READ_ALL_COLUMNS flag and removes any previously + * set column ids. + */ + public static void setReadAllColumns(Configuration conf) { + conf.setBoolean(READ_ALL_COLUMNS, true); + setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT); + setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + } + + /** + * Returns the READ_ALL_COLUMNS columns flag. + */ + public static boolean isReadAllColumns(Configuration conf) { + return conf.getBoolean(READ_ALL_COLUMNS, READ_ALL_COLUMNS_DEFAULT); + } + + /** + * Sets the READ_ALL_COLUMNS flag to false and overwrites column ids + * with the provided list. + */ + public static void setReadColumns(Configuration conf, List ids, List names) { + setReadColumnIDConf(conf, READ_COLUMN_IDS_CONF_STR_DEFAULT); + setReadColumnNamesConf(conf, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + appendReadColumns(conf, ids); + } + + /** + * Appends read columns' ids (start from zero). Once a column + * is included in the list, a underlying record reader of a columnar file format + * (e.g. RCFile and ORC) can know what columns are needed. + */ + public static void appendReadColumns(Configuration conf, List ids) { + String id = toReadColumnIDString(ids); + String old = conf.get(READ_COLUMN_IDS_CONF_STR, null); + String newConfStr = id; + if (old != null && !old.isEmpty()) { + newConfStr = newConfStr + StringUtils.COMMA_STR + old; + } + System.out.println("Read Column Ids=" + newConfStr); + setReadColumnIDConf(conf, newConfStr); + // Set READ_ALL_COLUMNS to false + conf.setBoolean(READ_ALL_COLUMNS, false); + } + + /** + * Appends read nested column's paths. Once a read nested column path + * is included in the list, a underlying record reader of a columnar file format + * (e.g. Parquet and ORC) can know what columns are needed. + */ + public static void appendNestedColumnPaths( + Configuration conf, + List paths) { + if (paths == null || paths.isEmpty()) { + return; + } + String pathsStr = StringUtils.join(StringUtils.COMMA_STR, + paths.toArray(new String[paths.size()])); + String old = conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, null); + String newConfStr = pathsStr; + if (old != null && !old.isEmpty()) { + newConfStr = newConfStr + StringUtils.COMMA_STR + old; + } + setReadNestedColumnPathConf(conf, newConfStr); + } + + + /** + * This method appends read column information to configuration to use for PPD. It is + * currently called with information from TSOP. Names come from TSOP input RowSchema, and + * IDs are the indexes inside the schema (which PPD assumes correspond to indexes inside the + * files to PPD in; something that would be invalid in many cases of schema evolution). + * @param conf Config to set values to. + * @param ids Column ids. + * @param names Column names. + */ + public static void appendReadColumns( + Configuration conf, List ids, List names, List groupPaths) { + if (ids.size() != names.size()) { + LOG.warn("Read column counts do not match: " + + ids.size() + " ids, " + names.size() + " names"); + } + appendReadColumns(conf, ids); + appendReadColumnNames(conf, names); + appendNestedColumnPaths(conf, groupPaths); + } + + public static void appendReadColumns( + StringBuilder readColumnsBuffer, StringBuilder readColumnNamesBuffer, List ids, + List names) { + CSV_JOINER.appendTo(readColumnsBuffer, ids); + CSV_JOINER.appendTo(readColumnNamesBuffer, names); + } + + /** + * Returns an array of column ids(start from zero) which is set in the given + * parameter conf. + */ + public static List getReadColumnIDs(Configuration conf) { + String skips = conf.get(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT); + String[] list = StringUtils.split(skips); + List result = new ArrayList(list.length); + for (String element : list) { + // it may contain duplicates, remove duplicates + Integer toAdd = Integer.parseInt(element); + if (!result.contains(toAdd)) { + result.add(toAdd); + } + // NOTE: some code uses this list to correlate with column names, and yet these lists may + // contain duplicates, which this call will remove and the other won't. As far as I can + // tell, no code will actually use these two methods together; all is good if the code + // gets the ID list without relying on this method. Or maybe it just works by magic. + } + return result; + } + + public static Set getNestedColumnPaths(Configuration conf) { + String skips = + conf.get(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT); + return new HashSet<>(Arrays.asList(StringUtils.split(skips))); + } + + public static String[] getReadColumnNames(Configuration conf) { + String colNames = conf.get(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + if (colNames != null && !colNames.isEmpty()) { + return colNames.split(","); + } + return new String[] {}; + } + + public static List getIOColumns(Configuration conf) { + String colNames = conf.get(IOConstants.COLUMNS, ""); + if (colNames != null && !colNames.isEmpty()) { + return Arrays.asList(colNames.split(",")); + } + return new ArrayList<>(); + } + + public static List getIOColumnTypes(Configuration conf) { + String colTypes = conf.get(IOConstants.COLUMNS_TYPES, ""); + if (colTypes != null && !colTypes.isEmpty()) { + return Arrays.asList(colTypes.split(",")); + } + return new ArrayList<>(); + } + + public static List> getIOColumnNameAndTypes(Configuration conf) { + List names = getIOColumns(conf); + List types = getIOColumnTypes(conf); + Preconditions.checkArgument(names.size() == types.size()); + return IntStream.range(0, names.size()).mapToObj(idx -> Pair.of(names.get(idx), types.get(idx))) + .collect(Collectors.toList()); + } + + public static void setIOColumnNameAndTypes(Configuration conf, List> colNamesAndTypes) { + String colNames = colNamesAndTypes.stream().map(e -> e.getKey()).collect(Collectors.joining(",")); + String colTypes = colNamesAndTypes.stream().map(e -> e.getValue()).collect(Collectors.joining(",")); + conf.set(IOConstants.COLUMNS, colNames); + conf.set(IOConstants.COLUMNS_TYPES, colTypes); + } + + private static void setReadColumnIDConf(Configuration conf, String id) { + if (id.trim().isEmpty()) { + conf.set(READ_COLUMN_IDS_CONF_STR, READ_COLUMN_IDS_CONF_STR_DEFAULT); + } else { + conf.set(READ_COLUMN_IDS_CONF_STR, id); + } + } + + private static void setReadColumnNamesConf(Configuration conf, String id) { + if (id.trim().isEmpty()) { + conf.set(READ_COLUMN_NAMES_CONF_STR, READ_COLUMN_NAMES_CONF_STR_DEFAULT); + } else { + conf.set(READ_COLUMN_NAMES_CONF_STR, id); + } + } + + private static void setReadNestedColumnPathConf( + Configuration conf, + String nestedColumnPaths) { + nestedColumnPaths = nestedColumnPaths.toLowerCase(); + if (nestedColumnPaths.trim().isEmpty()) { + conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, READ_NESTED_COLUMN_PATH_CONF_STR_DEFAULT); + } else { + conf.set(READ_NESTED_COLUMN_PATH_CONF_STR, nestedColumnPaths); + } + } + + private static void appendReadColumnNames(Configuration conf, List cols) { + String old = conf.get(READ_COLUMN_NAMES_CONF_STR, ""); + StringBuilder result = new StringBuilder(old); + boolean first = old.isEmpty(); + for (String col: cols) { + if (first) { + first = false; + } else { + result.append(','); + } + result.append(col); + } + System.out.println("Read Column Names=" + result); + conf.set(READ_COLUMN_NAMES_CONF_STR, result.toString()); + } + + private static String toReadColumnIDString(List ids) { + String id = ""; + for (int i = 0; i < ids.size(); i++) { + if (i == 0) { + id = id + ids.get(i); + } else { + id = id + StringUtils.COMMA_STR + ids.get(i); + } + } + return id; + } + +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnStichingRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnStichingRecordReader.java new file mode 100644 index 0000000000000..6bb10064fd228 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieColumnStichingRecordReader.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.hadoop; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; + +import java.io.IOException; +import java.util.Arrays; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class HoodieColumnStichingRecordReader implements RecordReader { + + private final RecordReader leftColsRecordReader; + private final RecordReader rightColsRecordReader; + + private final ArrayWritable values; + + public HoodieColumnStichingRecordReader(RecordReader left, + RecordReader right) { + this.leftColsRecordReader = left; + this.rightColsRecordReader = right; + + ArrayWritable leftW = leftColsRecordReader.createValue(); + ArrayWritable rightW = rightColsRecordReader.createValue(); + int numColumns = leftW.get().length + rightW.get().length; + if (rightW.getValueClass() != null) { + values = new ArrayWritable(rightW.getValueClass(), new Writable[numColumns]); + } else { + String[] vals = IntStream.range(0, numColumns).mapToObj(idx -> "").collect(Collectors.toList()) + .toArray(new String[0]); + values = new ArrayWritable(vals); + } + } + + @Override + public boolean next(NullWritable key, ArrayWritable value) throws IOException { + ArrayWritable left = leftColsRecordReader.createValue(); + ArrayWritable right = rightColsRecordReader.createValue(); + + boolean hasMoreOnLeft = leftColsRecordReader.next(leftColsRecordReader.createKey(), left); + boolean hasMoreOnRight = rightColsRecordReader.next(rightColsRecordReader.createKey(), right); + Preconditions.checkArgument(hasMoreOnLeft == hasMoreOnRight); + int i = 0; + for (;i < left.get().length; i++) { + value.get()[i] = left.get()[i]; + } + + for (int j = 0; j < right.get().length; j++) { + value.get()[i++] = right.get()[j]; + } + System.out.println("Left Record :" + Arrays.asList(left.get())); + System.out.println("Right Record :" + Arrays.asList(right.get())); + System.out.println("Stiched Record :" + Arrays.asList(value.get())); + return hasMoreOnLeft; + } + + @Override + public NullWritable createKey() { + return leftColsRecordReader.createKey(); + } + + @Override + public ArrayWritable createValue() { + return values; + } + + @Override + public long getPos() throws IOException { + return leftColsRecordReader.getPos(); + } + + @Override + public void close() throws IOException { + leftColsRecordReader.close(); + rightColsRecordReader.close(); + } + + @Override + public float getProgress() throws IOException { + return leftColsRecordReader.getProgress(); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java index 5d6e52c41abee..40b34bb82d267 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java @@ -25,6 +25,8 @@ import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.HoodieAvroUtils; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.DatasetNotFoundException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.InvalidDatasetException; @@ -33,10 +35,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RecordReader; @@ -47,10 +51,17 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.hudi.hadoop.HoodieColumnProjectionUtils.READ_NESTED_COLUMN_PATH_CONF_STR; + /** * HoodieInputFormat which understands the Hoodie File Structure and filters files based on the Hoodie Mode. If paths @@ -63,13 +74,16 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement private static final Logger LOG = LogManager.getLogger(HoodieParquetInputFormat.class); protected Configuration conf; + public boolean firstTime = true; @Override public FileStatus[] listStatus(JobConf job) throws IOException { + System.out.println("HoodieParquetInputFormat listing Status"); // Get all the file status from FileInputFormat and then do the filter FileStatus[] fileStatuses = super.listStatus(job); Map> groupedFileStatus = groupFileStatus(fileStatuses); LOG.info("Found a total of " + groupedFileStatus.size() + " groups"); + System.out.println("Found a total of " + groupedFileStatus.size() + " groups"); List returns = new ArrayList<>(); for (Map.Entry> entry : groupedFileStatus.entrySet()) { HoodieTableMetaClient metadata = entry.getKey(); @@ -103,7 +117,7 @@ public FileStatus[] listStatus(JobConf job) throws IOException { for (HoodieDataFile filteredFile : filteredFiles) { LOG.info("Processing incremental hoodie file - " + filteredFile.getPath()); filteredFile = checkFileStatus(filteredFile); - returns.add(filteredFile.getFileStatus()); + returns.add(getFileStatus(filteredFile)); } LOG.info("Total paths to process after hoodie incremental filter " + filteredFiles.size()); } else { @@ -115,12 +129,23 @@ public FileStatus[] listStatus(JobConf job) throws IOException { LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); } filteredFile = checkFileStatus(filteredFile); - returns.add(filteredFile.getFileStatus()); + returns.add(getFileStatus(filteredFile)); } } } return returns.toArray(new FileStatus[returns.size()]); + } + private static FileStatus getFileStatus(HoodieDataFile dataFile) throws IOException { + if (dataFile.getExternalDataFile().isPresent()) { + if (dataFile.getFileStatus() instanceof LocatedFileStatus) { + return new LocatedFileStatusWithExternalDataFile((LocatedFileStatus)dataFile.getFileStatus(), + dataFile.getExternalDataFile().get()); + } else { + return new FileStatusWithExternalDataFile(dataFile.getFileStatus(), dataFile.getExternalDataFile().get()); + } + } + return dataFile.getFileStatus(); } /** @@ -134,7 +159,7 @@ private HoodieDataFile checkFileStatus(HoodieDataFile dataFile) throws IOExcepti if (dataFile.getFileSize() == 0) { FileSystem fs = dataPath.getFileSystem(conf); LOG.info("Refreshing file status " + dataFile.getPath()); - return new HoodieDataFile(fs.getFileStatus(dataPath)); + return new HoodieDataFile(fs.getFileStatus(dataPath), dataFile.getExternalDataFile().orElse(null)); } return dataFile; } catch (IOException e) { @@ -196,6 +221,93 @@ public RecordReader getRecordReader(final InputSpli // ParquetInputFormat.setFilterPredicate(job, predicate); // clearOutExistingPredicate(job); // } + if (split instanceof ExternalDataFileSplit) { + ExternalDataFileSplit eSplit = (ExternalDataFileSplit)split; + System.out.println("ExternalDataFileSplit is " + eSplit); + String[] rawColNames = HoodieColumnProjectionUtils.getReadColumnNames(job); + List rawColIds = HoodieColumnProjectionUtils.getReadColumnIDs(job); + List> colsWithIndex = + IntStream.range(0, rawColIds.size()).mapToObj(idx -> Pair.of(rawColIds.get(idx), rawColNames[idx])) + .collect(Collectors.toList()); + + List> hoodieColsProjected = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() < HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .collect(Collectors.toList()); + // This always matches hive table description + List> colNameWithTypes = HoodieColumnProjectionUtils.getIOColumnNameAndTypes(job); + List> hoodieColNamesWithTypes = colNameWithTypes.subList(0, 5); + List> otherColNamesWithTypes = + colNameWithTypes.subList(5, colNameWithTypes.size()); + JobConf jobConf1 = new JobConf(job); + JobConf jobConf2 = new JobConf(job); + HoodieColumnProjectionUtils.setIOColumnNameAndTypes(jobConf1, hoodieColNamesWithTypes); + HoodieColumnProjectionUtils.setIOColumnNameAndTypes(jobConf2, otherColNamesWithTypes); + if (hoodieColsProjected.isEmpty()) { + // Adjust adjustedColsProjected + List adjustednonHoodieColsProjected = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() >= HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getKey() - HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .collect(Collectors.toList()); + List adjustednonHoodieColNamesProjected = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() >= HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getValue()) + .collect(Collectors.toList()); + HoodieColumnProjectionUtils.setReadColumns(jobConf1, adjustednonHoodieColsProjected, + adjustednonHoodieColNamesProjected); + return super.getRecordReader(split, jobConf1, reporter); + } else { + HoodieColumnProjectionUtils.setReadColumns(jobConf1, new ArrayList<>(), new ArrayList<>()); + HoodieColumnProjectionUtils.setReadColumns(jobConf2, new ArrayList<>(), new ArrayList<>()); + List hoodieColNames = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() < HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getValue()).collect(Collectors.toList()); + List hoodieColIds = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() < HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getKey()).collect(Collectors.toList()); + List nonHoodieColNames = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() >= HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getValue()).collect(Collectors.toList()); + List nonHoodieColIdsAdjusted = colsWithIndex.stream() + .filter(idxWithName -> idxWithName.getKey() >= HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .map(idxWithName -> idxWithName.getKey() - HoodieAvroUtils.NUM_HUDI_METADATA_COLS) + .collect(Collectors.toList()); + List groupCols = Arrays.asList(job.get(READ_NESTED_COLUMN_PATH_CONF_STR, "").split(",")); + HoodieColumnProjectionUtils.appendReadColumns(jobConf1, hoodieColIds, hoodieColNames, new ArrayList<>()); + HoodieColumnProjectionUtils.appendReadColumns(jobConf2, nonHoodieColIdsAdjusted, nonHoodieColNames, groupCols); + System.out.println("hoodieColNames=" + hoodieColNames + ", hoodieColIds=" + hoodieColIds); + System.out.println("SIZES : hoodieColNames=" + hoodieColNames.size() + + ", hoodieColIds=" + hoodieColIds.size()); + System.out.println("nonHoodieColNames=" + nonHoodieColNames + ", nonHoodieColIdsAdjusted=" + + nonHoodieColIdsAdjusted); + System.out.println("SIZES : nonHoodieColNames=" + nonHoodieColNames.size() + ", nonHoodieColIdsAdjusted=" + + nonHoodieColIdsAdjusted.size()); + FileSystem fs = FileSystem.get(job); + Path externalFile = new Path(eSplit.getSourceFileFullPath()); + FileStatus externalFileStatus = fs.getFileStatus(externalFile); + FileSplit rightSplit = + makeSplit(externalFile, 0, externalFileStatus.getLen(), new String[0], new String[0]); + System.out.println("Generating HoodieColumnStichingRecordReader for " + eSplit.getSourceFileFullPath() + + " and " + externalFileStatus); + if (firstTime) { + System.out.println("JobConf1="); + Iterator> e = jobConf1.iterator(); + while (e.hasNext()) { + Entry kv = e.next(); + System.out.println("\tKey=" + kv.getKey() + ", Value=" + kv.getValue()); + } + System.out.println("\n\n\nJobConf2="); + e = jobConf2.iterator(); + while (e.hasNext()) { + Entry kv = e.next(); + System.out.println("\tKey=" + kv.getKey() + ", Value=" + kv.getValue()); + } + firstTime = false; + } + return new HoodieColumnStichingRecordReader(super.getRecordReader(eSplit, jobConf1, reporter), + super.getRecordReader(rightSplit, jobConf2, reporter)); + } + } + System.out.println("EMPLOYING DEFAULT RECORD READER - " + split); return super.getRecordReader(split, job, reporter); } @@ -214,4 +326,40 @@ protected static HoodieTableMetaClient getTableMetaClient(FileSystem fs, Path da LOG.info("Reading hoodie metadata from path " + baseDir.toString()); return new HoodieTableMetaClient(fs.getConf(), baseDir.toString()); } + + @Override + protected boolean isSplitable(FileSystem fs, Path filename) { + return !(filename instanceof PathWithExternalDataFile); + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, + String[] hosts) { + FileSplit split = new FileSplit(file, start, length, hosts); + + if (file instanceof PathWithExternalDataFile) { + try { + System.out.println("Making external data split for " + file); + return new ExternalDataFileSplit(split, ((PathWithExternalDataFile)file).getExternalPath()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + return split; + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, + String[] hosts, String[] inMemoryHosts) { + FileSplit split = new FileSplit(file, start, length, hosts, inMemoryHosts); + if (file instanceof PathWithExternalDataFile) { + try { + System.out.println("Making external data split for " + file); + return new ExternalDataFileSplit(split, ((PathWithExternalDataFile)file).getExternalPath()); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + return split; + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithExternalDataFile.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithExternalDataFile.java new file mode 100644 index 0000000000000..58e0bbbbe6b7f --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/LocatedFileStatusWithExternalDataFile.java @@ -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.hadoop; + +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +public class LocatedFileStatusWithExternalDataFile extends LocatedFileStatus { + + private final String externalDataFilePath; + + public LocatedFileStatusWithExternalDataFile(LocatedFileStatus fileStatus, String externalDataFile) + throws IOException { + super(fileStatus, fileStatus.getBlockLocations()); + this.externalDataFilePath = externalDataFile; + } + + @Override + public Path getPath() { + return new PathWithExternalDataFile(super.getPath(), externalDataFilePath); + } +} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithExternalDataFile.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithExternalDataFile.java new file mode 100644 index 0000000000000..40beffd3feac4 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithExternalDataFile.java @@ -0,0 +1,35 @@ +/* + * 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.hadoop; + +import org.apache.hadoop.fs.Path; + +public class PathWithExternalDataFile extends Path { + + private final String externalPath; + + public PathWithExternalDataFile(Path path, String externalPath) { + super(path.getParent(), path.getName()); + this.externalPath = externalPath; + } + + public String getExternalPath() { + return externalPath; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 8e2cd234cf275..5245644cffb4a 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi; import org.apache.hudi.client.embedded.EmbeddedTimelineService; +import org.apache.hudi.client.utils.ClientUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -36,7 +37,6 @@ import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; import org.apache.hudi.index.HoodieIndex; -import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -46,7 +46,6 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * Utilities used throughout the data source. @@ -68,38 +67,14 @@ public static String getNullableNestedFieldValAsString(GenericRecord record, Str * Obtain value of the provided field as string, denoted by dot notation. e.g: a.b.c */ public static String getNestedFieldValAsString(GenericRecord record, String fieldName) { - Object obj = getNestedFieldVal(record, fieldName); - return (obj == null) ? null : obj.toString(); + return ClientUtils.getNestedFieldValAsString(record, fieldName); } /** * Obtain value of the provided field, denoted by dot notation. e.g: a.b.c */ public static Object getNestedFieldVal(GenericRecord record, String fieldName) { - String[] parts = fieldName.split("\\."); - GenericRecord valueNode = record; - int i = 0; - for (; i < parts.length; i++) { - String part = parts[i]; - Object val = valueNode.get(part); - if (val == null) { - break; - } - - // return, if last part of name - if (i == parts.length - 1) { - return val; - } else { - // VC: Need a test here - if (!(val instanceof GenericRecord)) { - throw new HoodieException("Cannot find a record at part value :" + part); - } - valueNode = (GenericRecord) val; - } - } - throw new HoodieException( - fieldName + "(Part -" + parts[i] + ") field not found in record. Acceptable fields were :" - + valueNode.getSchema().getFields().stream().map(Field::name).collect(Collectors.toList())); + return ClientUtils.getNestedFieldVal(record, fieldName); } /** diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 7dfb015efa7d8..18edd5393d5cb 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -23,6 +23,7 @@ import org.apache.hudi.HoodieWriteClient; import org.apache.hudi.KeyGenerator; import org.apache.hudi.WriteStatus; +import org.apache.hudi.bootstrap.HoodieBootstrapClient; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -213,22 +214,28 @@ public Option syncOnce() throws Exception { // Refresh Timeline refreshTimeline(); - Pair>> srcRecordsWithCkpt = readFromSource(commitTimelineOpt); + if ((cfg.operation == Operation.BOOTSTRAP) || (!commitTimelineOpt.isPresent() + || (commitTimelineOpt.get().filterCompletedInstants().empty() && cfg.enableBootstrap))) { + LOG.info("Bootstrap is enabled. Running it"); + HoodieBootstrapClient bootstrapClient = new HoodieBootstrapClient(jssc, getHoodieClientConfig(null)); + bootstrapClient.bootstrap(); + } else { + Pair>> srcRecordsWithCkpt = readFromSource(commitTimelineOpt); + + if (null != srcRecordsWithCkpt) { + // this is the first input batch. If schemaProvider not set, use it and register Avro Schema and start + // compactor + if (null == schemaProvider) { + // Set the schemaProvider if not user-provided + this.schemaProvider = srcRecordsWithCkpt.getKey(); + // Setup HoodieWriteClient and compaction now that we decided on schema + setupWriteClient(); + } - if (null != srcRecordsWithCkpt) { - // this is the first input batch. If schemaProvider not set, use it and register Avro Schema and start - // compactor - if (null == schemaProvider) { - // Set the schemaProvider if not user-provided - this.schemaProvider = srcRecordsWithCkpt.getKey(); - // Setup HoodieWriteClient and compaction now that we decided on schema - setupWriteClient(); + scheduledCompaction = writeToSink(srcRecordsWithCkpt.getRight().getRight(), + srcRecordsWithCkpt.getRight().getLeft(), metrics, overallTimerContext); } - - scheduledCompaction = writeToSink(srcRecordsWithCkpt.getRight().getRight(), - srcRecordsWithCkpt.getRight().getLeft(), metrics, overallTimerContext); } - // Clear persistent RDDs jssc.getPersistentRDDs().values().forEach(JavaRDD::unpersist); return scheduledCompaction; diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java index d4f3ec42ed671..6e4b831913b9b 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -35,7 +35,7 @@ import org.apache.hudi.utilities.HiveIncrementalPuller; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; -import org.apache.hudi.utilities.sources.JsonDFSSource; +import org.apache.hudi.utilities.sources.NoOpRowSource; import com.beust.jcommander.IStringConverter; import com.beust.jcommander.JCommander; @@ -112,7 +112,7 @@ private static HiveConf getDefaultHiveConf(Configuration cfg) { * @throws Exception */ public void sync() throws Exception { - if (cfg.continuousMode) { + if (cfg.continuousMode && (cfg.operation != Operation.BOOTSTRAP)) { deltaSyncService.start(this::onDeltaSyncShutdown); deltaSyncService.waitForShutdown(); LOG.info("Delta Sync shutting down"); @@ -137,7 +137,7 @@ private boolean onDeltaSyncShutdown(boolean error) { } public enum Operation { - UPSERT, INSERT, BULK_INSERT + UPSERT, INSERT, BULK_INSERT, BOOTSTRAP } private static class OperationConvertor implements IStringConverter { @@ -179,7 +179,7 @@ public static class Config implements Serializable { description = "Subclass of org.apache.hudi.utilities.sources to read data. " + "Built-in options: org.apache.hudi.utilities.sources.{JsonDFSSource (default), AvroDFSSource, " + "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}") - public String sourceClassName = JsonDFSSource.class.getName(); + public String sourceClassName = NoOpRowSource.class.getName(); @Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how" + " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record") @@ -271,6 +271,9 @@ public static class Config implements Serializable { @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; + @Parameter(names = {"--enable-bootstrap"}, description = "Bootstrap if bootstrap commit is not found") + public Boolean enableBootstrap = false; + public boolean isAsyncCompactionEnabled() { return continuousMode && !forceDisableCompaction && HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType)); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/NoOpRowSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/NoOpRowSource.java new file mode 100644 index 0000000000000..7b1f6fe260f13 --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/NoOpRowSource.java @@ -0,0 +1,43 @@ +/* + * 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.utilities.sources; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.TypedProperties; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.utilities.schema.SchemaProvider; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +public class NoOpRowSource extends RowSource { + + public NoOpRowSource(TypedProperties props, + JavaSparkContext sparkContext, SparkSession sparkSession, + SchemaProvider schemaProvider) { + super(props, sparkContext, sparkSession, schemaProvider); + } + + @Override + protected Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { + return Pair.of(Option.empty(), null); + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java index 4205144a4409c..975e6b2a9f8de 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java @@ -21,6 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.SimpleKeyGenerator; import org.apache.hudi.common.model.HoodieCommitMetadata; +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.HoodieTimeline; @@ -61,6 +62,7 @@ import org.apache.spark.sql.api.java.UDF4; import org.apache.spark.sql.functions; import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -331,7 +333,7 @@ public void testDatasetCreation() throws Exception { } @Test - public void testBulkInsertsAndUpserts() throws Exception { + public void testBulkInsertsUpsertsAndBootstrapWriteOnly() throws Exception { String datasetBasePath = dfsBasePath + "/test_dataset"; // Initial bulk insert @@ -357,6 +359,31 @@ public void testBulkInsertsAndUpserts() throws Exception { TestHelpers.assertCommitMetadata("00001", datasetBasePath, dfs, 2); List counts = TestHelpers.countsPerCommit(datasetBasePath + "/*/*.parquet", sqlContext); assertEquals(2000, counts.get(0).getLong(1)); + + // bootstrap + String newDatasetBasePath = dfsBasePath + "/test_dataset_bootstrapped"; + cfg.enableBootstrap = true; + cfg.configs.add(String.format("hoodie.bootstrap.source.base.path=%s", cfg.targetBasePath)); + cfg.configs.add(String.format("hoodie.bootstrap.recordkey.columns=%s", "_row_key")); + cfg.configs.add("hoodie.bootstrap.metadata.parallelism=5"); + cfg.targetBasePath = newDatasetBasePath; + new HoodieDeltaStreamer(cfg, jsc).sync(); + Dataset res = sqlContext.read().format("org.apache.hudi").load(newDatasetBasePath + "/*/*.parquet"); + LOG.info("Schema :"); + res.printSchema(); + + // 3000 because original source is treated as non-hoodie source when in fact it was a hoodie source + TestHelpers.assertRecordCount(3000, newDatasetBasePath + "/*/*.parquet", sqlContext); + TestHelpers.assertRecordCount(2000, datasetBasePath + "/*/*.parquet", sqlContext); + Assert.assertEquals(2000, res.select(HoodieRecord.RECORD_KEY_METADATA_FIELD).distinct().count()); + + StructField[] fields = res.schema().fields(); + Assert.assertEquals(5, fields.length); + Assert.assertEquals(HoodieRecord.COMMIT_TIME_METADATA_FIELD, fields[0].name()); + Assert.assertEquals(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, fields[1].name()); + Assert.assertEquals(HoodieRecord.RECORD_KEY_METADATA_FIELD, fields[2].name()); + Assert.assertEquals(HoodieRecord.PARTITION_PATH_METADATA_FIELD, fields[3].name()); + Assert.assertEquals(HoodieRecord.FILENAME_METADATA_FIELD, fields[4].name()); } @Test