9aman commented on code in PR #14920:
URL: https://github.com/apache/pinot/pull/14920#discussion_r1942248666


##########
pinot-server/src/main/java/org/apache/pinot/server/api/resources/reingestion/utils/StatelessRealtimeSegmentDataManager.java:
##########
@@ -0,0 +1,575 @@
+/**
+ * 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.server.api.resources.reingestion.utils;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.metrics.ServerMetrics;
+import org.apache.pinot.common.utils.FileUploadDownloadClient;
+import org.apache.pinot.common.utils.TarCompressionUtils;
+import org.apache.pinot.segment.local.data.manager.SegmentDataManager;
+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.MutableSegment;
+import org.apache.pinot.segment.spi.V1Constants;
+import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl;
+import org.apache.pinot.segment.spi.partition.PartitionFunctionFactory;
+import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths;
+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.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.retry.RetryPolicies;
+import org.apache.pinot.spi.utils.retry.RetryPolicy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Simplified Segment Data Manager for ingesting data from a start offset to 
an end offset.
+ */
+public class StatelessRealtimeSegmentDataManager extends SegmentDataManager {
+
+  private static final int DEFAULT_CAPACITY = 100_000;
+  private static final int DEFAULT_FETCH_TIMEOUT_MS = 5000;
+  public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = 
new FileUploadDownloadClient();
+
+  private final String _segmentName;
+  private final String _tableNameWithType;
+  private final int _partitionGroupId;
+  private final String _segmentNameStr;
+  private final SegmentZKMetadata _segmentZKMetadata;
+  private final TableConfig _tableConfig;
+  private final Schema _schema;
+  private final StreamConfig _streamConfig;
+  private final StreamPartitionMsgOffsetFactory _offsetFactory;
+  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 _shouldStop = new AtomicBoolean(false);
+  private final AtomicBoolean _isDoneConsuming = new AtomicBoolean(false);
+  private final StreamPartitionMsgOffset _startOffset;
+  private final StreamPartitionMsgOffset _endOffset;
+  private volatile StreamPartitionMsgOffset _currentOffset;
+  private volatile int _numRowsIndexed = 0;
+  private final String _segmentStoreUriStr;
+  private final int _fetchTimeoutMs;
+  private final TransformPipeline _transformPipeline;
+  private volatile boolean _isSuccess = false;
+  private volatile Throwable _consumptionException;
+  private final ServerMetrics _serverMetrics;
+
+  public StatelessRealtimeSegmentDataManager(String segmentName, String 
tableNameWithType, int partitionGroupId,
+      SegmentZKMetadata segmentZKMetadata, TableConfig tableConfig, Schema 
schema,
+      IndexLoadingConfig indexLoadingConfig, StreamConfig streamConfig, String 
startOffsetStr, String endOffsetStr,
+      ServerMetrics serverMetrics)
+      throws Exception {
+
+    _segmentName = segmentName;
+    _tableNameWithType = tableNameWithType;
+    _partitionGroupId = partitionGroupId;
+    _segmentZKMetadata = segmentZKMetadata;
+    _tableConfig = tableConfig;
+    _schema = schema;
+    _segmentStoreUriStr = indexLoadingConfig.getSegmentStoreURI();
+    _streamConfig = streamConfig;
+    _resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" 
+ System.currentTimeMillis());
+    _resourceDataDir = new File(FileUtils.getTempDirectory(), 
"resourceDataDir_" + System.currentTimeMillis());;
+    _serverMetrics = serverMetrics;
+    _logger = 
LoggerFactory.getLogger(StatelessRealtimeSegmentDataManager.class.getName() + 
"_" + _segmentName);
+
+    _offsetFactory = 
StreamConsumerFactoryProvider.create(_streamConfig).createStreamMsgOffsetFactory();
+    _startOffset = _offsetFactory.create(startOffsetStr);
+    _endOffset = _offsetFactory.create(endOffsetStr);
+
+    String clientId = getClientId();
+
+    // Temp dirs
+    _resourceTmpDir.mkdirs();
+    _resourceDataDir.mkdirs();
+
+    _consumerFactory = StreamConsumerFactoryProvider.create(_streamConfig);
+    _partitionMetadataProvider = 
_consumerFactory.createPartitionMetadataProvider(clientId, _partitionGroupId);
+    _segmentNameStr = _segmentZKMetadata.getSegmentName();
+
+    // Create a simple PartitionGroupConsumptionStatus
+    PartitionGroupConsumptionStatus partitionGroupConsumptionStatus =
+        new PartitionGroupConsumptionStatus(_partitionGroupId, 0, 
_startOffset, null, null);
+
+    _consumer = _consumerFactory.createPartitionGroupConsumer(clientId, 
partitionGroupConsumptionStatus);
+
+    // Initialize decoder
+    Set<String> fieldsToRead = 
IngestionUtils.getFieldsForRecordExtractor(_tableConfig, _schema);
+    _decoder = createDecoder(fieldsToRead);
+
+    // Fetch capacity from indexLoadingConfig or use default
+    int capacity = streamConfig.getFlushThresholdRows();
+    if (capacity <= 0) {
+      capacity = DEFAULT_CAPACITY;
+    }
+
+    // Fetch average number of multi-values from indexLoadingConfig
+    int avgNumMultiValues = indexLoadingConfig.getRealtimeAvgMultiValueCount();
+
+    // Load stats history, here we are using the same stats while as the 
RealtimeSegmentDataManager so that we are
+    // much more efficient in allocating buffers. It also works with empty file
+    String tableDataDir = indexLoadingConfig.getInstanceDataManagerConfig() != 
null
+        ? 
indexLoadingConfig.getInstanceDataManagerConfig().getInstanceDataDir() + 
File.separator + _tableNameWithType
+        : _resourceTmpDir.getAbsolutePath();
+    File statsHistoryFile = new File(tableDataDir, "segment-stats.ser");
+    RealtimeSegmentStatsHistory statsHistory = 
RealtimeSegmentStatsHistory.deserialzeFrom(statsHistoryFile);
+
+    // Initialize mutable segment with configurations
+    RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder =
+        new 
RealtimeSegmentConfig.Builder(indexLoadingConfig).setTableNameWithType(_tableNameWithType)
+            
.setSegmentName(_segmentName).setStreamName(_streamConfig.getTopicName())
+            
.setSegmentZKMetadata(_segmentZKMetadata).setStatsHistory(statsHistory).setSchema(_schema)
+            .setCapacity(capacity).setAvgNumMultiValues(avgNumMultiValues)
+            .setOffHeap(indexLoadingConfig.isRealtimeOffHeapAllocation())
+            
.setFieldConfigList(tableConfig.getFieldConfigList()).setConsumerDir(_resourceDataDir.getAbsolutePath())
+            .setMemoryManager(
+                new 
MmapMemoryManager(FileUtils.getTempDirectory().getAbsolutePath(), 
_segmentNameStr, _serverMetrics));
+
+    setPartitionParameters(realtimeSegmentConfigBuilder, 
_tableConfig.getIndexingConfig().getSegmentPartitionConfig());
+
+    _realtimeSegment = new 
MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), _serverMetrics);
+
+    _transformPipeline = new TransformPipeline(tableConfig, schema);
+
+    // Initialize fetch timeout
+    _fetchTimeoutMs =
+        _streamConfig.getFetchTimeoutMillis() > 0 ? 
_streamConfig.getFetchTimeoutMillis() : DEFAULT_FETCH_TIMEOUT_MS;
+  }
+
+  private String getClientId() {
+    return _tableNameWithType + "-" + _partitionGroupId;
+  }
+
+  public void startConsumption() {
+    // Start the consumer thread
+    _consumerThread = new Thread(new PartitionConsumer(), _segmentName);

Review Comment:
   Are we not logging this ?
   



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