bkonold commented on a change in pull request #1385: URL: https://github.com/apache/samza/pull/1385#discussion_r443890714
########## File path: samza-core/src/main/java/org/apache/samza/storage/SideInputTask.java ########## @@ -0,0 +1,124 @@ +/* + * 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 java.util.Collections; +import java.util.Set; +import org.apache.samza.checkpoint.OffsetManager; +import org.apache.samza.container.RunLoopTask; +import org.apache.samza.container.TaskInstanceMetrics; +import org.apache.samza.container.TaskName; +import org.apache.samza.scheduler.EpochTimeScheduler; +import org.apache.samza.system.IncomingMessageEnvelope; +import org.apache.samza.system.SystemStreamPartition; +import org.apache.samza.task.ReadableCoordinator; +import org.apache.samza.task.TaskCallback; +import org.apache.samza.task.TaskCallbackFactory; + + +/** + * This class encapsulates the processing logic for side input streams. It is executed by {@link org.apache.samza.container.RunLoop} + */ +public class SideInputTask implements RunLoopTask { + private final TaskName taskName; + private final Set<SystemStreamPartition> taskSSPs; + private final TaskSideInputHandler taskSideInputHandler; + private final TaskInstanceMetrics metrics; + + public SideInputTask( + TaskName taskName, + Set<SystemStreamPartition> taskSSPs, + TaskSideInputHandler taskSideInputHandler, + TaskInstanceMetrics metrics) { + this.taskName = taskName; + this.taskSSPs = taskSSPs; + this.taskSideInputHandler = taskSideInputHandler; + this.metrics = metrics; + } + + @Override + public TaskName taskName() { Review comment: Method names were left as-is when extracting `RunLoopTask` from `TaskInstance`. `TaskInstance` is written in scala and as a result, some of the "getter" method names here are named as the vals defined by the scala class. ########## File path: samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java ########## @@ -758,41 +744,50 @@ public void run() { sideInputSystemConsumers.register(ssp, startingOffset); taskInstanceMetrics.get(this.sspSideInputHandlers.get(ssp).getTaskName()).addOffsetGauge( ssp, ScalaJavaUtil.toScalaFunction(() -> this.sspSideInputHandlers.get(ssp).getLastProcessedOffset(ssp))); + sideInputTaskMetrics.get(this.sspSideInputHandlers.get(ssp).getTaskName()).addOffsetGauge( + ssp, ScalaJavaUtil.toScalaFunction(() -> this.sspSideInputHandlers.get(ssp).getLastProcessedOffset(ssp))); + } - SystemStreamMetadata systemStreamMetadata = streamMetadataCache.getSystemStreamMetadata(ssp.getSystemStream(), false); - SystemStreamMetadata.SystemStreamPartitionMetadata sspMetadata = - (systemStreamMetadata == null) ? null : systemStreamMetadata.getSystemStreamPartitionMetadata().get(ssp.getPartition()); + Map<TaskName, TaskSideInputHandler> taskSideInputHandlers = this.sspSideInputHandlers.values().stream() + .distinct() + .collect(Collectors.toMap(TaskSideInputHandler::getTaskName, Function.identity())); - // record a copy of the sspMetadata, to later check if its caught up - initialSideInputSSPMetadata.put(ssp, sspMetadata); + Map<TaskName, RunLoopTask> sideInputTasks = new HashMap<>(); + this.taskSideInputStoreSSPs.forEach((taskName, storesToSSPs) -> { + Set<SystemStreamPartition> taskSSPs = this.taskSideInputStoreSSPs.get(taskName).values().stream() + .flatMap(Set::stream) + .collect(Collectors.toSet()); - // check if the ssp is caught to upcoming, even at start - checkSideInputCaughtUp(ssp, startingOffset, SystemStreamMetadata.OffsetType.UPCOMING, false); - } + RunLoopTask sideInputTask = new SideInputTask(taskName, taskSSPs, taskSideInputHandlers.get(taskName), sideInputTaskMetrics.get(taskName)); + + sideInputTasks.put(taskName, sideInputTask); + }); // start the systemConsumers for consuming input this.sideInputSystemConsumers.start(); + TaskConfig taskConfig = new TaskConfig(this.config); + SamzaContainerMetrics sideInputContainerMetrics = + new SamzaContainerMetrics(SIDEINPUTS_METRICS_PREFIX + this.samzaContainerMetrics.source(), + this.samzaContainerMetrics.registry()); + + this.sideInputRunLoop = new RunLoop(sideInputTasks, Review comment: `RunLoopFactory` pulls construction values from config, not all of which we want to use 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: [email protected]
