noob-se7en commented on code in PR #14920: URL: https://github.com/apache/pinot/pull/14920#discussion_r1936058894
########## pinot-server/src/main/java/org/apache/pinot/server/api/resources/ReIngestionResource.java: ########## @@ -0,0 +1,526 @@ +/** + * 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.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +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.GET; +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.commons.lang3.tuple.Pair; +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.URIUtils; +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.core.data.manager.realtime.SegmentCompletionUtils; +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.realtime.ControllerLeaderLocator; +import org.apache.pinot.server.realtime.ServerSegmentCompletionProtocolHandler; +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.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; +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.HTTPS_PROTOCOL; +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); + + //TODO: Make this configurable + private static final int MAX_PARALLEL_REINGESTIONS = 8; + + // Tracks if a particular segment is currently being re-ingested + private static final ConcurrentHashMap<String, AtomicBoolean> + SEGMENT_INGESTION_MAP = new ConcurrentHashMap<>(); + + // Executor for asynchronous re-ingestion + private static final ExecutorService REINGESTION_EXECUTOR = + Executors.newFixedThreadPool(MAX_PARALLEL_REINGESTIONS); + + // Keep track of jobs by jobId => job info + private static final ConcurrentHashMap<String, ReIngestionJob> RUNNING_JOBS = new ConcurrentHashMap<>(); + public static final long CONSUMPTION_END_TIMEOUT_MS = 300000L; + public static final long UPLOAD_END_TIMEOUT_MS = 300000L; + + @Inject + private ServerInstance _serverInstance; + + /** + * Simple data class to hold job details. + */ + private static class ReIngestionJob { + private final String _jobId; + private final String _tableNameWithType; + private final String _segmentName; + private final long _startTimeMs; + + ReIngestionJob(String jobId, String tableNameWithType, String segmentName) { + _jobId = jobId; + _tableNameWithType = tableNameWithType; + _segmentName = segmentName; + _startTimeMs = System.currentTimeMillis(); + } + + public String getJobId() { + return _jobId; + } + + public String getTableNameWithType() { + return _tableNameWithType; + } + + public String getSegmentName() { + return _segmentName; + } + + public long getStartTimeMs() { + return _startTimeMs; + } + } + + /** + * New API to get all running re-ingestion jobs. + */ + @GET + @Path("/reingestSegment/jobs") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation("Get all running re-ingestion jobs along with job IDs") + public Response getAllRunningReingestionJobs() { + // Filter only the jobs still marked as running + List<ReIngestionJob> runningJobs = new ArrayList<>(RUNNING_JOBS.values()); + return Response.ok(runningJobs).build(); + } + + @POST + @Path("/reingestSegment") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Re-ingest segment asynchronously", notes = "Returns a jobId immediately; ingestion runs in " + + "background.") + @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) { + String tableNameWithType = request.getTableNameWithType(); + String segmentName = request.getSegmentName(); + + if (RUNNING_JOBS.size() >= MAX_PARALLEL_REINGESTIONS) { + return Response.status(Response.Status.TOO_MANY_REQUESTS) + .entity("Reingestion jobs parallel limit " + MAX_PARALLEL_REINGESTIONS + " reached.").build(); + } + + InstanceDataManager instanceDataManager = _serverInstance.getInstanceDataManager(); + if (instanceDataManager == null) { + throw new WebApplicationException("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, Schema, ZK metadata + 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); + } + + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentName); + if (segmentZKMetadata == null) { + throw new WebApplicationException("Segment metadata not found for segment: " + segmentName, + Response.Status.NOT_FOUND); + } + + // Grab start/end offsets + String startOffsetStr = segmentZKMetadata.getStartOffset(); + String endOffsetStr = segmentZKMetadata.getEndOffset(); + if (startOffsetStr == null || endOffsetStr == null) { + throw new WebApplicationException("Null start/end offset for segment: " + segmentName, + Response.Status.INTERNAL_SERVER_ERROR); + } + + // Check if this segment is already being re-ingested + AtomicBoolean isIngesting = SEGMENT_INGESTION_MAP.computeIfAbsent(segmentName, k -> new AtomicBoolean(false)); + if (!isIngesting.compareAndSet(false, true)) { + return Response.status(Response.Status.CONFLICT) + .entity("Re-ingestion for segment: " + segmentName + " is already in progress.") + .build(); + } + + // Generate a jobId for tracking + String jobId = UUID.randomUUID().toString(); + ReIngestionJob job = new ReIngestionJob(jobId, tableNameWithType, segmentName); + RUNNING_JOBS.put(jobId, job); + + // Send immediate success response with jobId + ReIngestionResponse immediateResponse = new ReIngestionResponse( + "Re-ingestion job submitted successfully with jobId: " + jobId); + Response response = Response.ok(immediateResponse).build(); + + // Kick off the actual work asynchronously + REINGESTION_EXECUTOR.submit(() -> { + try { + LLCSegmentName llcSegmentName = new LLCSegmentName(segmentName); + int partitionGroupId = llcSegmentName.getPartitionGroupId(); + + Map<String, String> streamConfigMap = IngestionConfigUtils.getStreamConfigMaps(tableConfig).get(0); + StreamConfig streamConfig = new StreamConfig(tableNameWithType, streamConfigMap); + + SimpleRealtimeSegmentDataManager manager = new SimpleRealtimeSegmentDataManager( + segmentName, tableNameWithType, partitionGroupId, segmentZKMetadata, tableConfig, schema, + indexLoadingConfig, streamConfig, startOffsetStr, endOffsetStr, _serverInstance.getServerMetrics()); + + doReIngestSegment(manager, segmentName, tableNameWithType, indexLoadingConfig, tableDataManager); + } catch (Exception e) { + LOGGER.error("Error during async re-ingestion for job {} (segment={})", jobId, segmentName, e); + } finally { + isIngesting.set(false); Review Comment: I was talking about at just the instant just after flag is set false. (Just looking from as a server API perspective ignoring however frequently this API is called) -- 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