yupeng9 commented on a change in pull request #6718:
URL: https://github.com/apache/incubator-pinot/pull/6718#discussion_r604482045



##########
File path: 
pinot-plugins/pinot-segment-writer/pinot-segment-writer-file-based/src/main/java/org/apache/pinot/plugin/segmentwriter/filebased/FileBasedSegmentWriter.java
##########
@@ -0,0 +1,197 @@
+/**
+ * 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.plugin.segmentwriter.filebased;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
+import org.apache.pinot.core.data.recordtransformer.RecordTransformer;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.util.IngestionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a local file as a buffer 
to collect {@link GenericRow}.
+ * The {@link GenericRow} are written to the buffer as AVRO records.
+ */
+@NotThreadSafe
+public class FileBasedSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(FileBasedSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && 
_tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils
+            
.isNotEmpty(_tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps 
in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = 
_tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() 
== 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", 
_tableNameWithType);
+    _batchConfig = new BatchConfig(_tableNameWithType, 
_batchIngestionConfig.getBatchConfigMaps().get(0));
+
+    
Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", 
BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = 
CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = 
SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d", _tableNameWithType, 
System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging 
dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: 
%s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for table: {}", 
FileBasedSegmentWriter.class.getName(), _tableNameWithType);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _recordWriter = new DataFileWriter<>(new 
GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, 
_reusableRecord, _fieldsToRead);
+    _recordWriter.append(_reusableRecord);
+  }
+
+  @Override
+  public URI flush()
+      throws IOException {
+
+    LOGGER.info("Beginning flush for table: {}", _tableNameWithType);
+    _recordWriter.close();
+
+    // Create temp dir for flush
+    File flushDir = new File(_stagingDir, "flush_dir_" + 
System.currentTimeMillis());
+    Preconditions.checkState(flushDir.mkdirs(), "Failed to create flush dir: 
%s", flushDir);
+
+    try {
+      // Segment dir
+      File segmentDir = new File(flushDir, "segment_dir");
+
+      // Make BatchIngestionConfig for flush
+      Map<String, String> batchConfigMapOverride = new 
HashMap<>(_batchConfig.getBatchConfigMap());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_DIR_URI, 
_bufferFile.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.OUTPUT_DIR_URI, 
segmentDir.getAbsolutePath());
+      batchConfigMapOverride.put(BatchConfigProperties.INPUT_FORMAT, 
BUFFER_FILE_FORMAT.toString());
+      BatchIngestionConfig batchIngestionConfig = new 
BatchIngestionConfig(Lists.newArrayList(batchConfigMapOverride),
+          _batchIngestionConfig.getSegmentIngestionType(), 
_batchIngestionConfig.getSegmentIngestionFrequency());
+
+      // Build segment
+      SegmentGeneratorConfig segmentGeneratorConfig =
+          IngestionUtils.generateSegmentGeneratorConfig(_tableConfig, _schema, 
batchIngestionConfig);
+      String segmentName = IngestionUtils.buildSegment(segmentGeneratorConfig);
+      LOGGER.info("Successfully built segment: {} for table: {}", segmentName, 
_tableNameWithType);
+
+      // Tar segment
+      File segmentTarFile = new File(_outputDirURI, segmentName + 
Constants.TAR_GZ_FILE_EXT);
+      if (!_batchConfig.isOverwriteOutput() && segmentTarFile.exists()) {
+        segmentTarFile = new File(_outputDirURI,
+            String.format("%s_%d%s", segmentName, System.currentTimeMillis(), 
Constants.TAR_GZ_FILE_EXT));
+      }
+      TarGzCompressionUtils.createTarGzFile(new File(segmentDir, segmentName), 
segmentTarFile);
+      LOGGER.info("Created segment tar: {} for segment: {} of table: {}", 
segmentTarFile.getAbsolutePath(), segmentName,
+          _tableNameWithType);
+
+      // Reset buffer and return segmentTar URI
+      resetBuffer();

Review comment:
       Thanks for calling this out.
   Is it possible to tell which exceptions are retriable and which are not? Or 
have an argument of `retryNum` so the retries are done within the 
implementation?

##########
File path: 
pinot-plugins/pinot-segment-writer/pinot-segment-writer-file-based/src/main/java/org/apache/pinot/plugin/segmentwriter/filebased/FileBasedSegmentWriter.java
##########
@@ -0,0 +1,214 @@
+/**
+ * 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.plugin.segmentwriter.filebased;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.pinot.common.utils.TarGzCompressionUtils;
+import org.apache.pinot.core.data.recordtransformer.CompositeTransformer;
+import org.apache.pinot.core.data.recordtransformer.RecordTransformer;
+import org.apache.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import org.apache.pinot.core.util.IngestionUtils;
+import org.apache.pinot.core.util.SegmentProcessorAvroUtils;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.FileFormat;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.ingestion.batch.BatchConfig;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.ingestion.batch.spec.Constants;
+import org.apache.pinot.spi.ingestion.segment.writer.SegmentWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * A {@link SegmentWriter} implementation that uses a local file as a buffer 
to collect {@link GenericRow}.
+ * The {@link GenericRow} are written to the buffer as AVRO records.
+ */
+@NotThreadSafe
+public class FileBasedSegmentWriter implements SegmentWriter {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(FileBasedSegmentWriter.class);
+  private static final FileFormat BUFFER_FILE_FORMAT = FileFormat.AVRO;
+
+  private TableConfig _tableConfig;
+  private String _tableNameWithType;
+  private BatchIngestionConfig _batchIngestionConfig;
+  private BatchConfig _batchConfig;
+  private String _outputDirURI;
+  private Schema _schema;
+  private Set<String> _fieldsToRead;
+  private RecordTransformer _recordTransformer;
+
+  private File _stagingDir;
+  private File _bufferFile;
+
+  private org.apache.avro.Schema _avroSchema;
+  private DataFileWriter<GenericData.Record> _recordWriter;
+  private GenericData.Record _reusableRecord;
+
+  @Override
+  public void init(TableConfig tableConfig, Schema schema)
+      throws Exception {
+    _tableConfig = tableConfig;
+    _tableNameWithType = _tableConfig.getTableName();
+
+    Preconditions.checkState(
+        _tableConfig.getIngestionConfig() != null && 
_tableConfig.getIngestionConfig().getBatchIngestionConfig() != null
+            && CollectionUtils
+            
.isNotEmpty(_tableConfig.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps()),
+        "Must provide ingestionConfig->batchIngestionConfig->batchConfigMaps 
in tableConfig for table: %s",
+        _tableNameWithType);
+    _batchIngestionConfig = 
_tableConfig.getIngestionConfig().getBatchIngestionConfig();
+    Preconditions.checkState(_batchIngestionConfig.getBatchConfigMaps().size() 
== 1,
+        "batchConfigMaps must contain only 1 BatchConfig for table: %s", 
_tableNameWithType);
+    _batchConfig = new BatchConfig(_tableNameWithType, 
_batchIngestionConfig.getBatchConfigMaps().get(0));
+
+    
Preconditions.checkState(StringUtils.isNotBlank(_batchConfig.getOutputDirURI()),
+        "Must provide: %s in batchConfigs for table: %s", 
BatchConfigProperties.OUTPUT_DIR_URI, _tableNameWithType);
+    _outputDirURI = _batchConfig.getOutputDirURI();
+    Files.createDirectories(Paths.get(_outputDirURI));
+
+    _schema = schema;
+    _fieldsToRead = _schema.getColumnNames();
+    _recordTransformer = 
CompositeTransformer.getDefaultTransformer(_tableConfig, _schema);
+    _avroSchema = 
SegmentProcessorAvroUtils.convertPinotSchemaToAvroSchema(_schema);
+    _reusableRecord = new GenericData.Record(_avroSchema);
+
+    // Create tmp dir
+    _stagingDir = new File(FileUtils.getTempDirectory(),
+        String.format("segment_writer_staging_%s_%d", _tableNameWithType, 
System.currentTimeMillis()));
+    Preconditions.checkState(_stagingDir.mkdirs(), "Failed to create staging 
dir: %s", _stagingDir.getAbsolutePath());
+
+    // Create buffer file
+    File bufferDir = new File(_stagingDir, "buffer_dir");
+    Preconditions.checkState(bufferDir.mkdirs(), "Failed to create buffer_dir: 
%s", bufferDir.getAbsolutePath());
+    _bufferFile = new File(bufferDir, "buffer_file");
+    resetBuffer();
+    LOGGER.info("Initialized {} for table: {}", 
FileBasedSegmentWriter.class.getName(), _tableNameWithType);
+  }
+
+  private void resetBuffer()
+      throws IOException {
+    FileUtils.deleteQuietly(_bufferFile);
+    _recordWriter = new DataFileWriter<>(new 
GenericDatumWriter<>(_avroSchema));
+    _recordWriter.create(_avroSchema, _bufferFile);
+  }
+
+  @Override
+  public void collect(GenericRow row)
+      throws IOException {
+    GenericRow transform = _recordTransformer.transform(row);
+    SegmentProcessorAvroUtils.convertGenericRowToAvroRecord(transform, 
_reusableRecord, _fieldsToRead);
+    _recordWriter.append(_reusableRecord);
+  }
+
+  /**
+   * Creates one Pinot segment using the {@link GenericRow}s collected in the 
AVRO file buffer,
+   * at the outputDirUri as specified in the tableConfig->batchConfigs.
+   * Successful invocation of this method means that the {@link GenericRow}s 
collected so far,
+   * are now available in the Pinot segment and not available in the buffer 
anymore.
+   *
+   * Successful completion of segment will return the segment URI.
+   * The buffer will be reset and ready to accept further records via 
<code>collect()</code>
+   *
+   * If an exception is throw, the buffer will not be reset

Review comment:
       thrown




-- 
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



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

Reply via email to