KKcorps commented on code in PR #14798: URL: https://github.com/apache/pinot/pull/14798#discussion_r1914057933
########## pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java: ########## @@ -0,0 +1,450 @@ +/** + * 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; + +import com.google.common.base.Function; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiKeyAuthDefinition; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Authorization; +import io.swagger.annotations.SecurityDefinition; +import io.swagger.annotations.SwaggerDefinition; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import javax.inject.Inject; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.HttpHeaders; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.apache.commons.io.FileUtils; +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.NameValuePair; +import org.apache.hc.core5.http.message.BasicHeader; +import org.apache.hc.core5.http.message.BasicNameValuePair; +import org.apache.pinot.common.auth.AuthProviderUtils; +import org.apache.pinot.common.exception.HttpErrorStatusException; +import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.utils.FileUploadDownloadClient; +import org.apache.pinot.common.utils.LLCSegmentName; +import org.apache.pinot.common.utils.SimpleHttpResponse; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.common.utils.http.HttpClient; +import org.apache.pinot.core.data.manager.InstanceDataManager; +import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.segment.local.data.manager.SegmentDataManager; +import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionRequest; +import org.apache.pinot.server.api.resources.reingestion.ReIngestionResponse; +import org.apache.pinot.server.api.resources.reingestion.utils.SimpleRealtimeSegmentDataManager; +import org.apache.pinot.server.starter.ServerInstance; +import org.apache.pinot.spi.auth.AuthProvider; +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.stream.StreamConfig; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.StringUtil; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.pinot.spi.utils.CommonConstants.DATABASE; +import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY; + + +@Api(tags = "ReIngestion", authorizations = {@Authorization(value = SWAGGER_AUTHORIZATION_KEY), + @Authorization(value = DATABASE)}) +@SwaggerDefinition(securityDefinition = @SecurityDefinition(apiKeyAuthDefinitions = { + @ApiKeyAuthDefinition(name = HttpHeaders.AUTHORIZATION, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, + key = SWAGGER_AUTHORIZATION_KEY, + description = "The format of the key is ```\"Basic <token>\" or \"Bearer <token>\"```"), + @ApiKeyAuthDefinition(name = DATABASE, in = ApiKeyAuthDefinition.ApiKeyLocation.HEADER, key = DATABASE, + description = "Database context passed through http header. If no context is provided 'default' database " + + "context will be considered.")})) +@Path("/") +public class ReIngestionResource { + private static final Logger LOGGER = LoggerFactory.getLogger(ReIngestionResource.class); + public static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + //TODO: Maximum number of concurrent re-ingestions allowed should be configurable + private static final int MAX_PARALLEL_REINGESTIONS = 10; + + // Map to track ongoing ingestion per segment + private static final ConcurrentHashMap<String, AtomicBoolean> SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + + // Semaphore to enforce global concurrency limit + private static final Semaphore REINGESTION_SEMAPHORE = new Semaphore(MAX_PARALLEL_REINGESTIONS); + + @Inject + private ServerInstance _serverInstance; + + @POST + @Path("/reIngestSegment") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Re-ingest segment", notes = "Re-ingest data for a segment from startOffset to endOffset and " + + "upload the segment") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Success", response = ReIngestionResponse.class), @ApiResponse(code = 500, + message = "Internal server error", response = ErrorInfo.class) + }) + public Response reIngestSegment(ReIngestionRequest request) { + try { + String tableNameWithType = request.getTableNameWithType(); + String segmentName = request.getSegmentName(); + + // Try to acquire a permit from the semaphore to ensure we don't exceed max concurrency + if (!REINGESTION_SEMAPHORE.tryAcquire()) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE) + .entity("Too many re-ingestions in progress. Please try again later.") + .build(); + } + + // Check if the segment is already being re-ingested + AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); + if (!isIngesting.compareAndSet(false, true)) { + // The segment is already being ingested + REINGESTION_SEMAPHORE.release(); + return Response.status(Response.Status.CONFLICT) + .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") + .build(); + } + + InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); + if (instanceDataManager == null) { + throw new WebApplicationException(new RuntimeException("Invalid server initialization"), + Response.Status.INTERNAL_SERVER_ERROR); + } + + TableDataManager tableDataManager = instanceDataManager.getTableDataManager(tableNameWithType); + if (tableDataManager == null) { + throw new WebApplicationException("Table data manager not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + IndexLoadingConfig indexLoadingConfig = tableDataManager.fetchIndexLoadingConfig(); + LOGGER.info("Executing re-ingestion for table: {}, segment: {}", tableNameWithType, segmentName); + + // Get TableConfig and Schema + TableConfig tableConfig = indexLoadingConfig.getTableConfig(); + if (tableConfig == null) { + throw new WebApplicationException("Table config not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + Schema schema = indexLoadingConfig.getSchema(); + if (schema == null) { + throw new WebApplicationException("Schema not found for table: " + tableNameWithType, + Response.Status.NOT_FOUND); + } + + // Fetch SegmentZKMetadata + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (segmentZKMetadata == null) { + throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, + Response.Status.NOT_FOUND); + } + + // Get startOffset, endOffset, partitionGroupId + String startOffsetStr = segmentZKMetadata.getStartOffset(); + String endOffsetStr = segmentZKMetadata.getEndOffset(); + + if (startOffsetStr == null || endOffsetStr == null) { + return Response.serverError().entity("Start offset or end offset is null for segment: " + segmentName).build(); + } + + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + int partitionGroupId = llcSegmentName.getPartitionGroupId(); + + Map<String, String> streamConfigMap; + try { + streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + } catch (Exception e) { + return Response.serverError().entity("Failed to get stream config for table: " + tableNameWithType).build(); + } + + StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); + + // Set up directories + File resourceTmpDir = new File(FileUtils.getTempDirectory(), "resourceTmpDir_" + System.currentTimeMillis()); + File resourceDataDir = new File(FileUtils.getTempDirectory(), "resourceDataDir_" + System.currentTimeMillis()); + + if (!resourceTmpDir.exists()) { + resourceTmpDir.mkdirs(); + } + if (!resourceDataDir.exists()) { + resourceDataDir.mkdirs(); + } + + LOGGER.info("Starting SimpleRealtimeSegmentDataManager..."); + // Instantiate SimpleRealtimeSegmentDataManager + SimpleRealtimeSegmentDataManager manager = + new SimpleRealtimeSegmentDataManager(segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, + tableConfig, schema, indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, resourceTmpDir, + resourceDataDir, _serverInstance.getServerMetrics()); + + try { + + manager.startConsumption(); + + waitForCondition((Void) -> manager.isDoneConsuming(), 1000, 300000, 0); + + manager.stopConsumption(); + + // After ingestion is complete, get the segment + if (!manager.isSuccess()) { + throw new Exception("Consumer failed to reingest data: " + manager.getConsumptionException()); + } + + LOGGER.info("Starting build for segment {}", segmentName); + SimpleRealtimeSegmentDataManager.SegmentBuildDescriptor segmentBuildDescriptor = + manager.buildSegmentInternal(); + + // Get the segment directory + File segmentTarFile = segmentBuildDescriptor.getSegmentTarFile(); + + if (segmentTarFile == null) { + throw new Exception("Failed to build segment: " + segmentName); + } + + //TODO: Find a way to get auth token here using injection instead of request param Review Comment: Address this TODO -- 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