bvaradar 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_r277035332
########## File path: hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java ########## @@ -0,0 +1,828 @@ +/* + * Copyright (c) 2018 Uber Technologies, Inc. ([email protected]) + * + * 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.google.common.base.Preconditions; +import com.uber.hoodie.common.model.CompactionOperation; +import com.uber.hoodie.common.model.FileSlice; +import com.uber.hoodie.common.model.HoodieDataFile; +import com.uber.hoodie.common.model.HoodieFileGroup; +import com.uber.hoodie.common.model.HoodieFileGroupId; +import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.table.HoodieTableMetaClient; +import com.uber.hoodie.common.table.HoodieTimeline; +import com.uber.hoodie.common.table.HoodieView; +import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; +import com.uber.hoodie.common.util.collection.Pair; +import com.uber.hoodie.exception.HoodieIOException; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +/** + * 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 views based on that + * (c) Thread-safety in loading and managing file system views for this dataset. + * (d) resetting file-system views + * The actual mechanism of fetching file slices from different view storages is delegated to sub-classes. + */ +public abstract class AbstractTableFileSystemView implements HoodieView, Serializable { + + private static Logger log = LogManager.getLogger(AbstractTableFileSystemView.class); + + protected HoodieTableMetaClient metaClient; + + // This is the commits that will be visible for all views extending this view + protected HoodieTimeline visibleActiveTimeline; + + // Used to concurrently load and populate partition views + private ConcurrentHashMap<String, Boolean> addedPartitions = new ConcurrentHashMap<>(4096); + + // Locks to control concurrency. Sync operations use write-lock blocking all fetch operations. + // For the common-case, we allow concurrent read of single or multiple partitions + private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock(); + private final ReadLock readLock = globalLock.readLock(); + private final WriteLock writeLock = globalLock.writeLock(); + + private String getPartitionPathFromFilePath(String fullPath) { + return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent()); + } + + /** + * Inisitalize the view. + */ + protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { + this.metaClient = metaClient; + this.visibleActiveTimeline = visibleActiveTimeline; + // Load Pending Compaction Operations + resetPendingCompactionOperations( + CompactionUtils.getAllPendingCompactionOperations(metaClient).values() + .stream().map(e -> Pair.of(e.getKey(), + CompactionOperation.convertFromAvroRecordInstance(e.getValue())))); + } + + /** + * Adds the provided statuses into the file system view, and also caches it inside this object. + */ + protected List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) { + long beginFgTs = System.currentTimeMillis(); + List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, visibleActiveTimeline, true); + long endFgTs = System.currentTimeMillis(); + // Make building FileGroup Map efficient for both InMemory and DiskBased stuctures. + fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).entrySet() + .forEach(entry -> { + String partition = entry.getKey(); + if (!isPartitionAvailableInStore(partition)) { + storePartitionView(partition, entry.getValue()); Review comment: There is parallelization in storing file-system view for the store due to addedPartitions.computeIfAbsent allowing concurrent store as long as they are not in the same hashmap bucket. We use the file-system view level write lock only for syncing ---------------------------------------------------------------- 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: [email protected] With regards, Apache Git Services
