jenniferdai commented on a change in pull request #4253: Add segment 
pre-processing Hadoop job
URL: https://github.com/apache/incubator-pinot/pull/4253#discussion_r290838353
 
 

 ##########
 File path: 
pinot-hadoop/src/main/java/org/apache/pinot/hadoop/job/SegmentPreprocessingJob.java
 ##########
 @@ -0,0 +1,504 @@
+/**
+ * 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.hadoop.job;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.zip.GZIPInputStream;
+import javax.annotation.Nullable;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.mapred.AvroKey;
+import org.apache.avro.mapred.AvroValue;
+import org.apache.avro.mapreduce.AvroJob;
+import org.apache.avro.mapreduce.AvroKeyOutputFormat;
+import org.apache.avro.mapreduce.AvroMultipleOutputs;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
+import org.apache.pinot.common.Utils;
+import org.apache.pinot.common.config.ColumnPartitionConfig;
+import org.apache.pinot.common.config.IndexingConfig;
+import org.apache.pinot.common.config.SegmentPartitionConfig;
+import org.apache.pinot.common.config.TableConfig;
+import org.apache.pinot.hadoop.io.CombineAvroKeyInputFormat;
+import org.apache.pinot.hadoop.job.mapper.SegmentPreprocessingMapper;
+import org.apache.pinot.hadoop.job.partitioners.GenericPartitioner;
+import org.apache.pinot.hadoop.job.reducer.SegmentPreprocessingReducer;
+import org.apache.pinot.hadoop.utils.PushLocation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.pinot.hadoop.job.JobConfigConstants.*;
+
+
+public class SegmentPreprocessingJob extends Configured {
+  private static final Logger _logger = 
LoggerFactory.getLogger(SegmentPreprocessingJob.class);
+  private final Properties _properties;
+  private Configuration _configuration;
+
+  private boolean _enablePartitioning = false;
+  private boolean _enableSorting = false;
+  private boolean _enableResizing = false;
+
+  private String _partitionColumn;
+  private int _numberOfPartitions;
+  private String _partitionFunction;
+  private String _sortedColumn;
+  private int _numberOfOutputFiles;
+
+  private final String _inputSegmentDir;
+  private final String _preprocessedOutputDir;
+  protected final String _rawTableName;
+
+  private final String _hdfsDefaultClasspathDir;
+  private final String _defaultPermissionsMask;
+  protected final List<PushLocation> _pushLocations;
+
+  private TableConfig _tableConfig;
+  protected FileSystem _fileSystem;
+
+  public SegmentPreprocessingJob(final Properties properties) {
+    _properties = properties;
+    _configuration = new Configuration();
+    setConf(_configuration);
+    _configuration.set("mapreduce.job.user.classpath.first", "true");
+    Utils.logVersions();
+    logProperties();
+
+    if (_properties.getProperty(ENABLE_PARTITIONING) != null) {
+      _enablePartitioning = 
Boolean.parseBoolean(_properties.getProperty(ENABLE_PARTITIONING));
+    }
+
+    if (_properties.getProperty(ENABLE_SORTING) != null) {
+      _enableSorting = 
Boolean.parseBoolean(_properties.getProperty(ENABLE_SORTING));
+    }
+
+    if (!_enablePartitioning && _properties.getProperty(ENABLE_RESIZING) != 
null) {
+      _enableResizing = 
Boolean.parseBoolean(_properties.getProperty(ENABLE_RESIZING));
+    }
+
+    // get input/output paths.
+    _inputSegmentDir = properties.getProperty(PATH_TO_INPUT);
+    _preprocessedOutputDir = properties.getProperty(PATH_TO_OUTPUT);
+    _rawTableName = 
Preconditions.checkNotNull(_properties.getProperty(JobConfigConstants.SEGMENT_TABLE_NAME));
+
+    _hdfsDefaultClasspathDir = 
_properties.getProperty("hdfs.default.classpath.dir", null);
+    _defaultPermissionsMask = 
_properties.getProperty(JobConfigConstants.DEFAULT_PERMISSIONS_MASK, null);
+
+    // Optional push location and table parameters. If set, will use the table 
config and schema from the push hosts.
+    String pushHostsString = 
_properties.getProperty(JobConfigConstants.PUSH_TO_HOSTS);
+    String pushPortString = 
_properties.getProperty(JobConfigConstants.PUSH_TO_PORT);
+    if (pushHostsString != null && pushPortString != null) {
+      _pushLocations =
+          PushLocation.getPushLocations(StringUtils.split(pushHostsString, 
','), Integer.parseInt(pushPortString));
+    } else {
+      _pushLocations = null;
+    }
+
+    
_logger.info("*********************************************************************");
+    _logger.info("enable.partitioning: {}", _enablePartitioning);
+    _logger.info("enable.sorting: {}", _enableSorting);
+    _logger.info("enable.resizing: {}", _enableResizing);
+    _logger.info("path.to.input: {}", _inputSegmentDir);
+    _logger.info("path.to.output: {}", _preprocessedOutputDir);
+    _logger.info("hdfs.default.classpath.dir: {}", _hdfsDefaultClasspathDir);
+    _logger.info("push.locations: {}", _pushLocations);
+    
_logger.info("*********************************************************************");
+  }
+
+  public void run() throws Exception {
+    if (!_enablePartitioning && !_enableSorting && !_enableResizing) {
+      _logger.info("Pre-computing job is disabled.");
+      return;
+    } else {
+      _logger.info("Starting {}", getClass().getSimpleName());
+    }
+
+    _fileSystem = FileSystem.get(_configuration);
+    final List<String> inputDataPath = 
Arrays.asList(_inputSegmentDir.split("\\s*,\\s*"));
+    Path outputDir = new Path(_preprocessedOutputDir);
+
+    if (_fileSystem.exists(outputDir)) {
+      _logger.warn("Found the output folder {}, deleting it", 
_preprocessedOutputDir);
+      _fileSystem.delete(outputDir, true);
+    }
+    setDirPermission(outputDir);
+
+    _tableConfig = getTableConfig();
+
+    if (_enablePartitioning) {
+      fetchPartitioningConfig();
+      _logger.info("partition column: {}", _partitionColumn);
+      _logger.info("number of partitions: {}", _numberOfPartitions);
+      _logger.info("partition column: {}", _partitionColumn);
+    }
+
+    if (_enableSorting) {
+      fetchSortingConfig();
+      _logger.info("sorted.column: {}", _sortedColumn);
+    }
+
+    if (_enableResizing) {
+      fetchResizingConfig();
+      _logger.info("minimum number of output files: {}", _numberOfOutputFiles);
+    }
+
+    _logger.info("Initializing a pre-computing job");
+    Job job = Job.getInstance(_configuration);
+
+    job.getConfiguration().set(JobContext.JOB_NAME, this.getClass().getName());
+    job.getConfiguration().set(MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST, 
"true");
+    
job.getConfiguration().set(FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
 "false");
+
+    job.setJarByClass(SegmentPreprocessingJob.class);
+
+    String hadoopTokenFileLocation = 
System.getenv("HADOOP_TOKEN_FILE_LOCATION");
+    if (hadoopTokenFileLocation != null) {
+      job.getConfiguration().set("mapreduce.job.credentials.binary", 
hadoopTokenFileLocation);
+    }
+
+    // Schema configs.
+    Schema schema = getSchema(inputDataPath.get(0));
+    _logger.info("Schema is: {}", schema.toString(true));
+
+    // Mapper configs.
+    job.setMapperClass(SegmentPreprocessingMapper.class);
+    job.setMapOutputKeyClass(AvroKey.class);
+    job.setMapOutputValueClass(AvroValue.class);
+    job.getConfiguration().setInt(JobContext.NUM_MAPS, inputDataPath.size());
+
+    // Reducer configs.
+    job.setReducerClass(SegmentPreprocessingReducer.class);
+    job.setOutputKeyClass(AvroKey.class);
+    job.setOutputValueClass(NullWritable.class);
+    if (_properties.getProperty(MAXIMUM_NUMBER_OF_RECORDS) != null) {
+      _logger.info("Setting {} to {}", MAXIMUM_NUMBER_OF_RECORDS, 
_properties.getProperty(MAXIMUM_NUMBER_OF_RECORDS));
+      job.getConfiguration().set(MAXIMUM_NUMBER_OF_RECORDS, 
_properties.getProperty(MAXIMUM_NUMBER_OF_RECORDS));
+    }
+    AvroMultipleOutputs.addNamedOutput(job, "avro", AvroKeyOutputFormat.class, 
schema);
+    AvroMultipleOutputs.setCountersEnabled(job, true);
+    LazyOutputFormat.setOutputFormatClass(job, AvroKeyOutputFormat.class);
+
+    // Input and output paths.
+    FileInputFormat.setInputPaths(job, _inputSegmentDir);
+    FileOutputFormat.setOutputPath(job, outputDir);
+
+    List<FileStatus> inputDataFiles = new ArrayList<>();
+    for (String inputPath : inputDataPath) {
+      Path inputPathPattern = new Path(inputPath);
+      FileStatus[] fileStatusArr = _fileSystem.globStatus(inputPathPattern);
+      for (FileStatus fileStatus : fileStatusArr) {
+        inputDataFiles.addAll(getDataFilesFromPath(_fileSystem, 
fileStatus.getPath()));
+      }
+    }
+    _logger.info("Total number of files to be pre-computed: {}", 
inputDataFiles.size());
+
+    // Set up mapper output key
+    Set<Schema.Field> fieldSet = new HashSet<>();
+
+    // Partition configs.
+    int numReduceTasks = (_numberOfPartitions != 0) ? _numberOfPartitions : 
inputDataFiles.size();
+    if (_partitionColumn != null) {
+      job.getConfiguration().set(ENABLE_PARTITIONING, "true");
+      job.setPartitionerClass(GenericPartitioner.class);
+      job.getConfiguration().set(PARTITION_COLUMN, _partitionColumn);
+      if (_partitionFunction != null) {
+        job.getConfiguration().set(PARTITION_FUNCTION, _partitionFunction);
+      }
+      job.getConfiguration().set(NUMBER_OF_PARTITIONS, 
Integer.toString(numReduceTasks));
+    } else {
+      if (_numberOfOutputFiles > 0) {
+        numReduceTasks = _numberOfOutputFiles;
+      }
+      // Partitioning is disabled. Adding hashcode as one of the fields to 
mapper output key.
+      // so that all the rows can be spread evenly.
+      addHashCodeField(fieldSet);
+    }
+    job.setInputFormatClass(CombineAvroKeyInputFormat.class);
+
+    _logger.info("Number of reduce tasks for pre-computing: {}", 
numReduceTasks);
+    job.setNumReduceTasks(numReduceTasks);
+
+    // Sort config.
+    if (_sortedColumn != null) {
+      _logger.info("Adding sorted column: {} to job config", _sortedColumn);
+      job.getConfiguration().set(SORTED_COLUMN, _sortedColumn);
+
+      // Sorting is enabled. Adding sorted column value to mapper output key.
+      Schema sortedColumnSchema = schema.getField(_sortedColumn).schema();
+      Schema sortedColumnAsKeySchema;
+      if (sortedColumnSchema.getType().equals(Schema.Type.UNION)) {
+        sortedColumnAsKeySchema = 
Schema.createUnion(sortedColumnSchema.getTypes());
+      } else {
+        sortedColumnAsKeySchema = Schema.create(sortedColumnSchema.getType());
+      }
+      Schema.Field columnField = new Schema.Field(_sortedColumn, 
sortedColumnAsKeySchema, "sortedColumn", null);
+      fieldSet.add(columnField);
+    } else {
+      // If sorting is disabled, hashcode will be the only factor for 
sort/group comparator.
+      addHashCodeField(fieldSet);
+    }
+
+    Schema mapperOutputKeySchema = Schema.createRecord("record", "", "", 
false);
+    mapperOutputKeySchema.setFields(new ArrayList<>(fieldSet));
+    _logger.info("Mapper output schema: {}", mapperOutputKeySchema);
+
+    AvroJob.setInputKeySchema(job, schema);
+    AvroJob.setMapOutputKeySchema(job, mapperOutputKeySchema);
+    AvroJob.setMapOutputValueSchema(job, schema);
+    AvroJob.setOutputKeySchema(job, schema);
+
+    // Since we aren't extending AbstractHadoopJob, we need to add the jars 
for the job to
+    // distributed cache ourselves. Take a look at how the 
addFilesToDistributedCache is
+    // implemented so that you know what it does.
+    _logger.info("HDFS class path: " + _hdfsDefaultClasspathDir);
+    if (_hdfsDefaultClasspathDir != null) {
+      _logger.info("Copying jars locally.");
+      addContainerLocalFiles(job, _hdfsDefaultClasspathDir, 
job.getConfiguration());
+    } else {
+      _logger.info("Property 'hdfs.default.classpath.dir' not specified.");
+    }
+
+    long startTime = System.currentTimeMillis();
+    // Submit the job for execution.
+    job.waitForCompletion(true);
+    if (!job.isSuccessful()) {
+      throw new RuntimeException("Job failed : " + job);
+    }
+
+    _logger.info("Finished pre-computing job in {}ms", 
(System.currentTimeMillis() - startTime));
+  }
+
+  @Nullable
+  private TableConfig getTableConfig()
+      throws IOException {
+    try (ControllerRestApi controllerRestApi = getControllerRestApi()) {
+      return controllerRestApi != null ? controllerRestApi.getTableConfig() : 
null;
+    }
+  }
+
+  /**
+   * Can be overridden to provide custom controller Rest API.
+   */
+  @Nullable
+  private ControllerRestApi getControllerRestApi() {
+    return _pushLocations != null ? new 
DefaultControllerRestApi(_pushLocations, _rawTableName) : null;
+  }
+
+  private void setDirPermission(Path dirPath)
+      throws IOException {
+    if (_defaultPermissionsMask != null) {
+      FsPermission permission = FsPermission.getDirDefault().applyUMask(new 
FsPermission(_defaultPermissionsMask));
+      _logger.info("Setting permission: {} to directory: {}", permission, 
dirPath);
+      _fileSystem.setPermission(dirPath, permission);
+    }
+  }
+
+  private void fetchPartitioningConfig() {
+    // Fetch partition info from table config.
+    SegmentPartitionConfig segmentPartitionConfig = 
_tableConfig.getIndexingConfig().getSegmentPartitionConfig();
+    if (segmentPartitionConfig != null) {
+      Map<String, ColumnPartitionConfig> columnPartitionMap = 
segmentPartitionConfig.getColumnPartitionMap();
+      assert columnPartitionMap.size() <= 1;
+      if (columnPartitionMap.size() == 1) {
+        _partitionColumn = columnPartitionMap.keySet().iterator().next();
+        _numberOfPartitions = 
segmentPartitionConfig.getNumPartitions(_partitionColumn);
+        _partitionFunction = 
segmentPartitionConfig.getFunctionName(_partitionColumn);
+      }
+    } else {
+      _logger.info("Segment partition config is null for table: {}", 
_tableConfig.getTableName());
+    }
+
+    // Check whether configs overwritten by Hadoop properties.
+    if (_properties.getProperty(PARTITION_COLUMN) != null) {
+      _logger.info("Overriding partition column to: {}", 
_properties.getProperty(PARTITION_COLUMN));
+      _partitionColumn = _properties.getProperty(PARTITION_COLUMN);
+    }
+    if (_properties.getProperty(NUMBER_OF_PARTITIONS) != null) {
+      _logger.info("Overriding partition number to: {}", 
_properties.getProperty(NUMBER_OF_PARTITIONS));
+      _numberOfPartitions = 
Integer.parseInt(_properties.getProperty(NUMBER_OF_PARTITIONS));
+    }
+  }
+
+  private void fetchSortingConfig() {
+    // Fetch sorting info from table config.
+    IndexingConfig indexingConfig = _tableConfig.getIndexingConfig();
+    List<String> sortedColumns = indexingConfig.getSortedColumn();
+    assert sortedColumns.size() <= 1;
 
 Review comment:
   hmmm, i am thinking along the lines of when i want to resize by default, 
then i will enable preprocessing by default - do you think we should just turn 
sorting on for everyone? I think it will break, esp because some users think we 
can sort more than one column

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to