> tags = new ImmutableList.Builder<>();
+ tags.add(new Tag<>(GobblinClusterMetricTagNames.APPLICATION_ID, this.applicationId));
+ tags.add(new Tag<>(GobblinClusterMetricTagNames.APPLICATION_NAME, this.applicationName));
+
+ // Intialize Gobblin metrics and start reporters
+ GobblinMetrics gobblinMetrics = GobblinMetrics.get(this.applicationId, null, tags.build());
+ try {
+ gobblinMetrics.startMetricReporting(ConfigUtils.configToProperties(config));
+ } catch (MultiReporterException ex) {
+ for (MetricReporterException e: ex.getExceptions()) {
+ LOGGER.error("Failed to start {} {} reporter.", e.getSinkType().name(), e.getReporterType().name(), e);
+ }
+ }
+
+ return gobblinMetrics;
+ }
+
+ private EventSubmitter buildEventSubmitter() {
+ return new EventSubmitter.Builder(this.gobblinMetrics.get().getMetricContext(),
+ GobblinYarnEventConstants.EVENT_NAMESPACE)
+ .build();
+ }
+
+ /**
+ * Request an allocation of containers. If numTargetContainers is larger than the max of current and expected number
+ * of containers then additional containers are requested.
+ *
+ * If numTargetContainers is less than the current number of allocated containers then release free containers.
+ * Shrinking is relative to the number of currently allocated containers since it takes time for containers
+ * to be allocated and assigned work and we want to avoid releasing a container prematurely before it is assigned
+ * work. This means that a container may not be released even though numTargetContainers is less than the requested
+ * number of containers. The intended usage is for the caller of this method to make periodic calls to attempt to
+ * adjust the cluster towards the desired number of containers.
+ *
+ * @param yarnContainerRequestBundle the desired containers information, including numbers, resource and helix tag
+ * @param inUseInstances a set of in use instances
+ * @return whether successfully requested the target number of containers
+ */
+ public synchronized boolean requestTargetNumberOfContainers(YarnContainerRequestBundle yarnContainerRequestBundle, Set inUseInstances) {
+ LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}",
+ yarnContainerRequestBundle.getTotalContainers(), inUseInstances.size(), this.containerMap.size());
+
+ int defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY);
+ int defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys. CONTAINER_CORES_KEY);
+ int workerPoolSize = ConfigUtils.getInt(config, "temporal.workerpool.size",2);
+ requestContainers(workerPoolSize, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores));
+
+ this.yarnContainerRequest = yarnContainerRequestBundle;
+ LOGGER.info("Current tag-container desired count:{}, tag-container allocated: {}",
+ yarnContainerRequestBundle.getHelixTagContainerCountMap(), this.allocatedContainerCountMap);
+ return true;
+ }
+
+ // Request initial containers with default resource and helix tag
+ private void requestInitialContainers(int containersRequested) {
+ YarnContainerRequestBundle initialYarnContainerRequest = new YarnContainerRequestBundle();
+ Resource capability = Resource.newInstance(this.requestedContainerMemoryMbs, this.requestedContainerCores);
+ initialYarnContainerRequest.add(this.helixInstanceTags, containersRequested, capability);
+ requestTargetNumberOfContainers(initialYarnContainerRequest, Collections.EMPTY_SET);
+ }
+
+ private void requestContainer(Optional preferredNode, Optional resourceOptional) {
+ Resource desiredResource = resourceOptional.or(Resource.newInstance(
+ this.requestedContainerMemoryMbs, this.requestedContainerCores));
+ requestContainer(preferredNode, desiredResource);
+ }
+
+ /**
+ * Request {@param numContainers} from yarn with the specified resource. Resources will be allocated without a preferred
+ * node
+ * @param numContainers
+ * @param resource
+ */
+ private void requestContainers(int numContainers, Resource resource) {
+ LOGGER.info("Requesting {} containers with resource={}", numContainers, resource);
+ IntStream.range(0, numContainers)
+ .forEach(i -> requestContainer(Optional.absent(), resource));
+ }
+
+ // Request containers with specific resource requirement
+ private void requestContainer(Optional preferredNode, Resource resource) {
+ // Fail if Yarn cannot meet container resource requirements
+ Preconditions.checkArgument(resource.getMemory() <= this.maxResourceCapacity.get().getMemory() &&
+ resource.getVirtualCores() <= this.maxResourceCapacity.get().getVirtualCores(),
+ "Resource requirement must less than the max resource capacity. Requested resource" + resource.toString()
+ + " exceed the max resource limit " + this.maxResourceCapacity.get().toString());
+
+ // Due to YARN-314, different resource capacity needs different priority, otherwise Yarn will not allocate container
+ Priority priority = Records.newRecord(Priority.class);
+ if(!resourcePriorityMap.containsKey(resource.toString())) {
+ resourcePriorityMap.put(resource.toString(), priorityNumGenerator.getAndIncrement());
+ }
+ int priorityNum = resourcePriorityMap.get(resource.toString());
+ priority.setPriority(priorityNum);
+
+ String[] preferredNodes = preferredNode.isPresent() ? new String[] {preferredNode.get()} : null;
+ this.amrmClientAsync.addContainerRequest(
+ new AMRMClient.ContainerRequest(resource, preferredNodes, null, priority));
+ }
+
+ protected ContainerLaunchContext newContainerLaunchContext(ContainerInfo containerInfo)
+ throws IOException {
+ Path appWorkDir = GobblinClusterUtils.getAppWorkDirPathFromConfig(this.config, this.fs, this.applicationName, this.applicationId);
+ Path containerWorkDir = new Path(appWorkDir, GobblinYarnConfigurationKeys.CONTAINER_WORK_DIR_NAME);
+
+ Map resourceMap = Maps.newHashMap();
+
+ addContainerLocalResources(new Path(appWorkDir, GobblinYarnConfigurationKeys.LIB_JARS_DIR_NAME), resourceMap);
+ addContainerLocalResources(new Path(containerWorkDir, GobblinYarnConfigurationKeys.APP_JARS_DIR_NAME), resourceMap);
+ addContainerLocalResources(
+ new Path(containerWorkDir, GobblinYarnConfigurationKeys.APP_FILES_DIR_NAME), resourceMap);
+
+ if (this.config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_FILES_REMOTE_KEY)) {
+ YarnHelixUtils.addRemoteFilesToLocalResources(this.config.getString(GobblinYarnConfigurationKeys.CONTAINER_FILES_REMOTE_KEY),
+ resourceMap, yarnConfiguration);
+ }
+ if (this.config.hasPath(GobblinYarnConfigurationKeys.CONTAINER_ZIPS_REMOTE_KEY)) {
+ YarnHelixUtils.addRemoteZipsToLocalResources(this.config.getString(GobblinYarnConfigurationKeys.CONTAINER_ZIPS_REMOTE_KEY),
+ resourceMap, yarnConfiguration);
+ }
+ ContainerLaunchContext containerLaunchContext = Records.newRecord(ContainerLaunchContext.class);
+ containerLaunchContext.setLocalResources(resourceMap);
+ containerLaunchContext.setEnvironment(YarnHelixUtils.getEnvironmentVariables(this.yarnConfiguration));
+ containerLaunchContext.setCommands(Arrays.asList(containerInfo.getStartupCommand()));
+
+ Map acls = new HashMap<>(1);
+ acls.put(ApplicationAccessType.VIEW_APP, this.appViewAcl);
+ containerLaunchContext.setApplicationACLs(acls);
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ containerLaunchContext.setTokens(this.tokens.duplicate());
+ }
+
+ return containerLaunchContext;
+ }
+
+ private void addContainerLocalResources(Path destDir, Map resourceMap) throws IOException {
+ if (!this.fs.exists(destDir)) {
+ LOGGER.warn(String.format("Path %s does not exist so no container LocalResource to add", destDir));
+ return;
+ }
+
+ FileStatus[] statuses = this.fs.listStatus(destDir);
+ if (statuses != null) {
+ for (FileStatus status : statuses) {
+ YarnHelixUtils.addFileAsLocalResource(this.fs, status.getPath(), LocalResourceType.FILE, resourceMap);
+ }
+ }
+ }
+
+
+ protected ByteBuffer getSecurityTokens() throws IOException {
+ Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
+ Closer closer = Closer.create();
+ try {
+ DataOutputBuffer dataOutputBuffer = closer.register(new DataOutputBuffer());
+ credentials.writeTokenStorageToStream(dataOutputBuffer);
+
+ // Remove the AM->RM token so that containers cannot access it
+ Iterator> tokenIterator = credentials.getAllTokens().iterator();
+ while (tokenIterator.hasNext()) {
+ Token> token = tokenIterator.next();
+ if (token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) {
+ tokenIterator.remove();
+ }
+ }
+
+ return ByteBuffer.wrap(dataOutputBuffer.getData(), 0, dataOutputBuffer.getLength());
+ } catch (Throwable t) {
+ throw closer.rethrow(t);
+ } finally {
+ closer.close();
+ }
+ }
+
+ @VisibleForTesting
+ protected String buildContainerCommand(Container container, String helixParticipantId, String helixInstanceTag) {
+ String containerProcessName = GobblinTemporalYarnTaskRunner.class.getSimpleName();
+ StringBuilder containerCommand = new StringBuilder()
+ .append(ApplicationConstants.Environment.JAVA_HOME.$()).append("/bin/java")
+ .append(" -Xmx").append((int) (container.getResource().getMemory() * this.jvmMemoryXmxRatio) -
+ this.jvmMemoryOverheadMbs).append("M")
+ .append(" -D").append(GobblinYarnConfigurationKeys.JVM_USER_TIMEZONE_CONFIG).append("=").append(this.containerTimezone)
+ .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_DIR_NAME).append("=").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR)
+ .append(" -D").append(GobblinYarnConfigurationKeys.GOBBLIN_YARN_CONTAINER_LOG_FILE_NAME).append("=").append(containerProcessName).append(".").append(ApplicationConstants.STDOUT)
+ .append(" ").append(JvmUtils.formatJvmArguments(this.containerJvmArgs))
+ .append(" ").append(GobblinTemporalYarnTaskRunner.class.getName())
+ .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME)
+ .append(" ").append(this.applicationName)
+ .append(" --").append(GobblinClusterConfigurationKeys.APPLICATION_ID_OPTION_NAME)
+ .append(" ").append(this.applicationId)
+ .append(" --").append(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_OPTION_NAME)
+ .append(" ").append(helixParticipantId);
+
+ if (!Strings.isNullOrEmpty(helixInstanceTag)) {
+ containerCommand.append(" --").append(GobblinClusterConfigurationKeys.HELIX_INSTANCE_TAGS_OPTION_NAME)
+ .append(" ").append(helixInstanceTag);
+ }
+
+ LOGGER.info("Building " + containerProcessName);
+ return containerCommand.append(" 1>").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR).append(File.separator).append(
+ containerProcessName).append(".").append(ApplicationConstants.STDOUT)
+ .append(" 2>").append(ApplicationConstants.LOG_DIR_EXPANSION_VAR).append(File.separator).append(
+ containerProcessName).append(".").append(ApplicationConstants.STDERR).toString();
+ }
+
+ /**
+ * Check the exit status of a completed container and see if the replacement container
+ * should try to be started on the same node. Some exit status indicates a disk or
+ * node failure and in such cases the replacement container should try to be started on
+ * a different node.
+ */
+ private boolean shouldStickToTheSameNode(int containerExitStatus) {
+ switch (containerExitStatus) {
+ case ContainerExitStatus.DISKS_FAILED:
+ return false;
+ case ContainerExitStatus.ABORTED:
+ // Mostly likely this exit status is due to node failures because the
+ // application itself will not release containers.
+ return false;
+ default:
+ // Stick to the same node for other cases if host affinity is enabled.
+ return this.containerHostAffinityEnabled;
+ }
+ }
+
+ /**
+ * Handle the completion of a container. A new container will be requested to replace the one
+ * that just exited. Depending on the exit status and if container host affinity is enabled,
+ * the new container may or may not try to be started on the same node.
+ *
+ * A container completes in either of the following conditions: 1) some error happens in the
+ * container and caused the container to exit, 2) the container gets killed due to some reason,
+ * for example, if it runs over the allowed amount of virtual or physical memory, 3) the gets
+ * preempted by the ResourceManager, or 4) the container gets stopped by the ApplicationMaster.
+ * A replacement container is needed in all but the last case.
+ */
+ protected void handleContainerCompletion(ContainerStatus containerStatus) {
+ ContainerInfo completedContainerInfo = this.containerMap.remove(containerStatus.getContainerId());
+ //Get the Helix instance name for the completed container. Because callbacks are processed asynchronously, we might
+ //encounter situations where handleContainerCompletion() is called before onContainersAllocated(), resulting in the
+ //containerId missing from the containersMap.
+ // We use removedContainerID to remember these containers and remove them from containerMap later when we call requestTargetNumberOfContainers method
+ if (completedContainerInfo == null) {
+ removedContainerID.putIfAbsent(containerStatus.getContainerId(), "");
+ }
+ String completedInstanceName = UNKNOWN_HELIX_INSTANCE;
+
+ String helixTag = completedContainerInfo == null ? helixInstanceTags : completedContainerInfo.getHelixTag();
+ if (completedContainerInfo != null) {
+ allocatedContainerCountMap.get(helixTag).decrementAndGet();
+ }
+
+ LOGGER.info(String.format("Container %s running Helix instance %s with tag %s has completed with exit status %d",
+ containerStatus.getContainerId(), completedInstanceName, helixTag, containerStatus.getExitStatus()));
+
+ if (!Strings.isNullOrEmpty(containerStatus.getDiagnostics())) {
+ LOGGER.info(String.format("Received the following diagnostics information for container %s: %s",
+ containerStatus.getContainerId(), containerStatus.getDiagnostics()));
+ }
+
+ switch(containerStatus.getExitStatus()) {
+ case(ContainerExitStatus.ABORTED):
+ if (handleAbortedContainer(containerStatus, completedContainerInfo, completedInstanceName)) {
+ return;
+ }
+ break;
+ case(1): // Same as linux exit status 1 Often occurs when launch_container.sh failed
+ LOGGER.info("Exit status 1. CompletedContainerInfo={}", completedContainerInfo);
+ break;
+ default:
+ break;
+ }
+
+ if (this.shutdownInProgress) {
+ return;
+ }
+ if(completedContainerInfo != null) {
+ this.helixInstanceRetryCount.putIfAbsent(completedInstanceName, new AtomicInteger(0));
+ int retryCount = this.helixInstanceRetryCount.get(completedInstanceName).incrementAndGet();
+
+ // Populate event metadata
+ Optional> eventMetadataBuilder = Optional.absent();
+ if (this.eventSubmitter.isPresent()) {
+ eventMetadataBuilder = Optional.of(buildContainerStatusEventMetadata(containerStatus));
+ eventMetadataBuilder.get().put(GobblinYarnEventConstants.EventMetadata.HELIX_INSTANCE_ID, completedInstanceName);
+ eventMetadataBuilder.get().put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_RETRY_ATTEMPT, retryCount + "");
+ }
+
+ if (this.helixInstanceMaxRetries > 0 && retryCount > this.helixInstanceMaxRetries) {
+ if (this.eventSubmitter.isPresent()) {
+ this.eventSubmitter.get()
+ .submit(GobblinYarnEventConstants.EventNames.HELIX_INSTANCE_COMPLETION, eventMetadataBuilder.get().build());
+ }
+
+ LOGGER.warn("Maximum number of retries has been achieved for Helix instance " + completedInstanceName);
+ return;
+ }
+
+ // Add the Helix instance name of the completed container to the set of unused
+ // instance names so they can be reused by a replacement container.
+ LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName);
+ this.unusedHelixInstanceNames.add(completedInstanceName);
+
+ if (this.eventSubmitter.isPresent()) {
+ this.eventSubmitter.get()
+ .submit(GobblinYarnEventConstants.EventNames.HELIX_INSTANCE_COMPLETION, eventMetadataBuilder.get().build());
+ }
+ }
+ Optional newContainerResource = completedContainerInfo != null ?
+ Optional.of(completedContainerInfo.getContainer().getResource()) : Optional.absent();
+ LOGGER.info("Requesting a new container to replace {} to run Helix instance {} with helix tag {} and resource {}",
+ containerStatus.getContainerId(), completedInstanceName, helixTag, newContainerResource.orNull());
+ this.eventBus.post(new NewContainerRequest(
+ shouldStickToTheSameNode(containerStatus.getExitStatus()) && completedContainerInfo != null ?
+ Optional.of(completedContainerInfo.getContainer()) : Optional.absent(), newContainerResource));
+ }
+
+ /**
+ * Handles containers aborted. This method handles 2 cases:
+ *
+ * -
+ * Case 1: Gobblin AM intentionally requested container to be released (often because the number of helix tasks
+ * has decreased due to decreased traffic)
+ *
+ * -
+ * Case 2: Unexpected hardware fault and the node is lost. Need to do specific Helix logic to ensure 2 helix tasks
+ * are not being run by the multiple containers
+ *
+ *
+ * @param containerStatus
+ * @param completedContainerInfo
+ * @param completedInstanceName
+ * @return if release request was intentionally released (Case 1)
+ */
+ private boolean handleAbortedContainer(ContainerStatus containerStatus, ContainerInfo completedContainerInfo,
+ String completedInstanceName) {
+
+ // Case 1: Container intentionally released
+ if (this.releasedContainerCache.getIfPresent(containerStatus.getContainerId()) != null) {
+ LOGGER.info("Container release requested, so not spawning a replacement for containerId {}", containerStatus.getContainerId());
+ if (completedContainerInfo != null) {
+ LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName);
+ this.unusedHelixInstanceNames.add(completedInstanceName);
+ }
+
+ return true;
+ }
+
+ // Case 2: Container release was not requested. Likely, the container was running on a node on which the NM died.
+ // In this case, RM assumes that the containers are "lost", even though the container process may still be
+ // running on the node. We need to ensure that the Helix instances running on the orphaned containers
+ // are fenced off from the Helix cluster to avoid double publishing and state being committed by the
+ // instances.
+ LOGGER.info("Container {} aborted due to lost NM", containerStatus.getContainerId());
+ return false;
+ }
+
+ private ImmutableMap.Builder buildContainerStatusEventMetadata(ContainerStatus containerStatus) {
+ ImmutableMap.Builder eventMetadataBuilder = new ImmutableMap.Builder<>();
+ eventMetadataBuilder.put(GobblinYarnMetricTagNames.CONTAINER_ID, containerStatus.getContainerId().toString());
+ eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_CONTAINER_STATE,
+ containerStatus.getState().toString());
+ if (ContainerExitStatus.INVALID != containerStatus.getExitStatus()) {
+ eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_EXIT_STATUS,
+ containerStatus.getExitStatus() + "");
+ }
+ if (!Strings.isNullOrEmpty(containerStatus.getDiagnostics())) {
+ eventMetadataBuilder.put(GobblinYarnEventConstants.EventMetadata.CONTAINER_STATUS_EXIT_DIAGNOSTICS,
+ containerStatus.getDiagnostics());
+ }
+
+ return eventMetadataBuilder;
+ }
+
+ /**
+ * Get the number of matching container requests for the specified resource memory and cores.
+ * Due to YARN-1902 and YARN-660, this API is not 100% accurate. {@link AMRMClientCallbackHandler#onContainersAllocated(List)}
+ * contains logic for best effort clean up of requests, and the resource tend to match the allocated container. So in practice the count is pretty accurate.
+ *
+ * This API call gets the count of container requests for containers that are > resource if there is no request with the exact same resource
+ * The RM can return containers that are larger (because of normalization etc).
+ * Container may be larger by memory or cpu (e.g. container (1000M, 3cpu) can fit request (1000M, 1cpu) or request (500M, 3cpu).
+ *
+ * Thankfully since each helix tag / resource has a different priority, matching requests for one helix tag / resource
+ * have complete isolation from another helix tag / resource
+ */
+ private int getMatchingRequestsCount(Resource resource) {
+ Integer priorityNum = resourcePriorityMap.get(resource.toString());
+ if (priorityNum == null) { // request has never been made with this resource
+ return 0;
+ }
+ Priority priority = Priority.newInstance(priorityNum);
+
+ // Each collection in the list represents a set of requests with each with the same resource requirement.
+ // The reason for differing resources can be due to normalization
+ List extends Collection> outstandingRequests = getAmrmClientAsync().getMatchingRequests(priority, ResourceRequest.ANY, resource);
+ return outstandingRequests == null ? 0 : outstandingRequests.stream()
+ .filter(Objects::nonNull)
+ .mapToInt(Collection::size)
+ .sum();
+ }
+
+ /**
+ * A custom implementation of {@link AMRMClientAsync.CallbackHandler}.
+ */
+ private class AMRMClientCallbackHandler implements AMRMClientAsync.CallbackHandler {
+
+ private volatile boolean done = false;
+
+ @Override
+ public void onContainersCompleted(List statuses) {
+ for (ContainerStatus containerStatus : statuses) {
+ handleContainerCompletion(containerStatus);
+ }
+ }
+
+ @Override
+ public void onContainersAllocated(List containers) {
+ for (final Container container : containers) {
+ String containerId = container.getId().toString();
+ String containerHelixTag = YarnHelixUtils.findHelixTagForContainer(container, allocatedContainerCountMap, yarnContainerRequest);
+ if (Strings.isNullOrEmpty(containerHelixTag)) {
+ containerHelixTag = helixInstanceTags;
+ }
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_ALLOCATION,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId);
+ }
+
+ LOGGER.info("Container {} has been allocated with resource {} for helix tag {}",
+ container.getId(), container.getResource(), containerHelixTag);
+
+ //Iterate over the (thread-safe) set of unused instances to find the first instance that is not currently live.
+ //Once we find a candidate instance, it is removed from the set.
+ String instanceName = null;
+
+ //Ensure that updates to unusedHelixInstanceNames are visible to other threads that might concurrently
+ //invoke the callback on container allocation.
+ synchronized (this) {
+ Iterator iterator = unusedHelixInstanceNames.iterator();
+ while (iterator.hasNext()) {
+ instanceName = iterator.next();
+ }
+ }
+
+ ContainerInfo containerInfo = new ContainerInfo(container, instanceName, containerHelixTag);
+ containerMap.put(container.getId(), containerInfo);
+ allocatedContainerCountMap.putIfAbsent(containerHelixTag, new AtomicInteger(0));
+ allocatedContainerCountMap.get(containerHelixTag).incrementAndGet();
+
+ // Find matching requests and remove the request (YARN-660). We the scheduler are responsible
+ // for cleaning up requests after allocation based on the design in the described ticket.
+ // YARN does not have a delta request API and the requests are not cleaned up automatically.
+ // Try finding a match first with the host as the resource name then fall back to any resource match.
+ // Also see YARN-1902. Container count will explode without this logic for removing container requests.
+ List extends Collection> matchingRequests = amrmClientAsync
+ .getMatchingRequests(container.getPriority(), container.getNodeHttpAddress(), container.getResource());
+
+ if (matchingRequests.isEmpty()) {
+ LOGGER.debug("Matching request by host {} not found", container.getNodeHttpAddress());
+
+ matchingRequests = amrmClientAsync
+ .getMatchingRequests(container.getPriority(), ResourceRequest.ANY, container.getResource());
+ }
+
+ if (!matchingRequests.isEmpty()) {
+ AMRMClient.ContainerRequest firstMatchingContainerRequest = matchingRequests.get(0).iterator().next();
+ LOGGER.debug("Found matching requests {}, removing first matching request {}",
+ matchingRequests, firstMatchingContainerRequest);
+
+ amrmClientAsync.removeContainerRequest(firstMatchingContainerRequest);
+ }
+
+ containerLaunchExecutor.submit(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ LOGGER.info("Starting container " + containerId);
+
+ nmClientAsync.startContainerAsync(container, newContainerLaunchContext(containerInfo));
+ } catch (IOException ioe) {
+ LOGGER.error("Failed to start container " + containerId, ioe);
+ }
+ }
+ });
+ }
+ }
+
+ @Override
+ public void onShutdownRequest() {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.SHUTDOWN_REQUEST);
+ }
+
+ LOGGER.info("Received shutdown request from the ResourceManager");
+ this.done = true;
+ eventBus.post(new ClusterManagerShutdownRequest());
+ }
+
+ @Override
+ public void onNodesUpdated(List updatedNodes) {
+ for (NodeReport nodeReport : updatedNodes) {
+ LOGGER.info("Received node update report: " + nodeReport);
+ }
+ }
+
+ @Override
+ public float getProgress() {
+ return this.done ? 1.0f : 0.0f;
+ }
+
+ @Override
+ public void onError(Throwable t) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.ERROR,
+ GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t));
+ }
+
+ LOGGER.error("Received error: " + t, t);
+ this.done = true;
+ eventBus.post(new ClusterManagerShutdownRequest());
+ }
+ }
+
+ /**
+ * A custom implementation of {@link NMClientAsync.CallbackHandler}.
+ */
+ class NMClientCallbackHandler implements NMClientAsync.CallbackHandler {
+
+ @Override
+ public void onContainerStarted(ContainerId containerId, Map allServiceResponse) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STARTED,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString());
+ }
+
+ LOGGER.info(String.format("Container %s has been started", containerId));
+ }
+
+ @Override
+ public void onContainerStatusReceived(ContainerId containerId, ContainerStatus containerStatus) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STATUS_RECEIVED,
+ buildContainerStatusEventMetadata(containerStatus).build());
+ }
+
+ LOGGER.info(String.format("Received container status for container %s: %s", containerId, containerStatus));
+ }
+
+ @Override
+ public void onContainerStopped(ContainerId containerId) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STOPPED,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString());
+ }
+
+ LOGGER.info(String.format("Container %s has been stopped", containerId));
+ if (containerMap.isEmpty()) {
+ synchronized (allContainersStopped) {
+ allContainersStopped.notify();
+ }
+ }
+ }
+
+ @Override
+ public void onStartContainerError(ContainerId containerId, Throwable t) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_START_ERROR,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(),
+ GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t));
+ }
+
+ LOGGER.error(String.format("Failed to start container %s due to error %s", containerId, t));
+ }
+
+ @Override
+ public void onGetContainerStatusError(ContainerId containerId, Throwable t) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_GET_STATUS_ERROR,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(),
+ GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t));
+ }
+
+ LOGGER.error(String.format("Failed to get status for container %s due to error %s", containerId, t));
+ }
+
+ @Override
+ public void onStopContainerError(ContainerId containerId, Throwable t) {
+ if (eventSubmitter.isPresent()) {
+ eventSubmitter.get().submit(GobblinYarnEventConstants.EventNames.CONTAINER_STOP_ERROR,
+ GobblinYarnMetricTagNames.CONTAINER_ID, containerId.toString(),
+ GobblinYarnEventConstants.EventMetadata.ERROR_EXCEPTION, Throwables.getStackTraceAsString(t));
+ }
+
+ LOGGER.error(String.format("Failed to stop container %s due to error %s", containerId, t));
+ }
+ }
+
+ // Class encapsulates Container instances, Helix participant IDs of the containers, Helix Tag, and
+ // initial startup command
+ @Getter
+ class ContainerInfo {
+ private final Container container;
+ private final String helixParticipantId;
+ private final String helixTag;
+ private final String startupCommand;
+
+ public ContainerInfo(Container container, String helixParticipantId, String helixTag) {
+ this.container = container;
+ this.helixParticipantId = helixParticipantId;
+ this.helixTag = helixTag;
+ this.startupCommand = YarnTemporalService.this.buildContainerCommand(container, helixParticipantId, helixTag);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("ContainerInfo{ container=%s, helixParticipantId=%s, helixTag=%s, startupCommand=%s }",
+ container.getId(), helixParticipantId, helixTag, startupCommand);
+ }
+ }
+}
diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle
index ffc81c7352f..a26183a1922 100644
--- a/gradle/scripts/dependencyDefinitions.gradle
+++ b/gradle/scripts/dependencyDefinitions.gradle
@@ -101,7 +101,7 @@ ext.externalDependency = [
"confluentJsonSerializer": "io.confluent:kafka-json-serializer:" + confluentVersion,
"zkClient": "com.101tec:zkclient:0.7",
"quartz": "org.quartz-scheduler:quartz:2.2.3",
- "temporal-sdk": "io.temporal:temporal-sdk:1.20.0",
+ "temporal-sdk": "io.temporal:temporal-sdk:1.18.1",
"testng": "org.testng:testng:6.14.3",
"junit": "junit:junit:4.13.2",
"mockserver":"org.mock-server:mockserver-netty:3.10.4",
@@ -215,6 +215,7 @@ ext.externalDependency = [
"postgresConnector": "org.postgresql:postgresql:42.1.4",
"testContainers": "org.testcontainers:testcontainers:1.17.3",
"testContainersMysql": "org.testcontainers:mysql:1.17.3",
+ "workflowClient.namespace": "gobblin-fastingest-internpoc",
"xz": "org.tukaani:xz:1.8"
]
From 593676c13b51d7248bf30f495c70797655ae6dc4 Mon Sep 17 00:00:00 2001
From: Matthew Ho
Date: Mon, 21 Aug 2023 15:45:13 -0700
Subject: [PATCH 4/8] Py milestone 3 (#8)
* create temporal workflow from GobblinTemporalJobLauncher
Still fixing temporal activities
changed path settings
changed workunitpath
* fixed workunitpath
* Add EventSubmitter
* Clean up code and stop writing orc files
* add MetricReporter in taskrunner and make worker size configurable
* add workflowid in gobblin task
* Add workerOptions to setMaxConcurrentWorkflowTaskExecutionSize
---------
Co-authored-by: Peiyingy
---
.../GobblinClusterConfigurationKeys.java | 1 +
.../GobblinTemporalClusterManager.java | 95 +---
.../cluster/GobblinTemporalJobLauncher.java | 431 ++++++++++++++++++
.../cluster/GobblinTemporalJobScheduler.java | 313 +++++++++++++
.../cluster/GobblinTemporalTaskRunner.java | 104 ++++-
.../apache/gobblin/cluster/SingleTask.java | 53 ++-
.../temporal/GobblinTemporalActivity.java | 11 +-
.../temporal/GobblinTemporalActivityImpl.java | 62 ++-
.../temporal/GobblinTemporalWorkflow.java | 12 +-
.../temporal/GobblinTemporalWorkflowImpl.java | 89 +++-
.../gobblin/cluster/temporal/Shared.java | 6 +-
.../apache/gobblin/metrics/MetricContext.java | 8 +-
.../gobblin/metrics/event/EventSubmitter.java | 9 +-
.../runtime/StreamModelTaskRunner.java | 12 +-
.../GobblinTemporalApplicationMaster.java | 2 +-
.../yarn/GobblinYarnConfigurationKeys.java | 1 +
.../gobblin/yarn/YarnTemporalService.java | 38 +-
gradle/scripts/dependencyDefinitions.gradle | 1 -
18 files changed, 1077 insertions(+), 171 deletions(-)
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
index 31a8547aa98..f0dedec42b7 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinClusterConfigurationKeys.java
@@ -222,4 +222,5 @@ public class GobblinClusterConfigurationKeys {
public static final String CONTAINER_ID_KEY = GOBBLIN_HELIX_PREFIX + "containerId";
public static final String GOBBLIN_CLUSTER_SYSTEM_PROPERTY_PREFIX = GOBBLIN_CLUSTER_PREFIX + "sysProps";
+ public static final String TEMPORAL_WORKER_SIZE = "temporal.worker.size";
}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java
index 4a910c09aad..232e4af2338 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalClusterManager.java
@@ -52,10 +52,6 @@
import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts;
import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext;
-import io.temporal.client.WorkflowClient;
-import io.temporal.client.WorkflowClientOptions;
-import io.temporal.client.WorkflowOptions;
-import io.temporal.client.WorkflowStub;
import io.temporal.serviceclient.WorkflowServiceStubs;
import io.temporal.serviceclient.WorkflowServiceStubsOptions;
import javax.net.ssl.KeyManagerFactory;
@@ -67,8 +63,6 @@
import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest;
-import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow;
-import org.apache.gobblin.cluster.temporal.Shared;
import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.instrumented.StandardMetricsBridge;
import org.apache.gobblin.metrics.Tag;
@@ -86,21 +80,6 @@
/**
* The central cluster manager for Gobblin Clusters.
- *
- *
- *
- * This class will initiates a graceful shutdown of the cluster in the following conditions:
- *
- *
- * - A shutdown request is received via a Helix message of subtype
- * {@link HelixMessageSubTypes#APPLICATION_MASTER_SHUTDOWN}. Upon receiving such a message,
- * it will call {@link #stop()} to initiate a graceful shutdown of the cluster
- * - The shutdown hook gets called. The shutdown hook will call {@link #stop()}, which will
- * start a graceful shutdown of the cluster.
- *
- *
- *
- * @author Yinan Li
*/
@Alpha
@Slf4j
@@ -136,6 +115,8 @@ public class GobblinTemporalClusterManager implements ApplicationLauncher, Stand
@Getter
private JobConfigurationManager jobConfigurationManager;
@Getter
+ private GobblinTemporalJobScheduler gobblinTemporalJobScheduler;
+ @Getter
private volatile boolean started = false;
protected final String clusterName;
@@ -144,9 +125,8 @@ public class GobblinTemporalClusterManager implements ApplicationLauncher, Stand
public GobblinTemporalClusterManager(String clusterName, String applicationId, Config sysConfig,
Optional appWorkDirOptional) throws Exception {
- // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration
+ // Set system properties passed in via application config.
// overrides such as sessionTimeout. In this case, the overrides specified
- // in the application configuration have to be extracted and set before initializing HelixManager.
GobblinClusterUtils.setSystemProperties(sysConfig);
//Add dynamic config
@@ -161,7 +141,7 @@ public GobblinTemporalClusterManager(String clusterName, String applicationId, C
this.fs = GobblinClusterUtils.buildFileSystem(this.config, new Configuration());
this.appWorkDir = appWorkDirOptional.isPresent() ? appWorkDirOptional.get()
: GobblinClusterUtils.getAppWorkDirPathFromConfig(this.config, this.fs, clusterName, applicationId);
- LOGGER.info("Configured GobblinClusterManager work dir to: {}", this.appWorkDir);
+ LOGGER.info("Configured GobblinTemporalClusterManager work dir to: {}", this.appWorkDir);
initializeAppLauncherAndServices();
}
@@ -195,6 +175,9 @@ private void initializeAppLauncherAndServices() throws Exception {
SchedulerService schedulerService = new SchedulerService(properties);
this.applicationLauncher.addService(schedulerService);
+ this.gobblinTemporalJobScheduler = buildGobblinTemporalJobScheduler(config, this.appWorkDir, getMetadataTags(clusterName, applicationId),
+ schedulerService);
+ this.applicationLauncher.addService(this.gobblinTemporalJobScheduler);
this.jobConfigurationManager = buildJobConfigurationManager(config);
this.applicationLauncher.addService(this.jobConfigurationManager);
@@ -234,7 +217,7 @@ private void stopAppLauncherAndServices() {
/**
- * Start the Gobblin Cluster Manager.
+ * Start the Gobblin Temporal Cluster Manager.
*/
// @Import(clazz = ClientSslContextFactory.class, prefix = ClientSslContextFactory.SCOPE_PREFIX)
@Override
@@ -263,49 +246,12 @@ public void run() {
this.idleProcessThread.start();
// Need this in case a kill is issued to the process so that the idle thread does not keep the process up
- // since GobblinClusterManager.stop() is not called this case.
+ // since GobblinTemporalClusterManager.stop() is not called this case.
Runtime.getRuntime().addShutdownHook(new Thread(() -> GobblinTemporalClusterManager.this.stopIdleProcessThread = true));
} else {
startAppLauncherAndServices();
}
this.started = true;
-
- try {
- initiateWorkflow();
- }catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- public void initiateWorkflow()
- throws Exception {
- LOGGER.info("Initiating Temporal Workflow");
- WorkflowServiceStubs workflowServiceStubs = createServiceStubs();
- WorkflowClient client =
- WorkflowClient.newInstance(
- workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build());
-
- /*
- * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute.
- */
- WorkflowOptions options = WorkflowOptions.newBuilder()
- .setTaskQueue(Shared.HELLO_WORLD_TASK_QUEUE)
- .build();
-
- // Create the workflow client stub. It is used to start our workflow execution.
- GobblinTemporalWorkflow workflow = client.newWorkflowStub(GobblinTemporalWorkflow.class, options);
-
- /*
- * Execute our workflow and wait for it to complete. The call to our getGreeting method is
- * synchronous.
- *
- * Replace the parameter "World" in the call to getGreeting() with your name.
- */
- String greeting = workflow.getGreeting("World");
-
- String workflowId = WorkflowStub.fromTyped(workflow).getExecution().getWorkflowId();
- // Display workflow execution results
- LOGGER.info(workflowId + " " + greeting);
}
public static WorkflowServiceStubs createServiceStubs()
@@ -351,7 +297,6 @@ public static WorkflowServiceStubs createServiceStubs()
// Set trust manager from trust store
KeyStore trustStore = KeyStore.getInstance("JKS");
File trustStoreFile = new File(config.getString(SSL_TRUSTSTORE_LOCATION));
- LOGGER.info("SSL_TRUSTSTORE_LOCATION " + config.getString(SSL_TRUSTSTORE_LOCATION));
String trustStorePassword = config.getString(SSL_TRUSTSTORE_PASSWORD);
trustStore.load(toInputStream(trustStoreFile), trustStorePassword.toCharArray());
@@ -365,8 +310,6 @@ public static WorkflowServiceStubs createServiceStubs()
.ciphers(SSL_CONFIG_DEFAULT_CIPHER_SUITES)
.build();
- LOGGER.info("SSLContext: " + sslContext);
-
return WorkflowServiceStubs.newServiceStubs(
WorkflowServiceStubsOptions.newBuilder()
.setTarget("1.nephos-temporal.corp-lca1.atd.corp.linkedin.com:7233")
@@ -375,6 +318,7 @@ public static WorkflowServiceStubs createServiceStubs()
.build());
}
+
/**
* Stop the Gobblin Cluster Manager.
*/
@@ -400,9 +344,15 @@ public synchronized void stop() {
}
- /**
- * Get additional {@link Tag}s required for any type of reporting.
- */
+ private GobblinTemporalJobScheduler buildGobblinTemporalJobScheduler(Config sysConfig, Path appWorkDir,
+ List extends Tag>> metadataTags, SchedulerService schedulerService) throws Exception {
+ return new GobblinTemporalJobScheduler(sysConfig,
+ this.eventBus,
+ appWorkDir,
+ metadataTags,
+ schedulerService);
+ }
+
private List extends Tag>> getMetadataTags(String applicationName, String applicationId) {
return Tag.fromMap(
new ImmutableMap.Builder().put(GobblinClusterMetricTagNames.APPLICATION_NAME, applicationName)
@@ -448,8 +398,7 @@ public Collection getStandardMetricsCollection() {
/**
* TODO for now the cluster id is hardcoded to 1 both here and in the {@link GobblinTaskRunner}. In the future, the
- * cluster id should be created by the {@link GobblinTemporalClusterManager} and passed to each {@link GobblinTaskRunner} via
- * Helix (at least that would be the easiest approach, there are certainly others ways to do it).
+ * cluster id should be created by the {@link GobblinTemporalClusterManager} and passed to each {@link GobblinTaskRunner}
*/
private static String getApplicationId() {
return "1";
@@ -496,10 +445,10 @@ public static void main(String[] args) throws Exception {
ConfigValueFactory.fromAnyRef(true));
}
- try (GobblinTemporalClusterManager gobblinClusterManager = new GobblinTemporalClusterManager(
+ try (GobblinTemporalClusterManager GobblinTemporalClusterManager = new GobblinTemporalClusterManager(
cmd.getOptionValue(GobblinClusterConfigurationKeys.APPLICATION_NAME_OPTION_NAME), getApplicationId(),
config, Optional.absent())) {
- gobblinClusterManager.start();
+ GobblinTemporalClusterManager.start();
}
} catch (ParseException pe) {
printUsage(options);
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
new file mode 100644
index 00000000000..5f79e596fcf
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
@@ -0,0 +1,431 @@
+/*
+ * 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.gobblin.cluster;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+
+import io.temporal.client.WorkflowClient;
+import io.temporal.client.WorkflowClientOptions;
+import io.temporal.client.WorkflowOptions;
+import io.temporal.serviceclient.WorkflowServiceStubs;
+import javax.annotation.Nullable;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow;
+import org.apache.gobblin.cluster.temporal.Shared;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.StateStore;
+import org.apache.gobblin.metrics.Tag;
+import org.apache.gobblin.metrics.event.CountEventBuilder;
+import org.apache.gobblin.metrics.event.JobEvent;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.rest.LauncherTypeEnum;
+import org.apache.gobblin.runtime.AbstractJobLauncher;
+import org.apache.gobblin.runtime.JobException;
+import org.apache.gobblin.runtime.JobLauncher;
+import org.apache.gobblin.runtime.JobState;
+import org.apache.gobblin.runtime.TaskStateCollectorService;
+import org.apache.gobblin.runtime.listeners.JobListener;
+import org.apache.gobblin.runtime.util.StateStores;
+import org.apache.gobblin.source.extractor.extract.kafka.KafkaSource;
+import org.apache.gobblin.source.workunit.MultiWorkUnit;
+import org.apache.gobblin.source.workunit.WorkUnit;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.Id;
+import org.apache.gobblin.util.JobLauncherUtils;
+import org.apache.gobblin.util.ParallelRunner;
+import org.apache.gobblin.util.PropertiesUtils;
+import org.apache.gobblin.util.SerializationUtils;
+
+import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs;
+
+
+/**
+ * An implementation of {@link JobLauncher} that launches a Gobblin job using the Temporal task framework.
+ *
+ *
+ * Each {@link WorkUnit} of the job is persisted to the {@link FileSystem} of choice and the path to the file
+ * storing the serialized {@link WorkUnit} is passed to the Temporal task running the {@link WorkUnit} as a
+ * user-defined property {@link GobblinClusterConfigurationKeys#WORK_UNIT_FILE_PATH}. Upon startup, the gobblin
+ * task reads the property for the file path and de-serializes the {@link WorkUnit} from the file.
+ *
+ *
+ *
+ * This class is instantiated by the {@link GobblinTemporalJobScheduler} on every job submission to launch the Gobblin job.
+ * The actual task execution happens in the {@link GobblinTemporalTaskRunner}, usually in a different process.
+ *
+ */
+@Alpha
+@Slf4j
+public class GobblinTemporalJobLauncher extends AbstractJobLauncher {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalJobLauncher.class);
+
+ private static final String WORK_UNIT_FILE_EXTENSION = ".wu";
+
+ private final FileSystem fs;
+ private final Path appWorkDir;
+ private final Path inputWorkUnitDir;
+ private final Path outputTaskStateDir;
+
+ // Number of ParallelRunner threads to be used for state serialization/deserialization
+ private final int stateSerDeRunnerThreads;
+
+ private final TaskStateCollectorService taskStateCollectorService;
+ private final ConcurrentHashMap runningMap;
+ @Getter
+ private final StateStores stateStores;
+
+ private WorkflowServiceStubs workflowServiceStubs;
+ private WorkflowClient client;
+
+ public GobblinTemporalJobLauncher(Properties jobProps, Path appWorkDir,
+ List extends Tag>> metadataTags, ConcurrentHashMap runningMap)
+ throws Exception {
+ super(jobProps, initBaseEventTags(jobProps, metadataTags));
+ LOGGER.debug("GobblinTemporalJobLauncher: jobProps {}, appWorkDir {}", jobProps, appWorkDir);
+ this.runningMap = runningMap;
+ this.appWorkDir = appWorkDir;
+ this.inputWorkUnitDir = new Path(appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME);
+ this.outputTaskStateDir = new Path(this.appWorkDir,
+ GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME + Path.SEPARATOR + this.jobContext.getJobId());
+
+ this.jobContext.getJobState().setJobLauncherType(LauncherTypeEnum.CLUSTER);
+
+ this.stateSerDeRunnerThreads = Integer.parseInt(jobProps.getProperty(ParallelRunner.PARALLEL_RUNNER_THREADS_KEY,
+ Integer.toString(ParallelRunner.DEFAULT_PARALLEL_RUNNER_THREADS)));
+
+ Config stateStoreJobConfig = ConfigUtils.propertiesToConfig(jobProps)
+ .withValue(ConfigurationKeys.STATE_STORE_FS_URI_KEY, ConfigValueFactory.fromAnyRef(
+ new URI(appWorkDir.toUri().getScheme(), null, appWorkDir.toUri().getHost(), appWorkDir.toUri().getPort(),
+ "/", null, null).toString()));
+
+ this.stateStores =
+ new StateStores(stateStoreJobConfig, appWorkDir, GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME,
+ appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir,
+ GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME);
+
+ URI fsUri = URI.create(jobProps.getProperty(ConfigurationKeys.FS_URI_KEY, ConfigurationKeys.LOCAL_FS_URI));
+ this.fs = FileSystem.get(fsUri, new Configuration());
+
+ this.taskStateCollectorService =
+ new TaskStateCollectorService(jobProps, this.jobContext.getJobState(), this.eventBus, this.eventSubmitter,
+ this.stateStores.getTaskStateStore(), this.outputTaskStateDir, this.getIssueRepository());
+
+ this.workflowServiceStubs = createServiceStubs();
+ this.client = WorkflowClient.newInstance(
+ workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build());
+
+ /*
+ * Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute.
+ */
+ startCancellationExecutor();
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ executeCancellation();
+ } finally {
+ super.close();
+ }
+ }
+
+ public String getJobId() {
+ return this.jobContext.getJobId();
+ }
+
+ @Override
+ protected void runWorkUnits(List workUnits) throws Exception {
+ try {
+ CountEventBuilder countEventBuilder = new CountEventBuilder(JobEvent.WORK_UNITS_CREATED, workUnits.size());
+ this.eventSubmitter.submit(countEventBuilder);
+ LOGGER.info("Emitting WorkUnitsCreated Count: " + countEventBuilder.getCount());
+
+ long workUnitStartTime = System.currentTimeMillis();
+ workUnits.forEach((k) -> k.setProp(ConfigurationKeys.WORK_UNIT_CREATION_TIME_IN_MILLIS, workUnitStartTime));
+
+ // Start the output TaskState collector service
+ this.taskStateCollectorService.startAsync().awaitRunning();
+
+ TimingEvent jobSubmissionTimer =
+ this.eventSubmitter.getTimingEvent(TimingEvent.RunJobTimings.HELIX_JOB_SUBMISSION);
+
+ if (!this.cancellationRequested) {
+ submitJobToTemporal(workUnits);
+ jobSubmissionTimer.stop();
+ LOGGER.info(String.format("Submitted job %s to Temporal", this.jobContext.getJobId()));
+ } else {
+ LOGGER.warn("Job {} not submitted to Temporal as it was requested to be cancelled.", this.jobContext.getJobId());
+ }
+
+ TimingEvent jobRunTimer = this.eventSubmitter.getTimingEvent(TimingEvent.RunJobTimings.HELIX_JOB_RUN);
+ jobRunTimer.stop();
+ LOGGER.info(String.format("Job %s completed", this.jobContext.getJobId()));
+ } finally {
+ // The last iteration of output TaskState collecting will run when the collector service gets stopped
+ this.taskStateCollectorService.stopAsync().awaitTerminated();
+ cleanupWorkingDirectory();
+ }
+ }
+
+ @Override
+ protected void executeCancellation() {
+ LOGGER.info("Cancel temporal workflow");
+ }
+
+ protected void removeTasksFromCurrentJob(List workUnitIdsToRemove) {
+ LOGGER.info("Temporal removeTasksFromCurrentJob");
+ }
+
+ protected void addTasksToCurrentJob(List workUnitsToAdd) {
+ LOGGER.info("Temporal addTasksToCurrentJob");
+ }
+
+ /**
+ * Submit a job to run.
+ */
+ private void submitJobToTemporal(List workUnits) throws Exception{
+ try (ParallelRunner stateSerDeRunner = new ParallelRunner(this.stateSerDeRunnerThreads, this.fs)) {
+ Path jobStateFilePath;
+
+ // write the job.state using the state store if present, otherwise serialize directly to the file
+ if (this.stateStores.haveJobStateStore()) {
+ jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(true, this.appWorkDir, this.jobContext.getJobId());
+ this.stateStores.getJobStateStore()
+ .put(jobStateFilePath.getParent().getName(), jobStateFilePath.getName(), this.jobContext.getJobState());
+ } else {
+ jobStateFilePath = GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId());
+ SerializationUtils.serializeState(this.fs, jobStateFilePath, this.jobContext.getJobState());
+ }
+
+ // Block on persistence of all workunits to be finished.
+ stateSerDeRunner.waitForTasks(Long.MAX_VALUE);
+
+ LOGGER.debug("GobblinTemporalJobLauncher.createTemporalJob: jobStateFilePath {}, jobState {} jobProperties {}",
+ jobStateFilePath, this.jobContext.getJobState().toString(), this.jobContext.getJobState().getProperties());
+
+ String jobStateFilePathStr = jobStateFilePath.toString();
+
+ List> futures = new ArrayList<>();
+ AtomicInteger multiTaskIdSequence = new AtomicInteger(0);
+ AtomicInteger workflowCount = new AtomicInteger(0);
+ int workflowSize = 100;
+ ExecutorService executor = Executors.newFixedThreadPool(workflowSize);
+
+ for (int i = 0; i < workflowSize; i++) {
+ WorkUnit workUnit = workUnits.get(i);
+ futures.add(CompletableFuture.runAsync(() -> {
+ try {
+ if (workUnit instanceof MultiWorkUnit) {
+ workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement()));
+ }
+ String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner);
+ String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement();
+ WorkflowOptions options = WorkflowOptions.newBuilder()
+ .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE)
+ .setWorkflowId(workflowId)
+ .build();
+ GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options);
+ LOGGER.info("Setting up temporal workflow {}", workflowId);
+ workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }, executor));
+ }
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();
+ }
+ }
+
+ public void launchJob(@Nullable JobListener jobListener) throws JobException {
+ LOGGER.info("Launching Temporal Job");
+ boolean isLaunched = false;
+ this.runningMap.putIfAbsent(this.jobContext.getJobName(), false);
+
+ Throwable errorInJobLaunching = null;
+ try {
+ if (this.runningMap.replace(this.jobContext.getJobName(), false, true)) {
+ LOGGER.info("Job {} will be executed, add into running map.", this.jobContext.getJobId());
+ isLaunched = true;
+ launchJobImpl(jobListener);
+ } else {
+ LOGGER.warn("Job {} will not be executed because other jobs are still running.", this.jobContext.getJobId());
+ }
+
+ // TODO: Better error handling. The current impl swallows exceptions for jobs that were started by this method call.
+ // One potential way to improve the error handling is to make this error swallowing conifgurable
+ } catch (Throwable t) {
+ errorInJobLaunching = t;
+ } finally {
+ if (isLaunched) {
+ if (this.runningMap.replace(this.jobContext.getJobName(), true, false)) {
+ LOGGER.info("Job {} is done, remove from running map.", this.jobContext.getJobId());
+ } else {
+ throw errorInJobLaunching == null ? new IllegalStateException(
+ "A launched job should have running state equal to true in the running map.")
+ : new RuntimeException("Failure in launching job:", errorInJobLaunching);
+ }
+ }
+ }
+ }
+
+
+ /**
+ * This method looks silly at first glance but exists for a reason.
+ *
+ * The method {@link GobblinTemporalJobLauncher#launchJob(JobListener)} contains boiler plate for handling exceptions and
+ * mutating the runningMap to communicate state back to the {@link GobblinTemporalJobScheduler}. The boiler plate swallows
+ * exceptions when launching the job because many use cases require that 1 job failure should not affect other jobs by causing the
+ * entire process to fail through an uncaught exception.
+ *
+ * This method is useful for unit testing edge cases where we expect {@link JobException}s during the underlying launch operation.
+ * It would be nice to not swallow exceptions, but the implications of doing that will require careful refactoring since
+ * the class {@link GobblinTemporalJobLauncher} and {@link GobblinTemporalJobScheduler} are shared for 2 quite different cases
+ * between GaaS and streaming. GaaS typically requiring many short lifetime workflows (where a failure is tolerated) and
+ * streaming requiring a small number of long running workflows (where failure to submit is unexpected and is not
+ * tolerated)
+ *
+ * @throws JobException
+ */
+ @VisibleForTesting
+ void launchJobImpl(@Nullable JobListener jobListener) throws JobException {
+ super.launchJob(jobListener);
+ }
+
+ /**
+ * Persist a single {@link WorkUnit} (flattened) to a file.
+ */
+ private String persistWorkUnit(final Path workUnitFileDir, final WorkUnit workUnit, ParallelRunner stateSerDeRunner) {
+ final StateStore stateStore;
+ String workUnitFileName = workUnit.getId();
+
+ if (workUnit instanceof MultiWorkUnit) {
+ workUnitFileName += MULTI_WORK_UNIT_FILE_EXTENSION;
+ stateStore = stateStores.getMwuStateStore();
+ } else {
+ workUnitFileName += WORK_UNIT_FILE_EXTENSION;
+ stateStore = stateStores.getWuStateStore();
+ }
+
+ Path workUnitFile = new Path(workUnitFileDir, workUnitFileName);
+ final String fileName = workUnitFile.getName();
+ final String storeName = workUnitFile.getParent().getName();
+ stateSerDeRunner.submitCallable(new Callable() {
+ @Override
+ public Void call() throws Exception {
+ stateStore.put(storeName, fileName, workUnit);
+ return null;
+ }
+ }, "Serialize state to store " + storeName + " file " + fileName);
+
+ return workUnitFile.toString();
+ }
+
+ /**
+ * Delete persisted {@link WorkUnit}s and {@link JobState} upon job completion.
+ */
+ private void cleanupWorkingDirectory() throws IOException {
+ LOGGER.info("Deleting persisted work units for job " + this.jobContext.getJobId());
+ stateStores.getWuStateStore().delete(this.jobContext.getJobId());
+
+ // delete the directory that stores the task state files
+ stateStores.getTaskStateStore().delete(outputTaskStateDir.getName());
+
+ LOGGER.info("Deleting job state file for job " + this.jobContext.getJobId());
+
+ if (this.stateStores.haveJobStateStore()) {
+ this.stateStores.getJobStateStore().delete(this.jobContext.getJobId());
+ } else {
+ Path jobStateFilePath =
+ GobblinClusterUtils.getJobStateFilePath(false, this.appWorkDir, this.jobContext.getJobId());
+ this.fs.delete(jobStateFilePath, false);
+ }
+ }
+
+ public static List extends Tag>> initBaseEventTags(Properties jobProps,
+ List extends Tag>> inputTags) {
+ List> metadataTags = Lists.newArrayList(inputTags);
+ String jobId;
+
+ // generate job id if not already set
+ if (jobProps.containsKey(ConfigurationKeys.JOB_ID_KEY)) {
+ jobId = jobProps.getProperty(ConfigurationKeys.JOB_ID_KEY);
+ } else {
+ jobId = JobLauncherUtils.newJobId(JobState.getJobNameFromProps(jobProps),
+ PropertiesUtils.getPropAsLong(jobProps, ConfigurationKeys.FLOW_EXECUTION_ID_KEY, System.currentTimeMillis()));
+ jobProps.put(ConfigurationKeys.JOB_ID_KEY, jobId);
+ }
+
+ String jobExecutionId = Long.toString(Id.Job.parse(jobId).getSequence());
+
+ // only inject flow tags if a flow name is defined
+ if (jobProps.containsKey(ConfigurationKeys.FLOW_NAME_KEY)) {
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD,
+ jobProps.getProperty(ConfigurationKeys.FLOW_GROUP_KEY, "")));
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD,
+ jobProps.getProperty(ConfigurationKeys.FLOW_NAME_KEY)));
+
+ // use job execution id if flow execution id is not present
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD,
+ jobProps.getProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY, jobExecutionId)));
+ }
+
+ if (jobProps.containsKey(ConfigurationKeys.JOB_CURRENT_ATTEMPTS)) {
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.CURRENT_ATTEMPTS_FIELD,
+ jobProps.getProperty(ConfigurationKeys.JOB_CURRENT_ATTEMPTS, "1")));
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.CURRENT_GENERATION_FIELD,
+ jobProps.getProperty(ConfigurationKeys.JOB_CURRENT_GENERATION, "1")));
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.SHOULD_RETRY_FIELD,
+ "false"));
+ }
+
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_GROUP_FIELD,
+ jobProps.getProperty(ConfigurationKeys.JOB_GROUP_KEY, "")));
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_NAME_FIELD,
+ jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY, "")));
+ metadataTags.add(new Tag<>(TimingEvent.FlowEventConstants.JOB_EXECUTION_ID_FIELD, jobExecutionId));
+
+ log.debug("AddAdditionalMetadataTags: metadataTags {}", metadataTags);
+
+ return metadataTags;
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java
new file mode 100644
index 00000000000..af00ca49958
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobScheduler.java
@@ -0,0 +1,313 @@
+/*
+ * 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.gobblin.cluster;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.typesafe.config.Config;
+
+import org.apache.gobblin.annotation.Alpha;
+import org.apache.gobblin.cluster.event.CancelJobConfigArrivalEvent;
+import org.apache.gobblin.cluster.event.DeleteJobConfigArrivalEvent;
+import org.apache.gobblin.cluster.event.NewJobConfigArrivalEvent;
+import org.apache.gobblin.cluster.event.UpdateJobConfigArrivalEvent;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.instrumented.Instrumented;
+import org.apache.gobblin.instrumented.StandardMetricsBridge;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.Tag;
+import org.apache.gobblin.runtime.JobException;
+import org.apache.gobblin.runtime.JobLauncher;
+import org.apache.gobblin.runtime.listeners.JobListener;
+import org.apache.gobblin.scheduler.JobScheduler;
+import org.apache.gobblin.scheduler.SchedulerService;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.PathUtils;
+import org.apache.gobblin.util.PropertiesUtils;
+
+
+/**
+ * An extension to {@link JobScheduler} that schedules and runs
+ * Gobblin jobs on Temporal.
+ *
+ * If the job should be launched from the scheduler node,
+ * {@link GobblinTemporalJobLauncher} is invoked.
+ *
+ */
+@Alpha
+public class GobblinTemporalJobScheduler extends JobScheduler implements StandardMetricsBridge {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalJobScheduler.class);
+ private static final String COMMON_JOB_PROPS = "gobblin.common.job.props";
+
+ private final Properties commonJobProperties;
+ private final EventBus eventBus;
+ private final Path appWorkDir;
+ private final List extends Tag>> metadataTags;
+ private final ConcurrentHashMap jobRunningMap;
+ private final MetricContext metricContext;
+ final GobblinHelixJobSchedulerMetrics jobSchedulerMetrics;
+ final GobblinHelixJobLauncherMetrics launcherMetrics;
+ final GobblinHelixPlanningJobLauncherMetrics planningJobLauncherMetrics;
+ final HelixJobsMapping jobsMapping;
+ private boolean startServicesCompleted;
+
+ public GobblinTemporalJobScheduler(Config sysConfig,
+ EventBus eventBus,
+ Path appWorkDir, List extends Tag>> metadataTags,
+ SchedulerService schedulerService) throws Exception {
+
+ super(ConfigUtils.configToProperties(sysConfig), schedulerService);
+ this.commonJobProperties = ConfigUtils.configToProperties(ConfigUtils.getConfigOrEmpty(sysConfig, COMMON_JOB_PROPS));
+ this.eventBus = eventBus;
+ this.jobRunningMap = new ConcurrentHashMap<>();
+ this.appWorkDir = appWorkDir;
+ this.metadataTags = metadataTags;
+ this.metricContext = Instrumented.getMetricContext(new org.apache.gobblin.configuration.State(properties), this.getClass());
+
+ int metricsWindowSizeInMin = ConfigUtils.getInt(sysConfig,
+ ConfigurationKeys.METRIC_TIMER_WINDOW_SIZE_IN_MINUTES,
+ ConfigurationKeys.DEFAULT_METRIC_TIMER_WINDOW_SIZE_IN_MINUTES);
+
+ this.launcherMetrics = new GobblinHelixJobLauncherMetrics("launcherInScheduler",
+ this.metricContext,
+ metricsWindowSizeInMin);
+
+ this.jobSchedulerMetrics = new GobblinHelixJobSchedulerMetrics(this.jobExecutor,
+ this.metricContext,
+ metricsWindowSizeInMin);
+
+ this.jobsMapping = new HelixJobsMapping(ConfigUtils.propertiesToConfig(properties),
+ PathUtils.getRootPath(appWorkDir).toUri(),
+ appWorkDir.toString());
+
+ this.planningJobLauncherMetrics = new GobblinHelixPlanningJobLauncherMetrics("planningLauncherInScheduler",
+ this.metricContext,
+ metricsWindowSizeInMin, this.jobsMapping);
+
+ this.startServicesCompleted = false;
+ }
+
+ @Override
+ public Collection getStandardMetricsCollection() {
+ return ImmutableList.of(this.launcherMetrics,
+ this.jobSchedulerMetrics,
+ this.planningJobLauncherMetrics);
+ }
+
+ @Override
+ protected void startUp() throws Exception {
+ this.eventBus.register(this);
+ super.startUp();
+ this.startServicesCompleted = true;
+ }
+
+ @Override
+ public void scheduleJob(Properties jobProps, JobListener jobListener) throws JobException {
+ try {
+ while (!startServicesCompleted) {
+ LOGGER.info("{} service is not fully up, waiting here...", this.getClass().getName());
+ Thread.sleep(1000);
+ }
+
+ scheduleJob(jobProps,
+ jobListener,
+ Maps.newHashMap(),
+ GobblinHelixJob.class);
+
+ } catch (Exception e) {
+ throw new JobException("Failed to schedule job " + jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), e);
+ }
+ }
+
+ @Override
+ protected void startServices() throws Exception {
+
+ boolean cleanAllDistJobs = PropertiesUtils.getPropAsBoolean(this.properties,
+ GobblinClusterConfigurationKeys.CLEAN_ALL_DIST_JOBS,
+ String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CLEAN_ALL_DIST_JOBS));
+
+ if (cleanAllDistJobs) {
+ for (org.apache.gobblin.configuration.State state : this.jobsMapping.getAllStates()) {
+ String jobUri = state.getId();
+ LOGGER.info("Delete mapping for job " + jobUri);
+ this.jobsMapping.deleteMapping(jobUri);
+ }
+ }
+ }
+
+ @Override
+ public void runJob(Properties jobProps, JobListener jobListener) throws JobException {
+ }
+
+ @Override
+ public GobblinTemporalJobLauncher buildJobLauncher(Properties jobProps)
+ throws Exception {
+ Properties combinedProps = new Properties();
+ combinedProps.putAll(properties);
+ combinedProps.putAll(jobProps);
+
+ return new GobblinTemporalJobLauncher(combinedProps,
+ this.appWorkDir,
+ this.metadataTags,
+ this.jobRunningMap);
+ }
+
+ @Subscribe
+ public void handleNewJobConfigArrival(NewJobConfigArrivalEvent newJobArrival) {
+ String jobUri = newJobArrival.getJobName();
+ LOGGER.info("Received new job configuration of job " + jobUri);
+ try {
+ Properties jobProps = new Properties();
+ jobProps.putAll(this.commonJobProperties);
+ jobProps.putAll(newJobArrival.getJobConfig());
+
+ // set uri so that we can delete this job later
+ jobProps.setProperty(GobblinClusterConfigurationKeys.JOB_SPEC_URI, jobUri);
+
+ this.jobSchedulerMetrics.updateTimeBeforeJobScheduling(jobProps);
+
+ if (jobProps.containsKey(ConfigurationKeys.JOB_SCHEDULE_KEY)) {
+ LOGGER.info("Scheduling job " + jobUri);
+ scheduleJob(jobProps,
+ new GobblinHelixJobLauncherListener(this.launcherMetrics));
+ } else {
+ LOGGER.info("No job schedule found, so running job " + jobUri);
+ GobblinHelixJobLauncherListener listener = new GobblinHelixJobLauncherListener(this.launcherMetrics);
+ JobLauncher launcher = buildJobLauncher(newJobArrival.getJobConfig());
+ launcher.launchJob(listener);
+ }
+ } catch (Exception je) {
+ LOGGER.error("Failed to schedule or run job " + jobUri, je);
+ }
+ }
+
+ @Subscribe
+ public void handleUpdateJobConfigArrival(UpdateJobConfigArrivalEvent updateJobArrival) {
+ LOGGER.info("Received update for job configuration of job " + updateJobArrival.getJobName());
+ try {
+ handleDeleteJobConfigArrival(new DeleteJobConfigArrivalEvent(updateJobArrival.getJobName(),
+ updateJobArrival.getJobConfig()));
+ } catch (Exception je) {
+ LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je);
+ }
+
+ try {
+ handleNewJobConfigArrival(new NewJobConfigArrivalEvent(updateJobArrival.getJobName(),
+ updateJobArrival.getJobConfig()));
+ } catch (Exception je) {
+ LOGGER.error("Failed to update job " + updateJobArrival.getJobName(), je);
+ }
+ }
+
+ private void waitForJobCompletion(String jobName) {
+ while (this.jobRunningMap.getOrDefault(jobName, false)) {
+ LOGGER.info("Waiting for job {} to stop...", jobName);
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ LOGGER.warn("Interrupted exception encountered: ", e);
+ }
+ }
+ }
+
+ @Subscribe
+ public void handleDeleteJobConfigArrival(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException {
+ LOGGER.info("Received delete for job configuration of job " + deleteJobArrival.getJobName());
+ try {
+ unscheduleJob(deleteJobArrival.getJobName());
+ cancelJobIfRequired(deleteJobArrival);
+ } catch (JobException je) {
+ LOGGER.error("Failed to unschedule job " + deleteJobArrival.getJobName());
+ }
+ }
+
+ @Subscribe
+ public void handleCancelJobConfigArrival(CancelJobConfigArrivalEvent cancelJobArrival)
+ throws InterruptedException {
+ String jobUri = cancelJobArrival.getJoburi();
+ LOGGER.info("Received cancel for job configuration of job " + jobUri);
+ Optional distributedJobMode;
+ Optional planningJob = Optional.empty();
+ Optional actualJob = Optional.empty();
+ boolean cancelByDelete = PropertiesUtils.getPropAsBoolean(this.commonJobProperties, GobblinClusterConfigurationKeys.CANCEL_HELIX_JOB_BY_DELETE,
+ String.valueOf(GobblinClusterConfigurationKeys.DEFAULT_CANCEL_HELIX_JOB_BY_DELETE));
+
+ this.jobSchedulerMetrics.numCancellationStart.incrementAndGet();
+
+ try {
+ distributedJobMode = this.jobsMapping.getDistributedJobMode(jobUri);
+ if (distributedJobMode.isPresent() && Boolean.parseBoolean(distributedJobMode.get())) {
+ planningJob = this.jobsMapping.getPlanningJobId(jobUri);
+ } else {
+ actualJob = this.jobsMapping.getActualJobId(jobUri);
+ }
+ } catch (IOException e) {
+ LOGGER.warn("jobsMapping could not be retrieved for job {}", jobUri);
+ return;
+ }
+
+ this.jobSchedulerMetrics.numCancellationStart.decrementAndGet();
+ }
+
+
+ private void cancelJobIfRequired(DeleteJobConfigArrivalEvent deleteJobArrival) throws InterruptedException {
+
+ }
+
+ /**
+ * This class is responsible for running non-scheduled jobs.
+ */
+ class NonScheduledJobRunner implements Runnable {
+ private final Properties jobProps;
+ private final GobblinHelixJobLauncherListener jobListener;
+ private final Long creationTimeInMillis;
+
+ public NonScheduledJobRunner(Properties jobProps,
+ GobblinHelixJobLauncherListener jobListener) {
+
+ this.jobProps = jobProps;
+ this.jobListener = jobListener;
+ this.creationTimeInMillis = System.currentTimeMillis();
+ }
+
+ @Override
+ public void run() {
+ try {
+ GobblinTemporalJobScheduler.this.jobSchedulerMetrics.updateTimeBeforeJobLaunching(this.jobProps);
+ GobblinTemporalJobScheduler.this.jobSchedulerMetrics.updateTimeBetweenJobSchedulingAndJobLaunching(this.creationTimeInMillis, System.currentTimeMillis());
+ GobblinTemporalJobScheduler.this.runJob(this.jobProps, this.jobListener);
+ } catch (JobException je) {
+ LOGGER.error("Failed to run job " + this.jobProps.getProperty(ConfigurationKeys.JOB_NAME_KEY), je);
+ }
+ }
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
index 1337dfd9aa3..0633530e60f 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
@@ -18,6 +18,8 @@
package org.apache.gobblin.cluster;
import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
@@ -51,6 +53,7 @@
import io.temporal.serviceclient.WorkflowServiceStubs;
import io.temporal.worker.Worker;
import io.temporal.worker.WorkerFactory;
+import io.temporal.worker.WorkerOptions;
import lombok.Getter;
import lombok.Setter;
@@ -58,19 +61,24 @@
import org.apache.gobblin.cluster.temporal.GobblinTemporalActivityImpl;
import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflowImpl;
import org.apache.gobblin.cluster.temporal.Shared;
+import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.configuration.State;
import org.apache.gobblin.instrumented.StandardMetricsBridge;
import org.apache.gobblin.metrics.GobblinMetrics;
+import org.apache.gobblin.metrics.MultiReporterException;
import org.apache.gobblin.metrics.RootMetricContext;
import org.apache.gobblin.metrics.event.EventSubmitter;
import org.apache.gobblin.metrics.event.GobblinEventBuilder;
+import org.apache.gobblin.metrics.reporter.util.MetricReportUtils;
import org.apache.gobblin.runtime.api.TaskEventMetadataGenerator;
+import org.apache.gobblin.util.ClassAliasResolver;
import org.apache.gobblin.util.ConfigUtils;
import org.apache.gobblin.util.FileUtils;
import org.apache.gobblin.util.HadoopUtils;
import org.apache.gobblin.util.JvmUtils;
import org.apache.gobblin.util.TaskEventMetadataUtils;
import org.apache.gobblin.util.event.ContainerHealthCheckFailureEvent;
+import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs;
@@ -80,18 +88,6 @@
* {@link org.apache.gobblin.source.workunit.WorkUnit}s.
*
*
- * This class presents a Helix participant that uses a to communicate with Helix.
- * It uses Helix task execution framework and details are encapsulated in {@link TaskRunnerSuiteBase}.
- *
- *
- *
- * This class responds to a graceful shutdown initiated by the {@link GobblinTemporalClusterManager} via
- * a Helix message of subtype {@link HelixMessageSubTypes#WORK_UNIT_RUNNER_SHUTDOWN}, or it does a
- * graceful shutdown when the shutdown hook gets called. In both cases, {@link #stop()} will be
- * called to start the graceful shutdown.
- *
- *
- *
* If for some reason, the container exits or gets killed, the {@link GobblinTemporalClusterManager} will
* be notified for the completion of the container and will start a new container to replace this one.
*
@@ -106,6 +102,8 @@ public class GobblinTemporalTaskRunner implements StandardMetricsBridge {
private static final Logger logger = LoggerFactory.getLogger(GobblinTemporalTaskRunner.class);
static final java.nio.file.Path CLUSTER_CONF_PATH = Paths.get("generated-gobblin-cluster.conf");
+
+ private static TaskRunnerSuiteBase.Builder builder;
private final Optional containerMetrics;
private final Path appWorkPath;
private boolean isTaskDriver;
@@ -124,15 +122,15 @@ public class GobblinTemporalTaskRunner implements StandardMetricsBridge {
protected final FileSystem fs;
protected final String applicationName;
protected final String applicationId;
+ protected final int temporalWorkerSize;
+ private final boolean isMetricReportingFailureFatal;
+ private final boolean isEventReportingFailureFatal;
public GobblinTemporalTaskRunner(String applicationName,
String applicationId,
String taskRunnerId,
Config config,
Optional appWorkDirOptional) throws Exception {
- // Set system properties passed in via application config. As an example, Helix uses System#getProperty() for ZK configuration
- // overrides such as sessionTimeout. In this case, the overrides specified
- // in the application configuration have to be extracted and set before initializing HelixManager.
GobblinClusterUtils.setSystemProperties(config);
//Add dynamic config
@@ -150,6 +148,17 @@ public GobblinTemporalTaskRunner(String applicationName,
logger.info("Configured GobblinTaskRunner work dir to: {}", this.appWorkPath.toString());
this.containerMetrics = buildContainerMetrics();
+ this.builder = initBuilder();
+ // The default worker size would be 1
+ this.temporalWorkerSize = ConfigUtils.getInt(config, GobblinClusterConfigurationKeys.TEMPORAL_WORKER_SIZE,1);
+
+ this.isMetricReportingFailureFatal = ConfigUtils.getBoolean(this.clusterConfig,
+ ConfigurationKeys.GOBBLIN_TASK_METRIC_REPORTING_FAILURE_FATAL,
+ ConfigurationKeys.DEFAULT_GOBBLIN_TASK_METRIC_REPORTING_FAILURE_FATAL);
+
+ this.isEventReportingFailureFatal = ConfigUtils.getBoolean(this.clusterConfig,
+ ConfigurationKeys.GOBBLIN_TASK_EVENT_REPORTING_FAILURE_FATAL,
+ ConfigurationKeys.DEFAULT_GOBBLIN_TASK_EVENT_REPORTING_FAILURE_FATAL);
logger.info("GobblinTaskRunner({}): applicationName {}, applicationId {}, taskRunnerId {}, config {}, appWorkDir {}",
this.isTaskDriver ? "taskDriver" : "worker",
@@ -160,6 +169,35 @@ public GobblinTemporalTaskRunner(String applicationName,
appWorkDirOptional);
}
+ public static TaskRunnerSuiteBase.Builder getBuilder() {
+ return builder;
+ }
+
+ private TaskRunnerSuiteBase.Builder initBuilder() throws ReflectiveOperationException {
+ String builderStr = ConfigUtils.getString(this.clusterConfig,
+ GobblinClusterConfigurationKeys.TASK_RUNNER_SUITE_BUILDER,
+ TaskRunnerSuiteBase.Builder.class.getName());
+
+ String hostName = "";
+ try {
+ hostName = InetAddress.getLocalHost().getHostName();
+ } catch (UnknownHostException e) {
+ logger.warn("Cannot find host name for Helix instance: {}");
+ }
+
+ TaskRunnerSuiteBase.Builder builder = GobblinConstructorUtils.invokeLongestConstructor(
+ new ClassAliasResolver(TaskRunnerSuiteBase.Builder.class)
+ .resolveClass(builderStr), this.clusterConfig);
+
+ return builder.setAppWorkPath(this.appWorkPath)
+ .setContainerMetrics(this.containerMetrics)
+ .setFileSystem(this.fs)
+ .setApplicationId(applicationId)
+ .setApplicationName(applicationName)
+ .setContainerId(taskRunnerId)
+ .setHostName(hostName);
+ }
+
private Path initAppWorkDir(Config config, Optional appWorkDirOptional) {
return appWorkDirOptional.isPresent() ? appWorkDirOptional.get() : GobblinClusterUtils
.getAppWorkDirPathFromConfig(config, this.fs, this.applicationName, this.applicationId);
@@ -181,21 +219,32 @@ public void start()
throws ContainerHealthCheckException {
logger.info("Calling start method in GobblinTemporalTaskRunner");
logger.info(String.format("Starting in container %s", this.taskRunnerId));
+
+ // Start metric reporting
+ initMetricReporter();
+
+ // Add a shutdown hook so the task scheduler gets properly shutdown
+ addShutdownHook();
+
try {
- initiateWorker();
+ for (int i = 0; i < this.temporalWorkerSize; i++) {
+ initiateWorker();
+ }
}catch (Exception e) {
logger.info(e + " for initiate workers");
throw new RuntimeException(e);
}
-
- // Add a shutdown hook so the task scheduler gets properly shutdown
- addShutdownHook();
}
private void initiateWorker() throws Exception{
logger.info("Starting Temporal Worker");
WorkflowServiceStubs service = createServiceStubs();
+ WorkerOptions workerOptions = WorkerOptions.newBuilder()
+ .setMaxConcurrentWorkflowTaskExecutionSize(1)
+ .setMaxConcurrentActivityExecutionSize(1)
+ .build();
+
// WorkflowClient can be used to start, signal, query, cancel, and terminate Workflows.
WorkflowClient client =
WorkflowClient.newInstance(
@@ -210,7 +259,7 @@ private void initiateWorker() throws Exception{
* Define the workflow worker. Workflow workers listen to a defined task queue and process
* workflows and activities.
*/
- Worker worker = factory.newWorker(Shared.HELLO_WORLD_TASK_QUEUE);
+ Worker worker = factory.newWorker(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE, workerOptions);
/*
* Register our workflow implementation with the worker.
@@ -233,6 +282,19 @@ private void initiateWorker() throws Exception{
logger.info("A new worker is started.");
}
+ private void initMetricReporter() {
+ if (this.containerMetrics.isPresent()) {
+ try {
+ this.containerMetrics.get()
+ .startMetricReportingWithFileSuffix(ConfigUtils.configToState(this.clusterConfig), this.taskRunnerId);
+ } catch (MultiReporterException ex) {
+ if (MetricReportUtils.shouldThrowException(logger, ex, this.isMetricReportingFailureFatal, this.isEventReportingFailureFatal)) {
+ throw new RuntimeException(ex);
+ }
+ }
+ }
+ }
+
public synchronized void stop() {
if (this.isStopped) {
logger.info("Gobblin Task runner is already stopped.");
@@ -320,7 +382,7 @@ private void submitEvent(ContainerHealthCheckFailureEvent event) {
EventSubmitter eventSubmitter = new EventSubmitter.Builder(RootMetricContext.get(), getClass().getPackage().getName()).build();
GobblinEventBuilder eventBuilder = new GobblinEventBuilder(event.getClass().getSimpleName());
State taskState = ConfigUtils.configToState(event.getConfig());
- //Add task metadata such as Helix taskId, containerId, and workflowId if configured
+ //Add task metadata such as taskId, containerId, and workflowId if configured
TaskEventMetadataGenerator taskEventMetadataGenerator = TaskEventMetadataUtils.getTaskEventMetadataGenerator(taskState);
eventBuilder.addAdditionalMetadata(taskEventMetadataGenerator.getMetadata(taskState, event.getClass().getSimpleName()));
eventBuilder.addAdditionalMetadata(event.getMetadata());
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
index 93caab41c73..67fdcd4f140 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/SingleTask.java
@@ -18,6 +18,9 @@
package org.apache.gobblin.cluster;
import java.io.IOException;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
@@ -39,6 +42,9 @@
import org.apache.gobblin.broker.gobblin_scopes.GobblinScopeTypes;
import org.apache.gobblin.broker.gobblin_scopes.JobScopeInstance;
import org.apache.gobblin.broker.iface.SharedResourcesBroker;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.metrics.event.GobblinEventBuilder;
import org.apache.gobblin.runtime.AbstractJobLauncher;
import org.apache.gobblin.runtime.GobblinMultiTaskAttempt;
import org.apache.gobblin.runtime.JobState;
@@ -72,22 +78,32 @@ public class SingleTask {
private JobState _jobState;
// Preventing Helix calling cancel before taskAttempt is created
- // Checking if taskAttempt is empty is not enough, since canceller runs in different thread as runner, the case to
+ // Checking if taskAttempt is empty is not enough, since canceller runs in different thread as runner, the case
// to avoid here is taskAttempt being created and start to run after cancel has been called.
private Condition _taskAttemptBuilt;
private Lock _lock;
+ private String workflowId;
- SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs,
+ public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs,
TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig) {
this(jobId, workUnitFilePath, jobStateFilePath, fs, taskAttemptBuilder, stateStores, dynamicConfig, false);
+ this.workflowId = "";
+ }
+
+ public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs,
+ TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig, String workflowId) {
+ this(jobId, workUnitFilePath, jobStateFilePath, fs, taskAttemptBuilder, stateStores, dynamicConfig, false);
+ _logger.info("Gobblin task workflowid: {}", workflowId);
+ this.workflowId = workflowId;
}
/**
* Do all heavy-lifting of initialization in constructor which could be retried if failed,
* see the example in {@link GobblinHelixTask}.
*/
- SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs,
+ public SingleTask(String jobId, Path workUnitFilePath, Path jobStateFilePath, FileSystem fs,
TaskAttemptBuilder taskAttemptBuilder, StateStores stateStores, Config dynamicConfig, boolean skipGetJobState) {
+ _logger.info("Constructing SingleTask");
_jobId = jobId;
_workUnitFilePath = workUnitFilePath;
_jobStateFilePath = jobStateFilePath;
@@ -109,23 +125,25 @@ public class SingleTask {
}
}
- public void run()
- throws IOException, InterruptedException {
+ public void run() throws IOException, InterruptedException {
+ _logger.info("Running SingleTask");
if (_jobState == null) {
throw new RuntimeException("jobState is null. Task may have already been cancelled.");
}
+ MetricContext metricContext = MetricContext.builder("SingleTaskContext").build();
+ EventSubmitter eventSubmitter = new EventSubmitter.Builder(metricContext, "gobblin.task").build();
+ submitEvent(eventSubmitter, "GobblinTaskStarted");
+
// Add dynamic configuration to the job state
_dynamicConfig.entrySet().forEach(e -> _jobState.setProp(e.getKey(), e.getValue().unwrapped().toString()));
Config jobConfig = getConfigFromJobState(_jobState);
- _logger.debug("SingleTask.run: jobId {} workUnitFilePath {} jobStateFilePath {} jobState {} jobConfig {}",
- _jobId, _workUnitFilePath, _jobStateFilePath, _jobState, jobConfig);
+ _logger.debug("SingleTask.run: jobId {} workUnitFilePath {} jobStateFilePath {} jobState {} jobConfig {}", _jobId, _workUnitFilePath, _jobStateFilePath, _jobState, jobConfig);
- try (SharedResourcesBroker globalBroker = SharedResourcesBrokerFactory
- .createDefaultTopLevelBroker(jobConfig, GobblinScopeTypes.GLOBAL.defaultScopeInstance())) {
+ try (SharedResourcesBroker globalBroker = SharedResourcesBrokerFactory.createDefaultTopLevelBroker(jobConfig, GobblinScopeTypes.GLOBAL.defaultScopeInstance())) {
SharedResourcesBroker jobBroker = getJobBroker(_jobState, globalBroker);
// Secure atomicity of taskAttempt's execution.
@@ -135,19 +153,34 @@ public void run()
_lock.lock();
try {
_taskAttemptBuilt.signal();
+ submitEvent(eventSubmitter, "GobblinTaskAttemptBuiltSignal");
} finally {
_lock.unlock();
}
// This is a blocking call.
- _taskAttempt.runAndOptionallyCommitTaskAttempt(GobblinMultiTaskAttempt.CommitPolicy.IMMEDIATE);
+ submitEvent(eventSubmitter, "GobblinTaskAttemptRunAndOptionallyCommit");
+ _taskAttempt.runAndOptionallyCommitTaskAttempt(GobblinMultiTaskAttempt.CommitPolicy.IMMEDIATE);
} finally {
_logger.info("Clearing all metrics object in cache.");
+ submitEvent(eventSubmitter, "GobblinTaskAttemptCleanMetrics");
_taskAttempt.cleanMetrics();
}
}
+ private void submitEvent(EventSubmitter eventSubmitter, String eventName) {
+ GobblinEventBuilder eventBuilder = new GobblinEventBuilder(eventName);
+ Instant instant = Instant.now();
+ DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")
+ .withZone(ZoneId.systemDefault());
+ String formattedDateTime = formatter.format(instant);
+ eventBuilder.addMetadata("EventType", eventName);
+ eventBuilder.addMetadata("WorkflowId", this.workflowId);
+ eventBuilder.addMetadata("EventTime", formattedDateTime);
+ eventSubmitter.submit(eventBuilder);
+ }
+
private SharedResourcesBroker getJobBroker(JobState jobState,
SharedResourcesBroker globalBroker) {
return globalBroker.newSubscopedBuilder(new JobScopeInstance(jobState.getJobName(), jobState.getJobId())).build();
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java
index 269ff094794..70cbcda2a38 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivity.java
@@ -15,16 +15,21 @@
* limitations under the License.
*/
-// @@@SNIPSTART hello-world-project-template-java-activity-interface
package org.apache.gobblin.cluster.temporal;
+import java.util.Properties;
+
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+
import io.temporal.activity.ActivityInterface;
+@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.ANY)
@ActivityInterface
public interface GobblinTemporalActivity {
- // Define your activity methods which can be called during workflow execution
String composeGreeting(String name);
+ // Method to run Gobblin Task in activity
+ void run(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath, String workflowId)
+ throws Exception;
}
-// @@@SNIPEND
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java
index c1c5225f503..d6ef6f6e467 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalActivityImpl.java
@@ -15,21 +15,79 @@
* limitations under the License.
*/
-// @@@SNIPSTART hello-world-project-template-java-activity
package org.apache.gobblin.cluster.temporal;
+import java.net.URI;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+
+import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys;
+import org.apache.gobblin.cluster.GobblinHelixTaskStateTracker;
+import org.apache.gobblin.cluster.GobblinTemporalTaskRunner;
+import org.apache.gobblin.cluster.SingleTask;
+import org.apache.gobblin.cluster.TaskAttemptBuilder;
+import org.apache.gobblin.cluster.TaskRunnerSuiteBase;
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.runtime.TaskExecutor;
+import org.apache.gobblin.runtime.TaskStateTracker;
+import org.apache.gobblin.runtime.util.StateStores;
+import org.apache.gobblin.util.ConfigUtils;
public class GobblinTemporalActivityImpl implements GobblinTemporalActivity {
private static final Logger LOGGER = LoggerFactory.getLogger(GobblinTemporalActivityImpl.class);
+ private TaskRunnerSuiteBase.Builder builder;
+ private StateStores stateStores;
+ private TaskAttemptBuilder taskAttemptBuilder;
+
+
@Override
public String composeGreeting(String name) {
LOGGER.info("Activity triggered");
return "Hello " + name + "!";
}
+ private TaskAttemptBuilder createTaskAttemptBuilder() {
+ Properties properties = ConfigUtils.configToProperties(builder.getConfig());
+ TaskStateTracker taskStateTracker = new GobblinHelixTaskStateTracker(properties);
+ TaskExecutor taskExecutor = new TaskExecutor(ConfigUtils.configToProperties(builder.getConfig()));
+ TaskAttemptBuilder taskAttemptBuilder = new TaskAttemptBuilder(taskStateTracker, taskExecutor);
+ taskAttemptBuilder.setTaskStateStore(this.stateStores.getTaskStateStore());
+ return taskAttemptBuilder;
+ }
+
+ @Override
+ public void run(Properties jobProps, String appWorkDirStr, String jobId, String workUnitFilePath, String jobStateFilePath, String workflowId)
+ throws Exception {
+ Path appWorkDir = new Path(appWorkDirStr);
+ this.builder = GobblinTemporalTaskRunner.getBuilder();
+
+ Config stateStoreJobConfig = ConfigUtils.propertiesToConfig(jobProps)
+ .withValue(ConfigurationKeys.STATE_STORE_FS_URI_KEY, ConfigValueFactory.fromAnyRef(
+ new URI(appWorkDir.toUri().getScheme(), null, appWorkDir.toUri().getHost(), appWorkDir.toUri().getPort(),
+ "/", null, null).toString()));
+
+ this.stateStores =
+ new StateStores(stateStoreJobConfig, appWorkDir, GobblinClusterConfigurationKeys.OUTPUT_TASK_STATE_DIR_NAME,
+ appWorkDir, GobblinClusterConfigurationKeys.INPUT_WORK_UNIT_DIR_NAME, appWorkDir,
+ GobblinClusterConfigurationKeys.JOB_STATE_DIR_NAME);
+
+ this.taskAttemptBuilder = createTaskAttemptBuilder();
+
+ // Dynamic config is considered as part of JobState in SingleTask
+ // Important to distinguish between dynamicConfig and Config
+ final Config dynamicConfig = builder.getDynamicConfig()
+ .withValue(GobblinClusterConfigurationKeys.TASK_RUNNER_HOST_NAME_KEY, ConfigValueFactory.fromAnyRef(builder.getHostName()))
+ .withValue(GobblinClusterConfigurationKeys.CONTAINER_ID_KEY, ConfigValueFactory.fromAnyRef(builder.getContainerId()))
+ .withValue(GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_KEY, ConfigValueFactory.fromAnyRef(builder.getInstanceName()));
+
+ SingleTask singleTask = new SingleTask(jobId, new Path(workUnitFilePath), new Path(jobStateFilePath), builder.getFs(), this.taskAttemptBuilder, this.stateStores, dynamicConfig, workflowId);
+ singleTask.run();
+ }
}
-// @@@SNIPEND
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java
index 7edfcd0e81c..351a9ec7f48 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflow.java
@@ -15,20 +15,26 @@
* limitations under the License.
*/
-// @@@SNIPSTART hello-world-project-template-java-workflow-interface
package org.apache.gobblin.cluster.temporal;
+import java.util.Properties;
+
+import io.temporal.workflow.QueryMethod;
import io.temporal.workflow.WorkflowInterface;
import io.temporal.workflow.WorkflowMethod;
+
@WorkflowInterface
public interface GobblinTemporalWorkflow {
+ @QueryMethod
+ String getGreeting(String name);
+
/**
* This is the method that is executed when the Workflow Execution is started. The Workflow
* Execution completes when this method finishes execution.
*/
@WorkflowMethod
- String getGreeting(String name);
+ void runTask(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath)
+ throws Exception;
}
-// @@@SNIPEND
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java
index c2f4e76f305..49d974384c6 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/GobblinTemporalWorkflowImpl.java
@@ -19,14 +19,35 @@
package org.apache.gobblin.cluster.temporal;
import java.time.Duration;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import com.google.protobuf.Timestamp;
+
import io.temporal.activity.ActivityOptions;
+import io.temporal.api.common.v1.WorkflowExecution;
+import io.temporal.api.history.v1.HistoryEvent;
+import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryRequest;
+import io.temporal.api.workflowservice.v1.GetWorkflowExecutionHistoryResponse;
import io.temporal.common.RetryOptions;
+import io.temporal.serviceclient.WorkflowServiceStubs;
import io.temporal.workflow.Workflow;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.event.EventSubmitter;
+import org.apache.gobblin.metrics.event.GobblinEventBuilder;
+
+import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs;
+
+
public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow {
/*
@@ -36,19 +57,17 @@ public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow {
*/
private final RetryOptions retryoptions = RetryOptions.newBuilder()
- .setInitialInterval(Duration.ofSeconds(1))
- .setMaximumInterval(Duration.ofSeconds(100))
- .setBackoffCoefficient(2)
- .setMaximumAttempts(500)
+ .setMaximumAttempts(1)
.build();
+ int yearPeriod = 365 * 24 * 60 * 60;
private final ActivityOptions options = ActivityOptions.newBuilder()
- .setStartToCloseTimeout(Duration.ofSeconds(60))
+ .setStartToCloseTimeout(Duration.ofSeconds(yearPeriod))
.setRetryOptions(retryoptions)
.build();
/*
- * Define the HelloWorldActivity stub. Activity stubs are proxies for activity invocations that
+ * Define the GobblinTemporalActivity stub. Activity stubs are proxies for activity invocations that
* are executed outside of the workflow thread on the activity worker, that can be on a
* different host. Temporal is going to dispatch the activity results back to the workflow and
* unblock the stub as soon as activity is completed on the activity worker.
@@ -60,7 +79,6 @@ public class GobblinTemporalWorkflowImpl implements GobblinTemporalWorkflow {
// This is the entry point to the Workflow.
@Override
public String getGreeting(String name) {
-
/**
* If there were other Activity methods they would be orchestrated here or from within other Activities.
* This is a blocking call that returns only after the activity has completed.
@@ -68,5 +86,58 @@ public String getGreeting(String name) {
LOGGER.info("Workflow triggered");
return activity.composeGreeting(name);
}
-}
-// @@@SNIPEND
+
+ @Override
+ public void runTask(Properties jobProps, String appWorkDir, String jobId, String workUnitFilePath, String jobStateFilePath)
+ throws Exception{
+ String workflowId = Workflow.getInfo().getWorkflowId();
+ String runId = Workflow.getInfo().getRunId();
+ WorkflowExecution execution = WorkflowExecution.newBuilder()
+ .setWorkflowId(workflowId)
+ .setRunId(runId)
+ .build();
+
+ ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1);
+
+ MetricContext metricContext = MetricContext.builder("TemporalWorkflowContext").build();
+ EventSubmitter eventSubmitter = new EventSubmitter.Builder(metricContext, "gobblin.temporal").build();
+
+ final long[] lastLoggedEventId = {0};
+ executorService.scheduleAtFixedRate(() -> {
+ try {
+ GetWorkflowExecutionHistoryRequest request =
+ GetWorkflowExecutionHistoryRequest.newBuilder().setNamespace("gobblin-fastingest-internpoc").setExecution(execution).build();
+
+ WorkflowServiceStubs workflowServiceStubs = createServiceStubs();
+ GetWorkflowExecutionHistoryResponse response =
+ workflowServiceStubs.blockingStub().getWorkflowExecutionHistory(request);
+
+ for (HistoryEvent event : response.getHistory().getEventsList()) {
+ // Only log events that are newer than the last one we logged
+ if (event.getEventId() > lastLoggedEventId[0]) {
+ Timestamp timestamp = event.getEventTime();
+ Instant instant = Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos());
+ DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss")
+ .withZone(ZoneId.systemDefault());
+ String formattedDateTime = formatter.format(instant);
+
+ GobblinEventBuilder eventBuilder = new GobblinEventBuilder("TemporalEvent");
+ eventBuilder.addMetadata("WorkflowId", workflowId);
+ eventBuilder.addMetadata("EventType", event.getEventType().name());
+ eventBuilder.addMetadata("EventTime", formattedDateTime);
+ // add metadata of workflow topic
+ eventSubmitter.submit(eventBuilder);
+
+ lastLoggedEventId[0] = event.getEventId();
+ }
+ }
+ } catch (Exception e) {
+ LOGGER.error("Error retrieving workflow history", e);
+ }
+ }, 0, 10, TimeUnit.SECONDS);
+
+ activity.run(jobProps, appWorkDir, jobId, workUnitFilePath, jobStateFilePath, workflowId);
+
+ executorService.shutdown();
+ }
+}
\ No newline at end of file
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
index 9aa869af570..b282d362751 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
@@ -15,13 +15,11 @@
* limitations under the License.
*/
-// @@@SNIPSTART hello-world-project-template-java-shared-constants
package org.apache.gobblin.cluster.temporal;
public interface Shared {
// Define the task queue name
- final String HELLO_WORLD_TASK_QUEUE = "HelloWorldTaskQueue";
+ final String GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue";
-}
-// @@@SNIPEND
+}
\ No newline at end of file
diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java
index 7a851e4a15e..31c820f2478 100644
--- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java
+++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/MetricContext.java
@@ -17,8 +17,6 @@
package org.apache.gobblin.metrics;
-import lombok.Getter;
-
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
@@ -38,15 +36,14 @@
import org.slf4j.LoggerFactory;
import com.codahale.metrics.Counter;
-import com.codahale.metrics.MetricFilter;
import com.codahale.metrics.Gauge;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Meter;
import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricFilter;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.MetricSet;
import com.codahale.metrics.Timer;
-
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Optional;
@@ -59,6 +56,8 @@
import com.google.common.io.Closer;
import com.google.common.util.concurrent.MoreExecutors;
+import lombok.Getter;
+
import org.apache.gobblin.metrics.context.NameConflictException;
import org.apache.gobblin.metrics.context.ReportableContext;
import org.apache.gobblin.metrics.notification.EventNotification;
@@ -211,6 +210,7 @@ public void submitEvent(GobblinTrackingEvent nonReusableEvent) {
EventNotification notification = new EventNotification(nonReusableEvent);
sendNotification(notification);
+ LOG.info("EventBuilder {} is submitted.", nonReusableEvent);
}
/**
diff --git a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java
index 891f980b7f9..73ea591b7e9 100644
--- a/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java
+++ b/gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/EventSubmitter.java
@@ -19,15 +19,18 @@
import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
+import lombok.Getter;
+
import org.apache.gobblin.metrics.GobblinTrackingEvent;
import org.apache.gobblin.metrics.MetricContext;
-import lombok.Getter;
-
/**
* Wrapper around Avro {@link org.apache.gobblin.metrics.GobblinTrackingEvent.Builder} simplifying handling {@link org.apache.gobblin.metrics.GobblinTrackingEvent}s.
@@ -47,6 +50,8 @@ public class EventSubmitter {
@Getter
private final Optional metricContext;
+ private static final Logger LOGGER = LoggerFactory.getLogger(EventSubmitter.class);
+
public static class Builder {
private final Optional metricContext;
private final Map metadata;
diff --git a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java
index 9e3a7f70cca..124d1286f05 100644
--- a/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java
+++ b/gobblin-runtime/src/main/java/org/apache/gobblin/runtime/StreamModelTaskRunner.java
@@ -28,6 +28,11 @@
import com.google.common.io.Closer;
import com.google.common.util.concurrent.Futures;
+import io.reactivex.Flowable;
+import io.reactivex.flowables.ConnectableFlowable;
+import io.reactivex.schedulers.Schedulers;
+import lombok.AllArgsConstructor;
+
import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.converter.Converter;
import org.apache.gobblin.fork.ForkOperator;
@@ -45,11 +50,6 @@
import org.apache.gobblin.writer.WatermarkManager;
import org.apache.gobblin.writer.WatermarkStorage;
-import io.reactivex.Flowable;
-import io.reactivex.flowables.ConnectableFlowable;
-import io.reactivex.schedulers.Schedulers;
-import lombok.AllArgsConstructor;
-
/**
* A helper class to run {@link Task} in stream mode. Prevents {@link Task} from loading reactivex classes when not
@@ -147,7 +147,7 @@ protected void run() throws Exception {
forkedStream = forkedStream.mapStream(f -> f.observeOn(Schedulers.from(this.taskExecutor.getForkExecutor()), false, bufferSize));
}
Fork fork = new Fork(this.taskContext, forkedStream.getGlobalMetadata().getSchema(), forkedStreams.getForkedStreams().size(), fidx, this.taskMode);
- fork.consumeRecordStream(forkedStream);
+ // fork.consumeRecordStream(forkedStream);
this.forks.put(Optional.of(fork), Optional.of(Futures.immediateFuture(null)));
this.task.configureStreamingFork(fork);
}
diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java
index 4ef48c9765a..00c8e746dcf 100644
--- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinTemporalApplicationMaster.java
@@ -55,7 +55,7 @@
/**
- * The Yarn ApplicationMaster class for Gobblin.
+ * The Yarn ApplicationMaster class for Gobblin using Temporal.
*
*
* This class runs the {@link YarnTemporalService} for all Yarn-related stuffs like ApplicationMaster registration
diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java
index 7088dfa996c..490ed72b94f 100644
--- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnConfigurationKeys.java
@@ -151,4 +151,5 @@ public class GobblinYarnConfigurationKeys {
//Config to control Heartbeat interval for Yarn AMRM client.
public static final String AMRM_HEARTBEAT_INTERVAL_SECS = GOBBLIN_YARN_PREFIX + "amRmHeartbeatIntervalSecs";
public static final Integer DEFAULT_AMRM_HEARTBEAT_INTERVAL_SECS = 15;
+ public static final String TEMPORAL_WORKERPOOL_SIZE = "temporal.workerpool.size";
}
diff --git a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java
index a45017b5c23..421a4a6300a 100644
--- a/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java
+++ b/gobblin-yarn/src/main/java/org/apache/gobblin/yarn/YarnTemporalService.java
@@ -422,12 +422,14 @@ private EventSubmitter buildEventSubmitter() {
* @return whether successfully requested the target number of containers
*/
public synchronized boolean requestTargetNumberOfContainers(YarnContainerRequestBundle yarnContainerRequestBundle, Set inUseInstances) {
- LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}",
- yarnContainerRequestBundle.getTotalContainers(), inUseInstances.size(), this.containerMap.size());
-
int defaultContainerMemoryMbs = config.getInt(GobblinYarnConfigurationKeys.CONTAINER_MEMORY_MBS_KEY);
int defaultContainerCores = config.getInt(GobblinYarnConfigurationKeys. CONTAINER_CORES_KEY);
- int workerPoolSize = ConfigUtils.getInt(config, "temporal.workerpool.size",2);
+ // making workerPoolSize configurable, the default value would be 10
+ int workerPoolSize = ConfigUtils.getInt(config, GobblinYarnConfigurationKeys.TEMPORAL_WORKERPOOL_SIZE,10);
+
+ LOGGER.info("Trying to set numTargetContainers={}, in-use helix instances count is {}, container map size is {}",
+ workerPoolSize, inUseInstances.size(), this.containerMap.size());
+
requestContainers(workerPoolSize, Resource.newInstance(defaultContainerMemoryMbs, defaultContainerCores));
this.yarnContainerRequest = yarnContainerRequestBundle;
@@ -701,42 +703,16 @@ protected void handleContainerCompletion(ContainerStatus containerStatus) {
Optional.of(completedContainerInfo.getContainer()) : Optional.absent(), newContainerResource));
}
- /**
- * Handles containers aborted. This method handles 2 cases:
- *
- * -
- * Case 1: Gobblin AM intentionally requested container to be released (often because the number of helix tasks
- * has decreased due to decreased traffic)
- *
- * -
- * Case 2: Unexpected hardware fault and the node is lost. Need to do specific Helix logic to ensure 2 helix tasks
- * are not being run by the multiple containers
- *
- *
- * @param containerStatus
- * @param completedContainerInfo
- * @param completedInstanceName
- * @return if release request was intentionally released (Case 1)
- */
private boolean handleAbortedContainer(ContainerStatus containerStatus, ContainerInfo completedContainerInfo,
String completedInstanceName) {
-
- // Case 1: Container intentionally released
if (this.releasedContainerCache.getIfPresent(containerStatus.getContainerId()) != null) {
LOGGER.info("Container release requested, so not spawning a replacement for containerId {}", containerStatus.getContainerId());
if (completedContainerInfo != null) {
LOGGER.info("Adding instance {} to the pool of unused instances", completedInstanceName);
this.unusedHelixInstanceNames.add(completedInstanceName);
}
-
return true;
}
-
- // Case 2: Container release was not requested. Likely, the container was running on a node on which the NM died.
- // In this case, RM assumes that the containers are "lost", even though the container process may still be
- // running on the node. We need to ensure that the Helix instances running on the orphaned containers
- // are fenced off from the Helix cluster to avoid double publishing and state being committed by the
- // instances.
LOGGER.info("Container {} aborted due to lost NM", containerStatus.getContainerId());
return false;
}
@@ -767,8 +743,6 @@ private ImmutableMap.Builder buildContainerStatusEventMetadata(C
* The RM can return containers that are larger (because of normalization etc).
* Container may be larger by memory or cpu (e.g. container (1000M, 3cpu) can fit request (1000M, 1cpu) or request (500M, 3cpu).
*
- * Thankfully since each helix tag / resource has a different priority, matching requests for one helix tag / resource
- * have complete isolation from another helix tag / resource
*/
private int getMatchingRequestsCount(Resource resource) {
Integer priorityNum = resourcePriorityMap.get(resource.toString());
diff --git a/gradle/scripts/dependencyDefinitions.gradle b/gradle/scripts/dependencyDefinitions.gradle
index a26183a1922..ba961bbf8f6 100644
--- a/gradle/scripts/dependencyDefinitions.gradle
+++ b/gradle/scripts/dependencyDefinitions.gradle
@@ -215,7 +215,6 @@ ext.externalDependency = [
"postgresConnector": "org.postgresql:postgresql:42.1.4",
"testContainers": "org.testcontainers:testcontainers:1.17.3",
"testContainersMysql": "org.testcontainers:mysql:1.17.3",
- "workflowClient.namespace": "gobblin-fastingest-internpoc",
"xz": "org.tukaani:xz:1.8"
]
From f49aedb1b42584b146b678d2f770faf0c88921df Mon Sep 17 00:00:00 2001
From: Yiming Yang
Date: Mon, 28 Aug 2023 22:39:04 -0700
Subject: [PATCH 5/8] Temporal POC
---
.../cluster/GobblinTemporalJobLauncher.java | 76 ++++++++----
.../cluster/GobblinTemporalTaskRunner.java | 43 ++-----
.../AbstractNestingExecWorkflowImpl.java | 116 ++++++++++++++++++
.../temporal/AbstractTemporalWorker.java | 42 +++++++
.../temporal/CollectionBackedTaskSpan.java | 51 ++++++++
.../cluster/temporal/IllustrationTask.java | 16 +++
.../temporal/IllustrationTaskActivity.java | 16 +++
.../IllustrationTaskActivityImpl.java | 12 ++
.../cluster/temporal/NestingExecWorker.java | 21 ++++
.../cluster/temporal/NestingExecWorkflow.java | 30 +++++
.../temporal/NestingExecWorkflowImpl.java | 34 +++++
.../gobblin/cluster/temporal/Shared.java | 1 -
.../temporal/SimpleGeneratedWorkload.java | 44 +++++++
.../TemporalWorkflowClientFactory.java | 92 ++++++++++++++
.../gobblin/cluster/temporal/WFAddr.java | 48 ++++++++
.../gobblin/cluster/temporal/Workload.java | 37 ++++++
16 files changed, 619 insertions(+), 60 deletions(-)
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java
create mode 100644 gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
index 5f79e596fcf..ee7b803b81d 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
@@ -28,6 +28,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -51,6 +52,11 @@
import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.cluster.temporal.GobblinTemporalWorkflow;
import org.apache.gobblin.cluster.temporal.Shared;
+import org.apache.gobblin.cluster.temporal.IllustrationTask;
+import org.apache.gobblin.cluster.temporal.Workload;
+import org.apache.gobblin.cluster.temporal.SimpleGeneratedWorkload;
+import org.apache.gobblin.cluster.temporal.NestingExecWorkflow;
+import org.apache.gobblin.cluster.temporal.WFAddr;
import org.apache.gobblin.configuration.ConfigurationKeys;
import org.apache.gobblin.metastore.StateStore;
import org.apache.gobblin.metrics.Tag;
@@ -76,7 +82,8 @@
import org.apache.gobblin.util.SerializationUtils;
import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs;
-
+import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createServiceInstance;
+import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createClientInstance;
/**
* An implementation of {@link JobLauncher} that launches a Gobblin job using the Temporal task framework.
@@ -150,10 +157,8 @@ public GobblinTemporalJobLauncher(Properties jobProps, Path appWorkDir,
new TaskStateCollectorService(jobProps, this.jobContext.getJobState(), this.eventBus, this.eventSubmitter,
this.stateStores.getTaskStateStore(), this.outputTaskStateDir, this.getIssueRepository());
- this.workflowServiceStubs = createServiceStubs();
- this.client = WorkflowClient.newInstance(
- workflowServiceStubs, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build());
-
+ this.workflowServiceStubs = createServiceInstance();
+ this.client = createClientInstance(workflowServiceStubs);
/*
* Set Workflow options such as WorkflowId and Task Queue so the worker knows where to list and which workflows to execute.
*/
@@ -246,27 +251,50 @@ private void submitJobToTemporal(List workUnits) throws Exception{
String jobStateFilePathStr = jobStateFilePath.toString();
List> futures = new ArrayList<>();
- AtomicInteger multiTaskIdSequence = new AtomicInteger(0);
- AtomicInteger workflowCount = new AtomicInteger(0);
- int workflowSize = 100;
- ExecutorService executor = Executors.newFixedThreadPool(workflowSize);
-
- for (int i = 0; i < workflowSize; i++) {
- WorkUnit workUnit = workUnits.get(i);
+ boolean fastIngest = false;
+ if (fastIngest) {
+ AtomicInteger multiTaskIdSequence = new AtomicInteger(0);
+ AtomicInteger workflowCount = new AtomicInteger(0);
+ int workflowSize = 100;
+ ExecutorService executor = Executors.newFixedThreadPool(workflowSize);
+ for (int i = 0; i < workflowSize; i++) {
+ WorkUnit workUnit = workUnits.get(i);
+ futures.add(CompletableFuture.runAsync(() -> {
+ try {
+ if (workUnit instanceof MultiWorkUnit) {
+ workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement()));
+ }
+ String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner);
+ String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement();
+ WorkflowOptions options = WorkflowOptions.newBuilder()
+ .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE)
+ .setWorkflowId(workflowId)
+ .build();
+ // TODO(yiyang): change up the workflow
+ GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options);
+ LOGGER.info("Setting up temporal workflow {}", workflowId);
+ workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }, executor));
+ }
+ } else {
+ // TODO(yiyang): how do we determine 100 tasks in total. what if more tasks come in
+ int numTasks = 100;
+ int maxBranchesPerTree = 20;
+ int maxSubTreesPerTree = 5;
+ // TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time
+ ExecutorService executor = Executors.newFixedThreadPool(100);
futures.add(CompletableFuture.runAsync(() -> {
try {
- if (workUnit instanceof MultiWorkUnit) {
- workUnit.setId(JobLauncherUtils.newMultiTaskId(this.jobContext.getJobId(), multiTaskIdSequence.getAndIncrement()));
- }
- String workUnitFilePathStr = persistWorkUnit(new Path(this.inputWorkUnitDir, this.jobContext.getJobId()), workUnit, stateSerDeRunner);
- String workflowId = workUnit.getProp(KafkaSource.TOPIC_NAME) + "_" + workflowCount.getAndIncrement();
- WorkflowOptions options = WorkflowOptions.newBuilder()
- .setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE)
- .setWorkflowId(workflowId)
- .build();
- GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options);
- LOGGER.info("Setting up temporal workflow {}", workflowId);
- workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr);
+ Workload workload = SimpleGeneratedWorkload.createAs(numTasks);
+ // WARNING: although type param must agree w/ that of `workload`, it's entirely unverified by type checker!
+ // ...and more to the point, mismatch would occur at runtime (`performWork` on whichever workflow underpins stub)!
+ WorkflowOptions options = WorkflowOptions.newBuilder().setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE).build();
+ NestingExecWorkflow workflow =
+ this.client.newWorkflowStub(NestingExecWorkflow.class, options);
+ workflow.performWork(WFAddr.ROOT, workload, 0, maxBranchesPerTree, maxSubTreesPerTree, Optional.empty());
} catch (Exception e) {
throw new RuntimeException(e);
}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
index 0633530e60f..de99b17d971 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalTaskRunner.java
@@ -33,6 +33,7 @@
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
+import org.apache.gobblin.cluster.temporal.NestingExecWorker;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -81,6 +82,8 @@
import org.apache.gobblin.util.reflection.GobblinConstructorUtils;
import static org.apache.gobblin.cluster.GobblinTemporalClusterManager.createServiceStubs;
+import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createClientInstance;
+import static org.apache.gobblin.cluster.temporal.TemporalWorkflowClientFactory.createServiceInstance;
/**
@@ -238,47 +241,17 @@ public void start()
private void initiateWorker() throws Exception{
logger.info("Starting Temporal Worker");
- WorkflowServiceStubs service = createServiceStubs();
+
+ WorkflowServiceStubs service = createServiceInstance();
+ WorkflowClient client = createClientInstance(service);
WorkerOptions workerOptions = WorkerOptions.newBuilder()
.setMaxConcurrentWorkflowTaskExecutionSize(1)
.setMaxConcurrentActivityExecutionSize(1)
.build();
- // WorkflowClient can be used to start, signal, query, cancel, and terminate Workflows.
- WorkflowClient client =
- WorkflowClient.newInstance(
- service, WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build());
-
- /*
- * Define the workflow factory. It is used to create workflow workers that poll specific Task Queues.
- */
- WorkerFactory factory = WorkerFactory.newInstance(client);
-
- /*
- * Define the workflow worker. Workflow workers listen to a defined task queue and process
- * workflows and activities.
- */
- Worker worker = factory.newWorker(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE, workerOptions);
-
- /*
- * Register our workflow implementation with the worker.
- * Workflow implementations must be known to the worker at runtime in
- * order to dispatch workflow tasks.
- */
- worker.registerWorkflowImplementationTypes(GobblinTemporalWorkflowImpl.class);
-
- /*
- * Register our Activity Types with the Worker. Since Activities are stateless and thread-safe,
- * the Activity Type is a shared instance.
- */
- worker.registerActivitiesImplementations(new GobblinTemporalActivityImpl());
-
- /*
- * Start all the workers registered for a specific task queue.
- * The started workers then start polling for workflows and activities.
- */
- factory.start();
+ NestingExecWorker worker = new NestingExecWorker(client, Shared.GOBBLIN_TEMPORAL_TASK_QUEUE);
+ worker.start();
logger.info("A new worker is started.");
}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java
new file mode 100644
index 00000000000..b04c4e9a355
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractNestingExecWorkflowImpl.java
@@ -0,0 +1,116 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.api.enums.v1.ParentClosePolicy;
+import io.temporal.workflow.Async;
+import io.temporal.workflow.ChildWorkflowOptions;
+import io.temporal.workflow.Promise;
+import io.temporal.workflow.Workflow;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.compress.utils.Lists;
+
+/** Core skeleton of {@link NestingExecWorkflow}: realizing classes need only define {@link #launchAsyncActivity} */
+@Slf4j
+public abstract class AbstractNestingExecWorkflowImpl implements NestingExecWorkflow {
+ @Override
+ public int performWork(
+ final WFAddr addr,
+ final Workload workload,
+ final int startIndex,
+ final int maxBranchesPerTree,
+ final int maxSubTreesPerTree,
+ final Optional maxSubTreesForCurrentTreeOverride) {
+ final int maxSubTreesForCurrent = maxSubTreesForCurrentTreeOverride.orElse(maxSubTreesPerTree);
+ final int maxLeaves = maxBranchesPerTree - maxSubTreesForCurrent;
+ final Optional> optSpan = workload.getSpan(startIndex, maxLeaves);
+ log.info("[" + addr + "] " + workload + " w/ start '" + startIndex + "'" + "; tree (" + maxBranchesPerTree + "/" + maxSubTreesPerTree + "): " + optSpan);
+ if (!optSpan.isPresent()) {
+ return 0;
+ } else {
+ final Workload.TaskSpan taskSpan = optSpan.get();
+ final Iterable iterable = () -> taskSpan;
+ final List> childActivities = StreamSupport.stream(iterable.spliterator(), false)
+ .map(t -> launchAsyncActivity(t))
+ .collect(Collectors.toList());
+ final List> childSubTrees = new ArrayList<>();
+ if (taskSpan.getNumElems() == maxLeaves) { // received as many as requested (did not stop short)
+ int subTreeId = 0;
+ for (int subTreeChildMaxSubTreesPerTree
+ : consolidateSubTreeGrandChildren(maxSubTreesForCurrent, maxBranchesPerTree, maxSubTreesPerTree)) {
+ // CAUTION: calc these *before* incrementing `subTreeId`!
+ final int childStartIndex = startIndex + maxLeaves + (maxBranchesPerTree * subTreeId);
+ final int nextChildId = maxLeaves + subTreeId;
+ final WFAddr childAddr = addr.createChild(nextChildId);
+ final NestingExecWorkflow child = createChildWorkflow(childAddr);
+ if (!workload.isIndexKnownToExceed(childStartIndex)) { // best-effort short-circuiting
+ childSubTrees.add(
+ Async.function(child::performWork, childAddr, workload, childStartIndex, maxBranchesPerTree,
+ maxSubTreesPerTree, Optional.of(subTreeChildMaxSubTreesPerTree)));
+ ++subTreeId;
+ }
+ }
+ }
+ final Promise allActivityChildren = Promise.allOf(childActivities);
+ allActivityChildren.get(); // ensure all complete prior to counting them in `overallActivitiesRollupCount`
+ // TODO: determine whether any benefit to unordered `::get` blocking for any next ready (perhaps no difference...)
+ final int descendantActivitiesRollupCount = childSubTrees.stream().map(Promise::get).reduce(0, (x, y) -> x + y);
+ final int overallActivitiesRollupCount = taskSpan.getNumElems() + descendantActivitiesRollupCount;
+ log.info("[" + addr + "] activites finished coordinating: " + overallActivitiesRollupCount);
+ return overallActivitiesRollupCount;
+ }
+ }
+
+ /** Factory for invoking the specific activity by providing it args via {@link Async::function} */
+ protected abstract Promise launchAsyncActivity(TASK task);
+
+ protected NestingExecWorkflow createChildWorkflow(final WFAddr childAddr) {
+ ChildWorkflowOptions childOpts = ChildWorkflowOptions.newBuilder()
+ .setParentClosePolicy(ParentClosePolicy.PARENT_CLOSE_POLICY_ABANDON)
+ .setWorkflowId("NestingExecWorkflow-" + childAddr)
+ .build();
+ return Workflow.newChildWorkflowStub(NestingExecWorkflow.class, childOpts);
+ }
+
+ /**
+ * "right-tilt" sub-tree's grandchildren, so final child gets all grandchildren (vs. constant grandchildren/child)
+ * i.e. NOT!:
+ * List naiveUniformity = Collections.nCopies(numSubTreesPerSubTree, numSubTreeChildren);
+ * @return each sub-tree's desired size, in ascending sub-tree order
+ */
+ protected static List consolidateSubTreeGrandChildren(
+ final int numSubTreesPerSubTree,
+ final int numChildrenTotal,
+ final int numSubTreeChildren
+ ) {
+ if (numSubTreesPerSubTree <= 0) {
+ return Lists.newArrayList();
+ } else if (isSqrt(numSubTreeChildren, numChildrenTotal)) {
+ // redistribute all grandchild sub-trees to pack every grandchild beneath the final child sub-tree
+ final List grandChildCounts = new ArrayList<>(Collections.nCopies(numSubTreesPerSubTree - 1, 0));
+ grandChildCounts.add(numChildrenTotal);
+ return grandChildCounts;
+ } else {
+ final int totalGrandChildSubTrees = numSubTreesPerSubTree * numSubTreeChildren;
+ final int numTreesWithSolelySubTreeBranches = totalGrandChildSubTrees / numChildrenTotal;
+ final int numSubTreesRemaining = totalGrandChildSubTrees % numChildrenTotal;
+ assert (numTreesWithSolelySubTreeBranches == 1 && numSubTreesRemaining == 0) || numTreesWithSolelySubTreeBranches == 0
+ : "present limitation: at most one sub-tree may use further branching: (found: numSubTreesPerSubTree: "
+ + numSubTreesPerSubTree + "; numChildrenTotal: " + numChildrenTotal + " / numSubTreeChildren: "
+ + numSubTreeChildren + ")";
+ final List grandChildCounts = new ArrayList<>(Collections.nCopies(numSubTreesPerSubTree - (numTreesWithSolelySubTreeBranches + 1), 0));
+ grandChildCounts.addAll(Collections.nCopies(Math.min(1, numSubTreesPerSubTree - numTreesWithSolelySubTreeBranches), numSubTreesRemaining));
+ grandChildCounts.addAll(Collections.nCopies(Math.min(numTreesWithSolelySubTreeBranches, numSubTreesPerSubTree), numChildrenTotal));
+ return grandChildCounts;
+ }
+ }
+
+ /** @return whether `maxSubTrees` == `Math.sqrt(maxBranches)` */
+ private static boolean isSqrt(int maxSubTrees, int maxBranches) {
+ return maxSubTrees > 0 && maxSubTrees * maxSubTrees == maxBranches;
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java
new file mode 100644
index 00000000000..0613eed020b
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/AbstractTemporalWorker.java
@@ -0,0 +1,42 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.client.WorkflowClient;
+import io.temporal.worker.Worker;
+import io.temporal.worker.WorkerOptions;
+import io.temporal.worker.WorkerFactory;
+public abstract class AbstractTemporalWorker {
+ private final WorkflowClient workflowClient;
+ private final String queueName;
+ private final WorkerFactory workerFactory;
+
+ public AbstractTemporalWorker(WorkflowClient client, String queue) {
+ workflowClient = client;
+ queueName = queue;
+ // Create a Worker factory that can be used to create Workers that poll specific Task Queues.
+ workerFactory = WorkerFactory.newInstance(workflowClient);
+ }
+
+ public void start() {
+ Worker worker = workerFactory.newWorker(queueName);
+ // This Worker hosts both Workflow and Activity implementations.
+ // Workflows are stateful, so you need to supply a type to create instances.
+ worker.registerWorkflowImplementationTypes(getWorkflowImplClasses());
+ // Activities are stateless and thread safe, so a shared instance is used.
+ worker.registerActivitiesImplementations(getActivityImplInstances());
+ // Start polling the Task Queue.
+ workerFactory.start();
+ }
+
+ /**
+ * Shuts down the worker.
+ */
+ public void shutdown() {
+ workerFactory.shutdown();
+ }
+
+ /** @return workflow types for *implementation* classes (not interface) */
+ protected abstract Class>[] getWorkflowImplClasses();
+
+ /** @return activity instances; NOTE: activities must be stateless and thread-safe, so a shared instance is used. */
+ protected abstract Object[] getActivityImplInstances();
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java
new file mode 100644
index 00000000000..a1b302afe97
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/CollectionBackedTaskSpan.java
@@ -0,0 +1,51 @@
+package org.apache.gobblin.cluster.temporal;
+import java.util.Iterator;
+import java.util.List;
+import lombok.NoArgsConstructor;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+
+/** Logical sub-sequence of `Task`s, backed for simplicity's sake by an in-memory collection */
+@NoArgsConstructor
+@RequiredArgsConstructor
+public class CollectionBackedTaskSpan implements Workload.TaskSpan {
+ @NonNull
+ private List elems;
+ // CAUTION: despite the "warning: @NonNull is meaningless on a primitive @lombok.RequiredArgsConstructor"...
+ // if removed, no two-arg ctor is generated, so syntax error on `new CollectionBackedTaskSpan(elems, startIndex)`
+ @NonNull
+ private int startingIndex;
+ private transient Iterator statefulDelegatee = null;
+
+ @Override
+ public int getNumElems() {
+ return elems.size();
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (statefulDelegatee == null) {
+ statefulDelegatee = elems.iterator();
+ }
+ return statefulDelegatee.hasNext();
+ }
+
+ @Override
+ public T next() {
+ if (statefulDelegatee == null) {
+ throw new IllegalStateException("first call `hasNext()`!");
+ }
+ return statefulDelegatee.next();
+ }
+
+ @Override
+ public String toString() {
+ return getClassNickname() + "(" + startingIndex + "... {+" + getNumElems() + "})";
+ }
+
+ protected String getClassNickname() {
+ // return getClass().getSimpleName();
+ return "TaskSpan";
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java
new file mode 100644
index 00000000000..43f1e6bd6f0
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTask.java
@@ -0,0 +1,16 @@
+package org.apache.gobblin.cluster.temporal;
+
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+
+/** Generally, this would specify what "work" needs performing plus how to perform; now just a unique name (to log) */
+@Data
+@NoArgsConstructor // IMPORTANT: for jackson (de)serialization
+@RequiredArgsConstructor
+public class IllustrationTask {
+ @NonNull
+ private String name;
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java
new file mode 100644
index 00000000000..d24217fe45f
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivity.java
@@ -0,0 +1,16 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.activity.ActivityInterface;
+import io.temporal.activity.ActivityMethod;
+/**
+ * Activity for processing {@link IllustrationTask}s
+ *
+ * CAUTION/FINDING: an `@ActivityInterface` must not be parameterized (e.g. here, by TASK), as doing so results in:
+ * io.temporal.failure.ApplicationFailure: message='class java.util.LinkedHashMap cannot be cast to class
+ * com.linkedin.temporal.app.work.IllustrationTask', type='java.lang.ClassCastException'
+ */
+@ActivityInterface
+public interface IllustrationTaskActivity {
+ @ActivityMethod
+ String doTask(IllustrationTask task);
+}
\ No newline at end of file
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java
new file mode 100644
index 00000000000..c170829f09e
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/IllustrationTaskActivityImpl.java
@@ -0,0 +1,12 @@
+package org.apache.gobblin.cluster.temporal;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class IllustrationTaskActivityImpl implements IllustrationTaskActivity {
+ @Override
+ public String doTask(final IllustrationTask task) {
+ log.info("Now performing - '" + task.getName() + "'");
+ return task.getName();
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java
new file mode 100644
index 00000000000..327a28c8d91
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorker.java
@@ -0,0 +1,21 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.client.WorkflowClient;
+import io.temporal.worker.Worker;
+import io.temporal.worker.WorkerOptions;
+import io.temporal.worker.WorkerFactory;
+public class NestingExecWorker extends AbstractTemporalWorker{
+ public NestingExecWorker(WorkflowClient workflowClient, String queueName) {
+ super(workflowClient, queueName);
+ }
+
+ @Override
+ protected Class>[] getWorkflowImplClasses() {
+ return new Class[] { NestingExecWorkflowImpl.class };
+ }
+
+ @Override
+ protected Object[] getActivityImplInstances() {
+ return new Object[] { new IllustrationTaskActivityImpl() };
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java
new file mode 100644
index 00000000000..8887d71b74c
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflow.java
@@ -0,0 +1,30 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.workflow.WorkflowInterface;
+import io.temporal.workflow.WorkflowMethod;
+import java.util.Optional;
+
+/**
+ * Process all `TASK`s of `workload`, from `startIndex` to the end by creating child workflows, where this and
+ * descendants should have at most `maxBranchesPerTree`, with at most `maxSubTreesPerTree` of those being child
+ * workflows. (Non-child-workflow branches being activities.)
+ *
+ * IMPORTANT: `Math.sqrt(maxBranchesPerTree) == maxSubTreesPerTree` provides a good rule-of-thumb; `maxSubTreesPerTree
+ * should not exceed that.
+ *
+ * @param the type of task for which to invoke an appropriate activity
+ * @param maxSubTreesForCurrentTreeOverride when the current tree should use different max sub-trees than descendants
+ */
+
+@WorkflowInterface
+public interface NestingExecWorkflow {
+ @WorkflowMethod
+ int performWork(
+ WFAddr addr,
+ Workload workload,
+ int startIndex,
+ int maxBranchesPerTree,
+ int maxSubTreesPerTree,
+ Optional maxSubTreesForCurrentTreeOverride
+ );
+}
\ No newline at end of file
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java
new file mode 100644
index 00000000000..4f16c1101cc
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/NestingExecWorkflowImpl.java
@@ -0,0 +1,34 @@
+package org.apache.gobblin.cluster.temporal;
+
+import io.temporal.activity.ActivityOptions;
+import io.temporal.common.RetryOptions;
+import io.temporal.workflow.Async;
+import io.temporal.workflow.Promise;
+import io.temporal.workflow.Workflow;
+import java.time.Duration;
+
+/** {@link com.linkedin.temporal.app.workflow.nesting.NestingExecWorkflow} for {@link IllustrationTask} */
+public class NestingExecWorkflowImpl
+ extends AbstractNestingExecWorkflowImpl {
+
+ // RetryOptions specify how to automatically handle retries when Activities fail.
+ private static final RetryOptions ACTIVITY_RETRY_OPTS = RetryOptions.newBuilder()
+ .setInitialInterval(Duration.ofSeconds(1))
+ .setMaximumInterval(Duration.ofSeconds(100))
+ .setBackoffCoefficient(2)
+ .setMaximumAttempts(3)
+ .build();
+
+ private static final ActivityOptions ACTIVITY_OPTS = ActivityOptions.newBuilder()
+ .setStartToCloseTimeout(Duration.ofSeconds(10))
+ .setRetryOptions(ACTIVITY_RETRY_OPTS)
+ .build();
+
+ private final IllustrationTaskActivity activityStub =
+ Workflow.newActivityStub(IllustrationTaskActivity.class, ACTIVITY_OPTS);
+
+ @Override
+ protected Promise launchAsyncActivity(final IllustrationTask t) {
+ return Async.function(activityStub::doTask, t);
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
index b282d362751..3d7e51a459e 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Shared.java
@@ -21,5 +21,4 @@ public interface Shared {
// Define the task queue name
final String GOBBLIN_TEMPORAL_TASK_QUEUE = "GobblinTemporalTaskQueue";
-
}
\ No newline at end of file
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java
new file mode 100644
index 00000000000..93461bfe27d
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/SimpleGeneratedWorkload.java
@@ -0,0 +1,44 @@
+package org.apache.gobblin.cluster.temporal;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import lombok.AccessLevel;
+
+/** Example, illustration workload that synthesizes tasks; genuine {@link Workload}s likely arise from query/calc */
+@lombok.AllArgsConstructor(access = AccessLevel.PRIVATE)
+@lombok.NoArgsConstructor // IMPORTANT: for jackson (de)serialization
+@lombok.ToString
+public class SimpleGeneratedWorkload implements Workload {
+ private int numTasks;
+
+ /** Factory method */
+ public static SimpleGeneratedWorkload createAs(final int numTasks) {
+ return new SimpleGeneratedWorkload(numTasks);
+ }
+
+ @Override
+ public Optional> getSpan(final int startIndex, final int numElements) {
+ if (startIndex >= numTasks || startIndex < 0) {
+ return Optional.empty();
+ } else {
+ List elems = IntStream.range(startIndex, Math.min(startIndex + numElements, numTasks))
+ .mapToObj(n -> new IllustrationTask("task-" + n + "-of-" + numTasks))
+ .collect(Collectors.toList());
+ return Optional.of(new CollectionBackedTaskSpan<>(elems, startIndex));
+ }
+ }
+
+ @Override
+ public boolean isIndexKnownToExceed(final int index) {
+ return isDefiniteSize() && index >= numTasks;
+ }
+
+ @Override
+ @JsonIgnore // (because no-arg method resembles 'java bean property')
+ public boolean isDefiniteSize() {
+ return true;
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
new file mode 100644
index 00000000000..76b6dd983c1
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
@@ -0,0 +1,92 @@
+package org.apache.gobblin.cluster.temporal;
+
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts;
+import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext;
+import io.temporal.client.WorkflowClient;
+import io.temporal.client.WorkflowClientOptions;
+import io.temporal.serviceclient.WorkflowServiceStubs;
+import io.temporal.serviceclient.WorkflowServiceStubsOptions;
+import org.apache.gobblin.cluster.GobblinClusterUtils;
+
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.File;
+import java.security.KeyStore;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.gobblin.security.ssl.SSLContextFactory.toInputStream;
+
+public class TemporalWorkflowClientFactory {
+
+ public static WorkflowServiceStubs createServiceInstance() throws Exception {
+ GobblinClusterUtils.setSystemProperties(ConfigFactory.load());
+ Config config = GobblinClusterUtils.addDynamicConfig(ConfigFactory.load());
+ String SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT = "gobblin.kafka.sharedConfig.";
+ String SSL_KEYMANAGER_ALGORITHM = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keymanager.algorithm";
+ String SSL_KEYSTORE_TYPE = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.type";
+ String SSL_KEYSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.keystore.location";
+ String SSL_KEY_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.key.password";
+ String SSL_TRUSTSTORE_LOCATION = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.location";
+ String SSL_TRUSTSTORE_PASSWORD = SHARED_KAFKA_CONFIG_PREFIX_WITH_DOT + "ssl.truststore.password";
+
+ List SSL_CONFIG_DEFAULT_SSL_PROTOCOLS = Collections.unmodifiableList(
+ Arrays.asList("TLSv1.2"));
+ List SSL_CONFIG_DEFAULT_CIPHER_SUITES = Collections.unmodifiableList(Arrays.asList(
+ // The following list is from https://github.com/netty/netty/blob/4.1/codec-http2/src/main/java/io/netty/handler/codec/http2/Http2SecurityUtil.java#L50
+ "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",
+
+ /* REQUIRED BY HTTP/2 SPEC */
+ "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",
+ /* REQUIRED BY HTTP/2 SPEC */
+
+ "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",
+ "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",
+ "TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256",
+ "TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256"
+ ));
+
+ String keyStoreType = config.getString(SSL_KEYSTORE_TYPE);
+ File keyStoreFile = new File(config.getString(SSL_KEYSTORE_LOCATION));
+ String keyStorePassword = config.getString(SSL_KEY_PASSWORD);
+
+ KeyStore keyStore = KeyStore.getInstance(keyStoreType);
+ keyStore.load(toInputStream(keyStoreFile), keyStorePassword.toCharArray());
+
+ // Set key manager from key store
+ String sslKeyManagerAlgorithm = config.getString(SSL_KEYMANAGER_ALGORITHM);
+ KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(sslKeyManagerAlgorithm);
+ keyManagerFactory.init(keyStore, keyStorePassword.toCharArray());
+
+ // Set trust manager from trust store
+ KeyStore trustStore = KeyStore.getInstance("JKS");
+ File trustStoreFile = new File(config.getString(SSL_TRUSTSTORE_LOCATION));
+
+ String trustStorePassword = config.getString(SSL_TRUSTSTORE_PASSWORD);
+ trustStore.load(toInputStream(trustStoreFile), trustStorePassword.toCharArray());
+ TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance("SunX509");
+ trustManagerFactory.init(trustStore);
+
+ SslContext sslContext = GrpcSslContexts.forClient()
+ .keyManager(keyManagerFactory)
+ .trustManager(trustManagerFactory)
+ .protocols(SSL_CONFIG_DEFAULT_SSL_PROTOCOLS)
+ .ciphers(SSL_CONFIG_DEFAULT_CIPHER_SUITES)
+ .build();
+
+ WorkflowServiceStubsOptions options = WorkflowServiceStubsOptions.newBuilder()
+ .setTarget("1.nephos-temporal.corp-lca1.atd.corp.linkedin.com:7233")
+ .setEnableHttps(true)
+ .setSslContext(sslContext)
+ .build();
+ return WorkflowServiceStubs.newServiceStubs(options);
+ }
+
+ public static WorkflowClient createClientInstance(WorkflowServiceStubs service) {
+ WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-gaas-poc").build();
+ return WorkflowClient.newInstance(service, options);
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java
new file mode 100644
index 00000000000..e7de7f347c0
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/WFAddr.java
@@ -0,0 +1,48 @@
+package org.apache.gobblin.cluster.temporal;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.List;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+
+
+/** Hierarchical address for nesting workflows (0-based). */
+@NoArgsConstructor // IMPORTANT: for jackson (de)serialization
+@RequiredArgsConstructor
+public class WFAddr {
+ public static final String SEP = ".";
+
+ /** initial, top-level address */
+ public static final WFAddr ROOT = new WFAddr(0);
+
+ @Getter
+ @NonNull // IMPORTANT: for jackson (de)serialization (which won't permit `final`)
+ private List segments;
+
+ public WFAddr(final int firstLevelOnly) {
+ this(Lists.newArrayList(firstLevelOnly));
+ }
+
+ /** @return 0-based depth */
+ @JsonIgnore // (because no-arg method resembles 'java bean property')
+ public int getDepth() {
+ return segments.size() - 1;
+ }
+
+ /** Create a child of the current `WFAddr` */
+ public WFAddr createChild(int childLevel) {
+ final List copy = new ArrayList<>(segments);
+ copy.add(childLevel);
+ return new WFAddr(copy);
+ }
+
+ @Override
+ public String toString() {
+ return Joiner.on(SEP).join(segments);
+ }
+}
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java
new file mode 100644
index 00000000000..2957eeb2a7f
--- /dev/null
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/Workload.java
@@ -0,0 +1,37 @@
+package org.apache.gobblin.cluster.temporal;
+
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import java.util.Iterator;
+import java.util.Optional;
+
+
+/**
+ * An assemblage of "work", modeled as sequential "task" specifications. Given Temporal's required determinism, tasks
+ * and task spans should remain unchanged, with stable sequential ordering. This need not constrain `Workload`s to
+ * eager, advance elaboration: "streaming" definition is possible, so long as producing a deterministic result.
+ *
+ * A actual, real-world workload might correspond to datastore contents, such as records serialized into HDFS files
+ * or ordered DB query results.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "@class") // to handle impls
+
+public interface Workload {
+
+ /**
+ * @return a sequential sub-sequence, from `startIndex` (0-based), unless it falls beyond the underlying sequence
+ * NOTE: this is a blocking call that forces elaboration: `TaskSpan.getNumElems() < numElements` signifies end of seq
+ */
+ Optional> getSpan(int startIndex, int numElements);
+
+ /** Non-blocking, best-effort advice: to support non-strict elaboration, does NOT guarantee `index` will not exceed */
+ boolean isIndexKnownToExceed(int index);
+
+ default boolean isDefiniteSize() {
+ return false;
+ }
+
+ /** Logical sub-sequence 'slice' of contiguous "tasks" */
+ public interface TaskSpan extends Iterator {
+ int getNumElems();
+ }
+}
\ No newline at end of file
From b21def1df5693986c918f5f735150dbd786ece64 Mon Sep 17 00:00:00 2001
From: Yiming Yang
Date: Tue, 29 Aug 2023 11:39:48 -0700
Subject: [PATCH 6/8] minor changes
---
.../org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java | 2 +-
.../gobblin/cluster/temporal/TemporalWorkflowClientFactory.java | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
index ee7b803b81d..239ce55c827 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
@@ -285,7 +285,7 @@ private void submitJobToTemporal(List workUnits) throws Exception{
int maxBranchesPerTree = 20;
int maxSubTreesPerTree = 5;
// TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time
- ExecutorService executor = Executors.newFixedThreadPool(100);
+ ExecutorService executor = Executors.newFixedThreadPool(1);
futures.add(CompletableFuture.runAsync(() -> {
try {
Workload workload = SimpleGeneratedWorkload.createAs(numTasks);
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
index 76b6dd983c1..7a958bb7858 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/temporal/TemporalWorkflowClientFactory.java
@@ -86,7 +86,7 @@ public static WorkflowServiceStubs createServiceInstance() throws Exception {
}
public static WorkflowClient createClientInstance(WorkflowServiceStubs service) {
- WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-gaas-poc").build();
+ WorkflowClientOptions options = WorkflowClientOptions.newBuilder().setNamespace("gobblin-fastingest-internpoc").build();
return WorkflowClient.newInstance(service, options);
}
}
From ec381f92c68499a2fa2e9270f9501788176d2734 Mon Sep 17 00:00:00 2001
From: Yiming Yang
Date: Thu, 31 Aug 2023 10:20:24 -0700
Subject: [PATCH 7/8] fix a few things
---
.../org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java | 2 --
1 file changed, 2 deletions(-)
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
index 239ce55c827..a823fafa726 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
@@ -280,11 +280,9 @@ private void submitJobToTemporal(List workUnits) throws Exception{
}, executor));
}
} else {
- // TODO(yiyang): how do we determine 100 tasks in total. what if more tasks come in
int numTasks = 100;
int maxBranchesPerTree = 20;
int maxSubTreesPerTree = 5;
- // TODO(yiyang): workflow size is also variable but we fix our thread pool ahead of time
ExecutorService executor = Executors.newFixedThreadPool(1);
futures.add(CompletableFuture.runAsync(() -> {
try {
From 2d08d53c1461ab21185e3bc37a04f139785b3e4a Mon Sep 17 00:00:00 2001
From: Yiming Yang
Date: Fri, 1 Sep 2023 13:58:01 -0700
Subject: [PATCH 8/8] make properties
---
.../gobblin/cluster/GobblinTemporalJobLauncher.java | 9 +++++----
1 file changed, 5 insertions(+), 4 deletions(-)
diff --git a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
index a823fafa726..6c062558619 100644
--- a/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
+++ b/gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinTemporalJobLauncher.java
@@ -270,7 +270,6 @@ private void submitJobToTemporal(List workUnits) throws Exception{
.setTaskQueue(Shared.GOBBLIN_TEMPORAL_TASK_QUEUE)
.setWorkflowId(workflowId)
.build();
- // TODO(yiyang): change up the workflow
GobblinTemporalWorkflow workflow = this.client.newWorkflowStub(GobblinTemporalWorkflow.class, options);
LOGGER.info("Setting up temporal workflow {}", workflowId);
workflow.runTask(jobProps, appWorkDir.toString(), getJobId(), workUnitFilePathStr, jobStateFilePathStr);
@@ -280,9 +279,11 @@ private void submitJobToTemporal(List workUnits) throws Exception{
}, executor));
}
} else {
- int numTasks = 100;
- int maxBranchesPerTree = 20;
- int maxSubTreesPerTree = 5;
+ // l0: (root)workflow
+ // l1: act0, act2, .... act14, (sub)workflow0, ... (sub)workflow4
+ int numTasks = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.size", 100);
+ int maxBranchesPerTree = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.maxBranchesPerTree", 20);
+ int maxSubTreesPerTree = PropertiesUtils.getPropAsInt(this.jobProps, "temporal.task.maxSubTreesPerTree", 5);
ExecutorService executor = Executors.newFixedThreadPool(1);
futures.add(CompletableFuture.runAsync(() -> {
try {