lnbest0707-uber commented on code in PR #14920:
URL: https://github.com/apache/pinot/pull/14920#discussion_r1972260409


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/writer/StatelessRealtimeSegmentWriter.java:
##########
@@ -0,0 +1,464 @@
+/**
+ * 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.segment.local.realtime.writer;
+
+import com.google.common.base.Preconditions;
+import java.io.Closeable;
+import java.io.File;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.annotation.Nullable;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.common.utils.LLCSegmentName;
+import org.apache.pinot.common.utils.TarCompressionUtils;
+import org.apache.pinot.segment.local.indexsegment.mutable.MutableSegmentImpl;
+import org.apache.pinot.segment.local.io.writer.impl.MmapMemoryManager;
+import 
org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter;
+import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig;
+import 
org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory;
+import org.apache.pinot.segment.local.segment.creator.TransformPipeline;
+import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig;
+import org.apache.pinot.segment.local.utils.IngestionUtils;
+import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory;
+import org.apache.pinot.spi.config.table.ColumnPartitionConfig;
+import org.apache.pinot.spi.config.table.SegmentPartitionConfig;
+import org.apache.pinot.spi.config.table.SegmentZKPropsConfig;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.stream.MessageBatch;
+import org.apache.pinot.spi.stream.PartitionGroupConsumer;
+import org.apache.pinot.spi.stream.PartitionGroupConsumptionStatus;
+import org.apache.pinot.spi.stream.StreamConfig;
+import org.apache.pinot.spi.stream.StreamConsumerFactory;
+import org.apache.pinot.spi.stream.StreamConsumerFactoryProvider;
+import org.apache.pinot.spi.stream.StreamDataDecoder;
+import org.apache.pinot.spi.stream.StreamDataDecoderImpl;
+import org.apache.pinot.spi.stream.StreamDataDecoderResult;
+import org.apache.pinot.spi.stream.StreamMessage;
+import org.apache.pinot.spi.stream.StreamMessageDecoder;
+import org.apache.pinot.spi.stream.StreamMetadataProvider;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffset;
+import org.apache.pinot.spi.stream.StreamPartitionMsgOffsetFactory;
+import org.apache.pinot.spi.utils.IngestionConfigUtils;
+import org.apache.pinot.spi.utils.builder.TableNameBuilder;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
+import org.apache.pinot.spi.utils.retry.RetryPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Segment writer to ingest streaming data from a start offset to an end 
offset.
+ *
+ * TODO:
+ *   1. Clean up this class and only keep the necessary parts.
+ *   2. Use a different segment impl for better performance because it doesn't 
need to serve queries.
+ */
+public class StatelessRealtimeSegmentWriter implements Closeable {
+
+  private static final int DEFAULT_CAPACITY = 100_000;
+  private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000;
+  public static final String SEGMENT_STATS_FILE_NAME = "segment-stats.ser";
+  public static final String RESOURCE_TMP_DIR_PREFIX = "resourceTmpDir_";
+  public static final String RESOURCE_DATA_DIR_PREFIX = "resourceDataDir_";
+
+  private final Semaphore _segBuildSemaphore;
+  private final String _segmentName;
+  private final String _tableNameWithType;
+  private final int _partitionGroupId;
+  private final SegmentZKMetadata _segmentZKMetadata;
+  private final TableConfig _tableConfig;
+  private final Schema _schema;
+  private final StreamConfig _streamConfig;
+  private final StreamConsumerFactory _consumerFactory;
+  private StreamMetadataProvider _partitionMetadataProvider;
+  private final PartitionGroupConsumer _consumer;
+  private final StreamDataDecoder _decoder;
+  private final MutableSegmentImpl _realtimeSegment;
+  private final File _resourceTmpDir;
+  private final File _resourceDataDir;
+  private final Logger _logger;
+  private Thread _consumerThread;
+  private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false);
+  private final StreamPartitionMsgOffset _startOffset;
+  private final StreamPartitionMsgOffset _endOffset;
+  private volatile StreamPartitionMsgOffset _currentOffset;
+  private final int _fetchTimeoutMs;
+  private final TransformPipeline _transformPipeline;
+  private volatile boolean _isSuccess = false;
+  private volatile Throwable _consumptionException;
+
+  public StatelessRealtimeSegmentWriter(SegmentZKMetadata segmentZKMetadata, 
IndexLoadingConfig indexLoadingConfig,
+      @Nullable Semaphore segBuildSemaphore)
+      throws Exception {
+    Preconditions.checkNotNull(indexLoadingConfig.getTableConfig(), "Table 
config must be set in index loading config");
+    Preconditions.checkNotNull(indexLoadingConfig.getSchema(), "Schema must be 
set in index loading config");
+    LLCSegmentName llcSegmentName = new 
LLCSegmentName(segmentZKMetadata.getSegmentName());
+
+    _segmentName = segmentZKMetadata.getSegmentName();
+    _partitionGroupId = llcSegmentName.getPartitionGroupId();

Review Comment:
   This will be incompatible with multi-stream ingestion. We need to use the 
streamPartitionGroupId to do the ingestion. Refer to the 
RealtimeSegmentDataManager



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

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

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