9aman commented on code in PR #14920: URL: https://github.com/apache/pinot/pull/14920#discussion_r1940948236
########## 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); + _consumerThread.start(); + } + + private StreamDataDecoder createDecoder(Set<String> fieldsToRead) + throws Exception { + AtomicReference<StreamDataDecoder> localStreamDataDecoder = new AtomicReference<>(); + RetryPolicy retryPolicy = RetryPolicies.exponentialBackoffRetryPolicy(5, 1000L, 1.2f); + retryPolicy.attempt(() -> { + try { + StreamMessageDecoder streamMessageDecoder = createMessageDecoder(fieldsToRead); + localStreamDataDecoder.set(new StreamDataDecoderImpl(streamMessageDecoder)); + return true; + } catch (Exception e) { + _logger.warn("Failed to create StreamMessageDecoder. Retrying...", e); + return false; + } + }); + return localStreamDataDecoder.get(); + } + + /** + * Creates a {@link StreamMessageDecoder} using properties in {@link StreamConfig}. + * + * @param fieldsToRead The fields to read from the source stream + * @return The initialized StreamMessageDecoder + */ + private StreamMessageDecoder createMessageDecoder(Set<String> fieldsToRead) { + String decoderClass = _streamConfig.getDecoderClass(); + try { + StreamMessageDecoder decoder = PluginManager.get().createInstance(decoderClass); + decoder.init(fieldsToRead, _streamConfig, _tableConfig, _schema); + return decoder; + } catch (Exception e) { + throw new RuntimeException( + "Caught exception while creating StreamMessageDecoder from stream config: " + _streamConfig, e); + } + } + + private class PartitionConsumer implements Runnable { + @Override + public void run() { Review Comment: ``` // TODO: // When reaching here, the current consuming segment has already acquired the consumer semaphore, but there is // no guarantee that the previous consuming segment is already persisted (replaced with immutable segment). It // can potentially cause the following problems: // 1. The snapshot for the previous consuming segment might not be taken since it is not persisted yet // 2. If the previous consuming segment is dropped but immutable segment is not downloaded and replaced yet, // it might cause inconsistency (especially for partial upsert because events are not consumed in sequence) // To address this problem, we should consider releasing the consumer semaphore after the consuming segment is // persisted. // Take upsert snapshot before starting consuming events if (_partitionUpsertMetadataManager != null) { if (_tableConfig.getUpsertMetadataTTL() > 0) { // If upsertMetadataTTL is enabled, we will remove expired primary keys from upsertMetadata // AFTER taking a snapshot. Taking the snapshot first is crucial to capture the final // state of each key before it exits the TTL window. Out-of-TTL segments are skipped in // the doAddSegment flow, and the snapshot is used to enableUpsert on the immutable out-of-TTL segment. // If no snapshot is found, the entire segment is marked as valid and queryable. _partitionUpsertMetadataManager.takeSnapshot(); _partitionUpsertMetadataManager.removeExpiredPrimaryKeys(); } else { // We should remove deleted-keys first and then take a snapshot. This is because the deletedKeysTTL // flow removes keys from the map and updates to remove valid doc IDs. By taking the snapshot immediately // after this process, we save one commit cycle, ensuring that the deletion of valid doc IDs is reflected // immediately _partitionUpsertMetadataManager.removeExpiredPrimaryKeys(); _partitionUpsertMetadataManager.takeSnapshot(); } } ``` Do we need to worry about 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