n3nash commented on a change in pull request #600: Timeline Service with Incremental View Syncing support URL: https://github.com/apache/incubator-hudi/pull/600#discussion_r274220577
########## File path: hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java ########## @@ -0,0 +1,329 @@ +/* + * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-gr...@uber.com) + * + * Licensed 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 com.uber.hoodie.common.table.view; + +import com.uber.hoodie.avro.model.HoodieCleanMetadata; +import com.uber.hoodie.avro.model.HoodieCompactionPlan; +import com.uber.hoodie.avro.model.HoodieRestoreMetadata; +import com.uber.hoodie.avro.model.HoodieRollbackMetadata; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieCommitMetadata; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.TimelineDiffHelper; +import com.uber.hoodie.common.util.TimelineDiffHelper.TimelineDiffResult; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.exception.HoodieException; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * Adds the capability to incrementally sync the changes to file-system view as and when new instants gets completed. + */ +public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTableFileSystemView { + + private static Logger log = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class); + + @Override + protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { + try { + TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline); + if (diffResult.canSyncIncrementally()) { + log.info("Doing incremental sync"); + runIncrementalSync(newTimeline, diffResult); + log.info("Finished incremental sync"); + // Reset timeline to latest + visibleActiveTimeline = newTimeline; + } else { + log.warn("Incremental Sync of timeline is deemed unsafe. Will revert to full syncing"); + super.runSync(oldTimeline, newTimeline); + } + } catch (Exception ioe) { + log.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe); + super.runSync(oldTimeline, newTimeline); + } + } + + /** + * Run incremental sync based on the diff result produced. + * + * @param timeline New Timeline + * @param diffResult Timeline Diff Result + */ + private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) { + + log.info("Timeline Diff Result is :" + diffResult); + + // First remove pending compaction instants which were completed + diffResult.getFinishedCompactionInstants().stream().forEach(instant -> { + try { + removePendingCompactionInstant(timeline, instant); + } catch (IOException e) { + throw new HoodieException(e); + } + }); + + // Add new completed instants found in the latest timeline + diffResult.getNewlySeenInstants().stream() + .filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) + .forEach(instant -> { + try { + if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION) + || instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) { + addCommitInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) { + addRestoreInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)) { + addCleanInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { + addPendingCompactionInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { + addRollbackInstant(timeline, instant); + } + } catch (IOException ioe) { + throw new HoodieException(ioe); + } + }); + } + + /** + * Remove Pending compaction instant + * + * @param timeline New Hoodie Timeline + * @param instant Compaction Instant to be removed + */ + private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { + log.info("Removing completed compaction instant (" + instant + ")"); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp()); + removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan) + .map(instantPair -> Pair.of(instantPair.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue())))); + } + + /** + * Add newly found compaction instant + * + * @param timeline Hoodie Timeline + * @param instant Compaction Instant + */ + private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { + log.info("Syncing pending compaction instant (" + instant + ")"); + HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp()); + List<Pair<String, CompactionOperation>> pendingOps = + CompactionUtils.getPendingCompactionOperations(instant, compactionPlan).map(p -> Pair.of(p.getValue().getKey(), + CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue()))).collect(Collectors.toList()); + // First, update Pending compaction instants + addPendingCompactionOperations(pendingOps.stream()); + + Map<String, List<Pair<String, HoodieFileGroup>>> partitionToFileGroups = + pendingOps.stream().map(opPair -> { + String compactionInstantTime = opPair.getKey(); + HoodieFileGroup fileGroup = new HoodieFileGroup(opPair.getValue().getFileGroupId(), timeline); + fileGroup.addNewFileSliceAtInstant(compactionInstantTime); + return Pair.of(compactionInstantTime, fileGroup); + }).collect(Collectors.groupingBy(x -> x.getValue().getPartitionPath())); + partitionToFileGroups.entrySet().forEach(entry -> { + if (isPartitionAvailableInStore(entry.getKey())) { + applyDeltaFileSlicesToPartitionView(entry.getKey(), + entry.getValue().stream().map(Pair::getValue).collect(Collectors.toList()), DeltaApplyMode.ADD); + } + }); + } + + /** + * Add newly found commit/delta-commit instant + * + * @param timeline Hoodie Timeline + * @param instant Instant + */ + private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { + log.info("Syncing committed instant (" + instant + ")"); + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), + HoodieCommitMetadata.class); + commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> { + String partition = entry.getKey(); + if (isPartitionAvailableInStore(partition)) { + log.info("Syncing partition (" + partition + ") of instant (" + instant + ")"); + FileStatus[] statuses = entry.getValue().stream().map(p -> { + FileStatus status = new FileStatus(p.getFileSizeInBytes(), false, 0, 0, 0, 0, null, null, null, Review comment: Should we do a listStatus here ? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services