bkonold commented on a change in pull request #1343: URL: https://github.com/apache/samza/pull/1343#discussion_r417024942
########## File path: samza-core/src/main/java/org/apache/samza/storage/TaskSideInputHandler.java ########## @@ -0,0 +1,331 @@ +/* + * 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.samza.storage; + + +import com.google.common.annotations.VisibleForTesting; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.samza.Partition; +import org.apache.samza.container.TaskName; +import org.apache.samza.executors.KeyBasedExecutorService; +import org.apache.samza.storage.kv.Entry; +import org.apache.samza.storage.kv.KeyValueStore; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.StreamMetadataCache; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemAdmins; +import org.apache.samza.system.SystemStream; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.task.TaskCallback; +import org.apache.samza.task.TaskCallbackFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; + +// TODO pick a better name for this class? + +/** + * This class encapsulates all processing logic / state for all side input SSPs within a task. + */ +public class TaskSideInputHandler { + private static final Logger LOG = LoggerFactory.getLogger(TaskSideInputHandler.class); + + private final StorageManagerUtil storageManagerUtil = new StorageManagerUtil(); + + private final TaskName taskName; + private final TaskSideInputStorageManager taskSideInputStorageManager; + private final Map<SystemStreamPartition, Set<String>> sspToStores; + private final Map<String, SideInputsProcessor> storeToProcessor; + private final SystemAdmins systemAdmins; + private final StreamMetadataCache streamMetadataCache; + // marks the offsets per SSP that must be bootstrapped to, inclusive. container startup will block until these offsets have been processed + private final Map<SystemStreamPartition, String> sspBootstrapOffsets; + + // these objects are SHARED WITH CONTAINER STORAGE MANAGER + // used to coordinate updates of checkpoint offsets by ContainerStorageManager + private final Map<SystemStreamPartition, Object> sspLockObjects; + // indicates the latest checkpoint per SSP. updated by ContainerStorageManager background thread + private final Map<SystemStreamPartition, Optional<String>> checkpointedOffsets; + // used to block container until each SSP reaches its bootstrap offset + private final CountDownLatch sideInputTasksCaughtUp; + + private final Map<SystemStreamPartition, String> startingOffsets; + private final Map<SystemStreamPartition, String> lastProcessedOffsets; + private final KeyBasedExecutorService checkpointedSSPExecutor; + private final KeyBasedExecutorService nonCheckpointedSSPExecutor; + + public TaskSideInputHandler( + TaskName taskName, + TaskSideInputStorageManager taskSideInputStorageManager, + Map<String, Set<SystemStreamPartition>> storeToSSPs, + Map<String, SideInputsProcessor> storeToProcessor, + SystemAdmins systemAdmins, + StreamMetadataCache streamMetadataCache, + CountDownLatch sideInputTasksCaughtUp, + Map<SystemStreamPartition, SystemStreamMetadata.SystemStreamPartitionMetadata> initialSSPMetadata, + Map<SystemStreamPartition, Object> sspLockObjects, + Map<SystemStreamPartition, Optional<String>> checkpointOffsets) { + this.taskName = taskName; + this.taskSideInputStorageManager = taskSideInputStorageManager; + this.systemAdmins = systemAdmins; + this.streamMetadataCache = streamMetadataCache; + this.sideInputTasksCaughtUp = sideInputTasksCaughtUp; + this.storeToProcessor = storeToProcessor; + this.sspLockObjects = Collections.unmodifiableMap(sspLockObjects); + this.checkpointedOffsets = Collections.unmodifiableMap(checkpointOffsets); + + this.sspToStores = storeToSSPs.entrySet().stream() + .flatMap(storeAndSSPs -> storeAndSSPs.getValue().stream() + .map(ssp -> new AbstractMap.SimpleImmutableEntry<>(ssp, storeAndSSPs.getKey()))) + .collect(Collectors.groupingBy( + Map.Entry::getKey, + Collectors.mapping(Map.Entry::getValue, Collectors.toSet()))); + + // for non-checkpointed SSPs, use their newest offset. for checkpointed SSPs, use their current checkpoint + this.sspBootstrapOffsets = new HashMap<>(); + initialSSPMetadata.entrySet().stream() + // only SSPs for this task + .filter(entry -> this.sspToStores.containsKey(entry.getKey())) + // that do not have checkpoints + .filter(entry -> !this.checkpointedOffsets.containsKey(entry.getKey())) + .forEach(entry -> this.sspBootstrapOffsets.put(entry.getKey(), entry.getValue().getNewestOffset())); + this.checkpointedOffsets.entrySet().stream() + .filter(entry -> entry.getValue().isPresent()) + .forEach(entry -> this.sspBootstrapOffsets.put(entry.getKey(), entry.getValue().get())); + + this.lastProcessedOffsets = taskSideInputStorageManager.getFileOffsets(); + this.startingOffsets = getStartingOffsets(this.lastProcessedOffsets, getOldestOffsets()); + this.startingOffsets.forEach((ssp, offset) -> checkCaughtUp(ssp, offset, true)); + + + Set<String> checkpointedStores = this.sspToStores.entrySet().stream() + .filter(sspAndStores -> this.checkpointedOffsets.containsKey(sspAndStores.getKey())) + .flatMap(sspAndStores -> sspAndStores.getValue().stream()) + .collect(Collectors.toSet()); + + Set<String> nonCheckpointedStores = this.sspToStores.entrySet().stream() + .filter(sspAndStores -> !this.checkpointedOffsets.containsKey(sspAndStores.getKey())) + .flatMap(sspAndStores -> sspAndStores.getValue().stream()) + .collect(Collectors.toSet()); + + this.checkpointedSSPExecutor = new KeyBasedExecutorService(Math.max(1, checkpointedStores.size())); + this.nonCheckpointedSSPExecutor = new KeyBasedExecutorService(Math.max(1, nonCheckpointedStores.size())); + } + + public void process(IncomingMessageEnvelope envelope, TaskCallbackFactory callbackFactory) { + TaskCallback callback = callbackFactory.createCallback(); Review comment: I don't think we can move this all the way down to `SideInputsProcessor`, because for some SSPs, for a message to be considered "processed", a number of `SideInputsProcessor`s will need to be invoked. So the callback factory needs to be a scope higher than that. Timeout is a problem though, and I think we do need to move this down at least to after the block waiting on the lock. What do you think? ---------------------------------------------------------------- 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]
