From 03ab3be487d6de6476b2c9bb93fd7d712bb827bb Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 9 Aug 2024 14:10:06 -0700 Subject: [PATCH 01/43] #12117 Support for Server API to check for Segments reload https://github.com/apache/pinot/issues/12117 --- .../SegmentColumnMismatchResponse.java | 48 +++++++++++++++++++ .../api/resources/SegmentMetadataFetcher.java | 16 +++++++ .../server/api/resources/TablesResource.java | 40 ++++++++++++++++ 3 files changed, 104 insertions(+) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java new file mode 100644 index 000000000000..bd83ee4acb24 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java @@ -0,0 +1,48 @@ +/** + * 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.common.restlet.resources; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + + +/** + * This class gives the data of a server if there exists any segments that need to be reloaded + * + * It has details of server id and returns true/false if there are any segments to be reloaded or not. + */ +public class SegmentColumnMismatchResponse { + boolean _isMismatch; + String _serverInstanceId; + + @JsonCreator + public SegmentColumnMismatchResponse(@JsonProperty("isMismatch") boolean isMismatch, + @JsonProperty("serverInstanceId") String serverInstanceId) { + this._isMismatch = isMismatch; + this._serverInstanceId = serverInstanceId; + } + + public String getServerInstanceId() { + return _serverInstanceId; + } + + public boolean getMismatch() { + return _isMismatch; + } +} diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java index c557d0681b8f..540fdce10836 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java @@ -90,6 +90,22 @@ public static String getSegmentMetadata(SegmentDataManager segmentDataManager, L return JsonUtils.objectToString(segmentMetadataJson); } + /** + * This is a helper method that fetches the segment column names for a given segment. + * @param columns Columns to include for metadata + */ + public static Set getSegmentColumns(SegmentDataManager segmentDataManager, List columns) { + IndexSegment segment = segmentDataManager.getSegment(); + Set columnSet; + if (columns.size() == 1 && columns.get(0).equals("*")) { + // Making code consistent and returning metadata and indexes only for non-virtual columns. + columnSet = segment.getPhysicalColumnNames(); + } else { + columnSet = new HashSet<>(columns); + } + return columnSet; + } + /** * Get the JSON object with the segment column's indexing metadata. * Lists all the columns if the parameter columnSet is null. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index b0f3b5dc94a0..c98afe5a0f63 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -63,6 +63,7 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; +import org.apache.pinot.common.restlet.resources.SegmentColumnMismatchResponse; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; @@ -94,9 +95,11 @@ import org.apache.pinot.server.access.AccessControlFactory; import org.apache.pinot.server.api.AdminApiApplication; import org.apache.pinot.server.starter.ServerInstance; +import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.ConsumerPartitionState; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.apache.pinot.spi.utils.JsonUtils; @@ -952,4 +955,41 @@ public TableSegmentValidationInfo validateTableSegmentState( } return new TableSegmentValidationInfo(true, maxEndTimeMs); } + + @GET + @Path("/tables/{tableName}/segments/mismatch") + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Checks if there is any mismatch of columns in a segment", notes = "Returns true if reload is required on" + + " any segment in a given server") + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Success", response = TableSegments.class), @ApiResponse(code = 500, + message = "Server initialization error", response = ErrorInfo.class) + }) + public String checkMismatchedSegments( + @ApiParam(value = "Table Name with type", required = true) @PathParam("tableName") String tableName, + @ApiParam(value = "Column name", allowMultiple = true) @QueryParam("columns") @DefaultValue("") + List columns, @Context HttpHeaders headers) { + tableName = DatabaseUtils.translateTableName(tableName, headers); + TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableName); + Pair tableConfigSchema = tableDataManager.fetchTableConfigAndSchema(); + Schema schema = tableConfigSchema.getValue(); + Set schemaColumns = schema.getPhysicalColumnNames(); + List segmentDataManagers = tableDataManager.acquireAllSegments(); + try { + boolean mismatchCheck = false; + for (SegmentDataManager segmentDataManager : segmentDataManagers) { + Set segmentColumns = SegmentMetadataFetcher.getSegmentColumns(segmentDataManager, columns); + if (!segmentColumns.containsAll(schemaColumns)) { + mismatchCheck = true; + break; + } + } + return ResourceUtils.convertToJsonString( + new SegmentColumnMismatchResponse(mismatchCheck, tableDataManager.getInstanceId())); + } finally { + for (SegmentDataManager segmentDataManager : segmentDataManagers) { + tableDataManager.releaseSegment(segmentDataManager); + } + } + } } From 557eb7382bf604eca7b723e7de161a10b7f9e91f Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 9 Aug 2024 14:37:53 -0700 Subject: [PATCH 02/43] checkstyle exception checkstyle exception --- .../restlet/resources/SegmentColumnMismatchResponse.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java index bd83ee4acb24..32995afb7cfb 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java @@ -34,8 +34,8 @@ public class SegmentColumnMismatchResponse { @JsonCreator public SegmentColumnMismatchResponse(@JsonProperty("isMismatch") boolean isMismatch, @JsonProperty("serverInstanceId") String serverInstanceId) { - this._isMismatch = isMismatch; - this._serverInstanceId = serverInstanceId; + _isMismatch = isMismatch; + _serverInstanceId = serverInstanceId; } public String getServerInstanceId() { From ab901a3b953696c662e7ab0f278f0224cfba8d2b Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 9 Aug 2024 14:47:47 -0700 Subject: [PATCH 03/43] line characters checkstyle line characters checkstyle --- .../org/apache/pinot/server/api/resources/TablesResource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index c98afe5a0f63..41b98ecfd4ef 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -959,8 +959,8 @@ public TableSegmentValidationInfo validateTableSegmentState( @GET @Path("/tables/{tableName}/segments/mismatch") @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Checks if there is any mismatch of columns in a segment", notes = "Returns true if reload is required on" - + " any segment in a given server") + @ApiOperation(value = "Checks if there is any mismatch of columns in a segment", notes = + "Returns true if reload is required on" + " any segment in a given server") @ApiResponses(value = { @ApiResponse(code = 200, message = "Success", response = TableSegments.class), @ApiResponse(code = 500, message = "Server initialization error", response = ErrorInfo.class) From 4e4a961c953a68d8edda7d10d5ff731d0bad849f Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 12 Aug 2024 16:47:50 -0700 Subject: [PATCH 04/43] segments reload check including indexes metadata segments reload check including indexes metadata --- .../data/manager/BaseTableDataManager.java | 41 +++++++++++++++++++ .../local/data/manager/TableDataManager.java | 13 ++++++ .../server/api/resources/TablesResource.java | 11 +++-- 3 files changed, 61 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 97b7530b046c..e13731b15042 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1013,6 +1013,47 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } + @Override + public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) { + String segmentName = zkMetadata.getSegmentName(); + + // Creates the SegmentDirectory object to access the segment metadata. + // The metadata is null if the segment doesn't exist yet. + SegmentDirectory segmentDirectory = + tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); + SegmentMetadataImpl segmentMetadata = (segmentDirectory == null) ? null : segmentDirectory.getSegmentMetadata(); + + // If the segment doesn't exist on server or its CRC has changed, then we + // need to fall back to download the segment from deep store to load it. + if (segmentMetadata == null || !hasSameCRC(zkMetadata, segmentMetadata)) { + if (segmentMetadata == null) { + _logger.info("Segment: {} does not exist", segmentName); + } else if (!hasSameCRC(zkMetadata, segmentMetadata)) { + _logger.info("Segment: {} has CRC changed from: {} to: {}", segmentName, segmentMetadata.getCrc(), + zkMetadata.getCrc()); + } + closeSegmentDirectoryQuietly(segmentDirectory); + return true; + } + + try { + Schema schema = indexLoadingConfig.getSchema(); + //if the reload of the segment is not needed then return false + if (!ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema)) { + _logger.info("Segment: {} is consistent with latest table config and schema", segmentName); + return false; + } else { + //if re processing or reload is needed on a segment then return true + _logger.info("Segment: {} needs reprocess to reflect latest table config and schema", segmentName); + return true; + } + } catch (Exception e) { + _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); + closeSegmentDirectoryQuietly(segmentDirectory); + return false; + } + } + @Nullable private SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java index 677d659fff72..eba077697a7e 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java @@ -122,6 +122,19 @@ void addNewOnlineSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexL */ boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig); + /** + * + * First tries to load a segment, and then processes to check if the segment reload is needed for the following + * three scenarios: + * 1) If the index config version and segment config version are different + * 2) If the default columns is updated according to the schema. + * 3) If there is addition or deletion of indexes + * 4) If there is any change in star trees + * 5) If there is any change in column min, max values + * @return true if the reload is needed on the segment + */ + boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig); + /** * Downloads a segment and loads it into the table. * NOTE: This method is part of the implementation detail of {@link #addOnlineSegment(String)}. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 41b98ecfd4ef..b09300febab6 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -85,6 +85,8 @@ import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; +import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; @@ -972,14 +974,15 @@ public String checkMismatchedSegments( tableName = DatabaseUtils.translateTableName(tableName, headers); TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableName); Pair tableConfigSchema = tableDataManager.fetchTableConfigAndSchema(); - Schema schema = tableConfigSchema.getValue(); - Set schemaColumns = schema.getPhysicalColumnNames(); + IndexLoadingConfig indexLoadingConfig = + tableDataManager.getIndexLoadingConfig(tableConfigSchema.getLeft(), tableConfigSchema.getRight()); List segmentDataManagers = tableDataManager.acquireAllSegments(); try { boolean mismatchCheck = false; for (SegmentDataManager segmentDataManager : segmentDataManagers) { - Set segmentColumns = SegmentMetadataFetcher.getSegmentColumns(segmentDataManager, columns); - if (!segmentColumns.containsAll(schemaColumns)) { + SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentDataManager.getSegmentName()); + if (tableDataManager.checkReloadSegment(segmentZKMetadata, + indexLoadingConfig)) { mismatchCheck = true; break; } From e3ea6a82502393eb587f120e9622d514f98757de Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 12 Aug 2024 16:57:57 -0700 Subject: [PATCH 05/43] checkstyle unused import removal checkstyle unused import removal --- .../org/apache/pinot/server/api/resources/TablesResource.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index b09300febab6..771d9e4cc069 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -85,7 +85,6 @@ import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; -import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.ImmutableSegment; From de792605eabfeb14e264dad0d60de2ead20ef7e0 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 12 Aug 2024 17:00:12 -0700 Subject: [PATCH 06/43] remove unused code to get the columns remove unused code to get the columns --- .../api/resources/SegmentMetadataFetcher.java | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java index 540fdce10836..c557d0681b8f 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/SegmentMetadataFetcher.java @@ -90,22 +90,6 @@ public static String getSegmentMetadata(SegmentDataManager segmentDataManager, L return JsonUtils.objectToString(segmentMetadataJson); } - /** - * This is a helper method that fetches the segment column names for a given segment. - * @param columns Columns to include for metadata - */ - public static Set getSegmentColumns(SegmentDataManager segmentDataManager, List columns) { - IndexSegment segment = segmentDataManager.getSegment(); - Set columnSet; - if (columns.size() == 1 && columns.get(0).equals("*")) { - // Making code consistent and returning metadata and indexes only for non-virtual columns. - columnSet = segment.getPhysicalColumnNames(); - } else { - columnSet = new HashSet<>(columns); - } - return columnSet; - } - /** * Get the JSON object with the segment column's indexing metadata. * Lists all the columns if the parameter columnSet is null. From 36beb11bfcc00920816872364a9fea1476adec35 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 15:19:24 -0700 Subject: [PATCH 07/43] refactoring and review comments fixes --- ....java => SegmentsReloadCheckResponse.java} | 14 +++--- .../data/manager/BaseTableDataManager.java | 48 +++++++++++-------- .../local/data/manager/TableDataManager.java | 14 +++--- .../server/api/resources/TablesResource.java | 40 ++++------------ 4 files changed, 50 insertions(+), 66 deletions(-) rename pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/{SegmentColumnMismatchResponse.java => SegmentsReloadCheckResponse.java} (83%) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java similarity index 83% rename from pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java rename to pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java index 32995afb7cfb..fe4843f6ac34 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentColumnMismatchResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java @@ -27,14 +27,14 @@ * * It has details of server id and returns true/false if there are any segments to be reloaded or not. */ -public class SegmentColumnMismatchResponse { - boolean _isMismatch; - String _serverInstanceId; +public class SegmentsReloadCheckResponse { + private final boolean _isReload; + private final String _serverInstanceId; @JsonCreator - public SegmentColumnMismatchResponse(@JsonProperty("isMismatch") boolean isMismatch, + public SegmentsReloadCheckResponse(@JsonProperty("isReload") boolean isReload, @JsonProperty("serverInstanceId") String serverInstanceId) { - _isMismatch = isMismatch; + _isReload = isReload; _serverInstanceId = serverInstanceId; } @@ -42,7 +42,7 @@ public String getServerInstanceId() { return _serverInstanceId; } - public boolean getMismatch() { - return _isMismatch; + public boolean getReload() { + return _isReload; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index e13731b15042..2f91b2e99664 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1013,38 +1013,24 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } + /** + * This method checks if any reload is needed on a segment. + * Scans through indices, columns, startree index and min max values to achieve the result + * True if reload is needed and false if no reload is needed. + */ @Override public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) { String segmentName = zkMetadata.getSegmentName(); - - // Creates the SegmentDirectory object to access the segment metadata. - // The metadata is null if the segment doesn't exist yet. SegmentDirectory segmentDirectory = tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); - SegmentMetadataImpl segmentMetadata = (segmentDirectory == null) ? null : segmentDirectory.getSegmentMetadata(); - - // If the segment doesn't exist on server or its CRC has changed, then we - // need to fall back to download the segment from deep store to load it. - if (segmentMetadata == null || !hasSameCRC(zkMetadata, segmentMetadata)) { - if (segmentMetadata == null) { - _logger.info("Segment: {} does not exist", segmentName); - } else if (!hasSameCRC(zkMetadata, segmentMetadata)) { - _logger.info("Segment: {} has CRC changed from: {} to: {}", segmentName, segmentMetadata.getCrc(), - zkMetadata.getCrc()); - } - closeSegmentDirectoryQuietly(segmentDirectory); - return true; - } try { Schema schema = indexLoadingConfig.getSchema(); //if the reload of the segment is not needed then return false if (!ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema)) { - _logger.info("Segment: {} is consistent with latest table config and schema", segmentName); return false; } else { //if re processing or reload is needed on a segment then return true - _logger.info("Segment: {} needs reprocess to reflect latest table config and schema", segmentName); return true; } } catch (Exception e) { @@ -1055,8 +1041,7 @@ public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConf } @Nullable - private SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, - IndexLoadingConfig indexLoadingConfig) { + SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) { try { return initSegmentDirectory(segmentName, segmentCrc, indexLoadingConfig); } catch (Exception e) { @@ -1065,6 +1050,27 @@ private SegmentDirectory tryInitSegmentDirectory(String segmentName, String segm } } + @Override + public boolean needReloadSegments() { + IndexLoadingConfig indexLoadingConfig = fetchIndexLoadingConfig(); + List segmentDataManagers = acquireAllSegments(); + boolean mismatchCheck = false; + try { + for (SegmentDataManager segmentDataManager : segmentDataManagers) { + SegmentZKMetadata segmentZKMetadata = fetchZKMetadata(segmentDataManager.getSegmentName()); + if (checkReloadSegment(segmentZKMetadata, indexLoadingConfig)) { + mismatchCheck = true; + break; + } + } + } finally { + for (SegmentDataManager segmentDataManager : segmentDataManagers) { + releaseSegment(segmentDataManager); + } + } + return mismatchCheck; + } + private SegmentDirectory initSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) throws Exception { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java index eba077697a7e..e54908bd9fb2 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java @@ -124,17 +124,17 @@ void addNewOnlineSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexL /** * - * First tries to load a segment, and then processes to check if the segment reload is needed for the following - * three scenarios: - * 1) If the index config version and segment config version are different - * 2) If the default columns is updated according to the schema. - * 3) If there is addition or deletion of indexes - * 4) If there is any change in star trees - * 5) If there is any change in column min, max values + * Checks for a particular segment, reload is needed or not * @return true if the reload is needed on the segment */ boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig); + /** + * Check if reload is needed for any of the segments of a table + * @return true if reload is needed for any of the segments and false otherwise + */ + boolean needReloadSegments(); + /** * Downloads a segment and loads it into the table. * NOTE: This method is part of the implementation detail of {@link #addOnlineSegment(String)}. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 771d9e4cc069..110a1205fefc 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -63,7 +63,7 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; -import org.apache.pinot.common.restlet.resources.SegmentColumnMismatchResponse; +import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; @@ -85,7 +85,6 @@ import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; -import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.ImmutableSegment; import org.apache.pinot.segment.spi.IndexSegment; @@ -96,11 +95,9 @@ import org.apache.pinot.server.access.AccessControlFactory; import org.apache.pinot.server.api.AdminApiApplication; import org.apache.pinot.server.starter.ServerInstance; -import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; -import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.stream.ConsumerPartitionState; import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel; import org.apache.pinot.spi.utils.JsonUtils; @@ -958,40 +955,21 @@ public TableSegmentValidationInfo validateTableSegmentState( } @GET - @Path("/tables/{tableName}/segments/mismatch") + @Path("/tables/{tableName}/segments/reload") @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Checks if there is any mismatch of columns in a segment", notes = - "Returns true if reload is required on" + " any segment in a given server") + @ApiOperation(value = "Checks if reload is needed on any segment", notes = "Returns true if reload is required on" + + " any segment in this server") @ApiResponses(value = { @ApiResponse(code = 200, message = "Success", response = TableSegments.class), @ApiResponse(code = 500, message = "Server initialization error", response = ErrorInfo.class) }) - public String checkMismatchedSegments( + public String checkSegmentsReload( @ApiParam(value = "Table Name with type", required = true) @PathParam("tableName") String tableName, - @ApiParam(value = "Column name", allowMultiple = true) @QueryParam("columns") @DefaultValue("") - List columns, @Context HttpHeaders headers) { + @Context HttpHeaders headers) { tableName = DatabaseUtils.translateTableName(tableName, headers); TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableName); - Pair tableConfigSchema = tableDataManager.fetchTableConfigAndSchema(); - IndexLoadingConfig indexLoadingConfig = - tableDataManager.getIndexLoadingConfig(tableConfigSchema.getLeft(), tableConfigSchema.getRight()); - List segmentDataManagers = tableDataManager.acquireAllSegments(); - try { - boolean mismatchCheck = false; - for (SegmentDataManager segmentDataManager : segmentDataManagers) { - SegmentZKMetadata segmentZKMetadata = tableDataManager.fetchZKMetadata(segmentDataManager.getSegmentName()); - if (tableDataManager.checkReloadSegment(segmentZKMetadata, - indexLoadingConfig)) { - mismatchCheck = true; - break; - } - } - return ResourceUtils.convertToJsonString( - new SegmentColumnMismatchResponse(mismatchCheck, tableDataManager.getInstanceId())); - } finally { - for (SegmentDataManager segmentDataManager : segmentDataManagers) { - tableDataManager.releaseSegment(segmentDataManager); - } - } + boolean isSegmentsReload = tableDataManager.needReloadSegments(); + return ResourceUtils.convertToJsonString( + new SegmentsReloadCheckResponse(isSegmentsReload, tableDataManager.getInstanceId())); } } From 0ee036fbee72a998ec406e78f6988424559a7f9c Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 15:23:33 -0700 Subject: [PATCH 08/43] checkstyle changes for the condition --- .../core/data/manager/BaseTableDataManager.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 2f91b2e99664..590dea4d1193 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1026,13 +1026,8 @@ public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConf try { Schema schema = indexLoadingConfig.getSchema(); - //if the reload of the segment is not needed then return false - if (!ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema)) { - return false; - } else { - //if re processing or reload is needed on a segment then return true - return true; - } + //if re processing or reload is needed on a segment then return true + return ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema); } catch (Exception e) { _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); closeSegmentDirectoryQuietly(segmentDirectory); @@ -1041,7 +1036,8 @@ public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConf } @Nullable - SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) { + SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, + IndexLoadingConfig indexLoadingConfig) { try { return initSegmentDirectory(segmentName, segmentCrc, indexLoadingConfig); } catch (Exception e) { From c8ba979d7705a0bd6a234d7f772a61d87a5f688f Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 15:35:22 -0700 Subject: [PATCH 09/43] checkstyle exception for import --- .../org/apache/pinot/server/api/resources/TablesResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 110a1205fefc..8f1432e6cd7f 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -63,8 +63,8 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; -import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; +import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; import org.apache.pinot.common.restlet.resources.TableSegments; From e1c7c6421282bbd0d6c8ce967b336aeb0b77a3dd Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 16:14:16 -0700 Subject: [PATCH 10/43] change the flag name --- .../pinot/core/data/manager/BaseTableDataManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 590dea4d1193..843e4844caee 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1050,12 +1050,12 @@ SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, public boolean needReloadSegments() { IndexLoadingConfig indexLoadingConfig = fetchIndexLoadingConfig(); List segmentDataManagers = acquireAllSegments(); - boolean mismatchCheck = false; + boolean segmentsReloadCheck = false; try { for (SegmentDataManager segmentDataManager : segmentDataManagers) { SegmentZKMetadata segmentZKMetadata = fetchZKMetadata(segmentDataManager.getSegmentName()); if (checkReloadSegment(segmentZKMetadata, indexLoadingConfig)) { - mismatchCheck = true; + segmentsReloadCheck = true; break; } } @@ -1064,7 +1064,7 @@ public boolean needReloadSegments() { releaseSegment(segmentDataManager); } } - return mismatchCheck; + return segmentsReloadCheck; } private SegmentDirectory initSegmentDirectory(String segmentName, String segmentCrc, From b9e4fbf990cc2c595d9dcf40537f17208821e735 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 17:55:26 -0700 Subject: [PATCH 11/43] review comments on variable names and response --- .../resources/SegmentsReloadCheckResponse.java | 4 ++-- .../data/manager/BaseTableDataManager.java | 18 ++++++++++-------- .../local/data/manager/TableDataManager.java | 6 ++++-- .../server/api/resources/TablesResource.java | 7 ++++--- 4 files changed, 20 insertions(+), 15 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java index fe4843f6ac34..32a08b09c4a0 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java @@ -32,9 +32,9 @@ public class SegmentsReloadCheckResponse { private final String _serverInstanceId; @JsonCreator - public SegmentsReloadCheckResponse(@JsonProperty("isReload") boolean isReload, + public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, @JsonProperty("serverInstanceId") String serverInstanceId) { - _isReload = isReload; + _isReload = needReload; _serverInstanceId = serverInstanceId; } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 843e4844caee..da3fb07626cb 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1019,7 +1019,8 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading * True if reload is needed and false if no reload is needed. */ @Override - public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) { + public boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) + throws Exception { String segmentName = zkMetadata.getSegmentName(); SegmentDirectory segmentDirectory = tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); @@ -1031,12 +1032,12 @@ public boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConf } catch (Exception e) { _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); closeSegmentDirectoryQuietly(segmentDirectory); - return false; + throw new Exception(String.format("Failed to perform reload check on the segment: %s", segmentName), e); } } @Nullable - SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, + private SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) { try { return initSegmentDirectory(segmentName, segmentCrc, indexLoadingConfig); @@ -1047,15 +1048,16 @@ SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, } @Override - public boolean needReloadSegments() { + public boolean needReloadSegments() + throws Exception { IndexLoadingConfig indexLoadingConfig = fetchIndexLoadingConfig(); List segmentDataManagers = acquireAllSegments(); - boolean segmentsReloadCheck = false; + boolean needReload = false; try { for (SegmentDataManager segmentDataManager : segmentDataManagers) { SegmentZKMetadata segmentZKMetadata = fetchZKMetadata(segmentDataManager.getSegmentName()); - if (checkReloadSegment(segmentZKMetadata, indexLoadingConfig)) { - segmentsReloadCheck = true; + if (needReloadSegment(segmentZKMetadata, indexLoadingConfig)) { + needReload = true; break; } } @@ -1064,7 +1066,7 @@ public boolean needReloadSegments() { releaseSegment(segmentDataManager); } } - return segmentsReloadCheck; + return needReload; } private SegmentDirectory initSegmentDirectory(String segmentName, String segmentCrc, diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java index e54908bd9fb2..6af11d645bb2 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java @@ -127,13 +127,15 @@ void addNewOnlineSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexL * Checks for a particular segment, reload is needed or not * @return true if the reload is needed on the segment */ - boolean checkReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig); + boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) + throws Exception; /** * Check if reload is needed for any of the segments of a table * @return true if reload is needed for any of the segments and false otherwise */ - boolean needReloadSegments(); + boolean needReloadSegments() + throws Exception; /** * Downloads a segment and loads it into the table. diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 8f1432e6cd7f..962d8b595341 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -965,11 +965,12 @@ public TableSegmentValidationInfo validateTableSegmentState( }) public String checkSegmentsReload( @ApiParam(value = "Table Name with type", required = true) @PathParam("tableName") String tableName, - @Context HttpHeaders headers) { + @Context HttpHeaders headers) + throws Exception { tableName = DatabaseUtils.translateTableName(tableName, headers); TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableName); - boolean isSegmentsReload = tableDataManager.needReloadSegments(); + boolean needReload = tableDataManager.needReloadSegments(); return ResourceUtils.convertToJsonString( - new SegmentsReloadCheckResponse(isSegmentsReload, tableDataManager.getInstanceId())); + new SegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); } } From eef4de5daa45e1736312fafd08f696896c5b4c5b Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 17:57:58 -0700 Subject: [PATCH 12/43] test to check for reload segment when adding index --- .../manager/BaseTableDataManagerTest.java | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java index 1282e493fd95..a15254636699 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java @@ -49,6 +49,7 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; +import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -289,6 +290,32 @@ public void testReloadSegmentAddIndex() assertTrue(hasInvertedIndex(indexDir, LONG_COLUMN)); } + @Test + public void testNeedReloadSegmentAddIndex() + throws Exception { + File indexDir = createSegment(SegmentVersion.v3, 5); + long crc = getCRC(indexDir); + assertFalse(hasInvertedIndex(indexDir, STRING_COLUMN)); + assertFalse(hasInvertedIndex(indexDir, LONG_COLUMN)); + + // Same CRCs so load the local segment directory directly. + SegmentZKMetadata zkMetadata = mock(SegmentZKMetadata.class); + when(zkMetadata.getCrc()).thenReturn(crc); + when(zkMetadata.getSegmentName()).thenReturn(SEGMENT_NAME); + SegmentMetadata localMetadata = mock(SegmentMetadata.class); + when(localMetadata.getCrc()).thenReturn(Long.toString(crc)); + SegmentDirectory segmentDirectory = mock(SegmentDirectory.class); + // Require to add indices. + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); + indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(Arrays.asList(STRING_COLUMN, LONG_COLUMN))); + BaseTableDataManager tableDataManager = createTableManager(); + assertTrue(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); + tableDataManager.reloadSegment(SEGMENT_NAME, indexLoadingConfig, zkMetadata, localMetadata, null, false); + assertFalse(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); + assertTrue(hasInvertedIndex(indexDir, STRING_COLUMN)); + assertTrue(hasInvertedIndex(indexDir, LONG_COLUMN)); + } + @Test public void testReloadSegmentForceDownload() throws Exception { From 9b904c6ebabf987915a03433edbfc97aed5b3902 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 18:05:57 -0700 Subject: [PATCH 13/43] Remove unimplemented method and make private --- .../pinot/core/data/manager/BaseTableDataManager.java | 9 +-------- .../segment/local/data/manager/TableDataManager.java | 8 -------- 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index da3fb07626cb..1198c679e17e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1013,18 +1013,11 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } - /** - * This method checks if any reload is needed on a segment. - * Scans through indices, columns, startree index and min max values to achieve the result - * True if reload is needed and false if no reload is needed. - */ - @Override - public boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) + private boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) throws Exception { String segmentName = zkMetadata.getSegmentName(); SegmentDirectory segmentDirectory = tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); - try { Schema schema = indexLoadingConfig.getSchema(); //if re processing or reload is needed on a segment then return true diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java index 6af11d645bb2..480b2ba70ba8 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/data/manager/TableDataManager.java @@ -122,14 +122,6 @@ void addNewOnlineSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexL */ boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig); - /** - * - * Checks for a particular segment, reload is needed or not - * @return true if the reload is needed on the segment - */ - boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) - throws Exception; - /** * Check if reload is needed for any of the segments of a table * @return true if reload is needed for any of the segments and false otherwise From 38992b1f7206f19868fdf79b5a2df13c5ff4b426 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 18:21:04 -0700 Subject: [PATCH 14/43] Change error responses --- .../core/data/manager/BaseTableDataManager.java | 4 +++- .../pinot/server/api/resources/TablesResource.java | 12 ++++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 1198c679e17e..6c5999efdad8 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -41,6 +41,8 @@ import java.util.concurrent.locks.Lock; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; @@ -1025,7 +1027,7 @@ private boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConf } catch (Exception e) { _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); closeSegmentDirectoryQuietly(segmentDirectory); - throw new Exception(String.format("Failed to perform reload check on the segment: %s", segmentName), e); + throw new Exception(String.format("Failed to perform reload check on the segment %s", segmentName)); } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 962d8b595341..ac9c096dac41 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -961,15 +961,19 @@ public TableSegmentValidationInfo validateTableSegmentState( + " any segment in this server") @ApiResponses(value = { @ApiResponse(code = 200, message = "Success", response = TableSegments.class), @ApiResponse(code = 500, - message = "Server initialization error", response = ErrorInfo.class) + message = "Internal Server error", response = ErrorInfo.class) }) public String checkSegmentsReload( @ApiParam(value = "Table Name with type", required = true) @PathParam("tableName") String tableName, - @Context HttpHeaders headers) - throws Exception { + @Context HttpHeaders headers) { tableName = DatabaseUtils.translateTableName(tableName, headers); TableDataManager tableDataManager = ServerResourceUtils.checkGetTableDataManager(_serverInstance, tableName); - boolean needReload = tableDataManager.needReloadSegments(); + boolean needReload = false; + try { + needReload = tableDataManager.needReloadSegments(); + } catch (Exception e) { + throw new WebApplicationException(e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR); + } return ResourceUtils.convertToJsonString( new SegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); } From d10b8b25d319515cf742c70a77b43a0588cd20da Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 18:24:14 -0700 Subject: [PATCH 15/43] remove unused imports remove unused imports --- .../apache/pinot/core/data/manager/BaseTableDataManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 6c5999efdad8..1b65c4634cfb 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -41,8 +41,6 @@ import java.util.concurrent.locks.Lock; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import javax.ws.rs.WebApplicationException; -import javax.ws.rs.core.Response; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; From 12656ecadf6d54bd6abfb0f00ad5fafca06a4460 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 18:31:06 -0700 Subject: [PATCH 16/43] Remove serverInstanceId from the response Remove serverInstanceId from the response --- .../resources/SegmentsReloadCheckResponse.java | 18 +++++------------- .../server/api/resources/TablesResource.java | 2 +- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java index 32a08b09c4a0..660dbacc759c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java @@ -28,21 +28,13 @@ * It has details of server id and returns true/false if there are any segments to be reloaded or not. */ public class SegmentsReloadCheckResponse { - private final boolean _isReload; - private final String _serverInstanceId; + private final boolean _needReload; @JsonCreator - public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, - @JsonProperty("serverInstanceId") String serverInstanceId) { - _isReload = needReload; - _serverInstanceId = serverInstanceId; + public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload) { + _needReload = needReload; } - - public String getServerInstanceId() { - return _serverInstanceId; - } - - public boolean getReload() { - return _isReload; + public boolean getNeedReload() { + return _needReload; } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index ac9c096dac41..a1001fcf539d 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -975,6 +975,6 @@ public String checkSegmentsReload( throw new WebApplicationException(e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR); } return ResourceUtils.convertToJsonString( - new SegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); + new SegmentsReloadCheckResponse(needReload)); } } From b987f0e4f78b0841447313e5d3861be64bdc1736 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 13 Aug 2024 18:43:12 -0700 Subject: [PATCH 17/43] Fix the test Fix the test --- .../apache/pinot/core/data/manager/BaseTableDataManager.java | 2 +- .../pinot/core/data/manager/BaseTableDataManagerTest.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 1b65c4634cfb..684cc97cca3b 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1013,7 +1013,7 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } - private boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) + boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) throws Exception { String segmentName = zkMetadata.getSegmentName(); SegmentDirectory segmentDirectory = diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java index a15254636699..290aaf61134a 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java @@ -312,8 +312,6 @@ public void testNeedReloadSegmentAddIndex() assertTrue(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); tableDataManager.reloadSegment(SEGMENT_NAME, indexLoadingConfig, zkMetadata, localMetadata, null, false); assertFalse(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); - assertTrue(hasInvertedIndex(indexDir, STRING_COLUMN)); - assertTrue(hasInvertedIndex(indexDir, LONG_COLUMN)); } @Test From c6aa9ca1e1a16e23eafb3b9a37bf7416a4a86a93 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Wed, 14 Aug 2024 18:46:53 -0700 Subject: [PATCH 18/43] Add server instance id back --- .../resources/SegmentsReloadCheckResponse.java | 16 ++++++++++++---- .../server/api/resources/TablesResource.java | 4 ++-- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java index 660dbacc759c..dcda4d026466 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java @@ -29,12 +29,20 @@ */ public class SegmentsReloadCheckResponse { private final boolean _needReload; + private final String _serverInstanceId; - @JsonCreator - public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload) { - _needReload = needReload; + public String getServerInstanceId() { + return _serverInstanceId; } - public boolean getNeedReload() { + + public boolean getReload() { return _needReload; } + + @JsonCreator + public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, + @JsonProperty("serverInstanceId") String serverInstanceId) { + _needReload = needReload; + _serverInstanceId = serverInstanceId; + } } diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index a1001fcf539d..61787d2c0349 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -955,7 +955,7 @@ public TableSegmentValidationInfo validateTableSegmentState( } @GET - @Path("/tables/{tableName}/segments/reload") + @Path("/tables/{tableName}/segments/needReload") @Produces(MediaType.APPLICATION_JSON) @ApiOperation(value = "Checks if reload is needed on any segment", notes = "Returns true if reload is required on" + " any segment in this server") @@ -975,6 +975,6 @@ public String checkSegmentsReload( throw new WebApplicationException(e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR); } return ResourceUtils.convertToJsonString( - new SegmentsReloadCheckResponse(needReload)); + new SegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); } } From 8848fa65480b4ed88d42f221f60528cc6c6e089e Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 01:14:42 -0700 Subject: [PATCH 19/43] review comments fixes --- .../SegmentsReloadCheckResponse.java | 13 ++--- .../common/utils/SerializerResponseTest.java | 48 +++++++++++++++++++ .../data/manager/BaseTableDataManager.java | 18 ++++--- .../manager/BaseTableDataManagerTest.java | 23 --------- .../integration/tests/TlsIntegrationTest.java | 1 + 5 files changed, 67 insertions(+), 36 deletions(-) create mode 100644 pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java index dcda4d026466..13d02e22e76a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java @@ -29,20 +29,21 @@ */ public class SegmentsReloadCheckResponse { private final boolean _needReload; - private final String _serverInstanceId; + private final String _instanceId; - public String getServerInstanceId() { - return _serverInstanceId; + public String getInstanceId() { + return _instanceId; } - public boolean getReload() { + @JsonProperty("needReload") + public boolean getNeedReload() { return _needReload; } @JsonCreator public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, - @JsonProperty("serverInstanceId") String serverInstanceId) { + @JsonProperty("instanceId") String instanceId) { _needReload = needReload; - _serverInstanceId = serverInstanceId; + _instanceId = instanceId; } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java new file mode 100644 index 000000000000..e4be6683cee5 --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -0,0 +1,48 @@ +package org.apache.pinot.common.utils; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.grpc.internal.JsonUtil; +import java.io.IOException; +import org.apache.pinot.common.restlet.resources.ResourceUtils; +import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import org.apache.pinot.spi.utils.JsonUtils; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + + +public class SerializerResponseTest { + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Test + public void testSerialization() + throws JsonProcessingException { + // Given + boolean needReload = true; + String instanceId = "instance123"; + SegmentsReloadCheckResponse response = new SegmentsReloadCheckResponse(needReload, instanceId); + String responseString = ResourceUtils.convertToJsonString(response); + + assertNotNull(responseString); + assertEquals("{\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + "}", responseString); + } + + @Test + public void testDeserialization() + throws Exception { + // Given + String json = "{\"needReload\":true,\"instanceId\":\"instance123\"}"; + + // When + JsonNode jsonNode = JsonUtils.stringToJsonNode(json); + + // Then + assertNotNull(jsonNode); + assertTrue(jsonNode.get("needReload").asBoolean()); + assertEquals("instance123", jsonNode.get("instanceId").asText()); + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 684cc97cca3b..e8d206dbb187 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -61,6 +61,7 @@ import org.apache.pinot.core.util.PeerServerSegmentFinder; import org.apache.pinot.segment.local.data.manager.SegmentDataManager; import org.apache.pinot.segment.local.data.manager.TableDataManager; +import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentImpl; import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.index.loader.LoaderUtils; @@ -1013,15 +1014,16 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } - boolean needReloadSegment(SegmentZKMetadata zkMetadata, IndexLoadingConfig indexLoadingConfig) + private boolean needReloadSegment(String segmentName, String crc, IndexLoadingConfig indexLoadingConfig) throws Exception { - String segmentName = zkMetadata.getSegmentName(); - SegmentDirectory segmentDirectory = - tryInitSegmentDirectory(segmentName, String.valueOf(zkMetadata.getCrc()), indexLoadingConfig); + SegmentDirectory segmentDirectory = tryInitSegmentDirectory(segmentName, crc, indexLoadingConfig); try { Schema schema = indexLoadingConfig.getSchema(); //if re processing or reload is needed on a segment then return true - return ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema); + if (segmentDirectory != null) { + return ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema); + } + return false; } catch (Exception e) { _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); closeSegmentDirectoryQuietly(segmentDirectory); @@ -1048,8 +1050,10 @@ public boolean needReloadSegments() boolean needReload = false; try { for (SegmentDataManager segmentDataManager : segmentDataManagers) { - SegmentZKMetadata segmentZKMetadata = fetchZKMetadata(segmentDataManager.getSegmentName()); - if (needReloadSegment(segmentZKMetadata, indexLoadingConfig)) { + String crcData = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); + IndexSegment segment = segmentDataManager.getSegment(); + if (segment instanceof ImmutableSegmentImpl && needReloadSegment(segment.getSegmentName(), crcData, + indexLoadingConfig)) { needReload = true; break; } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java index 290aaf61134a..0ec114b275ac 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java @@ -291,29 +291,6 @@ public void testReloadSegmentAddIndex() } @Test - public void testNeedReloadSegmentAddIndex() - throws Exception { - File indexDir = createSegment(SegmentVersion.v3, 5); - long crc = getCRC(indexDir); - assertFalse(hasInvertedIndex(indexDir, STRING_COLUMN)); - assertFalse(hasInvertedIndex(indexDir, LONG_COLUMN)); - - // Same CRCs so load the local segment directory directly. - SegmentZKMetadata zkMetadata = mock(SegmentZKMetadata.class); - when(zkMetadata.getCrc()).thenReturn(crc); - when(zkMetadata.getSegmentName()).thenReturn(SEGMENT_NAME); - SegmentMetadata localMetadata = mock(SegmentMetadata.class); - when(localMetadata.getCrc()).thenReturn(Long.toString(crc)); - SegmentDirectory segmentDirectory = mock(SegmentDirectory.class); - // Require to add indices. - IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(); - indexLoadingConfig.setInvertedIndexColumns(new HashSet<>(Arrays.asList(STRING_COLUMN, LONG_COLUMN))); - BaseTableDataManager tableDataManager = createTableManager(); - assertTrue(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); - tableDataManager.reloadSegment(SEGMENT_NAME, indexLoadingConfig, zkMetadata, localMetadata, null, false); - assertFalse(tableDataManager.needReloadSegment(zkMetadata, indexLoadingConfig)); - } - @Test public void testReloadSegmentForceDownload() throws Exception { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java index ad31328f5828..0bf2121134de 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java @@ -225,6 +225,7 @@ protected void overrideServerConf(PinotConfiguration serverConf) { int nettyTlsPort = NetUtils.findOpenPort(internalAdminPort + 1); serverConf.setProperty("pinot.server.nettytls.port", nettyTlsPort); _nextServerPort = nettyTlsPort + 1; + _serverBaseApiUrl = "http://localhost:" + _nextServerPort; serverConf.setProperty("pinot.server.segment.uploader.protocol", "https"); } From 048eb501ea38ba6eb74ebda6a3f4aae6e34a8d13 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 01:18:31 -0700 Subject: [PATCH 20/43] review comments fixes --- .../pinot/core/data/manager/BaseTableDataManagerTest.java | 2 -- .../org/apache/pinot/integration/tests/TlsIntegrationTest.java | 1 - 2 files changed, 3 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java index 0ec114b275ac..1282e493fd95 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/BaseTableDataManagerTest.java @@ -49,7 +49,6 @@ import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.index.metadata.SegmentMetadataImpl; -import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; import org.apache.pinot.spi.config.instance.InstanceDataManagerConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -290,7 +289,6 @@ public void testReloadSegmentAddIndex() assertTrue(hasInvertedIndex(indexDir, LONG_COLUMN)); } - @Test @Test public void testReloadSegmentForceDownload() throws Exception { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java index 0bf2121134de..ad31328f5828 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/TlsIntegrationTest.java @@ -225,7 +225,6 @@ protected void overrideServerConf(PinotConfiguration serverConf) { int nettyTlsPort = NetUtils.findOpenPort(internalAdminPort + 1); serverConf.setProperty("pinot.server.nettytls.port", nettyTlsPort); _nextServerPort = nettyTlsPort + 1; - _serverBaseApiUrl = "http://localhost:" + _nextServerPort; serverConf.setProperty("pinot.server.segment.uploader.protocol", "https"); } From e439b54ef685ed60301df3f143bdfa763bf7a487 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 01:20:01 -0700 Subject: [PATCH 21/43] remove unnecessary code here --- .../apache/pinot/common/utils/SerializerResponseTest.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java index e4be6683cee5..6e590dbb4266 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -1,10 +1,6 @@ package org.apache.pinot.common.utils; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.grpc.internal.JsonUtil; -import java.io.IOException; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; import org.apache.pinot.spi.utils.JsonUtils; @@ -16,11 +12,9 @@ public class SerializerResponseTest { - private final ObjectMapper objectMapper = new ObjectMapper(); @Test - public void testSerialization() - throws JsonProcessingException { + public void testSerialization() { // Given boolean needReload = true; String instanceId = "instance123"; From 028449c81dd3140ebc1154b7153bb8fb7ae2ebfb Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 01:27:08 -0700 Subject: [PATCH 22/43] add license header to the class --- .../common/utils/SerializerResponseTest.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java index 6e590dbb4266..911ad02c456b 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -1,3 +1,21 @@ +/** + * 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.common.utils; import com.fasterxml.jackson.databind.JsonNode; @@ -11,6 +29,10 @@ import static org.testng.Assert.assertTrue; +/** + * Tests some of the serializer and deserialization responses from SegmentsReloadCheckResponse class + * needReload will have to be carefully evaluated + */ public class SerializerResponseTest { @Test From 5c50ccacbe9efd6e9ff99cac10ff28f30bc986bf Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 16:25:29 -0700 Subject: [PATCH 23/43] Add Controller Side API and test in integration Add Controller Side API and test in integration --- .../TableSegmentsReloadCheckResponse.java | 31 ++++++++++++ .../PinotSegmentRestletResource.java | 49 +++++++++++++++++++ .../helix/ControllerRequestClient.java | 12 +++++ .../util/ServerSegmentMetadataReader.java | 44 +++++++++++++++++ .../controller/util/TableMetadataReader.java | 26 ++++++++++ .../controller/helix/ControllerTest.java | 5 ++ .../tests/BaseClusterIntegrationTestSet.java | 12 ++++- .../builder/ControllerRequestURLBuilder.java | 5 ++ 8 files changed, 182 insertions(+), 2 deletions(-) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java new file mode 100644 index 000000000000..d169812e4953 --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -0,0 +1,31 @@ +package org.apache.pinot.common.restlet.resources; + +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + + +/** + * This class gives list of the details from each server if there exists any segments that need to be reloaded + * + * It has details of reload flag which returns true if reload is needed on table and additional details of the + * servers which need reload. + */ +public class TableSegmentsReloadCheckResponse { + boolean _needReload; + Map _serverToSegmentsReloadList; + + public Map getServerToSegmentsReloadList() { + return _serverToSegmentsReloadList; + } + + public boolean isNeedReload() { + return _needReload; + } + + public TableSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, + Map serverToSegmentsReloadList) { + _needReload = needReload; + _serverToSegmentsReloadList = serverToSegmentsReloadList; + } +} diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java index c75469d68c3a..1a66e9e65e4c 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java @@ -69,6 +69,8 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.controllerjob.ControllerJobType; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; +import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import org.apache.pinot.common.restlet.resources.TableSegmentsReloadCheckResponse; import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.common.utils.URIUtils; @@ -822,6 +824,53 @@ public String getServerMetadata( return segmentsMetadata; } + @GET + @Path("segments/{tableNameWithType}/needReload") + @Authorize(targetType = TargetType.TABLE, paramName = "tableNameWithType", action = Actions.Table.GET_METADATA) + @Produces(MediaType.APPLICATION_JSON) + @ApiOperation(value = "Get the metadata of reload segments information from the servers", notes = "Returns true if " + + "reload is needed on the table from any one of the servers") + public String getTableReloadMetadata( + @ApiParam(value = "Table name with type", required = true, example = "myTable_REALTIME") + @PathParam("tableNameWithType") String tableNameWithType, + @QueryParam("verbose") @DefaultValue("false") boolean verbose, @Context HttpHeaders headers) { + String tableName = TableNameBuilder.extractRawTableName(tableNameWithType); + TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); + LOGGER.info("Received a request to fetch metadata for all segments for table {}", tableName); + try { + TableMetadataReader tableMetadataReader = + new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager); + Map needReloadMetadata = + tableMetadataReader.getServerCheckSegmentsReloadMetadata(tableName, tableType, + _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); + boolean needReload = false; + for (Map.Entry entry : needReloadMetadata.entrySet()) { + if (entry.getValue().get("needReload").booleanValue()) { + needReload = true; + break; + } + } + Map serverResponses = new HashMap<>(); + TableSegmentsReloadCheckResponse tableNeedReloadResponse; + if (verbose) { + for (Map.Entry entry : needReloadMetadata.entrySet()) { + serverResponses.put(entry.getKey(), + new SegmentsReloadCheckResponse(entry.getValue().get("needReload").booleanValue(), + entry.getValue().get("instanceId").asText())); + } + tableNeedReloadResponse = new TableSegmentsReloadCheckResponse(needReload, serverResponses); + } else { + tableNeedReloadResponse = new TableSegmentsReloadCheckResponse(needReload, serverResponses); + } + return JsonUtils.objectToPrettyString(tableNeedReloadResponse); + } catch (InvalidConfigException e) { + throw new ControllerApplicationException(LOGGER, e.getMessage(), Status.BAD_REQUEST); + } catch (IOException ioe) { + throw new ControllerApplicationException(LOGGER, "Error parsing Pinot server response: " + ioe.getMessage(), + Status.INTERNAL_SERVER_ERROR, ioe); + } + } + @GET @Path("segments/{tableName}/zkmetadata") @Authorize(targetType = TargetType.TABLE, paramName = "tableName", action = Actions.Table.GET_METADATA) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java index 39d553aa6592..d2b58393d5e6 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java @@ -201,6 +201,18 @@ public String reloadTable(String tableName, TableType tableType, boolean forceDo } } + public String needReloadTable(String tableNameWithType) + throws IOException { + try { + SimpleHttpResponse simpleHttpResponse = HttpClient.wrapAndThrowHttpException( + _httpClient.sendGetRequest(new URI(_controllerRequestURLBuilder.forTableNeedReload(tableNameWithType)), + _headers, null)); + return simpleHttpResponse.getResponse(); + } catch (HttpErrorStatusException | URISyntaxException e) { + throw new IOException(e); + } + } + public void reloadSegment(String tableName, String segmentName, boolean forceReload) throws IOException { try { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java index b3fd851ff41d..f85bf0f0de7a 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java @@ -214,6 +214,45 @@ public List getSegmentMetadataFromServer(String tableNameWithType, return segmentsMetadata; } + /** + * This method is called when the API request is to fetch data about segment reload of the table. + * This method makes a MultiGet call to all servers that host their respective segments and gets the results. + * This method will return metadata of all the servers along with need reload flag. + * In future additional details can also be added + * @return list of servers and the boolean check if reload is needed. + */ + public List getCheckReloadSegmentsFromServer(String tableNameWithType, Set serverInstances, + BiMap endpoints, int timeoutMs) { + LOGGER.debug("Checking if reload is needed on segments from servers for table {}.", tableNameWithType); + List serverURLs = new ArrayList<>(); + for (String serverInstance : serverInstances) { + serverURLs.add(generateCheckReloadSegmentsServerURL(tableNameWithType, endpoints.get(serverInstance))); + } + BiMap endpointsToServers = endpoints.inverse(); + CompletionServiceHelper completionServiceHelper = + new CompletionServiceHelper(_executor, _connectionManager, endpointsToServers); + CompletionServiceHelper.CompletionServiceResponse serviceResponse = + completionServiceHelper.doMultiGetRequest(serverURLs, tableNameWithType, true, timeoutMs); + List serversNeedReloadResponses = new ArrayList<>(); + + int failedParses = 0; + for (Map.Entry streamResponse : serviceResponse._httpResponses.entrySet()) { + try { + String needReloadSegmentsMetadata = streamResponse.getValue(); + serversNeedReloadResponses.add(needReloadSegmentsMetadata); + } catch (Exception e) { + failedParses++; + LOGGER.error("Unable to parse server {} response due to an error: ", streamResponse.getKey(), e); + } + } + if (failedParses != 0) { + LOGGER.error("Unable to parse server {} / {} response due to an error: ", failedParses, serverURLs.size()); + } + + LOGGER.debug("Retrieved metadata of reload check from servers."); + return serversNeedReloadResponses; + } + /** * This method is called when the API request is to fetch validDocId metadata for a list segments of the given table. * This method will pick one server randomly that hosts the target segment and fetch the segment metadata result. @@ -375,6 +414,11 @@ private String generateSegmentMetadataServerURL(String tableNameWithType, String return String.format("%s/tables/%s/segments/%s/metadata?%s", endpoint, tableNameWithType, segmentName, paramsStr); } + private String generateCheckReloadSegmentsServerURL(String tableNameWithType, String endpoint) { + tableNameWithType = URLEncoder.encode(tableNameWithType, StandardCharsets.UTF_8); + return String.format("%s/tables/%s/segments/needReload", endpoint, tableNameWithType); + } + @Deprecated private String generateValidDocIdsURL(String tableNameWithType, String segmentName, String validDocIdsType, String endpoint) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index adf7e3a7b7b1..71cf74e823e5 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,8 +33,12 @@ import org.apache.pinot.common.exception.InvalidConfigException; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; +import org.apache.pinot.controller.api.resources.ResourceUtils; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** @@ -44,6 +49,7 @@ * the column indexes available. */ public class TableMetadataReader { + private static final Logger LOGGER = LoggerFactory.getLogger(TableMetadataReader.class); private final Executor _executor; private final HttpClientConnectionManager _connectionManager; private final PinotHelixResourceManager _pinotHelixResourceManager; @@ -55,6 +61,26 @@ public TableMetadataReader(Executor executor, HttpClientConnectionManager connec _pinotHelixResourceManager = helixResourceManager; } + public Map getServerCheckSegmentsReloadMetadata(String tableName, TableType tableType, int timeoutMs) + throws InvalidConfigException, IOException { + String tableNameWithType = + ResourceUtils.getExistingTableNamesWithType(_pinotHelixResourceManager, tableName, tableType, LOGGER).get(0); + List serverInstances = _pinotHelixResourceManager.getServerInstancesForTable(tableName, tableType); + Set serverInstanceSet = new HashSet<>(serverInstances); + BiMap endpoints = _pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverInstanceSet); + ServerSegmentMetadataReader serverSegmentMetadataReader = + new ServerSegmentMetadataReader(_executor, _connectionManager); + List segmentsMetadata = + serverSegmentMetadataReader.getCheckReloadSegmentsFromServer(tableNameWithType, serverInstanceSet, endpoints, + timeoutMs); + Map response = new HashMap<>(); + for (String segmentMetadata : segmentsMetadata) { + JsonNode responseJson = JsonUtils.stringToJsonNode(segmentMetadata); + response.put(responseJson.get("instanceId").asText(), responseJson); + } + return response; + } + /** * This api takes in list of segments for which we need the metadata. */ diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java index 466de32d2500..97fd3baaa372 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java @@ -721,6 +721,11 @@ public String reloadOfflineTable(String tableName, boolean forceDownload) return getControllerRequestClient().reloadTable(tableName, TableType.OFFLINE, forceDownload); } + public String needReloadOfflineTable(String tableNameWithType) + throws IOException { + return getControllerRequestClient().needReloadTable(tableNameWithType); + } + public void reloadOfflineSegment(String tableName, String segmentName, boolean forceDownload) throws IOException { getControllerRequestClient().reloadSegment(tableName, segmentName, forceDownload); diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 09310cb2431c..77e2ca62dead 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -732,9 +732,14 @@ public void testReload(boolean includeOfflineTable) // Upload the schema with extra columns addSchema(schema); - + String tableNameWithType = TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName); // Reload the table if (includeOfflineTable) { + //test controller api which gives responses if reload is needed on any of the server segments when default columns are added. + String needBeforeReloadResponse = needReloadOfflineTable(tableNameWithType); + JsonNode jsonNeedReloadResponse = JsonUtils.stringToJsonNode(needBeforeReloadResponse); + //test to check if reload is needed i.e true + assertEquals(jsonNeedReloadResponse.get("needReload").asBoolean(), true); reloadOfflineTable(rawTableName); } reloadRealtimeTable(rawTableName); @@ -762,7 +767,6 @@ public void testReload(boolean includeOfflineTable) JsonNode resultTable = queryResponse.get("resultTable"); assertEquals(resultTable.get("dataSchema").get("columnNames").size(), schema.size()); assertEquals(resultTable.get("rows").size(), 10); - // Test aggregation query to include querying all segemnts (including realtime) String aggregationQuery = "SELECT SUMMV(NewIntMVDimension) FROM " + rawTableName; queryResponse = postQuery(aggregationQuery); @@ -778,6 +782,10 @@ public void testReload(boolean includeOfflineTable) queryResponse = postQuery(countStarQuery); assertEquals(queryResponse.get("exceptions").size(), 0); assertEquals(queryResponse.get("resultTable").get("rows").get(0).get(0).asLong(), countStarResult); + String needAfterReloadResponse = needReloadOfflineTable(tableNameWithType); + JsonNode jsonNeedReloadResponseAfter = JsonUtils.stringToJsonNode(needAfterReloadResponse); + //test to check if reload is needed i.e false after reload is finished + assertEquals(jsonNeedReloadResponseAfter.get("needReload").asBoolean(), false); } private DimensionFieldSpec constructNewDimension(FieldSpec.DataType dataType, boolean singleValue) { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java index f4133fee59d2..47ca6606ed35 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java @@ -239,6 +239,11 @@ public String forTableReload(String tableName, TableType tableType, boolean forc return StringUtil.join("/", _baseUrl, "segments", tableName, query); } + public String forTableNeedReload(String tableNameWithType) { + String query = String.format("needReload?verbose=%s", "false"); + return StringUtil.join("/", _baseUrl, "segments", tableNameWithType, query); + } + public String forTableRebalanceStatus(String jobId) { return StringUtil.join("/", _baseUrl, "rebalanceStatus", jobId); } From 83665803a0c477df761c8bf1705451e4d2b51659 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 17:12:02 -0700 Subject: [PATCH 24/43] Refactoring and checkstyle fixes --- .../TableSegmentsReloadCheckResponse.java | 24 +++++++++++++++++-- .../common/utils/SerializerResponseTest.java | 22 +++++++++++++---- .../PinotSegmentRestletResource.java | 11 +++------ .../util/ServerSegmentMetadataReader.java | 6 ++--- 4 files changed, 44 insertions(+), 19 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index d169812e4953..ab02c8b57f58 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -1,5 +1,24 @@ +/** + * 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.common.restlet.resources; +import com.fasterxml.jackson.annotation.JsonCreaoadotor; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; @@ -9,7 +28,7 @@ * This class gives list of the details from each server if there exists any segments that need to be reloaded * * It has details of reload flag which returns true if reload is needed on table and additional details of the - * servers which need reload. + * respective servers. */ public class TableSegmentsReloadCheckResponse { boolean _needReload; @@ -23,8 +42,9 @@ public boolean isNeedReload() { return _needReload; } + @JsonCreator public TableSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, - Map serverToSegmentsReloadList) { + @JsonProperty("serverToSegmentsReloadList") Map serverToSegmentsReloadList) { _needReload = needReload; _serverToSegmentsReloadList = serverToSegmentsReloadList; } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java index 911ad02c456b..d005ca26d3ab 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -19,8 +19,10 @@ package org.apache.pinot.common.utils; import com.fasterxml.jackson.databind.JsonNode; +import java.util.HashMap; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import org.apache.pinot.common.restlet.resources.TableSegmentsReloadCheckResponse; import org.apache.pinot.spi.utils.JsonUtils; import org.testng.annotations.Test; @@ -41,9 +43,14 @@ public void testSerialization() { boolean needReload = true; String instanceId = "instance123"; SegmentsReloadCheckResponse response = new SegmentsReloadCheckResponse(needReload, instanceId); + TableSegmentsReloadCheckResponse tableResponse = new TableSegmentsReloadCheckResponse(needReload, new HashMap<>()); String responseString = ResourceUtils.convertToJsonString(response); + String tableResponseString = ResourceUtils.convertToJsonString(tableResponse); assertNotNull(responseString); + assertNotNull(tableResponseString); + assertEquals("{\n" + " \"needReload\" : true,\n" + " \"serverToSegmentsReloadList\" : { }\n" + "}", + tableResponseString); assertEquals("{\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + "}", responseString); } @@ -51,13 +58,18 @@ public void testSerialization() { public void testDeserialization() throws Exception { // Given - String json = "{\"needReload\":true,\"instanceId\":\"instance123\"}"; - - // When - JsonNode jsonNode = JsonUtils.stringToJsonNode(json); - + boolean needReload = true; + String instanceId = "instance123"; + SegmentsReloadCheckResponse response = new SegmentsReloadCheckResponse(needReload, instanceId); + TableSegmentsReloadCheckResponse tableResponse = new TableSegmentsReloadCheckResponse(needReload, new HashMap<>()); + String responseString = ResourceUtils.convertToJsonString(response); + JsonNode jsonNode = JsonUtils.stringToJsonNode(responseString); + String jsonResponse = JsonUtils.objectToPrettyString(tableResponse); + JsonNode jsonNodeTableResponse = JsonUtils.stringToJsonNode(jsonResponse); // Then assertNotNull(jsonNode); + assertNotNull(jsonNodeTableResponse); + assertTrue(jsonNodeTableResponse.get("needReload").asBoolean()); assertTrue(jsonNode.get("needReload").asBoolean()); assertEquals("instance123", jsonNode.get("instanceId").asText()); } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java index 1a66e9e65e4c..815a78f16e9a 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java @@ -828,7 +828,7 @@ public String getServerMetadata( @Path("segments/{tableNameWithType}/needReload") @Authorize(targetType = TargetType.TABLE, paramName = "tableNameWithType", action = Actions.Table.GET_METADATA) @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Get the metadata of reload segments information from the servers", notes = "Returns true if " + @ApiOperation(value = "Gets the metadata of reload segments check from servers hosting the table", notes = "Returns true if " + "reload is needed on the table from any one of the servers") public String getTableReloadMetadata( @ApiParam(value = "Table name with type", required = true, example = "myTable_REALTIME") @@ -843,13 +843,8 @@ public String getTableReloadMetadata( Map needReloadMetadata = tableMetadataReader.getServerCheckSegmentsReloadMetadata(tableName, tableType, _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); - boolean needReload = false; - for (Map.Entry entry : needReloadMetadata.entrySet()) { - if (entry.getValue().get("needReload").booleanValue()) { - needReload = true; - break; - } - } + boolean needReload = needReloadMetadata.values().stream() + .anyMatch(value -> value.get("needReload").booleanValue()); Map serverResponses = new HashMap<>(); TableSegmentsReloadCheckResponse tableNeedReloadResponse; if (verbose) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java index f85bf0f0de7a..781140a978ba 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/ServerSegmentMetadataReader.java @@ -218,8 +218,7 @@ public List getSegmentMetadataFromServer(String tableNameWithType, * This method is called when the API request is to fetch data about segment reload of the table. * This method makes a MultiGet call to all servers that host their respective segments and gets the results. * This method will return metadata of all the servers along with need reload flag. - * In future additional details can also be added - * @return list of servers and the boolean check if reload is needed. + * In future additional details like segments list can also be added */ public List getCheckReloadSegmentsFromServer(String tableNameWithType, Set serverInstances, BiMap endpoints, int timeoutMs) { @@ -238,8 +237,7 @@ public List getCheckReloadSegmentsFromServer(String tableNameWithType, S int failedParses = 0; for (Map.Entry streamResponse : serviceResponse._httpResponses.entrySet()) { try { - String needReloadSegmentsMetadata = streamResponse.getValue(); - serversNeedReloadResponses.add(needReloadSegmentsMetadata); + serversNeedReloadResponses.add(streamResponse.getValue()); } catch (Exception e) { failedParses++; LOGGER.error("Unable to parse server {} response due to an error: ", streamResponse.getKey(), e); From 25ab04c46313eff363dbbae5cdee20a564745c80 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 17:13:22 -0700 Subject: [PATCH 25/43] import issue --- .../restlet/resources/TableSegmentsReloadCheckResponse.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index ab02c8b57f58..348da90b5148 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -18,7 +18,7 @@ */ package org.apache.pinot.common.restlet.resources; -import com.fasterxml.jackson.annotation.JsonCreaoadotor; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; From 760450f0d7609da1ce9a46d8e012173bc937318f Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 17:13:53 -0700 Subject: [PATCH 26/43] unused import removal --- .../restlet/resources/TableSegmentsReloadCheckResponse.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index 348da90b5148..2f313b782187 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; import java.util.Map; From ff0f1a1a68301f2ca7bb9b7260ce7f8674957456 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 17:18:17 -0700 Subject: [PATCH 27/43] checkstyle fixes --- .../api/resources/PinotSegmentRestletResource.java | 8 ++++---- .../apache/pinot/controller/util/TableMetadataReader.java | 3 ++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java index 815a78f16e9a..082a63729454 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java @@ -828,8 +828,8 @@ public String getServerMetadata( @Path("segments/{tableNameWithType}/needReload") @Authorize(targetType = TargetType.TABLE, paramName = "tableNameWithType", action = Actions.Table.GET_METADATA) @Produces(MediaType.APPLICATION_JSON) - @ApiOperation(value = "Gets the metadata of reload segments check from servers hosting the table", notes = "Returns true if " - + "reload is needed on the table from any one of the servers") + @ApiOperation(value = "Gets the metadata of reload segments check from servers hosting the table", notes = + "Returns true if " + "reload is needed on the table from any one of the servers") public String getTableReloadMetadata( @ApiParam(value = "Table name with type", required = true, example = "myTable_REALTIME") @PathParam("tableNameWithType") String tableNameWithType, @@ -843,8 +843,8 @@ public String getTableReloadMetadata( Map needReloadMetadata = tableMetadataReader.getServerCheckSegmentsReloadMetadata(tableName, tableType, _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); - boolean needReload = needReloadMetadata.values().stream() - .anyMatch(value -> value.get("needReload").booleanValue()); + boolean needReload = + needReloadMetadata.values().stream().anyMatch(value -> value.get("needReload").booleanValue()); Map serverResponses = new HashMap<>(); TableSegmentsReloadCheckResponse tableNeedReloadResponse; if (verbose) { diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index 71cf74e823e5..bddb438eb7de 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -61,7 +61,8 @@ public TableMetadataReader(Executor executor, HttpClientConnectionManager connec _pinotHelixResourceManager = helixResourceManager; } - public Map getServerCheckSegmentsReloadMetadata(String tableName, TableType tableType, int timeoutMs) + public Map getServerCheckSegmentsReloadMetadata(String tableName, TableType tableType, + int timeoutMs) throws InvalidConfigException, IOException { String tableNameWithType = ResourceUtils.getExistingTableNamesWithType(_pinotHelixResourceManager, tableName, tableType, LOGGER).get(0); From a5880c2d8613db4b0fc3d91a3a5ebb1cebcad328 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Fri, 16 Aug 2024 17:22:18 -0700 Subject: [PATCH 28/43] checkstyle exception fix --- .../pinot/integration/tests/BaseClusterIntegrationTestSet.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 77e2ca62dead..6102d5339225 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -735,7 +735,8 @@ public void testReload(boolean includeOfflineTable) String tableNameWithType = TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName); // Reload the table if (includeOfflineTable) { - //test controller api which gives responses if reload is needed on any of the server segments when default columns are added. + //test controller api which gives responses if reload is needed on any of the server segments when default + // columns are added. String needBeforeReloadResponse = needReloadOfflineTable(tableNameWithType); JsonNode jsonNeedReloadResponse = JsonUtils.stringToJsonNode(needBeforeReloadResponse); //test to check if reload is needed i.e true From a0006c903df078e4fb13dd4405e6ce348df309c9 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Sat, 17 Aug 2024 16:50:43 -0700 Subject: [PATCH 29/43] Refactoring changes and test failures fixes Refactoring changes and test failures fixes --- .../data/manager/BaseTableDataManager.java | 10 ++++--- .../tests/BaseClusterIntegrationTestSet.java | 27 ++++++++++++++----- .../immutable/ImmutableSegmentImpl.java | 12 +++++++++ 3 files changed, 38 insertions(+), 11 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index e8d206dbb187..6a9ae39f71d3 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1052,10 +1052,12 @@ public boolean needReloadSegments() for (SegmentDataManager segmentDataManager : segmentDataManagers) { String crcData = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); IndexSegment segment = segmentDataManager.getSegment(); - if (segment instanceof ImmutableSegmentImpl && needReloadSegment(segment.getSegmentName(), crcData, - indexLoadingConfig)) { - needReload = true; - break; + if (segment instanceof ImmutableSegmentImpl) { + ImmutableSegmentImpl immutableSegment = (ImmutableSegmentImpl) segment; + if (immutableSegment.needReloadSegment(segment.getSegmentName(), crcData, indexLoadingConfig)) { + needReload = true; + break; + } } } } finally { diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 6102d5339225..1a22509697c0 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -732,17 +732,24 @@ public void testReload(boolean includeOfflineTable) // Upload the schema with extra columns addSchema(schema); - String tableNameWithType = TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName); + String tableNameWithTypeOffline = TableNameBuilder.forType(TableType.OFFLINE).tableNameWithType(rawTableName); + String tableNameWithTypeRealtime = TableNameBuilder.forType(TableType.REALTIME).tableNameWithType(rawTableName); // Reload the table if (includeOfflineTable) { //test controller api which gives responses if reload is needed on any of the server segments when default - // columns are added. - String needBeforeReloadResponse = needReloadOfflineTable(tableNameWithType); + // columns are added + String needBeforeReloadResponse = needReloadOfflineTable(tableNameWithTypeOffline); JsonNode jsonNeedReloadResponse = JsonUtils.stringToJsonNode(needBeforeReloadResponse); //test to check if reload is needed i.e true assertEquals(jsonNeedReloadResponse.get("needReload").asBoolean(), true); reloadOfflineTable(rawTableName); } + //test controller api which gives responses if reload is needed on any of the server segments when default + // columns are added + String needBeforeReloadResponseRealtime = needReloadOfflineTable(tableNameWithTypeRealtime); + JsonNode jsonNeedReloadResponseRealTime = JsonUtils.stringToJsonNode(needBeforeReloadResponseRealtime); + //test to check if reload is needed i.e true + assertTrue(jsonNeedReloadResponseRealTime.get("needReload").asBoolean()); reloadRealtimeTable(rawTableName); // Wait for all segments to finish reloading, and test querying the new columns @@ -783,10 +790,16 @@ public void testReload(boolean includeOfflineTable) queryResponse = postQuery(countStarQuery); assertEquals(queryResponse.get("exceptions").size(), 0); assertEquals(queryResponse.get("resultTable").get("rows").get(0).get(0).asLong(), countStarResult); - String needAfterReloadResponse = needReloadOfflineTable(tableNameWithType); - JsonNode jsonNeedReloadResponseAfter = JsonUtils.stringToJsonNode(needAfterReloadResponse); - //test to check if reload is needed i.e false after reload is finished - assertEquals(jsonNeedReloadResponseAfter.get("needReload").asBoolean(), false); + if (includeOfflineTable) { + String needAfterReloadResponse = needReloadOfflineTable(tableNameWithTypeOffline); + JsonNode jsonNeedReloadResponseAfter = JsonUtils.stringToJsonNode(needAfterReloadResponse); + //test to check if reload on offline table is needed i.e false after reload is finished + assertFalse(jsonNeedReloadResponseAfter.get("needReload").asBoolean()); + } + String needAfterReloadResponseRealtime = needReloadOfflineTable(tableNameWithTypeRealtime); + JsonNode jsonNeedReloadResponseRealtimeAfter = JsonUtils.stringToJsonNode(needAfterReloadResponseRealtime); + //test to check if reload on real time table is needed i.e false after reload is finished + assertFalse(jsonNeedReloadResponseRealtimeAfter.get("needReload").asBoolean()); } private DimensionFieldSpec constructNewDimension(FieldSpec.DataType dataType, boolean singleValue) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java index ff49a36c8dc1..7110734f333c 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java @@ -33,6 +33,7 @@ import org.apache.pinot.common.utils.HashUtil; import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader; import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader; import org.apache.pinot.segment.local.startree.v2.store.StarTreeIndexContainer; @@ -54,6 +55,7 @@ import org.apache.pinot.segment.spi.index.startree.StarTreeV2; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; +import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; import org.roaringbitmap.buffer.MutableRoaringBitmap; @@ -173,6 +175,16 @@ private File getValidDocIdsSnapshotFile() { V1Constants.VALID_DOC_IDS_SNAPSHOT_FILE_NAME); } + public boolean needReloadSegment(String segmentName, String crc, IndexLoadingConfig indexLoadingConfig) + throws Exception { + Schema schema = indexLoadingConfig.getSchema(); + //if re processing or reload is needed on a segment then return true + if (_segmentDirectory != null) { + return ImmutableSegmentLoader.needPreprocess(_segmentDirectory, indexLoadingConfig, schema); + } + return false; + } + @Override public I getIndex(String column, IndexType type) { ColumnIndexContainer container = _indexContainerMap.get(column); From 84bb84bd571f24c78f6b573fc96c5b2d0d4f9c3b Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Thu, 22 Aug 2024 10:40:59 -0700 Subject: [PATCH 30/43] Code Readability and serializer tests Code Readability and serializer tests --- ...=> ServerSegmentsReloadCheckResponse.java} | 6 +- .../TableSegmentsReloadCheckResponse.java | 12 ++-- .../common/utils/SerializerResponseTest.java | 59 ++++++++++++------- .../PinotSegmentRestletResource.java | 8 +-- .../server/api/resources/TablesResource.java | 4 +- 5 files changed, 55 insertions(+), 34 deletions(-) rename pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/{SegmentsReloadCheckResponse.java => ServerSegmentsReloadCheckResponse.java} (86%) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java similarity index 86% rename from pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java rename to pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java index 13d02e22e76a..169dc0d8f67a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/SegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java @@ -19,6 +19,7 @@ package org.apache.pinot.common.restlet.resources; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; @@ -27,7 +28,8 @@ * * It has details of server id and returns true/false if there are any segments to be reloaded or not. */ -public class SegmentsReloadCheckResponse { +@JsonIgnoreProperties(ignoreUnknown = true) +public class ServerSegmentsReloadCheckResponse { private final boolean _needReload; private final String _instanceId; @@ -41,7 +43,7 @@ public boolean getNeedReload() { } @JsonCreator - public SegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, + public ServerSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, @JsonProperty("instanceId") String instanceId) { _needReload = needReload; _instanceId = instanceId; diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index 2f313b782187..c67f4e655a8f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -19,6 +19,7 @@ package org.apache.pinot.common.restlet.resources; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; @@ -29,12 +30,13 @@ * It has details of reload flag which returns true if reload is needed on table and additional details of the * respective servers. */ +@JsonIgnoreProperties(ignoreUnknown = true) public class TableSegmentsReloadCheckResponse { boolean _needReload; - Map _serverToSegmentsReloadList; + Map _serverToSegmentsCheckReloadList; - public Map getServerToSegmentsReloadList() { - return _serverToSegmentsReloadList; + public Map getServerToSegmentsCheckReloadList() { + return _serverToSegmentsCheckReloadList; } public boolean isNeedReload() { @@ -43,8 +45,8 @@ public boolean isNeedReload() { @JsonCreator public TableSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, - @JsonProperty("serverToSegmentsReloadList") Map serverToSegmentsReloadList) { + @JsonProperty("serverToSegmentsCheckReloadList") Map serverToSegmentsCheckReloadList) { _needReload = needReload; - _serverToSegmentsReloadList = serverToSegmentsReloadList; + _serverToSegmentsCheckReloadList = serverToSegmentsCheckReloadList; } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java index d005ca26d3ab..440f8fd1fd15 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -18,15 +18,18 @@ */ package org.apache.pinot.common.utils; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; import java.util.HashMap; -import org.apache.pinot.common.restlet.resources.ResourceUtils; -import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import java.util.Map; +import org.apache.pinot.common.restlet.resources.ServerSegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.TableSegmentsReloadCheckResponse; import org.apache.pinot.spi.utils.JsonUtils; import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -38,39 +41,53 @@ public class SerializerResponseTest { @Test - public void testSerialization() { + public void testSerialization() + throws IOException { // Given boolean needReload = true; String instanceId = "instance123"; - SegmentsReloadCheckResponse response = new SegmentsReloadCheckResponse(needReload, instanceId); - TableSegmentsReloadCheckResponse tableResponse = new TableSegmentsReloadCheckResponse(needReload, new HashMap<>()); - String responseString = ResourceUtils.convertToJsonString(response); - String tableResponseString = ResourceUtils.convertToJsonString(tableResponse); + ServerSegmentsReloadCheckResponse response = new ServerSegmentsReloadCheckResponse(needReload, instanceId); + Map serversResponse = new HashMap<>(); + serversResponse.put(instanceId, response); + TableSegmentsReloadCheckResponse tableResponse = new TableSegmentsReloadCheckResponse(needReload, serversResponse); + String responseString = JsonUtils.objectToPrettyString(response); + String tableResponseString = JsonUtils.objectToPrettyString(tableResponse); assertNotNull(responseString); assertNotNull(tableResponseString); - assertEquals("{\n" + " \"needReload\" : true,\n" + " \"serverToSegmentsReloadList\" : { }\n" + "}", + JsonNode tableResponseJsonNode = JsonUtils.stringToJsonNode(tableResponseString); + assertTrue(tableResponseJsonNode.get("needReload").asBoolean()); + + JsonNode serversList = + tableResponseJsonNode.get("serverToSegmentsCheckReloadList"); + JsonNode serverResp = serversList.get("instance123"); + assertEquals(serverResp.get("instanceId").asText(), "instance123"); + assertTrue(serverResp.get("needReload").asBoolean()); + + assertEquals("{\n" + " \"needReload\" : true,\n" + " \"serverToSegmentsCheckReloadList\" : {\n" + + " \"instance123\" : {\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + + " }\n" + " }\n" + "}", tableResponseString); assertEquals("{\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + "}", responseString); + } @Test public void testDeserialization() throws Exception { - // Given - boolean needReload = true; - String instanceId = "instance123"; - SegmentsReloadCheckResponse response = new SegmentsReloadCheckResponse(needReload, instanceId); - TableSegmentsReloadCheckResponse tableResponse = new TableSegmentsReloadCheckResponse(needReload, new HashMap<>()); - String responseString = ResourceUtils.convertToJsonString(response); - JsonNode jsonNode = JsonUtils.stringToJsonNode(responseString); - String jsonResponse = JsonUtils.objectToPrettyString(tableResponse); - JsonNode jsonNodeTableResponse = JsonUtils.stringToJsonNode(jsonResponse); + String jsonResponse = "{\n" + " \"needReload\": false,\n" + " \"serverToSegmentsCheckReloadList\": {\n" + + " \"Server_10.0.0.215_7050\": {\n" + " \"needReload\": false,\n" + + " \"instanceId\": \"Server_10.0.0.215_7050\"\n" + " }\n" + " }\n" + "}"; + JsonNode jsonNode = JsonUtils.stringToJsonNode(jsonResponse); + TableSegmentsReloadCheckResponse tableReloadResponse = JsonUtils.stringToObject( + jsonResponse, + new TypeReference() {} + ); // Then assertNotNull(jsonNode); - assertNotNull(jsonNodeTableResponse); - assertTrue(jsonNodeTableResponse.get("needReload").asBoolean()); - assertTrue(jsonNode.get("needReload").asBoolean()); - assertEquals("instance123", jsonNode.get("instanceId").asText()); + assertFalse(tableReloadResponse.isNeedReload()); + assertNotNull(tableReloadResponse.getServerToSegmentsCheckReloadList()); + Map serverSegmentReloadResp = tableReloadResponse.getServerToSegmentsCheckReloadList(); + assertEquals(serverSegmentReloadResp.get("Server_10.0.0.215_7050").getNeedReload(), false); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java index 082a63729454..b5f560f63962 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java @@ -69,7 +69,7 @@ import org.apache.pinot.common.metadata.ZKMetadataProvider; import org.apache.pinot.common.metadata.controllerjob.ControllerJobType; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; -import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import org.apache.pinot.common.restlet.resources.ServerSegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.TableSegmentsReloadCheckResponse; import org.apache.pinot.common.utils.DatabaseUtils; import org.apache.pinot.common.utils.LLCSegmentName; @@ -836,7 +836,7 @@ public String getTableReloadMetadata( @QueryParam("verbose") @DefaultValue("false") boolean verbose, @Context HttpHeaders headers) { String tableName = TableNameBuilder.extractRawTableName(tableNameWithType); TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); - LOGGER.info("Received a request to fetch metadata for all segments for table {}", tableName); + LOGGER.info("Received a request to check reload for all servers hosting segments for table {}", tableName); try { TableMetadataReader tableMetadataReader = new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager); @@ -845,12 +845,12 @@ public String getTableReloadMetadata( _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); boolean needReload = needReloadMetadata.values().stream().anyMatch(value -> value.get("needReload").booleanValue()); - Map serverResponses = new HashMap<>(); + Map serverResponses = new HashMap<>(); TableSegmentsReloadCheckResponse tableNeedReloadResponse; if (verbose) { for (Map.Entry entry : needReloadMetadata.entrySet()) { serverResponses.put(entry.getKey(), - new SegmentsReloadCheckResponse(entry.getValue().get("needReload").booleanValue(), + new ServerSegmentsReloadCheckResponse(entry.getValue().get("needReload").booleanValue(), entry.getValue().get("instanceId").asText())); } tableNeedReloadResponse = new TableSegmentsReloadCheckResponse(needReload, serverResponses); diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java index 633c09eb74d3..ce85ec3f3123 100644 --- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java +++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java @@ -64,7 +64,7 @@ import org.apache.pinot.common.response.server.TableIndexMetadataResponse; import org.apache.pinot.common.restlet.resources.ResourceUtils; import org.apache.pinot.common.restlet.resources.SegmentConsumerInfo; -import org.apache.pinot.common.restlet.resources.SegmentsReloadCheckResponse; +import org.apache.pinot.common.restlet.resources.ServerSegmentsReloadCheckResponse; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.TableSegmentValidationInfo; import org.apache.pinot.common.restlet.resources.TableSegments; @@ -977,6 +977,6 @@ public String checkSegmentsReload( throw new WebApplicationException(e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR); } return ResourceUtils.convertToJsonString( - new SegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); + new ServerSegmentsReloadCheckResponse(needReload, tableDataManager.getInstanceId())); } } From 54916624669d8f8cea4822b7eec76c3ce6e3daa7 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Thu, 22 Aug 2024 10:45:08 -0700 Subject: [PATCH 31/43] checkstyle fixes --- .../common/utils/SerializerResponseTest.java | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java index 440f8fd1fd15..1e1b8f2653be 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java @@ -58,18 +58,15 @@ public void testSerialization() JsonNode tableResponseJsonNode = JsonUtils.stringToJsonNode(tableResponseString); assertTrue(tableResponseJsonNode.get("needReload").asBoolean()); - JsonNode serversList = - tableResponseJsonNode.get("serverToSegmentsCheckReloadList"); + JsonNode serversList = tableResponseJsonNode.get("serverToSegmentsCheckReloadList"); JsonNode serverResp = serversList.get("instance123"); assertEquals(serverResp.get("instanceId").asText(), "instance123"); assertTrue(serverResp.get("needReload").asBoolean()); assertEquals("{\n" + " \"needReload\" : true,\n" + " \"serverToSegmentsCheckReloadList\" : {\n" - + " \"instance123\" : {\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" - + " }\n" + " }\n" + "}", - tableResponseString); + + " \"instance123\" : {\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + + " }\n" + " }\n" + "}", tableResponseString); assertEquals("{\n" + " \"needReload\" : true,\n" + " \"instanceId\" : \"instance123\"\n" + "}", responseString); - } @Test @@ -79,15 +76,15 @@ public void testDeserialization() + " \"Server_10.0.0.215_7050\": {\n" + " \"needReload\": false,\n" + " \"instanceId\": \"Server_10.0.0.215_7050\"\n" + " }\n" + " }\n" + "}"; JsonNode jsonNode = JsonUtils.stringToJsonNode(jsonResponse); - TableSegmentsReloadCheckResponse tableReloadResponse = JsonUtils.stringToObject( - jsonResponse, - new TypeReference() {} - ); + TableSegmentsReloadCheckResponse tableReloadResponse = + JsonUtils.stringToObject(jsonResponse, new TypeReference() { + }); // Then assertNotNull(jsonNode); assertFalse(tableReloadResponse.isNeedReload()); assertNotNull(tableReloadResponse.getServerToSegmentsCheckReloadList()); - Map serverSegmentReloadResp = tableReloadResponse.getServerToSegmentsCheckReloadList(); + Map serverSegmentReloadResp = + tableReloadResponse.getServerToSegmentsCheckReloadList(); assertEquals(serverSegmentReloadResp.get("Server_10.0.0.215_7050").getNeedReload(), false); } } From 8173c1ccf52882c90902b7f325fbd3394ea58b24 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Thu, 22 Aug 2024 10:48:43 -0700 Subject: [PATCH 32/43] checkstyle fixes --- .../restlet/resources/TableSegmentsReloadCheckResponse.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index c67f4e655a8f..4b7daa35447a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -45,7 +45,8 @@ public boolean isNeedReload() { @JsonCreator public TableSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, - @JsonProperty("serverToSegmentsCheckReloadList") Map serverToSegmentsCheckReloadList) { + @JsonProperty("serverToSegmentsCheckReloadList") + Map serverToSegmentsCheckReloadList) { _needReload = needReload; _serverToSegmentsCheckReloadList = serverToSegmentsCheckReloadList; } From 3649c78373b337d566039554332b0e45c4e1b8d1 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Thu, 22 Aug 2024 14:01:15 -0700 Subject: [PATCH 33/43] remove unused method --- .../core/data/manager/BaseTableDataManager.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index ab4745a60d79..349c67d61565 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1014,23 +1014,6 @@ public boolean tryLoadExistingSegment(SegmentZKMetadata zkMetadata, IndexLoading } } - private boolean needReloadSegment(String segmentName, String crc, IndexLoadingConfig indexLoadingConfig) - throws Exception { - SegmentDirectory segmentDirectory = tryInitSegmentDirectory(segmentName, crc, indexLoadingConfig); - try { - Schema schema = indexLoadingConfig.getSchema(); - //if re processing or reload is needed on a segment then return true - if (segmentDirectory != null) { - return ImmutableSegmentLoader.needPreprocess(segmentDirectory, indexLoadingConfig, schema); - } - return false; - } catch (Exception e) { - _logger.error("Failed to check if reload is needed for a segment {}", segmentName, e); - closeSegmentDirectoryQuietly(segmentDirectory); - throw new Exception(String.format("Failed to perform reload check on the segment %s", segmentName)); - } - } - @Nullable private SegmentDirectory tryInitSegmentDirectory(String segmentName, String segmentCrc, IndexLoadingConfig indexLoadingConfig) { From a14af52d85ec3970a0d92dc19cc5c060fc0ba151 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Thu, 22 Aug 2024 14:19:16 -0700 Subject: [PATCH 34/43] add json properties --- .../restlet/resources/ServerSegmentsReloadCheckResponse.java | 4 +++- .../restlet/resources/TableSegmentsReloadCheckResponse.java | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java index 169dc0d8f67a..b654fbcf5f36 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java @@ -30,14 +30,16 @@ */ @JsonIgnoreProperties(ignoreUnknown = true) public class ServerSegmentsReloadCheckResponse { + @JsonProperty("needReload") private final boolean _needReload; + + @JsonProperty("instanceId") private final String _instanceId; public String getInstanceId() { return _instanceId; } - @JsonProperty("needReload") public boolean getNeedReload() { return _needReload; } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index 4b7daa35447a..c8be9da61376 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -32,7 +32,9 @@ */ @JsonIgnoreProperties(ignoreUnknown = true) public class TableSegmentsReloadCheckResponse { + @JsonProperty("needReload") boolean _needReload; + @JsonProperty("serverToSegmentsCheckReloadList") Map _serverToSegmentsCheckReloadList; public Map getServerToSegmentsCheckReloadList() { From ad11da4fade56c4f5c6d7e6e7040037765740e6d Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 26 Aug 2024 15:19:17 -0700 Subject: [PATCH 35/43] Review Comments Fixes --- .../ServerSegmentsReloadCheckResponse.java | 8 ++-- .../TableSegmentsReloadCheckResponse.java | 8 ++-- ...a => SegmentsReloadCheckResponseTest.java} | 4 +- .../PinotSegmentRestletResource.java | 9 ++-- .../helix/ControllerRequestClient.java | 7 ++- .../controller/util/TableMetadataReader.java | 10 ++--- .../controller/helix/ControllerTest.java | 4 +- .../data/manager/BaseTableDataManager.java | 3 +- .../tests/BaseClusterIntegrationTestSet.java | 45 ++++++++++++++----- .../immutable/ImmutableSegmentImpl.java | 9 +--- .../builder/ControllerRequestURLBuilder.java | 4 +- 11 files changed, 61 insertions(+), 50 deletions(-) rename pinot-common/src/test/java/org/apache/pinot/common/utils/{SerializerResponseTest.java => SegmentsReloadCheckResponseTest.java} (98%) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java index b654fbcf5f36..2469bda4041f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/ServerSegmentsReloadCheckResponse.java @@ -36,12 +36,12 @@ public class ServerSegmentsReloadCheckResponse { @JsonProperty("instanceId") private final String _instanceId; - public String getInstanceId() { - return _instanceId; + public boolean isNeedReload() { + return _needReload; } - public boolean getNeedReload() { - return _needReload; + public String getInstanceId() { + return _instanceId; } @JsonCreator diff --git a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java index c8be9da61376..bd201870a41d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/restlet/resources/TableSegmentsReloadCheckResponse.java @@ -37,14 +37,14 @@ public class TableSegmentsReloadCheckResponse { @JsonProperty("serverToSegmentsCheckReloadList") Map _serverToSegmentsCheckReloadList; - public Map getServerToSegmentsCheckReloadList() { - return _serverToSegmentsCheckReloadList; - } - public boolean isNeedReload() { return _needReload; } + public Map getServerToSegmentsCheckReloadList() { + return _serverToSegmentsCheckReloadList; + } + @JsonCreator public TableSegmentsReloadCheckResponse(@JsonProperty("needReload") boolean needReload, @JsonProperty("serverToSegmentsCheckReloadList") diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/SegmentsReloadCheckResponseTest.java similarity index 98% rename from pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java rename to pinot-common/src/test/java/org/apache/pinot/common/utils/SegmentsReloadCheckResponseTest.java index 1e1b8f2653be..f63a607a1221 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/SerializerResponseTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/SegmentsReloadCheckResponseTest.java @@ -38,7 +38,7 @@ * Tests some of the serializer and deserialization responses from SegmentsReloadCheckResponse class * needReload will have to be carefully evaluated */ -public class SerializerResponseTest { +public class SegmentsReloadCheckResponseTest { @Test public void testSerialization() @@ -85,6 +85,6 @@ public void testDeserialization() assertNotNull(tableReloadResponse.getServerToSegmentsCheckReloadList()); Map serverSegmentReloadResp = tableReloadResponse.getServerToSegmentsCheckReloadList(); - assertEquals(serverSegmentReloadResp.get("Server_10.0.0.215_7050").getNeedReload(), false); + assertEquals(serverSegmentReloadResp.get("Server_10.0.0.215_7050").isNeedReload(), false); } } diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java index b5f560f63962..74c09d8da709 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java @@ -829,19 +829,18 @@ public String getServerMetadata( @Authorize(targetType = TargetType.TABLE, paramName = "tableNameWithType", action = Actions.Table.GET_METADATA) @Produces(MediaType.APPLICATION_JSON) @ApiOperation(value = "Gets the metadata of reload segments check from servers hosting the table", notes = - "Returns true if " + "reload is needed on the table from any one of the servers") + "Returns true if reload is needed on the table from any one of the servers") public String getTableReloadMetadata( @ApiParam(value = "Table name with type", required = true, example = "myTable_REALTIME") @PathParam("tableNameWithType") String tableNameWithType, @QueryParam("verbose") @DefaultValue("false") boolean verbose, @Context HttpHeaders headers) { - String tableName = TableNameBuilder.extractRawTableName(tableNameWithType); - TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); - LOGGER.info("Received a request to check reload for all servers hosting segments for table {}", tableName); + tableNameWithType = DatabaseUtils.translateTableName(tableNameWithType, headers); + LOGGER.info("Received a request to check reload for all servers hosting segments for table {}", tableNameWithType); try { TableMetadataReader tableMetadataReader = new TableMetadataReader(_executor, _connectionManager, _pinotHelixResourceManager); Map needReloadMetadata = - tableMetadataReader.getServerCheckSegmentsReloadMetadata(tableName, tableType, + tableMetadataReader.getServerCheckSegmentsReloadMetadata(tableNameWithType, _controllerConf.getServerAdminRequestTimeoutSeconds() * 1000); boolean needReload = needReloadMetadata.values().stream().anyMatch(value -> value.get("needReload").booleanValue()); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java index d2b58393d5e6..32a53b5e8b1b 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java @@ -201,12 +201,11 @@ public String reloadTable(String tableName, TableType tableType, boolean forceDo } } - public String needReloadTable(String tableNameWithType) + public String checkIfReloadIsNeeded(String tableNameWithType, Boolean verbose) throws IOException { try { - SimpleHttpResponse simpleHttpResponse = HttpClient.wrapAndThrowHttpException( - _httpClient.sendGetRequest(new URI(_controllerRequestURLBuilder.forTableNeedReload(tableNameWithType)), - _headers, null)); + SimpleHttpResponse simpleHttpResponse = HttpClient.wrapAndThrowHttpException(_httpClient.sendGetRequest( + new URI(_controllerRequestURLBuilder.forTableNeedReload(tableNameWithType, verbose)), _headers, null)); return simpleHttpResponse.getResponse(); } catch (HttpErrorStatusException | URISyntaxException e) { throw new IOException(e); diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index bddb438eb7de..ac06d958a2fe 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -33,10 +33,10 @@ import org.apache.pinot.common.exception.InvalidConfigException; import org.apache.pinot.common.restlet.resources.TableMetadataInfo; import org.apache.pinot.common.restlet.resources.ValidDocIdsMetadataInfo; -import org.apache.pinot.controller.api.resources.ResourceUtils; import org.apache.pinot.controller.helix.core.PinotHelixResourceManager; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.JsonUtils; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,12 +61,10 @@ public TableMetadataReader(Executor executor, HttpClientConnectionManager connec _pinotHelixResourceManager = helixResourceManager; } - public Map getServerCheckSegmentsReloadMetadata(String tableName, TableType tableType, - int timeoutMs) + public Map getServerCheckSegmentsReloadMetadata(String tableNameWithType, int timeoutMs) throws InvalidConfigException, IOException { - String tableNameWithType = - ResourceUtils.getExistingTableNamesWithType(_pinotHelixResourceManager, tableName, tableType, LOGGER).get(0); - List serverInstances = _pinotHelixResourceManager.getServerInstancesForTable(tableName, tableType); + TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableNameWithType); + List serverInstances = _pinotHelixResourceManager.getServerInstancesForTable(tableNameWithType, tableType); Set serverInstanceSet = new HashSet<>(serverInstances); BiMap endpoints = _pinotHelixResourceManager.getDataInstanceAdminEndpoints(serverInstanceSet); ServerSegmentMetadataReader serverSegmentMetadataReader = diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java index 97fd3baaa372..263913df7fb1 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java @@ -721,9 +721,9 @@ public String reloadOfflineTable(String tableName, boolean forceDownload) return getControllerRequestClient().reloadTable(tableName, TableType.OFFLINE, forceDownload); } - public String needReloadOfflineTable(String tableNameWithType) + public String needReloadTable(String tableNameWithType, Boolean verbose) throws IOException { - return getControllerRequestClient().needReloadTable(tableNameWithType); + return getControllerRequestClient().checkIfReloadIsNeeded(tableNameWithType, verbose); } public void reloadOfflineSegment(String tableName, String segmentName, boolean forceDownload) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java index 349c67d61565..56d2cb35d61f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java @@ -1033,11 +1033,10 @@ public boolean needReloadSegments() boolean needReload = false; try { for (SegmentDataManager segmentDataManager : segmentDataManagers) { - String crcData = segmentDataManager.getSegment().getSegmentMetadata().getCrc(); IndexSegment segment = segmentDataManager.getSegment(); if (segment instanceof ImmutableSegmentImpl) { ImmutableSegmentImpl immutableSegment = (ImmutableSegmentImpl) segment; - if (immutableSegment.needReloadSegment(segment.getSegmentName(), crcData, indexLoadingConfig)) { + if (immutableSegment.isReloadNeeded(indexLoadingConfig)) { needReload = true; break; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 1a22509697c0..9ab1434159e4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -738,18 +738,28 @@ public void testReload(boolean includeOfflineTable) if (includeOfflineTable) { //test controller api which gives responses if reload is needed on any of the server segments when default // columns are added - String needBeforeReloadResponse = needReloadOfflineTable(tableNameWithTypeOffline); - JsonNode jsonNeedReloadResponse = JsonUtils.stringToJsonNode(needBeforeReloadResponse); + String needBeforeReloadResponseWithNoVerbose = needReloadTable(tableNameWithTypeOffline, false); + String needBeforeReloadResponseWithVerbose = needReloadTable(tableNameWithTypeOffline, true); + JsonNode jsonNeedReloadResponseWithNoVerbose = JsonUtils.stringToJsonNode(needBeforeReloadResponseWithNoVerbose); + JsonNode jsonNeedReloadResponseWithVerbose = JsonUtils.stringToJsonNode(needBeforeReloadResponseWithVerbose); //test to check if reload is needed i.e true - assertEquals(jsonNeedReloadResponse.get("needReload").asBoolean(), true); + assertTrue(jsonNeedReloadResponseWithNoVerbose.get("needReload").asBoolean()); + assertTrue(jsonNeedReloadResponseWithVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); reloadOfflineTable(rawTableName); } //test controller api which gives responses if reload is needed on any of the server segments when default // columns are added - String needBeforeReloadResponseRealtime = needReloadOfflineTable(tableNameWithTypeRealtime); - JsonNode jsonNeedReloadResponseRealTime = JsonUtils.stringToJsonNode(needBeforeReloadResponseRealtime); + String needBeforeReloadResponseRealtimeWithNoVerbose = needReloadTable(tableNameWithTypeRealtime, false); + String needBeforeReloadResponseRealtimeWithVerbose = needReloadTable(tableNameWithTypeRealtime, true); + JsonNode jsonNeedReloadResponseRealTimeWithNoVerbose = + JsonUtils.stringToJsonNode(needBeforeReloadResponseRealtimeWithNoVerbose); + JsonNode jsonNeedReloadResponseRealTimeWithVerbose = + JsonUtils.stringToJsonNode(needBeforeReloadResponseRealtimeWithVerbose); //test to check if reload is needed i.e true - assertTrue(jsonNeedReloadResponseRealTime.get("needReload").asBoolean()); + assertTrue(jsonNeedReloadResponseRealTimeWithNoVerbose.get("needReload").asBoolean()); + assertTrue(jsonNeedReloadResponseRealTimeWithVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealTimeWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); reloadRealtimeTable(rawTableName); // Wait for all segments to finish reloading, and test querying the new columns @@ -791,15 +801,26 @@ public void testReload(boolean includeOfflineTable) assertEquals(queryResponse.get("exceptions").size(), 0); assertEquals(queryResponse.get("resultTable").get("rows").get(0).get(0).asLong(), countStarResult); if (includeOfflineTable) { - String needAfterReloadResponse = needReloadOfflineTable(tableNameWithTypeOffline); - JsonNode jsonNeedReloadResponseAfter = JsonUtils.stringToJsonNode(needAfterReloadResponse); + String needAfterReloadResponseWithNoVerbose = needReloadTable(tableNameWithTypeOffline, false); + String needAfterReloadResponseWithVerbose = needReloadTable(tableNameWithTypeOffline, true); + JsonNode jsonNeedReloadResponseAfterWithNoVerbose = + JsonUtils.stringToJsonNode(needAfterReloadResponseWithNoVerbose); + JsonNode jsonNeedReloadResponseAfterWithVerbose = JsonUtils.stringToJsonNode(needAfterReloadResponseWithVerbose); //test to check if reload on offline table is needed i.e false after reload is finished - assertFalse(jsonNeedReloadResponseAfter.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseAfterWithNoVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseAfterWithVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealTimeWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); } - String needAfterReloadResponseRealtime = needReloadOfflineTable(tableNameWithTypeRealtime); - JsonNode jsonNeedReloadResponseRealtimeAfter = JsonUtils.stringToJsonNode(needAfterReloadResponseRealtime); + String needAfterReloadResponseRealtimeWithNoVerbose = needReloadTable(tableNameWithTypeRealtime, false); + String needAfterReloadResponseRealTimeWithVerbose = needReloadTable(tableNameWithTypeRealtime, true); + JsonNode jsonNeedReloadResponseRealtimeAfterWithNoVerbose = + JsonUtils.stringToJsonNode(needAfterReloadResponseRealtimeWithNoVerbose); + JsonNode jsonNeedReloadResponseRealtimeAfterWithVerbose = + JsonUtils.stringToJsonNode(needAfterReloadResponseRealTimeWithVerbose); //test to check if reload on real time table is needed i.e false after reload is finished - assertFalse(jsonNeedReloadResponseRealtimeAfter.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealtimeAfterWithNoVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealtimeAfterWithNoVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealtimeAfterWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); } private DimensionFieldSpec constructNewDimension(FieldSpec.DataType dataType, boolean singleValue) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java index 3f3b1fe65559..9684e0ec0e0a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java @@ -55,7 +55,6 @@ import org.apache.pinot.segment.spi.index.startree.StarTreeV2; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; -import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; import org.roaringbitmap.buffer.MutableRoaringBitmap; @@ -175,14 +174,10 @@ private File getValidDocIdsSnapshotFile() { V1Constants.VALID_DOC_IDS_SNAPSHOT_FILE_NAME); } - public boolean needReloadSegment(String segmentName, String crc, IndexLoadingConfig indexLoadingConfig) + public boolean isReloadNeeded(IndexLoadingConfig indexLoadingConfig) throws Exception { - Schema schema = indexLoadingConfig.getSchema(); //if re processing or reload is needed on a segment then return true - if (_segmentDirectory != null) { - return ImmutableSegmentLoader.needPreprocess(_segmentDirectory, indexLoadingConfig, schema); - } - return false; + return ImmutableSegmentLoader.needPreprocess(_segmentDirectory, indexLoadingConfig, indexLoadingConfig.getSchema()); } @Override diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java index 47ca6606ed35..1efbb8b30934 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java @@ -239,8 +239,8 @@ public String forTableReload(String tableName, TableType tableType, boolean forc return StringUtil.join("/", _baseUrl, "segments", tableName, query); } - public String forTableNeedReload(String tableNameWithType) { - String query = String.format("needReload?verbose=%s", "false"); + public String forTableNeedReload(String tableNameWithType, Boolean verbose) { + String query = String.format("needReload?verbose=%s", verbose); return StringUtil.join("/", _baseUrl, "segments", tableNameWithType, query); } From 468b36ddc0bc89967d5b9f95aa12dde445f72090 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 26 Aug 2024 15:33:15 -0700 Subject: [PATCH 36/43] Naming fixes --- .../helix/ControllerRequestClient.java | 2 +- .../pinot/controller/helix/ControllerTest.java | 4 ++-- .../tests/BaseClusterIntegrationTestSet.java | 16 ++++++++-------- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java index 32a53b5e8b1b..df556a99356e 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java @@ -201,7 +201,7 @@ public String reloadTable(String tableName, TableType tableType, boolean forceDo } } - public String checkIfReloadIsNeeded(String tableNameWithType, Boolean verbose) + public String checkIfTableReloadIsNeeded(String tableNameWithType, Boolean verbose) throws IOException { try { SimpleHttpResponse simpleHttpResponse = HttpClient.wrapAndThrowHttpException(_httpClient.sendGetRequest( diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java index 263913df7fb1..2d5fd1d77e5a 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java @@ -721,9 +721,9 @@ public String reloadOfflineTable(String tableName, boolean forceDownload) return getControllerRequestClient().reloadTable(tableName, TableType.OFFLINE, forceDownload); } - public String needReloadTable(String tableNameWithType, Boolean verbose) + public String checkIfReloadIsNeeded(String tableNameWithType, Boolean verbose) throws IOException { - return getControllerRequestClient().checkIfReloadIsNeeded(tableNameWithType, verbose); + return getControllerRequestClient().checkIfTableReloadIsNeeded(tableNameWithType, verbose); } public void reloadOfflineSegment(String tableName, String segmentName, boolean forceDownload) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 9ab1434159e4..4927e18b3b52 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -738,8 +738,8 @@ public void testReload(boolean includeOfflineTable) if (includeOfflineTable) { //test controller api which gives responses if reload is needed on any of the server segments when default // columns are added - String needBeforeReloadResponseWithNoVerbose = needReloadTable(tableNameWithTypeOffline, false); - String needBeforeReloadResponseWithVerbose = needReloadTable(tableNameWithTypeOffline, true); + String needBeforeReloadResponseWithNoVerbose = checkIfReloadIsNeeded(tableNameWithTypeOffline, false); + String needBeforeReloadResponseWithVerbose = checkIfReloadIsNeeded(tableNameWithTypeOffline, true); JsonNode jsonNeedReloadResponseWithNoVerbose = JsonUtils.stringToJsonNode(needBeforeReloadResponseWithNoVerbose); JsonNode jsonNeedReloadResponseWithVerbose = JsonUtils.stringToJsonNode(needBeforeReloadResponseWithVerbose); //test to check if reload is needed i.e true @@ -750,8 +750,8 @@ public void testReload(boolean includeOfflineTable) } //test controller api which gives responses if reload is needed on any of the server segments when default // columns are added - String needBeforeReloadResponseRealtimeWithNoVerbose = needReloadTable(tableNameWithTypeRealtime, false); - String needBeforeReloadResponseRealtimeWithVerbose = needReloadTable(tableNameWithTypeRealtime, true); + String needBeforeReloadResponseRealtimeWithNoVerbose = checkIfReloadIsNeeded(tableNameWithTypeRealtime, false); + String needBeforeReloadResponseRealtimeWithVerbose = checkIfReloadIsNeeded(tableNameWithTypeRealtime, true); JsonNode jsonNeedReloadResponseRealTimeWithNoVerbose = JsonUtils.stringToJsonNode(needBeforeReloadResponseRealtimeWithNoVerbose); JsonNode jsonNeedReloadResponseRealTimeWithVerbose = @@ -801,8 +801,8 @@ public void testReload(boolean includeOfflineTable) assertEquals(queryResponse.get("exceptions").size(), 0); assertEquals(queryResponse.get("resultTable").get("rows").get(0).get(0).asLong(), countStarResult); if (includeOfflineTable) { - String needAfterReloadResponseWithNoVerbose = needReloadTable(tableNameWithTypeOffline, false); - String needAfterReloadResponseWithVerbose = needReloadTable(tableNameWithTypeOffline, true); + String needAfterReloadResponseWithNoVerbose = checkIfReloadIsNeeded(tableNameWithTypeOffline, false); + String needAfterReloadResponseWithVerbose = checkIfReloadIsNeeded(tableNameWithTypeOffline, true); JsonNode jsonNeedReloadResponseAfterWithNoVerbose = JsonUtils.stringToJsonNode(needAfterReloadResponseWithNoVerbose); JsonNode jsonNeedReloadResponseAfterWithVerbose = JsonUtils.stringToJsonNode(needAfterReloadResponseWithVerbose); @@ -811,8 +811,8 @@ public void testReload(boolean includeOfflineTable) assertFalse(jsonNeedReloadResponseAfterWithVerbose.get("needReload").asBoolean()); assertFalse(jsonNeedReloadResponseRealTimeWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); } - String needAfterReloadResponseRealtimeWithNoVerbose = needReloadTable(tableNameWithTypeRealtime, false); - String needAfterReloadResponseRealTimeWithVerbose = needReloadTable(tableNameWithTypeRealtime, true); + String needAfterReloadResponseRealtimeWithNoVerbose = checkIfReloadIsNeeded(tableNameWithTypeRealtime, false); + String needAfterReloadResponseRealTimeWithVerbose = checkIfReloadIsNeeded(tableNameWithTypeRealtime, true); JsonNode jsonNeedReloadResponseRealtimeAfterWithNoVerbose = JsonUtils.stringToJsonNode(needAfterReloadResponseRealtimeWithNoVerbose); JsonNode jsonNeedReloadResponseRealtimeAfterWithVerbose = From d1b330a89a035ad37c6b820cf471f5bf3ea7bf46 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Mon, 26 Aug 2024 16:07:50 -0700 Subject: [PATCH 37/43] Naming changes --- .../apache/pinot/controller/helix/ControllerRequestClient.java | 2 +- .../java/org/apache/pinot/controller/helix/ControllerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java index df556a99356e..32a53b5e8b1b 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/ControllerRequestClient.java @@ -201,7 +201,7 @@ public String reloadTable(String tableName, TableType tableType, boolean forceDo } } - public String checkIfTableReloadIsNeeded(String tableNameWithType, Boolean verbose) + public String checkIfReloadIsNeeded(String tableNameWithType, Boolean verbose) throws IOException { try { SimpleHttpResponse simpleHttpResponse = HttpClient.wrapAndThrowHttpException(_httpClient.sendGetRequest( diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java index 2d5fd1d77e5a..98ddbac73ab1 100644 --- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java +++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java @@ -723,7 +723,7 @@ public String reloadOfflineTable(String tableName, boolean forceDownload) public String checkIfReloadIsNeeded(String tableNameWithType, Boolean verbose) throws IOException { - return getControllerRequestClient().checkIfTableReloadIsNeeded(tableNameWithType, verbose); + return getControllerRequestClient().checkIfReloadIsNeeded(tableNameWithType, verbose); } public void reloadOfflineSegment(String tableName, String segmentName, boolean forceDownload) From ff056b081498b5c278c377c907461e9eaee8e0ae Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi <45308220+deepthi912@users.noreply.github.com> Date: Tue, 27 Aug 2024 17:17:25 -0700 Subject: [PATCH 38/43] Update to Boolean variable Co-authored-by: Xiaotian (Jackie) Jiang <17555551+Jackie-Jiang@users.noreply.github.com> --- .../pinot/spi/utils/builder/ControllerRequestURLBuilder.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java index 1efbb8b30934..19bed50b68a7 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/builder/ControllerRequestURLBuilder.java @@ -239,7 +239,7 @@ public String forTableReload(String tableName, TableType tableType, boolean forc return StringUtil.join("/", _baseUrl, "segments", tableName, query); } - public String forTableNeedReload(String tableNameWithType, Boolean verbose) { + public String forTableNeedReload(String tableNameWithType, boolean verbose) { String query = String.format("needReload?verbose=%s", verbose); return StringUtil.join("/", _baseUrl, "segments", tableNameWithType, query); } From 33a2408cde009dd5d8eda159c649f6e67d994029 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 27 Aug 2024 17:20:29 -0700 Subject: [PATCH 39/43] Changes in names --- .../org/apache/pinot/controller/util/TableMetadataReader.java | 1 - .../local/indexsegment/immutable/ImmutableSegmentImpl.java | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index ac06d958a2fe..091607deed83 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -49,7 +49,6 @@ * the column indexes available. */ public class TableMetadataReader { - private static final Logger LOGGER = LoggerFactory.getLogger(TableMetadataReader.class); private final Executor _executor; private final HttpClientConnectionManager _connectionManager; private final PinotHelixResourceManager _pinotHelixResourceManager; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java index 9684e0ec0e0a..48651a17ac03 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java @@ -174,9 +174,9 @@ private File getValidDocIdsSnapshotFile() { V1Constants.VALID_DOC_IDS_SNAPSHOT_FILE_NAME); } + //if re processing or reload is needed on a segment then return true public boolean isReloadNeeded(IndexLoadingConfig indexLoadingConfig) throws Exception { - //if re processing or reload is needed on a segment then return true return ImmutableSegmentLoader.needPreprocess(_segmentDirectory, indexLoadingConfig, indexLoadingConfig.getSchema()); } From 5f555e7ecccb78e789034229a74e12a12667e73f Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 27 Aug 2024 17:31:25 -0700 Subject: [PATCH 40/43] checkstyle fixes --- .../org/apache/pinot/controller/util/TableMetadataReader.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java index 091607deed83..a7a53d421da9 100644 --- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java +++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/TableMetadataReader.java @@ -37,8 +37,6 @@ import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** From efa1766f0cba0621f001179b0818af903862fda4 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 27 Aug 2024 20:56:32 -0700 Subject: [PATCH 41/43] change the format of the comment --- .../local/indexsegment/immutable/ImmutableSegmentImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java index 48651a17ac03..14546d7ba626 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java @@ -174,7 +174,9 @@ private File getValidDocIdsSnapshotFile() { V1Constants.VALID_DOC_IDS_SNAPSHOT_FILE_NAME); } - //if re processing or reload is needed on a segment then return true + /** + * if re processing or reload is needed on a segment then return true + */ public boolean isReloadNeeded(IndexLoadingConfig indexLoadingConfig) throws Exception { return ImmutableSegmentLoader.needPreprocess(_segmentDirectory, indexLoadingConfig, indexLoadingConfig.getSchema()); From 00d222046b3c111e009f42ceb9359ca3a8e05753 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Tue, 27 Aug 2024 22:18:40 -0700 Subject: [PATCH 42/43] test case fix --- .../pinot/integration/tests/BaseClusterIntegrationTestSet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index 4927e18b3b52..c0fa8e80b0d4 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -819,7 +819,7 @@ public void testReload(boolean includeOfflineTable) JsonUtils.stringToJsonNode(needAfterReloadResponseRealTimeWithVerbose); //test to check if reload on real time table is needed i.e false after reload is finished assertFalse(jsonNeedReloadResponseRealtimeAfterWithNoVerbose.get("needReload").asBoolean()); - assertFalse(jsonNeedReloadResponseRealtimeAfterWithNoVerbose.get("needReload").asBoolean()); + assertFalse(jsonNeedReloadResponseRealtimeAfterWithVerbose.get("needReload").asBoolean()); assertFalse(jsonNeedReloadResponseRealtimeAfterWithVerbose.get("serverToSegmentsCheckReloadList").isEmpty()); } From aa3a722dfd74ebc4af6872297ddd07f8fe76f9e1 Mon Sep 17 00:00:00 2001 From: Chaitanya Deepthi Date: Wed, 28 Aug 2024 10:15:53 -0700 Subject: [PATCH 43/43] test case random failures --- .../pinot/integration/tests/BaseClusterIntegrationTestSet.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java index c0fa8e80b0d4..ca27ed4ef46a 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTestSet.java @@ -817,6 +817,7 @@ public void testReload(boolean includeOfflineTable) JsonUtils.stringToJsonNode(needAfterReloadResponseRealtimeWithNoVerbose); JsonNode jsonNeedReloadResponseRealtimeAfterWithVerbose = JsonUtils.stringToJsonNode(needAfterReloadResponseRealTimeWithVerbose); + //test to check if reload on real time table is needed i.e false after reload is finished assertFalse(jsonNeedReloadResponseRealtimeAfterWithNoVerbose.get("needReload").asBoolean()); assertFalse(jsonNeedReloadResponseRealtimeAfterWithVerbose.get("needReload").asBoolean());