You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/12/17 02:45:57 UTC

[GitHub] [incubator-pinot] npawar commented on a change in pull request #6340: Adding Pinot minion segment generation and push task.

npawar commented on a change in pull request #6340:
URL: https://github.com/apache/incubator-pinot/pull/6340#discussion_r544765893



##########
File path: pinot-tools/src/main/resources/examples/minions/batch/airlineStats/airlineStats_offline_table_config.json
##########
@@ -0,0 +1,45 @@
+{
+  "tableName": "airlineStats",
+  "tableType": "OFFLINE",
+  "segmentsConfig": {
+    "timeColumnName": "DaysSinceEpoch",
+    "timeType": "DAYS",
+    "segmentPushType": "APPEND",
+    "segmentAssignmentStrategy": "BalanceNumSegmentAssignmentStrategy",
+    "replication": "1"
+  },
+  "tenants": {},
+  "tableIndexConfig": {
+    "loadMode": "MMAP"
+  },
+  "metadata": {
+    "customConfigs": {}
+  },
+  "ingestionConfig": {
+    "batchIngestionConfig": {
+      "segmentIngestionType": "APPEND",
+      "segmentIngestionFrequency": "DAILY",
+      "batchConfigMaps": [
+        {
+          "inputDirURI": "s3://my.s3.bucket/batch/airlineStats/rawdata/",
+          "fs.className": "org.apache.pinot.plugin.filesystem.S3PinotFS",

Review comment:
       aren't these expected to be "input.fs.className" and "input.fs.prop.region" ? Will this work without input/output prefix?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/SegmentGenerationAndPushTaskGenerator.java
##########
@@ -0,0 +1,267 @@
+/**
+ * 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.pinot.controller.helix.core.minion.generator;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.FileSystems;
+import java.nio.file.PathMatcher;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.minion.ClusterInfoAccessor;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class SegmentGenerationAndPushTaskGenerator implements PinotTaskGenerator {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentGenerationAndPushTaskGenerator.class);
+
+  private final ClusterInfoAccessor _clusterInfoAccessor;
+
+  public SegmentGenerationAndPushTaskGenerator(ClusterInfoAccessor clusterInfoAccessor) {
+    _clusterInfoAccessor = clusterInfoAccessor;
+  }
+
+  @Override
+  public String getTaskType() {
+    return MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE;
+  }
+
+  @Override
+  public List<PinotTaskConfig> generateTasks(List<TableConfig> tableConfigs) {
+    List<PinotTaskConfig> pinotTaskConfigs = new ArrayList<>();
+
+    for (TableConfig tableConfig : tableConfigs) {
+      // Only generate tasks for OFFLINE tables
+      String offlineTableName = tableConfig.getTableName();
+      if (tableConfig.getTableType() != TableType.OFFLINE) {
+        LOGGER.warn("Skip generating SegmentGenerationAndPushTask for non-OFFLINE table: {}", offlineTableName);
+        continue;
+      }
+
+      TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+      Preconditions.checkNotNull(tableTaskConfig);
+      Map<String, String> taskConfigs =
+          tableTaskConfig.getConfigsForTaskType(MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE);
+      if (tableConfigs == null) {
+        LOGGER.warn("Skip null task config for table: {}", offlineTableName);
+      }
+
+      // Get max number of tasks for this table
+      int tableMaxNumTasks;
+      String tableMaxNumTasksConfig = taskConfigs.get(MinionConstants.TABLE_MAX_NUM_TASKS_KEY);
+      if (tableMaxNumTasksConfig != null) {
+        try {
+          tableMaxNumTasks = Integer.parseInt(tableMaxNumTasksConfig);
+        } catch (NumberFormatException e) {
+          tableMaxNumTasks = Integer.MAX_VALUE;
+        }
+      } else {
+        tableMaxNumTasks = Integer.MAX_VALUE;
+      }
+
+      // Generate tasks
+      int tableNumTasks = 0;
+      // Generate up to tableMaxNumTasks tasks each time for each table
+      if (tableNumTasks == tableMaxNumTasks) {
+        break;
+      }
+      String batchSegmentIngestionType = IngestionConfigUtils.getBatchSegmentIngestionType(tableConfig);
+      String batchSegmentIngestionFrequency = IngestionConfigUtils.getBatchSegmentIngestionFrequency(tableConfig);
+      BatchIngestionConfig batchIngestionConfig = tableConfig.getIngestionConfig().getBatchIngestionConfig();
+      List<Map<String, String>> batchConfigMaps = batchIngestionConfig.getBatchConfigMaps();
+      for (Map<String, String> batchConfigMap : batchConfigMaps) {
+        try {
+          URI inputDirURI = getDirectoryUri(batchConfigMap.get(BatchConfigProperties.INPUT_DIR_URI));
+          updateRecordReaderConfigs(batchConfigMap);
+          List<OfflineSegmentZKMetadata> offlineSegmentsMetadata = Collections.emptyList();
+          // For append mode, we don't create segments for input file URIs already created.
+          if (BatchConfigProperties.SegmentIngestionType.APPEND.name().equalsIgnoreCase(batchSegmentIngestionType)) {
+            offlineSegmentsMetadata = this._clusterInfoAccessor.getOfflineSegmentsMetadata(offlineTableName);
+          }
+          List<URI> inputFileURIs = getInputFilesFromDirectory(batchConfigMap, inputDirURI,
+              getExistingSegmentInputFiles(offlineSegmentsMetadata));
+
+          for (URI inputFileURI : inputFileURIs) {
+            Map<String, String> singleFileGenerationTaskConfig =
+                getSingleFileGenerationTaskConfig(offlineTableName, tableNumTasks, batchConfigMap, inputFileURI);
+            pinotTaskConfigs.add(new PinotTaskConfig(MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE,
+                singleFileGenerationTaskConfig));
+            tableNumTasks++;
+
+            // Generate up to tableMaxNumTasks tasks each time for each table
+            if (tableNumTasks == tableMaxNumTasks) {
+              break;
+            }
+          }
+        } catch (Exception e) {
+          LOGGER.error("Unable to generate the SegmentGenerationAndPush task. [ table configs: {}, task configs: {} ]",
+              tableConfig, taskConfigs, e);
+        }
+      }
+    }
+    return pinotTaskConfigs;
+  }
+
+  private Map<String, String> getSingleFileGenerationTaskConfig(String offlineTableName, int sequenceID,
+      Map<String, String> batchConfigMap, URI inputFileURI)
+      throws JsonProcessingException, URISyntaxException {
+
+    URI inputDirURI = getDirectoryUri(batchConfigMap.get(BatchConfigProperties.INPUT_DIR_URI));
+    URI outputDirURI = getDirectoryUri(batchConfigMap.get(BatchConfigProperties.OUTPUT_DIR_URI));
+    String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap);
+
+    Map<String, String> singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap);
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.INPUT_FILE_URI, inputFileURI.toString());
+    URI outputSegmentDirURI = getRelativeOutputPath(inputDirURI, inputFileURI, outputDirURI);
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString());
+    singleFileGenerationTaskConfig
+        .put(BatchConfigProperties.SCHEMA, JsonUtils.objectToString(_clusterInfoAccessor.getTableSchema(offlineTableName)));
+    singleFileGenerationTaskConfig
+        .put(BatchConfigProperties.TABLE_CONFIGS, JsonUtils.objectToString(_clusterInfoAccessor.getTableConfig(offlineTableName)));
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.SEQUENCE_ID, String.valueOf(sequenceID));
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.SEGMENT_NAME_GENERATOR_TYPE, BatchConfigProperties.SegmentNameGeneratorType.SIMPLE);
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode);
+    singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl());
+    return singleFileGenerationTaskConfig;
+  }
+
+  private void updateRecordReaderConfigs(Map<String, String> batchConfigMap) {
+    String inputFormat = batchConfigMap.get(BatchConfigProperties.INPUT_FORMAT);
+    String recordReaderClassName = PluginManager.get().getRecordReaderClassName(inputFormat);
+    if (recordReaderClassName != null) {
+      batchConfigMap.putIfAbsent(BatchConfigProperties.RECORD_READER_CLASS, recordReaderClassName);
+    }
+    String recordReaderConfigClassName = PluginManager.get().getRecordReaderConfigClassName(inputFormat);
+    if (recordReaderConfigClassName != null) {
+      batchConfigMap.putIfAbsent(BatchConfigProperties.RECORD_READER_CONFIG_CLASS, recordReaderConfigClassName);
+    }
+  }
+
+  private List<URI> getInputFilesFromDirectory(Map<String, String> batchConfigMap, URI inputDirURI,
+      Set<String> existingSegmentInputFileURIs) {
+    String inputDirURIScheme = inputDirURI.getScheme();
+    if (!PinotFSFactory.isSchemeSupported(inputDirURIScheme)) {
+      String fsClass = batchConfigMap.get(BatchConfigProperties.INPUT_FS_CLASS);
+      PinotConfiguration fsProps = IngestionConfigUtils.getFsProps(batchConfigMap);
+      PinotFSFactory.register(inputDirURIScheme, fsClass, fsProps);
+    }
+    PinotFS inputDirFS = PinotFSFactory.create(inputDirURIScheme);
+
+    String includeFileNamePattern = batchConfigMap.get(BatchConfigProperties.INCLUDE_FILE_NAME_PATTERN);
+    String excludeFileNamePattern = batchConfigMap.get(BatchConfigProperties.EXCLUDE_FILE_NAME_PATTERN);
+
+    //Get list of files to process
+    String[] files;
+    try {
+      files = inputDirFS.listFiles(inputDirURI, true);
+    } catch (IOException e) {
+      LOGGER.error("Unable to list files under URI: " + inputDirURI, e);
+      return Collections.emptyList();
+    }
+    PathMatcher includeFilePathMatcher = null;
+    if (includeFileNamePattern != null) {
+      includeFilePathMatcher = FileSystems.getDefault().getPathMatcher(includeFileNamePattern);
+    }
+    PathMatcher excludeFilePathMatcher = null;
+    if (excludeFileNamePattern != null) {
+      excludeFilePathMatcher = FileSystems.getDefault().getPathMatcher(excludeFileNamePattern);
+    }
+    List<URI> inputFileURIs = new ArrayList<>();
+    for (String file : files) {
+      if (includeFilePathMatcher != null) {
+        if (!includeFilePathMatcher.matches(Paths.get(file))) {
+          continue;
+        }
+      }
+      if (excludeFilePathMatcher != null) {
+        if (excludeFilePathMatcher.matches(Paths.get(file))) {
+          continue;
+        }
+      }
+      try {
+        URI inputFileURI = new URI(file);
+        if (inputFileURI.getScheme() == null) {
+          inputFileURI = new File(file).toURI();
+        }
+        if (inputDirFS.isDirectory(inputFileURI) || existingSegmentInputFileURIs.contains(inputFileURI.toString())) {
+          continue;
+        }
+        inputFileURIs.add(inputFileURI);
+      } catch (Exception e) {
+        continue;
+      }
+    }
+    return inputFileURIs;
+  }
+
+  private Set<String> getExistingSegmentInputFiles(List<OfflineSegmentZKMetadata> offlineSegmentsMetadata) {
+    Set<String> existingSegmentInputFiles = new HashSet<>();
+    for (OfflineSegmentZKMetadata metadata : offlineSegmentsMetadata) {
+      if ((metadata.getCustomMap() != null) && metadata.getCustomMap()
+          .containsKey(BatchConfigProperties.INPUT_DATA_FILE_URI_KEY)) {
+        existingSegmentInputFiles.add(metadata.getCustomMap().get(BatchConfigProperties.INPUT_DATA_FILE_URI_KEY));
+      }
+    }
+    return existingSegmentInputFiles;
+  }
+
+  private URI getDirectoryUri(String uriStr)
+      throws URISyntaxException {
+    URI uri = new URI(uriStr);
+    if (uri.getScheme() == null) {
+      uri = new File(uriStr).toURI();
+    }
+    return uri;
+  }
+
+  public static URI getRelativeOutputPath(URI baseInputDir, URI inputFile, URI outputDir) {

Review comment:
       this method can be private?

##########
File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/SegmentGenerationAndPushTaskGenerator.java
##########
@@ -0,0 +1,267 @@
+/**
+ * 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.pinot.controller.helix.core.minion.generator;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.FileSystems;
+import java.nio.file.PathMatcher;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
+import org.apache.pinot.controller.helix.core.minion.ClusterInfoAccessor;
+import org.apache.pinot.core.common.MinionConstants;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableTaskConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class SegmentGenerationAndPushTaskGenerator implements PinotTaskGenerator {

Review comment:
       please add some java docs on the Generator and the Executor and ExecutorFactory for the new task, explaining which configs are looked at and how they are used

##########
File path: pinot-minion/src/main/java/org/apache/pinot/minion/executor/SegmentGenerationAndPushTaskExecutor.java
##########
@@ -0,0 +1,256 @@
+/**
+ * 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.pinot.minion.executor;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.minion.PinotTaskConfig;
+import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationTaskRunner;
+import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationUtils;
+import org.apache.pinot.plugin.ingestion.batch.common.SegmentPushUtils;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.filesystem.LocalPinotFS;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.batch.spec.PinotClusterSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.PushJobSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.RecordReaderSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationTaskSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.SegmentNameGeneratorSpec;
+import org.apache.pinot.spi.ingestion.batch.spec.TableSpec;
+import org.apache.pinot.spi.utils.DataSizeUtils;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.apache.pinot.spi.utils.retry.AttemptsExceededException;
+import org.apache.pinot.spi.utils.retry.RetriableOperationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class SegmentGenerationAndPushTaskExecutor extends BaseTaskExecutor {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SegmentGenerationAndPushTaskExecutor.class);
+
+  private static final PinotFS LOCAL_PINOT_FS = new LocalPinotFS();
+  private static final int DEFUALT_PUSH_ATTEMPTS = 5;
+  private static final int DEFAULT_PUSH_PARALLELISM = 1;
+  private static final long DEFAULT_PUSH_RETRY_INTERVAL_MILLIS = 1000L;
+
+  @Override
+  public Object executeTask(PinotTaskConfig pinotTaskConfig) {
+    Map<String, String> taskConfigs = pinotTaskConfig.getConfigs();
+    SegmentGenerationAndPushResult.Builder resultBuilder = new SegmentGenerationAndPushResult.Builder();
+    File localTempDir = new File(FileUtils.getTempDirectory(), "pinot-" + UUID.randomUUID());
+
+    try {
+      // Generate Pinot Segment
+      SegmentGenerationTaskSpec taskSpec = generateTaskSpec(taskConfigs, localTempDir);
+      SegmentGenerationTaskRunner taskRunner = new SegmentGenerationTaskRunner(taskSpec);
+      String segmentName = taskRunner.run();
+
+      // Tar segment directory to compress file
+      File localSegmentTarFile = tarSegmentDir(taskSpec, segmentName);
+
+      //move segment to output PinotFS
+      URI outputSegmentTarURI = moveSegmentToOutputPinotFS(taskConfigs, localSegmentTarFile);
+
+      resultBuilder.setSegmentName(segmentName);
+      // Segment push task
+      pushSegment(taskSpec.getTableConfig().get(BatchConfigProperties.TABLE_NAME).asText(), taskConfigs,
+          outputSegmentTarURI);
+      resultBuilder.setSucceed(true);
+    } catch (Exception e) {
+      resultBuilder.setException(e);
+    } finally {
+      // Cleanup output dir
+      FileUtils.deleteQuietly(localTempDir);
+    }
+    return resultBuilder.build();
+  }
+
+  private void pushSegment(String tableName, Map<String, String> taskConfigs, URI outputSegmentTarURI)
+      throws Exception {
+    String pushMode = taskConfigs.get(BatchConfigProperties.PUSH_MODE);
+
+    PushJobSpec pushJobSpec = new PushJobSpec();
+    pushJobSpec.setPushAttempts(DEFUALT_PUSH_ATTEMPTS);
+    pushJobSpec.setPushParallelism(DEFAULT_PUSH_PARALLELISM);
+    pushJobSpec.setPushRetryIntervalMillis(DEFAULT_PUSH_RETRY_INTERVAL_MILLIS);
+    pushJobSpec.setSegmentUriPrefix(taskConfigs.get(BatchConfigProperties.PUSH_SEGMENT_URI_PREFIX));
+    pushJobSpec.setSegmentUriSuffix(taskConfigs.get(BatchConfigProperties.PUSH_SEGMENT_URI_SUFFIX));
+
+    SegmentGenerationJobSpec spec = generatePushJobSpec(tableName, taskConfigs, pushJobSpec);
+
+    URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI));
+    PinotFS outputFileFS = getPinotFS(outputSegmentDirURI);
+    switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) {
+      case TAR:
+        try {
+          SegmentPushUtils.pushSegments(spec, LOCAL_PINOT_FS, Arrays.asList(outputSegmentTarURI.toString()));
+        } catch (RetriableOperationException | AttemptsExceededException e) {
+          throw new RuntimeException(e);
+        }
+        break;
+      case URI:
+        try {
+          List<String> segmentUris = new ArrayList<>();
+          URI updatedURI = SegmentPushUtils
+              .generateSegmentTarURI(outputSegmentDirURI, outputSegmentTarURI, pushJobSpec.getSegmentUriPrefix(),
+                  pushJobSpec.getSegmentUriSuffix());
+          segmentUris.add(updatedURI.toString());
+          SegmentPushUtils.sendSegmentUris(spec, segmentUris);
+        } catch (RetriableOperationException | AttemptsExceededException e) {
+          throw new RuntimeException(e);
+        }
+        break;
+      case METADATA:
+        try {
+          Map<String, String> segmentUriToTarPathMap = SegmentPushUtils
+              .getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec.getSegmentUriPrefix(),
+                  pushJobSpec.getSegmentUriSuffix(), new String[]{outputSegmentTarURI.toString()});
+          SegmentPushUtils.sendSegmentUriAndMetadata(spec, outputFileFS, segmentUriToTarPathMap);
+        } catch (RetriableOperationException | AttemptsExceededException e) {
+          throw new RuntimeException(e);
+        }
+        break;
+      default:
+        throw new UnsupportedOperationException("Unrecognized push mode - " + pushMode);
+    }
+  }
+
+  private SegmentGenerationJobSpec generatePushJobSpec(String tableName, Map<String, String> taskConfigs,
+      PushJobSpec pushJobSpec) {
+
+    TableSpec tableSpec = new TableSpec();
+    tableSpec.setTableName(tableName);
+
+    PinotClusterSpec pinotClusterSpec = new PinotClusterSpec();
+    pinotClusterSpec.setControllerURI(taskConfigs.get(BatchConfigProperties.PUSH_CONTROLLER_URI));
+    PinotClusterSpec[] pinotClusterSpecs = new PinotClusterSpec[]{pinotClusterSpec};
+
+    SegmentGenerationJobSpec spec = new SegmentGenerationJobSpec();
+    spec.setPushJobSpec(pushJobSpec);
+    spec.setTableSpec(tableSpec);
+    spec.setPinotClusterSpecs(pinotClusterSpecs);
+    return spec;
+  }
+
+  private URI moveSegmentToOutputPinotFS(Map<String, String> taskConfigs, File localSegmentTarFile)
+      throws Exception {
+    URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI));
+    PinotFS outputFileFS = getPinotFS(outputSegmentDirURI);

Review comment:
       Doesn't the output FS also need to be registered if not present? Or is the assumption that it will be same as input fs?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org