From f57cf708697c1bf688671f5296dc836ff2e115da Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 14 Jun 2024 10:24:21 -0700 Subject: [PATCH 01/23] Initial support for bootstrap segments. - Adds a new API in the coordinator. - All processes that have storage locations configured (including tasks) talk to the coordinator if they can, and fetch bootstrap segments from it. - Then load the segments onto the segment cache as part of startup. - This addresses the segment bootstrapping logic required by processes before they can start serving queries or ingesting. This patch also lays the foundation to speed up upgrades. --- .../apache/druid/error/DruidException.java | 13 +- .../druid/client/JsonParserIterator.java | 10 ++ .../client/coordinator/CoordinatorClient.java | 7 + .../coordinator/CoordinatorClientImpl.java | 24 +++ .../rpc/indexing/OverlordClientImpl.java | 3 - .../loading/LocalDataSegmentPuller.java | 7 - .../coordination/LoadableDataSegment.java | 4 +- .../coordination/SegmentLoadDropHandler.java | 110 ++++++++++++-- .../server/coordinator/DruidCoordinator.java | 23 +++ .../server/coordinator/duty/RunRules.java | 14 +- .../loading/StrategicSegmentAssigner.java | 10 ++ .../druid/server/http/MetadataResource.java | 32 ++++ .../CoordinatorClientImplTest.java | 52 +++++++ .../coordinator/NoopCoordinatorClient.java | 7 + .../SegmentLoadDropHandlerCacheTest.java | 21 ++- .../SegmentLoadDropHandlerTest.java | 75 +++++++++- .../coordination/TestCoordinatorClient.java | 51 +++++++ .../coordination/ZkCoordinatorTest.java | 4 +- .../coordinator/DruidCoordinatorTest.java | 15 +- .../server/coordinator/duty/RunRulesTest.java | 140 +++++++++++++++++- .../server/http/MetadataResourceTest.java | 33 +++++ 21 files changed, 598 insertions(+), 57 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index c4ec6bcb115f..5e85e164a935 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -315,19 +315,19 @@ public enum Persona } /** - * Category of error. The simplest way to describe this is that it exists as a classification of errors that + * Category of error. The simplest way to describe this is that it exists as a classification of errors that * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException */ public enum Category { /** * Means that the exception is being created defensively, because we want to validate something but expect that - * it should never actually be hit. Using this category is good to provide an indication to future reviewers and + * it should never actually be hit. Using this category is good to provide an indication to future reviewers and * developers that the case being checked is not intended to actually be able to occur in the wild. */ DEFENSIVE(500), /** - * Means that the input provided was malformed in some way. Generally speaking, it is hoped that errors of this + * Means that the input provided was malformed in some way. Generally speaking, it is hoped that errors of this * category have messages written either targeting the USER or ADMIN personas as those are the general users * of the APIs who could generate invalid inputs. */ @@ -340,9 +340,8 @@ public enum Category * Means that an action that was attempted is forbidden */ FORBIDDEN(403), - /** - * Means that the requsted requested resource cannot be found. + * Means that the requested resource cannot be found. */ NOT_FOUND(404), /** @@ -357,6 +356,10 @@ public enum Category * Indicates a server-side failure of some sort at runtime */ RUNTIME_FAILURE(500), + /** + * Means that the requested resource could not be found due to a transient unavailability. + */ + UNAVAILABLE(503), /** * A timeout happened */ diff --git a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java index 00dea3dff0e0..7aa887743979 100644 --- a/server/src/main/java/org/apache/druid/client/JsonParserIterator.java +++ b/server/src/main/java/org/apache/druid/client/JsonParserIterator.java @@ -86,6 +86,16 @@ public JsonParserIterator( this.hasTimeout = timeoutAt > -1; } + /** + * Bypasses Jackson serialization to prevent materialization of results from the {@code future} in memory at once. + * A shortened version of {@link #JsonParserIterator(JavaType, Future, String, Query, String, ObjectMapper)} + * where the URL and host parameters, used solely for logging/errors, are not known. + */ + public JsonParserIterator(JavaType typeRef, Future future, ObjectMapper objectMapper) + { + this(typeRef, future, "", null, "", objectMapper); + } + @Override public boolean hasNext() { diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index aeccee8043ba..3113d7ea4f5a 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; @@ -58,6 +59,12 @@ public interface CoordinatorClient */ ListenableFuture> fetchDataSourceInformation(Set datasources); + /** + * Fetch bootstrap segments from the coordiantor. The results must be streamed back to the caller as the + * result set can be large. + */ + ListenableFuture> fetchBootstrapSegments(); + /** * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy */ diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 93c22bbdbff8..414aee485084 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -21,17 +21,23 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; +import org.apache.druid.client.JsonParserIterator; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; +import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.server.coordination.LoadableDataSegment; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; @@ -42,6 +48,7 @@ public class CoordinatorClientImpl implements CoordinatorClient { + private static final Logger log = new Logger(CoordinatorClientImpl.class); private final ServiceClient client; private final ObjectMapper jsonMapper; @@ -156,6 +163,23 @@ public ListenableFuture> fetchDataSourceInformation( ); } + @Override + public ListenableFuture> fetchBootstrapSegments() + { + final String path = "/druid/coordinator/v1/metadata/bootstrapSegments"; + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, path), + new InputStreamResponseHandler() + ), + in -> new JsonParserIterator<>( + jsonMapper.getTypeFactory().constructType(LoadableDataSegment.class), + Futures.immediateFuture(in), + jsonMapper + ) + ); + } + @Override public CoordinatorClientImpl withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index 35276aa723dc..3e3d86ca5f25 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -323,9 +323,6 @@ private JsonParserIterator asJsonParserIterator(final InputStream in, fin return new JsonParserIterator<>( jsonMapper.getTypeFactory().constructType(clazz), Futures.immediateFuture(in), - "", // We don't know URL at this point, but it's OK to use empty; it's used for logs/errors - null, - "", // We don't know host at this point, but it's OK to use empty; it's used for logs/errors jsonMapper ); } diff --git a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPuller.java b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPuller.java index 0f27dac9e1bb..27807a5c3c87 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPuller.java +++ b/server/src/main/java/org/apache/druid/segment/loading/LocalDataSegmentPuller.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.loading; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; import com.google.common.io.Files; import org.apache.druid.java.util.common.FileUtils; @@ -117,12 +116,6 @@ public boolean delete() private static final Logger log = new Logger(LocalDataSegmentPuller.class); - @VisibleForTesting - public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException - { - getSegmentFiles(getFile(segment), dir); - } - public FileUtils.FileCopyResult getSegmentFiles(final File sourceFile, final File dir) throws SegmentLoadingException { if (sourceFile.isDirectory()) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java index 4f4f7a5b1d19..2a633b6ad275 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java +++ b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; @@ -59,8 +58,7 @@ public LoadableDataSegment( @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, @JsonProperty("binaryVersion") Integer binaryVersion, - @JsonProperty("size") long size, - @JacksonInject PruneSpecsHolder pruneSpecsHolder + @JsonProperty("size") long size ) { super( diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index bcd88ee7ee93..69c4d9403c20 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -29,24 +29,35 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.rpc.HttpResponseException; +import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -83,6 +94,8 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final SegmentManager segmentManager; private final ScheduledExecutorService exec; private final ServerTypeConfig serverTypeConfig; + private final CoordinatorClient coordinatorClient; + private final ServiceEmitter emitter; private final ConcurrentSkipListSet segmentsToDelete; private volatile boolean started = false; @@ -103,7 +116,9 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - ServerTypeConfig serverTypeConfig + ServerTypeConfig serverTypeConfig, + CoordinatorClient coordinatorClient, + ServiceEmitter emitter ) { this( @@ -115,7 +130,9 @@ public SegmentLoadDropHandler( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") ), - serverTypeConfig + serverTypeConfig, + coordinatorClient, + emitter ); } @@ -126,7 +143,9 @@ public SegmentLoadDropHandler( DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, ScheduledExecutorService exec, - ServerTypeConfig serverTypeConfig + ServerTypeConfig serverTypeConfig, + CoordinatorClient coordinatorClient, + ServiceEmitter emitter ) { this.config = config; @@ -135,6 +154,8 @@ public SegmentLoadDropHandler( this.segmentManager = segmentManager; this.exec = exec; this.serverTypeConfig = serverTypeConfig; + this.coordinatorClient = coordinatorClient; + this.emitter = emitter; this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); @@ -151,7 +172,7 @@ public void start() throws IOException log.info("Starting..."); try { if (segmentManager.canHandleSegments()) { - bootstrapCachedSegments(); + loadSegmentsOnBootstrap(); } if (shouldAnnounce()) { @@ -207,14 +228,24 @@ public Map getRowCountDistributionPerDataso } /** - * Bulk loading of cached segments into page cache during bootstrap. + * Bulk loading of the following segments into the page cache at startup: + *
  • Previously cached segments
  • + *
  • Bootstrap segments from the coordinator
  • */ - private void bootstrapCachedSegments() throws IOException + private void loadSegmentsOnBootstrap() throws IOException { + final List startupSegments = new ArrayList<>(); + final Stopwatch stopwatch = Stopwatch.createStarted(); - final List segments = segmentManager.getCachedSegments(); + startupSegments.addAll(segmentManager.getCachedSegments()); + + final List bootstrapSegments = getBootstrapSegments(); + log.info("Found [%d] bootstrap cachedSegments from the coordinator. They are [%s]", bootstrapSegments.size(), bootstrapSegments); + // Add all elements from both lists to the new list + startupSegments.addAll(bootstrapSegments); + - // Start a temporary thread pool to load segments into page cache during bootstrap + // Start a temporary thread pool to load cachedSegments into page cache during bootstrap final ExecutorService loadingExecutor = Execs.multiThreaded( config.getNumBootstrapThreads(), "Segment-Load-Startup-%s" ); @@ -224,11 +255,11 @@ private void bootstrapCachedSegments() throws IOException backgroundSegmentAnnouncer.startAnnouncing(); - final int numSegments = segments.size(); + final int numSegments = startupSegments.size(); final CountDownLatch latch = new CountDownLatch(numSegments); final AtomicInteger counter = new AtomicInteger(0); final CopyOnWriteArrayList failedSegments = new CopyOnWriteArrayList<>(); - for (final DataSegment segment : segments) { + for (final DataSegment segment : startupSegments) { loadingExecutor.submit( () -> { try { @@ -269,7 +300,7 @@ private void bootstrapCachedSegments() throws IOException latch.await(); if (failedSegments.size() > 0) { - log.makeAlert("%,d errors seen while loading segments", failedSegments.size()) + log.makeAlert("%,d errors seen while loading segments on startup", failedSegments.size()) .addData("failedSegments", failedSegments) .emit(); } @@ -282,8 +313,8 @@ private void bootstrapCachedSegments() throws IOException backgroundSegmentAnnouncer.finishAnnouncing(); } catch (SegmentLoadingException e) { - log.makeAlert(e, "Failed to load segments -- likely problem with announcing.") - .addData("numSegments", segments.size()) + log.makeAlert(e, "Failed to load cachedSegments -- likely problem with announcing.") + .addData("numSegments", startupSegments.size()) .emit(); } finally { @@ -292,7 +323,58 @@ private void bootstrapCachedSegments() throws IOException // At this stage, all tasks have been submitted, send a shutdown command to cleanup any resources alloted // for the bootstrapping function. segmentManager.shutdownBootstrap(); - log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", segments.size(), stopwatch.millisElapsed()); + log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", startupSegments.size(), stopwatch.millisElapsed()); + } + } + + private List getBootstrapSegments() throws IOException + { + final List bootstrapSegments; + try (CloseableIterator iterator = fetchBootstrapSegments()) { + bootstrapSegments = ImmutableList.copyOf(iterator); + } + return bootstrapSegments; + } + + /** + * Fetches all broadcast segments. + */ + private CloseableIterator fetchBootstrapSegments() + { + // So think about the failure mechanisms. + // If the endpoint doesn't exist as with first scenario, handle gracefuly (404) + // If the endpoint exists, but a 503, then keep retrying with an appropriate strategy. + // Any other exception, then bomb? Or fail gracefully and wait for the usual coordinator route to assigning + // bootstrap segments. + final Stopwatch stopwatch = Stopwatch.createStarted(); + log.info("Fetching bootstrap segments from the coordinator"); + + final ListenableFuture> bootstrapSegments = + coordinatorClient.withRetryPolicy(StandardRetryPolicy.aboutAnHour()).fetchBootstrapSegments(); + + try { + return FutureUtils.getUnchecked(bootstrapSegments, true); + } + catch (Exception e) { + log.info("Exception and retry as necessary. Cause[%s]", e.getCause()); + if (e.getCause() instanceof HttpResponseException && + ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + // NOTE: this is to support backward compatibility, when the coordinator is running on an older version and + // doesn't have the "bootstrapSegments" endpoint. + log.warn("Bootstrap segments endpoint is not available in the coordinator."); + return CloseableIterators.withEmptyBaggage(Collections.emptyIterator()); + } + // All retriable errors are handled by the service client itself. For any other error code/exception, + // we just bail. TODO: evaluate if this is the right thing. Should we fail startup or start gracefully? + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Could not fetch bootstrap segments from the coordinator at startup."); + } + finally { + stopwatch.stop(); + final long fetchRunMillis = stopwatch.millisElapsed(); + emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/time", fetchRunMillis)); + log.info("Fetching bootstrap segments completed in [%d] ms", fetchRunMillis); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 0787bc8f7d4f..1a85d7c608e9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -37,6 +37,7 @@ import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.DateTimes; @@ -83,6 +84,7 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; @@ -174,6 +176,12 @@ public class DruidCoordinator */ private volatile SegmentReplicationStatus segmentReplicationStatus = null; + /** + * Used to determine broadcast segments. Similar to {@link #segmentReplicationStatus}, this might contain stale + * information if the coordinator runs are delayed. + */ + private volatile StrategicSegmentAssigner segmentAssigner = null; + public static final String HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP = "HistoricalManagementDuties"; private static final String METADATA_STORE_MANAGEMENT_DUTIES_DUTY_GROUP = "MetadataStoreManagementDuties"; private static final String INDEXING_SERVICE_DUTIES_DUTY_GROUP = "IndexingServiceDuties"; @@ -315,6 +323,20 @@ public Map getDatasourceToLoadStatus() return loadStatus; } + /** + * @return the set of broadcast segments. If the coordinator hasn't initialized fully, a retriable exception + * will be thrown. + */ + public Set getBroadcastSegments() + { + if (segmentAssigner == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.UNAVAILABLE) + .build("bootstrap segments not available yet."); + } + return segmentAssigner.getBroadcastSegments(); + } + @Nullable public Integer getReplicationFactor(SegmentId segmentId) { @@ -798,6 +820,7 @@ private class UpdateReplicationStatus implements CoordinatorDuty @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { + segmentAssigner = params.getSegmentAssigner(); segmentReplicationStatus = params.getSegmentReplicationStatus(); // Collect stats for unavailable and under-replicated segments diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index 60f46aca6e29..61061c527e68 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -66,12 +66,6 @@ public RunRules(SegmentDeleteHandler deleteHandler) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - final DruidCluster cluster = params.getDruidCluster(); - if (cluster.isEmpty()) { - log.warn("Cluster has no servers. Not running any rules."); - return params; - } - final Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); final Set usedSegments = params.getUsedSegments(); log.info( @@ -109,7 +103,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) processSegmentDeletes(segmentAssigner, params.getCoordinatorStats()); alertForSegmentsWithNoRules(datasourceToSegmentsWithNoRule); - alertForInvalidRules(segmentAssigner); + alertForInvalidRules(segmentAssigner, params.getDruidCluster()); return params.buildFromExisting() .withBroadcastDatasources(getBroadcastDatasources(params)) @@ -145,8 +139,12 @@ private void alertForSegmentsWithNoRules(Object2IntOpenHashMap datasourc ); } - private void alertForInvalidRules(StrategicSegmentAssigner segmentAssigner) + private void alertForInvalidRules(StrategicSegmentAssigner segmentAssigner, DruidCluster druidCluster) { + if (druidCluster.isEmpty()) { + log.warn("Cluster has no servers."); + return; + } segmentAssigner.getDatasourceToInvalidLoadTiers().forEach( (datasource, invalidTiers) -> log.makeAlert( "Load rules for datasource[%s] refer to invalid tiers[%s]." diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java index 1c2a867c4faf..f58e74cdb2aa 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.client.DruidServer; +import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.balancer.BalancerStrategy; @@ -55,6 +56,7 @@ @NotThreadSafe public class StrategicSegmentAssigner implements SegmentActionHandler { + private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class); private final SegmentLoadQueueManager loadQueueManager; private final DruidCluster cluster; private final CoordinatorRunStats stats; @@ -69,6 +71,7 @@ public class StrategicSegmentAssigner implements SegmentActionHandler private final Map tierToHistoricalCount = new HashMap<>(); private final Map> segmentsToDelete = new HashMap<>(); private final Map> segmentsWithZeroRequiredReplicas = new HashMap<>(); + private final Set broadcastSegments = new HashSet<>(); public StrategicSegmentAssigner( SegmentLoadQueueManager loadQueueManager, @@ -361,6 +364,8 @@ public void broadcastSegment(DataSegment segment) entry -> replicaCountMap.computeIfAbsent(segment.getId(), entry.getKey()) .setRequired(entry.getIntValue(), entry.getIntValue()) ); + + broadcastSegments.add(segment); } @Override @@ -398,6 +403,11 @@ private boolean loadBroadcastSegment(DataSegment segment, ServerHolder server) return false; } + public Set getBroadcastSegments() + { + return broadcastSegments; + } + /** * Drops the broadcast segment if it is loaded on the given server. * Returns true only if the segment was successfully queued for drop on the server. diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index d8b00c318db6..164896a1afb0 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -471,4 +471,36 @@ public Response getDataSourceInformation( ); return Response.status(Response.Status.OK).entity(authorizedDataSourceInformation).build(); } + + /** + * @return all bootstrap segments determined by the coordinator. Currently, only the set of broadcast segments + * is returned. + * @implNote this currently accepts an empty POST, but is future-proof to other scenarios. + */ + @GET + @Path("/bootstrapSegments") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getBootstrapSegments() + { + try { + log.info("Hmm call to bootstrap segments.."); + Set broadcastSegments = coordinator.getBroadcastSegments(); + log.info( + "Number of bootstrap segments coordinator is returning [%d] and they are [%s]", + broadcastSegments.size(), + broadcastSegments + ); + return Response.status(Response.Status.OK).entity(broadcastSegments).build(); + } + catch (DruidException e) { + return ServletResourceUtils.buildErrorResponseFrom(e); + } + catch (Exception e) { + return Response + .serverError() + .entity(ImmutableMap.of("error", "Exception occurred.", "message", Throwables.getRootCause(e).toString())) + .build(); + } + } } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index 62af96d4e0eb..95fe5e68c07a 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -181,6 +181,58 @@ public void test_fetchUsedSegments() throws Exception ); } + @Test + public void test_fetchBootstrapSegments() throws Exception + { + final DataSegment segment1 = + DataSegment.builder() + .dataSource("xyz") + .interval(Intervals.of("1000/2000")) + .version("1") + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(); + final DataSegment segment2 = + DataSegment.builder() + .dataSource("xyz") + .interval(Intervals.of("2000/3000")) + .version("1") + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(); + final List segments = ImmutableList.of(segment1, segment2); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/metadata/bootstrapSegments"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(segments) + ); + + Assert.assertEquals( + segments, + ImmutableList.copyOf(coordinatorClient.fetchBootstrapSegments().get()) + ); + } + + @Test + public void test_fetchEmptyBootstrapSegments() throws Exception + { + final List segments = ImmutableList.of(); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/metadata/bootstrapSegments"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + jsonMapper.writeValueAsBytes(segments) + ); + + Assert.assertEquals( + segments, + ImmutableList.copyOf(coordinatorClient.fetchBootstrapSegments().get()) + ); + } + @Test public void test_fetchDataSourceInformation() throws Exception { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 7b7d253ef6d5..bd42b337e4a9 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; @@ -62,6 +63,12 @@ public ListenableFuture> fetchDataSourceInformation( throw new UnsupportedOperationException(); } + @Override + public ListenableFuture> fetchBootstrapSegments() + { + throw new UnsupportedOperationException(); + } + @Override public CoordinatorClient withRetryPolicy(ServiceRetryPolicy retryPolicy) { diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java index b7ce3b8e0588..f6b1c39c59da 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerCacheTest.java @@ -23,6 +23,8 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -34,7 +36,6 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; -import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -63,6 +64,8 @@ public class SegmentLoadDropHandlerCacheTest private SegmentManager segmentManager; private SegmentLoaderConfig loaderConfig; private SegmentLocalCacheManager cacheManager; + private TestCoordinatorClient coordinatorClient; + private ServiceEmitter emitter; private ObjectMapper objectMapper; @Before @@ -100,7 +103,9 @@ public List getLocations() segmentManager = new SegmentManager(cacheManager); segmentAnnouncer = new TestDataSegmentAnnouncer(); serverAnnouncer = new TestDataServerAnnouncer(); - EmittingLogger.registerEmitter(new NoopServiceEmitter()); + coordinatorClient = new TestCoordinatorClient(); + emitter = new StubServiceEmitter(); + EmittingLogger.registerEmitter(emitter); } @Test @@ -122,7 +127,9 @@ public void testLoadStartStopWithEmptyLocations() throws IOException segmentAnnouncer, serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.BROKER) + new ServerTypeConfig(ServerType.BROKER), + coordinatorClient, + emitter ); loadDropHandler.start(); @@ -140,7 +147,9 @@ public void testLoadStartStop() throws IOException segmentAnnouncer, serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.BROKER) + new ServerTypeConfig(ServerType.BROKER), + coordinatorClient, + emitter ); loadDropHandler.start(); @@ -171,7 +180,9 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException segmentAnnouncer, serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL) + new ServerTypeConfig(ServerType.HISTORICAL), + coordinatorClient, + emitter ); // Start the load drop handler diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 1a776c6c34a1..41bdb4c79a1e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -22,12 +22,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; import org.apache.druid.segment.loading.NoopSegmentCacheManager; @@ -37,7 +40,6 @@ import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; import org.apache.druid.server.coordination.SegmentChangeStatus.State; -import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import org.junit.Assert; @@ -72,6 +74,8 @@ public class SegmentLoadDropHandlerTest private List scheduledRunnable; private SegmentLoaderConfig segmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; + private TestCoordinatorClient coordinatorClient; + private ServiceEmitter emitter; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -136,7 +140,9 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) }; }; - EmittingLogger.registerEmitter(new NoopServiceEmitter()); + coordinatorClient = new TestCoordinatorClient(); + emitter = new StubServiceEmitter(); + EmittingLogger.registerEmitter(emitter); } /** @@ -293,6 +299,51 @@ public void testLoadCache() throws Exception Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } + @Test + public void testLoadBootstrapSegments() throws Exception + { + Set segments = new HashSet<>(); + for (int i = 0; i < COUNT; ++i) { + segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01"))); + segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-01"))); + segments.add(makeSegment("test_two" + i, "1", Intervals.of("P1d/2011-04-02"))); + } + + final TestCoordinatorClient coordinatorClient = new TestCoordinatorClient(segments); + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager, coordinatorClient); + + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); + + for (int i = 0; i < COUNT; ++i) { + Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test" + i).longValue()); + Assert.assertEquals(2L, segmentManager.getDataSourceCounts().get("test_two" + i).longValue()); + } + + final ImmutableList expectedBootstrapSegments = ImmutableList.copyOf(segments); + + Assert.assertEquals(expectedBootstrapSegments, segmentAnnouncer.getObservedSegments()); + + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); + Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegmentsLoadedIntoPageCache); + + + handler.stop(); + + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); + } + @Test public void testStartStop() throws Exception { @@ -467,7 +518,8 @@ public int getDropSegmentDelayMillis() final SegmentLoadDropHandler handler = initSegmentLoadDropHandler( noAnnouncerSegmentLoaderConfig, - segmentManager + segmentManager, + coordinatorClient ); handler.start(); @@ -543,12 +595,21 @@ public int getDropSegmentDelayMillis() Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager, TestCoordinatorClient coordinatorClient) + { + return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager, coordinatorClient); + } + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager) { - return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager); + return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager, coordinatorClient); } - private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentLoaderConfig config, SegmentManager segmentManager) + private SegmentLoadDropHandler initSegmentLoadDropHandler( + SegmentLoaderConfig config, + SegmentManager segmentManager, + CoordinatorClient coordinatorClient + ) { return new SegmentLoadDropHandler( config, @@ -556,7 +617,9 @@ private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentLoaderConfig co serverAnnouncer, segmentManager, scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - new ServerTypeConfig(ServerType.HISTORICAL) + new ServerTypeConfig(ServerType.HISTORICAL), + coordinatorClient, + emitter ); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java new file mode 100644 index 000000000000..5389e20840b7 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java @@ -0,0 +1,51 @@ +/* + * 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.druid.server.coordination; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.timeline.DataSegment; + +import java.util.HashSet; +import java.util.Set; + +class TestCoordinatorClient extends NoopCoordinatorClient +{ + private final Set bootstrapSegments; + + TestCoordinatorClient() + { + this(new HashSet<>()); + } + + TestCoordinatorClient(final Set bootstrapSegments) + { + this.bootstrapSegments = bootstrapSegments; + } + + @Override + public ListenableFuture> fetchBootstrapSegments() + { + return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index f0f2bd3b4e44..c5ca17c5a676 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -106,7 +106,9 @@ public void testLoadDrop() throws Exception EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), EasyMock.createNiceMock(SegmentManager.class), EasyMock.createNiceMock(ScheduledExecutorService.class), - new ServerTypeConfig(ServerType.HISTORICAL) + new ServerTypeConfig(ServerType.HISTORICAL), + null, + null ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index a5ed1616a6d0..c31a8f8e8970 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -39,6 +39,8 @@ import org.apache.druid.curator.CuratorUtils; import org.apache.druid.curator.discovery.LatchableServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; @@ -74,6 +76,7 @@ import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; +import org.hamcrest.MatcherAssert; import org.joda.time.Duration; import org.junit.After; import org.junit.Assert; @@ -82,6 +85,7 @@ import javax.annotation.Nullable; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; @@ -266,7 +270,14 @@ public void testCoordinatorRun() throws Exception coordinator.start(); Assert.assertNull(coordinator.getReplicationFactor(dataSegment.getId())); - + MatcherAssert.assertThat( + Assert.assertThrows(DruidException.class, () -> coordinator.getBroadcastSegments()), + new DruidExceptionMatcher( + DruidException.Persona.OPERATOR, + DruidException.Category.UNAVAILABLE, + "general" + ).expectMessageIs("bootstrap segments not available yet.") + ); // Wait for this coordinator to become leader leaderAnnouncerLatch.await(); @@ -293,6 +304,7 @@ public void testCoordinatorRun() throws Exception coordinator.getDatasourceToUnavailableSegmentCount(); Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size()); Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource)); + Assert.assertEquals(0, coordinator.getBroadcastSegments().size()); Map> underReplicationCountsPerDataSourcePerTier = coordinator.getTierToDatasourceToUnderReplicatedCount(false); @@ -571,6 +583,7 @@ public void testComputeUnderReplicationCountsPerDataSourcePerTierForSegmentsWith coordinatorRunLatch.await(); Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getDatasourceToLoadStatus()); + Assert.assertEquals(new HashSet<>(dataSegments.values()), coordinator.getBroadcastSegments()); // Under-replicated counts are updated only after the next coordinator run Map> underReplicationCountsPerDataSourcePerTier = diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index a284a27c0071..951d703ec3bf 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -46,8 +46,11 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; +import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; +import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; +import org.apache.druid.server.coordinator.rules.IntervalBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.IntervalDropRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -69,9 +72,6 @@ import java.util.List; import java.util.Set; -/** - * - */ public class RunRulesTest { private static final long SERVER_SIZE_10GB = 10L << 30; @@ -315,6 +315,11 @@ private ServerHolder createServerHolder(String name, String tier, LoadQueuePeon return new ServerHolder(createHistorical(name, tier).toImmutableDruidServer(), peon); } + private ServerHolder createServerHolder(String name, String tier, LoadQueuePeon peon, boolean isDecommmisionning) + { + return new ServerHolder(createHistorical(name, tier).toImmutableDruidServer(), peon, isDecommmisionning); + } + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams( DruidCluster druidCluster, DataSegment segment @@ -482,7 +487,6 @@ public void testRunTwoTiersTierDoesNotExist() @Test public void testRunRuleDoesNotExist() { - EasyMock .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) .andReturn( @@ -524,6 +528,134 @@ public void testRunRuleDoesNotExist() EasyMock.verify(mockPeon); } + @Test + public void testRunRuleWithNoServers() + { + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Lists.newArrayList( + new IntervalBroadcastDistributionRule( + Intervals.of("2012-01-01/2012-01-02") + ), + new IntervalBroadcastDistributionRule( + Intervals.of("2012-01-02/2012-01-03") + ) + ) + ) + .atLeastOnce(); + + EasyMock.replay(databaseRuleManager, mockPeon); + + DruidCluster emptyCluster = DruidCluster.builder().build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(emptyCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertFalse(stats.hasStat(Stats.Segments.ASSIGNED)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + Assert.assertEquals(0, emitter.getAlerts().size()); + + StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + Assert.assertNotNull(segmentAssigner); + Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); + + EasyMock.verify(mockPeon); + } + + @Test + public void testBroadcastLoad() + { + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn( + Lists.newArrayList( + new IntervalBroadcastDistributionRule( + Intervals.of("2012-01-01/2012-01-02") + ), + new IntervalBroadcastDistributionRule( + Intervals.of("2012-01-02/2012-01-03") + ) + ) + ) + .atLeastOnce(); + + mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + + EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.replay(databaseRuleManager, mockPeon); + + DruidCluster druidCluster = DruidCluster + .builder() + .add(createServerHolder("serverNorm", "normal", mockPeon)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); + + StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + Assert.assertNotNull(segmentAssigner); + Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); + + EasyMock.verify(mockPeon); + } + + @Test + public void testBroadcastDrop() + { + EasyMock + .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) + .andReturn(Collections.singletonList(new ForeverBroadcastDistributionRule())) + .atLeastOnce(); + + mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().atLeastOnce(); + + EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); + EasyMock.replay(databaseRuleManager, mockPeon); + + DruidServer server = createHistorical("serverNorm", "normal"); + for (DataSegment segment : usedSegments) { + server.addDataSegment(segment); + } + + DruidCluster druidCluster = DruidCluster + .builder() + .add(new ServerHolder(server.toImmutableDruidServer(), mockPeon, true)) + .build(); + + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) + .withSegmentAssignerUsing(loadQueueManager) + .build(); + + CoordinatorRunStats stats = runDutyAndGetStats(params); + + Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); + Assert.assertFalse(stats.hasStat(Stats.Segments.ASSIGNED)); + + StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); + Assert.assertNotNull(segmentAssigner); + Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); + + EasyMock.verify(mockPeon); + } + @Test public void testDropRemove() { diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index 1b86bbca4536..ff3bb2d6654d 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -486,6 +486,39 @@ public void testGetSegment() ); } + @Test + public void testGetBootstrapSegmentsWithEmptySet() + { + Mockito.doReturn(ImmutableSet.of()).when(coordinator).getBroadcastSegments(); + + Response response = metadataResource.getBootstrapSegments(); + final List observedSegments = extractResponseList(response); + Assert.assertEquals(0, observedSegments.size()); + } + + @Test + public void testGetBootstrapSegmentsWithNonEmptySet() + { + Mockito.doReturn(ImmutableSet.of(segments[0], segments[1])).when(coordinator).getBroadcastSegments(); + + Response response = metadataResource.getBootstrapSegments(); + final List observedSegments = extractResponseList(response); + Assert.assertEquals(2, observedSegments.size()); + } + + @Test + public void testGetBootstrapSegmentsWithNullSet() + { + Mockito.doThrow(DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.UNAVAILABLE) + .build("something went wrong")) + .when(coordinator).getBroadcastSegments(); + + Response response = metadataResource.getBootstrapSegments(); + Assert.assertEquals(503, response.getStatus()); + Assert.assertEquals("something went wrong", getExceptionMessage(response)); + } + private List extractResponseList(Response response) { return Lists.newArrayList( From a36351134bfa06af0a0a6bb2b2d31ff1e5434475 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 14 Jun 2024 13:47:04 -0700 Subject: [PATCH 02/23] Fail open by default if there are any errors talking to the coordinator. --- .../coordination/SegmentLoadDropHandler.java | 77 ++++++------------- 1 file changed, 24 insertions(+), 53 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 69c4d9403c20..447cfe61bfe8 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -31,10 +31,8 @@ import com.google.inject.Inject; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.ServerTypeConfig; -import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; @@ -44,20 +42,16 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.rpc.HttpResponseException; -import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -172,7 +166,7 @@ public void start() throws IOException log.info("Starting..."); try { if (segmentManager.canHandleSegments()) { - loadSegmentsOnBootstrap(); + loadSegmentsOnStartup(); } if (shouldAnnounce()) { @@ -232,18 +226,15 @@ public Map getRowCountDistributionPerDataso *
  • Previously cached segments
  • *
  • Bootstrap segments from the coordinator
  • */ - private void loadSegmentsOnBootstrap() throws IOException + private void loadSegmentsOnStartup() throws IOException { final List startupSegments = new ArrayList<>(); - - final Stopwatch stopwatch = Stopwatch.createStarted(); startupSegments.addAll(segmentManager.getCachedSegments()); + startupSegments.addAll(getBootstrapSegments()); - final List bootstrapSegments = getBootstrapSegments(); - log.info("Found [%d] bootstrap cachedSegments from the coordinator. They are [%s]", bootstrapSegments.size(), bootstrapSegments); - // Add all elements from both lists to the new list - startupSegments.addAll(bootstrapSegments); + log.info("Server type[%s]", serverTypeConfig.getServerType()); + final Stopwatch stopwatch = Stopwatch.createStarted(); // Start a temporary thread pool to load cachedSegments into page cache during bootstrap final ExecutorService loadingExecutor = Execs.multiThreaded( @@ -313,7 +304,7 @@ private void loadSegmentsOnBootstrap() throws IOException backgroundSegmentAnnouncer.finishAnnouncing(); } catch (SegmentLoadingException e) { - log.makeAlert(e, "Failed to load cachedSegments -- likely problem with announcing.") + log.makeAlert(e, "Failed to load startup segments -- likely problem with announcing.") .addData("numSegments", startupSegments.size()) .emit(); } @@ -323,59 +314,39 @@ private void loadSegmentsOnBootstrap() throws IOException // At this stage, all tasks have been submitted, send a shutdown command to cleanup any resources alloted // for the bootstrapping function. segmentManager.shutdownBootstrap(); - log.info("Cache load of [%d] bootstrap segments took [%,d]ms.", startupSegments.size(), stopwatch.millisElapsed()); + log.info("Cache load of [%d] startup segments took [%,d]ms.", startupSegments.size(), stopwatch.millisElapsed()); } } - private List getBootstrapSegments() throws IOException + private List getBootstrapSegments() { - final List bootstrapSegments; - try (CloseableIterator iterator = fetchBootstrapSegments()) { - bootstrapSegments = ImmutableList.copyOf(iterator); - } - return bootstrapSegments; - } - - /** - * Fetches all broadcast segments. - */ - private CloseableIterator fetchBootstrapSegments() - { - // So think about the failure mechanisms. - // If the endpoint doesn't exist as with first scenario, handle gracefuly (404) - // If the endpoint exists, but a 503, then keep retrying with an appropriate strategy. - // Any other exception, then bomb? Or fail gracefully and wait for the usual coordinator route to assigning - // bootstrap segments. + log.info("Fetching bootstrap segments from the coordinator."); final Stopwatch stopwatch = Stopwatch.createStarted(); - log.info("Fetching bootstrap segments from the coordinator"); - final ListenableFuture> bootstrapSegments = - coordinatorClient.withRetryPolicy(StandardRetryPolicy.aboutAnHour()).fetchBootstrapSegments(); + final ListenableFuture> bootstrapSegmentsFuture = + coordinatorClient.fetchBootstrapSegments(); - try { - return FutureUtils.getUnchecked(bootstrapSegments, true); + List bootstrapSegments = new ArrayList<>(); + + try (CloseableIterator iterator = FutureUtils.getUnchecked(bootstrapSegmentsFuture, true)) { + bootstrapSegments = ImmutableList.copyOf(iterator); } catch (Exception e) { - log.info("Exception and retry as necessary. Cause[%s]", e.getCause()); - if (e.getCause() instanceof HttpResponseException && - ((HttpResponseException) e.getCause()).getResponse().getStatus().equals(HttpResponseStatus.NOT_FOUND)) { - // NOTE: this is to support backward compatibility, when the coordinator is running on an older version and - // doesn't have the "bootstrapSegments" endpoint. - log.warn("Bootstrap segments endpoint is not available in the coordinator."); - return CloseableIterators.withEmptyBaggage(Collections.emptyIterator()); - } - // All retriable errors are handled by the service client itself. For any other error code/exception, - // we just bail. TODO: evaluate if this is the right thing. Should we fail startup or start gracefully? - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(e, "Could not fetch bootstrap segments from the coordinator at startup."); + // By default, we "fail open" when there is any error -- finding the coordinator, or if the API endpoint cannot + // be found during rolling upgrades, or even if it's irrecoverable. + log.warn(e, "Error fetching bootstrap segments from the coordinator. " + + "Returning an empty set of bootstrap segments."); + bootstrapSegments = ImmutableList.of(); } finally { stopwatch.stop(); final long fetchRunMillis = stopwatch.millisElapsed(); emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/time", fetchRunMillis)); - log.info("Fetching bootstrap segments completed in [%d] ms", fetchRunMillis); + emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/count", bootstrapSegments.size())); + log.info("Fetching [%d] bootstrap segments completed in [%d] ms", bootstrapSegments.size(), fetchRunMillis); } + + return bootstrapSegments; } @Override From 97e97348037b590485da7b9c0676483f1544f782 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 14 Jun 2024 19:55:33 -0700 Subject: [PATCH 03/23] Add test for failure scenario and cleanup logs. --- .../coordination/SegmentLoadDropHandler.java | 11 +++---- .../druid/server/http/MetadataResource.java | 6 ---- .../SegmentLoadDropHandlerTest.java | 31 ++++++++++++++++--- 3 files changed, 32 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 447cfe61bfe8..d78a52319122 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -232,8 +232,6 @@ private void loadSegmentsOnStartup() throws IOException startupSegments.addAll(segmentManager.getCachedSegments()); startupSegments.addAll(getBootstrapSegments()); - log.info("Server type[%s]", serverTypeConfig.getServerType()); - final Stopwatch stopwatch = Stopwatch.createStarted(); // Start a temporary thread pool to load cachedSegments into page cache during bootstrap @@ -318,17 +316,18 @@ private void loadSegmentsOnStartup() throws IOException } } + /** + * @return a list of bootstrap segments. When bootstrap segments cannot be found, an empty list is returned. + */ private List getBootstrapSegments() { log.info("Fetching bootstrap segments from the coordinator."); final Stopwatch stopwatch = Stopwatch.createStarted(); - final ListenableFuture> bootstrapSegmentsFuture = - coordinatorClient.fetchBootstrapSegments(); - List bootstrapSegments = new ArrayList<>(); - try (CloseableIterator iterator = FutureUtils.getUnchecked(bootstrapSegmentsFuture, true)) { + try (final CloseableIterator iterator = + FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true)) { bootstrapSegments = ImmutableList.copyOf(iterator); } catch (Exception e) { diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 164896a1afb0..5f4c5432e46f 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -484,13 +484,7 @@ public Response getDataSourceInformation( public Response getBootstrapSegments() { try { - log.info("Hmm call to bootstrap segments.."); Set broadcastSegments = coordinator.getBroadcastSegments(); - log.info( - "Number of bootstrap segments coordinator is returning [%d] and they are [%s]", - broadcastSegments.size(), - broadcastSegments - ); return Response.status(Response.Status.OK).entity(broadcastSegments).build(); } catch (DruidException e) { diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 41bdb4c79a1e..a935b2bff64e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.MapUtils; @@ -302,7 +303,7 @@ public void testLoadCache() throws Exception @Test public void testLoadBootstrapSegments() throws Exception { - Set segments = new HashSet<>(); + final Set segments = new HashSet<>(); for (int i = 0; i < COUNT; ++i) { segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01"))); segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02"))); @@ -334,9 +335,31 @@ public void testLoadBootstrapSegments() throws Exception Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegments); - Assert.assertEquals(ImmutableList.of(), cacheManager.observedSegmentsLoadedIntoPageCache); + handler.stop(); + + Assert.assertEquals(0, serverAnnouncer.getObservedCount()); + Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); + } + + @Test + public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception + { + final TestSegmentCacheManager cacheManager = new TestSegmentCacheManager(); + final SegmentManager segmentManager = new SegmentManager(cacheManager); + + final SegmentLoadDropHandler handler = initSegmentLoadDropHandler(segmentManager, new NoopCoordinatorClient()); + + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + handler.start(); + + Assert.assertEquals(1, serverAnnouncer.getObservedCount()); + Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); + + Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); + Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); handler.stop(); @@ -595,7 +618,7 @@ public int getDropSegmentDelayMillis() Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } - private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager, TestCoordinatorClient coordinatorClient) + private SegmentLoadDropHandler initSegmentLoadDropHandler(SegmentManager segmentManager, CoordinatorClient coordinatorClient) { return initSegmentLoadDropHandler(segmentLoaderConfig, segmentManager, coordinatorClient); } From a4ed4b1000fe0c7d3a5f8dddd9168e67a82bf7fd Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 06:50:47 -0700 Subject: [PATCH 04/23] Cleanup and add debug log --- .../druid/server/coordination/SegmentLoadDropHandler.java | 2 +- .../org/apache/druid/server/http/MetadataResource.java | 4 +--- .../druid/server/coordination/ZkCoordinatorTest.java | 4 ++-- .../server/lookup/cache/LookupCoordinatorManagerTest.java | 7 ++++++- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d78a52319122..9f3302de5321 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -234,7 +234,7 @@ private void loadSegmentsOnStartup() throws IOException final Stopwatch stopwatch = Stopwatch.createStarted(); - // Start a temporary thread pool to load cachedSegments into page cache during bootstrap + // Start a temporary thread pool to load segments into page cache during bootstrap final ExecutorService loadingExecutor = Execs.multiThreaded( config.getNumBootstrapThreads(), "Segment-Load-Startup-%s" ); diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 5f4c5432e46f..b48bd2acc673 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -473,9 +473,7 @@ public Response getDataSourceInformation( } /** - * @return all bootstrap segments determined by the coordinator. Currently, only the set of broadcast segments - * is returned. - * @implNote this currently accepts an empty POST, but is future-proof to other scenarios. + * @return all bootstrap segments determined by the coordinator. */ @GET @Path("/bootstrapSegments") diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index c5ca17c5a676..9f5291af598f 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -107,8 +107,8 @@ public void testLoadDrop() throws Exception EasyMock.createNiceMock(SegmentManager.class), EasyMock.createNiceMock(ScheduledExecutorService.class), new ServerTypeConfig(ServerType.HISTORICAL), - null, - null + new TestCoordinatorClient(), + new NoopServiceEmitter() ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index c94d8e436683..0c69cce8924d 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.java.util.http.client.response.SequenceInputStreamResponseHandler; @@ -61,6 +62,8 @@ public class LookupCoordinatorManagerTest { + private static final EmittingLogger log = new EmittingLogger(LookupCoordinatorManagerTest.class); + private final ObjectMapper mapper = new DefaultObjectMapper(); private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createStrictMock(DruidNodeDiscoveryProvider.class); private final LookupNodeDiscovery lookupNodeDiscovery = EasyMock.createStrictMock( @@ -142,7 +145,9 @@ public void setUp() @After public void tearDown() { - Assert.assertEquals(0, SERVICE_EMITTER.getEvents().size()); + List events = SERVICE_EMITTER.getEvents(); + log.info("Events: [%s]", events); + Assert.assertEquals(0, events.size()); SERVICE_EMITTER.flush(); } From 7459a8a15ef524052024bb50a2c955f154f4dbe9 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 09:53:13 -0700 Subject: [PATCH 05/23] Assert the events so we know the list exactly. --- .../server/lookup/cache/LookupCoordinatorManagerTest.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 0c69cce8924d..f7ac37994a8b 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.core.EventMap; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.java.util.http.client.response.SequenceInputStreamResponseHandler; @@ -147,6 +148,10 @@ public void tearDown() { List events = SERVICE_EMITTER.getEvents(); log.info("Events: [%s]", events); + for (Event event : events) { + Assert.assertEquals(EventMap.builder().put("foo", "bar").build(), event.toMap()); + } + Assert.assertEquals(ImmutableList.of(), events); Assert.assertEquals(0, events.size()); SERVICE_EMITTER.flush(); } From f0b874ae565bc52b2baf358d233b7b9f526b0e8a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 12:40:16 -0700 Subject: [PATCH 06/23] Revert RunRules test. The rules aren't evaluated if there are no clusters. --- .../druid/server/coordinator/duty/RunRules.java | 14 ++++++++------ .../server/coordinator/duty/RunRulesTest.java | 16 +--------------- 2 files changed, 9 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index 61061c527e68..60f46aca6e29 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -66,6 +66,12 @@ public RunRules(SegmentDeleteHandler deleteHandler) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { + final DruidCluster cluster = params.getDruidCluster(); + if (cluster.isEmpty()) { + log.warn("Cluster has no servers. Not running any rules."); + return params; + } + final Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); final Set usedSegments = params.getUsedSegments(); log.info( @@ -103,7 +109,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) processSegmentDeletes(segmentAssigner, params.getCoordinatorStats()); alertForSegmentsWithNoRules(datasourceToSegmentsWithNoRule); - alertForInvalidRules(segmentAssigner, params.getDruidCluster()); + alertForInvalidRules(segmentAssigner); return params.buildFromExisting() .withBroadcastDatasources(getBroadcastDatasources(params)) @@ -139,12 +145,8 @@ private void alertForSegmentsWithNoRules(Object2IntOpenHashMap datasourc ); } - private void alertForInvalidRules(StrategicSegmentAssigner segmentAssigner, DruidCluster druidCluster) + private void alertForInvalidRules(StrategicSegmentAssigner segmentAssigner) { - if (druidCluster.isEmpty()) { - log.warn("Cluster has no servers."); - return; - } segmentAssigner.getDatasourceToInvalidLoadTiers().forEach( (datasource, invalidTiers) -> log.makeAlert( "Load rules for datasource[%s] refer to invalid tiers[%s]." diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index 951d703ec3bf..a2bf5efa3e6d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -531,20 +531,6 @@ public void testRunRuleDoesNotExist() @Test public void testRunRuleWithNoServers() { - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Lists.newArrayList( - new IntervalBroadcastDistributionRule( - Intervals.of("2012-01-01/2012-01-02") - ), - new IntervalBroadcastDistributionRule( - Intervals.of("2012-01-02/2012-01-03") - ) - ) - ) - .atLeastOnce(); - EasyMock.replay(databaseRuleManager, mockPeon); DruidCluster emptyCluster = DruidCluster.builder().build(); @@ -563,7 +549,7 @@ public void testRunRuleWithNoServers() StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); Assert.assertNotNull(segmentAssigner); - Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); + Assert.assertEquals(0, segmentAssigner.getBroadcastSegments().size()); EasyMock.verify(mockPeon); } From d2a1817dab916c734c57c217297bd12a374095ee Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 12:41:36 -0700 Subject: [PATCH 07/23] Revert RunRulesTest too. --- .../server/coordinator/duty/RunRulesTest.java | 126 +----------------- 1 file changed, 4 insertions(+), 122 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index a2bf5efa3e6d..a284a27c0071 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -46,11 +46,8 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; -import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; -import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; -import org.apache.druid.server.coordinator.rules.IntervalBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.IntervalDropRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; @@ -72,6 +69,9 @@ import java.util.List; import java.util.Set; +/** + * + */ public class RunRulesTest { private static final long SERVER_SIZE_10GB = 10L << 30; @@ -315,11 +315,6 @@ private ServerHolder createServerHolder(String name, String tier, LoadQueuePeon return new ServerHolder(createHistorical(name, tier).toImmutableDruidServer(), peon); } - private ServerHolder createServerHolder(String name, String tier, LoadQueuePeon peon, boolean isDecommmisionning) - { - return new ServerHolder(createHistorical(name, tier).toImmutableDruidServer(), peon, isDecommmisionning); - } - private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams( DruidCluster druidCluster, DataSegment segment @@ -487,6 +482,7 @@ public void testRunTwoTiersTierDoesNotExist() @Test public void testRunRuleDoesNotExist() { + EasyMock .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) .andReturn( @@ -528,120 +524,6 @@ public void testRunRuleDoesNotExist() EasyMock.verify(mockPeon); } - @Test - public void testRunRuleWithNoServers() - { - EasyMock.replay(databaseRuleManager, mockPeon); - - DruidCluster emptyCluster = DruidCluster.builder().build(); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(emptyCluster) - .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) - .withSegmentAssignerUsing(loadQueueManager) - .build(); - - CoordinatorRunStats stats = runDutyAndGetStats(params); - - Assert.assertFalse(stats.hasStat(Stats.Segments.ASSIGNED)); - Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); - - Assert.assertEquals(0, emitter.getAlerts().size()); - - StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); - Assert.assertNotNull(segmentAssigner); - Assert.assertEquals(0, segmentAssigner.getBroadcastSegments().size()); - - EasyMock.verify(mockPeon); - } - - @Test - public void testBroadcastLoad() - { - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn( - Lists.newArrayList( - new IntervalBroadcastDistributionRule( - Intervals.of("2012-01-01/2012-01-02") - ), - new IntervalBroadcastDistributionRule( - Intervals.of("2012-01-02/2012-01-03") - ) - ) - ) - .atLeastOnce(); - - mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - - EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); - EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); - EasyMock.replay(databaseRuleManager, mockPeon); - - DruidCluster druidCluster = DruidCluster - .builder() - .add(createServerHolder("serverNorm", "normal", mockPeon)) - .build(); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) - .withSegmentAssignerUsing(loadQueueManager) - .build(); - - CoordinatorRunStats stats = runDutyAndGetStats(params); - - Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.ASSIGNED, "normal", DATASOURCE)); - Assert.assertFalse(stats.hasStat(Stats.Segments.DROPPED)); - - StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); - Assert.assertNotNull(segmentAssigner); - Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); - - EasyMock.verify(mockPeon); - } - - @Test - public void testBroadcastDrop() - { - EasyMock - .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) - .andReturn(Collections.singletonList(new ForeverBroadcastDistributionRule())) - .atLeastOnce(); - - mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); - EasyMock.expectLastCall().atLeastOnce(); - - EasyMock.expect(mockPeon.getSegmentsInQueue()).andReturn(Collections.emptySet()).anyTimes(); - EasyMock.expect(mockPeon.getSegmentsMarkedToDrop()).andReturn(Collections.emptySet()).anyTimes(); - EasyMock.replay(databaseRuleManager, mockPeon); - - DruidServer server = createHistorical("serverNorm", "normal"); - for (DataSegment segment : usedSegments) { - server.addDataSegment(segment); - } - - DruidCluster druidCluster = DruidCluster - .builder() - .add(new ServerHolder(server.toImmutableDruidServer(), mockPeon, true)) - .build(); - - DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) - .withBalancerStrategy(new CostBalancerStrategy(balancerExecutor)) - .withSegmentAssignerUsing(loadQueueManager) - .build(); - - CoordinatorRunStats stats = runDutyAndGetStats(params); - - Assert.assertEquals(24L, stats.getSegmentStat(Stats.Segments.DROPPED, "normal", DATASOURCE)); - Assert.assertFalse(stats.hasStat(Stats.Segments.ASSIGNED)); - - StrategicSegmentAssigner segmentAssigner = params.getSegmentAssigner(); - Assert.assertNotNull(segmentAssigner); - Assert.assertEquals(usedSegments.size(), segmentAssigner.getBroadcastSegments().size()); - - EasyMock.verify(mockPeon); - } - @Test public void testDropRemove() { From 6a85f997fa5de7c61fd99816be009c84a89b5bce Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 14:19:27 -0700 Subject: [PATCH 08/23] Remove debug info. --- .../server/coordination/SegmentLoadDropHandler.java | 2 +- .../lookup/cache/LookupCoordinatorManagerTest.java | 12 +----------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 9f3302de5321..d92e498bbcc5 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -342,7 +342,7 @@ private List getBootstrapSegments() final long fetchRunMillis = stopwatch.millisElapsed(); emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/time", fetchRunMillis)); emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/count", bootstrapSegments.size())); - log.info("Fetching [%d] bootstrap segments completed in [%d] ms", bootstrapSegments.size(), fetchRunMillis); + log.info("Fetching [%d] bootstrap segments completed in [%d]ms.", bootstrapSegments.size(), fetchRunMillis); } return bootstrapSegments; diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index f7ac37994a8b..c94d8e436683 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -35,8 +35,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.core.Event; -import org.apache.druid.java.util.emitter.core.EventMap; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.java.util.http.client.response.SequenceInputStreamResponseHandler; @@ -63,8 +61,6 @@ public class LookupCoordinatorManagerTest { - private static final EmittingLogger log = new EmittingLogger(LookupCoordinatorManagerTest.class); - private final ObjectMapper mapper = new DefaultObjectMapper(); private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createStrictMock(DruidNodeDiscoveryProvider.class); private final LookupNodeDiscovery lookupNodeDiscovery = EasyMock.createStrictMock( @@ -146,13 +142,7 @@ public void setUp() @After public void tearDown() { - List events = SERVICE_EMITTER.getEvents(); - log.info("Events: [%s]", events); - for (Event event : events) { - Assert.assertEquals(EventMap.builder().put("foo", "bar").build(), event.toMap()); - } - Assert.assertEquals(ImmutableList.of(), events); - Assert.assertEquals(0, events.size()); + Assert.assertEquals(0, SERVICE_EMITTER.getEvents().size()); SERVICE_EMITTER.flush(); } From a8316da22fd540802852f1e5c8bac2b4bf3a120b Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 19:03:49 -0700 Subject: [PATCH 09/23] Make the API POST and update log. --- .../apache/druid/client/coordinator/CoordinatorClientImpl.java | 2 +- .../druid/server/coordination/SegmentLoadDropHandler.java | 2 +- .../java/org/apache/druid/server/http/MetadataResource.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 414aee485084..f3ae01f78102 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -169,7 +169,7 @@ public ListenableFuture> fetchBootstrapSegments() final String path = "/druid/coordinator/v1/metadata/bootstrapSegments"; return FutureUtils.transform( client.asyncRequest( - new RequestBuilder(HttpMethod.GET, path), + new RequestBuilder(HttpMethod.POST, path), new InputStreamResponseHandler() ), in -> new JsonParserIterator<>( diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d92e498bbcc5..3c00f1e743ae 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -342,7 +342,7 @@ private List getBootstrapSegments() final long fetchRunMillis = stopwatch.millisElapsed(); emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/time", fetchRunMillis)); emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/count", bootstrapSegments.size())); - log.info("Fetching [%d] bootstrap segments completed in [%d]ms.", bootstrapSegments.size(), fetchRunMillis); + log.info("Fetched [%d] bootstrap segments in [%d]ms.", bootstrapSegments.size(), fetchRunMillis); } return bootstrapSegments; diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index b48bd2acc673..b313a8b00a57 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -475,7 +475,7 @@ public Response getDataSourceInformation( /** * @return all bootstrap segments determined by the coordinator. */ - @GET + @POST @Path("/bootstrapSegments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) From cbd4c4e51dec0d63e069ad8efaf5a7398498a4ff Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Mon, 17 Jun 2024 22:16:31 -0700 Subject: [PATCH 10/23] Fix up UTs. --- .../druid/server/coordinator/DruidCoordinator.java | 4 ++-- .../apache/druid/server/http/MetadataResource.java | 2 +- .../coordinator/CoordinatorClientImplTest.java | 4 ++-- .../druid/server/http/MetadataResourceTest.java | 14 +++++++------- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 1a85d7c608e9..a3da6902ded7 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -324,8 +324,8 @@ public Map getDatasourceToLoadStatus() } /** - * @return the set of broadcast segments. If the coordinator hasn't initialized fully, a retriable exception - * will be thrown. + * @return the set of broadcast segments. If the coordinator hasn't initialized fully and run a cycle of the historical + * duties, a retryable exception will be thrown. */ public Set getBroadcastSegments() { diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index b313a8b00a57..981340e87d92 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -482,7 +482,7 @@ public Response getDataSourceInformation( public Response getBootstrapSegments() { try { - Set broadcastSegments = coordinator.getBroadcastSegments(); + final Set broadcastSegments = coordinator.getBroadcastSegments(); return Response.status(Response.Status.OK).entity(broadcastSegments).build(); } catch (DruidException e) { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index 95fe5e68c07a..c8166c3a5fa3 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -203,7 +203,7 @@ public void test_fetchBootstrapSegments() throws Exception final List segments = ImmutableList.of(segment1, segment2); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/metadata/bootstrapSegments"), + new RequestBuilder(HttpMethod.POST, "/druid/coordinator/v1/metadata/bootstrapSegments"), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), jsonMapper.writeValueAsBytes(segments) @@ -221,7 +221,7 @@ public void test_fetchEmptyBootstrapSegments() throws Exception final List segments = ImmutableList.of(); serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/metadata/bootstrapSegments"), + new RequestBuilder(HttpMethod.POST, "/druid/coordinator/v1/metadata/bootstrapSegments"), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), jsonMapper.writeValueAsBytes(segments) diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index ff3bb2d6654d..89a06c206291 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -487,27 +487,27 @@ public void testGetSegment() } @Test - public void testGetBootstrapSegmentsWithEmptySet() + public void testGetBootstrapSegments() { - Mockito.doReturn(ImmutableSet.of()).when(coordinator).getBroadcastSegments(); + Mockito.doReturn(ImmutableSet.of(segments[0], segments[1])).when(coordinator).getBroadcastSegments(); Response response = metadataResource.getBootstrapSegments(); final List observedSegments = extractResponseList(response); - Assert.assertEquals(0, observedSegments.size()); + Assert.assertEquals(2, observedSegments.size()); } @Test - public void testGetBootstrapSegmentsWithNonEmptySet() + public void testEmptyGetBootstrapSegments() { - Mockito.doReturn(ImmutableSet.of(segments[0], segments[1])).when(coordinator).getBroadcastSegments(); + Mockito.doReturn(ImmutableSet.of()).when(coordinator).getBroadcastSegments(); Response response = metadataResource.getBootstrapSegments(); final List observedSegments = extractResponseList(response); - Assert.assertEquals(2, observedSegments.size()); + Assert.assertEquals(0, observedSegments.size()); } @Test - public void testGetBootstrapSegmentsWithNullSet() + public void testGetBootstrapSegmentsWhenCoordinatorIsUninitialized() { Mockito.doThrow(DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.UNAVAILABLE) From 9086f880d48fe366936c58c8496aebed461e9fff Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 18 Jun 2024 06:58:28 -0700 Subject: [PATCH 11/23] Throw 503 from MetadataResource; clean up exception handling and DruidException. --- .../apache/druid/error/DruidException.java | 4 ---- .../coordinator/CoordinatorClientImpl.java | 2 ++ .../coordination/SegmentLoadDropHandler.java | 4 +--- .../server/coordinator/DruidCoordinator.java | 11 +++++------ .../druid/server/http/MetadataResource.java | 19 +++++++------------ .../coordinator/DruidCoordinatorTest.java | 13 ++----------- .../server/http/MetadataResourceTest.java | 8 ++------ 7 files changed, 19 insertions(+), 42 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index 5e85e164a935..3985d92a0adb 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -356,10 +356,6 @@ public enum Category * Indicates a server-side failure of some sort at runtime */ RUNTIME_FAILURE(500), - /** - * Means that the requested resource could not be found due to a transient unavailability. - */ - UNAVAILABLE(503), /** * A timeout happened */ diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index f3ae01f78102..85153ff09ae8 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -173,6 +173,8 @@ public ListenableFuture> fetchBootstrapSegments() new InputStreamResponseHandler() ), in -> new JsonParserIterator<>( + // We specifically use LoadableDataSegment instead of DataSegment so the callers can correctly load the + // returned set of segments, as the load specs are guaranteed not to be pruned. jsonMapper.getTypeFactory().constructType(LoadableDataSegment.class), Futures.immediateFuture(in), jsonMapper diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 3c00f1e743ae..725426e3a149 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -333,9 +333,7 @@ private List getBootstrapSegments() catch (Exception e) { // By default, we "fail open" when there is any error -- finding the coordinator, or if the API endpoint cannot // be found during rolling upgrades, or even if it's irrecoverable. - log.warn(e, "Error fetching bootstrap segments from the coordinator. " - + "Returning an empty set of bootstrap segments."); - bootstrapSegments = ImmutableList.of(); + log.warn("Error fetching bootstrap segments from the coordinator: [%s]. ", e.getMessage()); } finally { stopwatch.stop(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index a3da6902ded7..0e2dbe316ffd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -37,7 +37,6 @@ import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.curator.discovery.ServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; -import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.DateTimes; @@ -324,15 +323,15 @@ public Map getDatasourceToLoadStatus() } /** - * @return the set of broadcast segments. If the coordinator hasn't initialized fully and run a cycle of the historical - * duties, a retryable exception will be thrown. + * @return the set of broadcast segments as determined by the coordinator at a point in time. + * If the coordinator runs haven't triggered or are delayed, this returned information + * may be stale. */ + @Nullable public Set getBroadcastSegments() { if (segmentAssigner == null) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.UNAVAILABLE) - .build("bootstrap segments not available yet."); + return null; } return segmentAssigner.getBroadcastSegments(); } diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 981340e87d92..806d9467849f 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -481,18 +481,13 @@ public Response getDataSourceInformation( @ResourceFilters(DatasourceResourceFilter.class) public Response getBootstrapSegments() { - try { - final Set broadcastSegments = coordinator.getBroadcastSegments(); - return Response.status(Response.Status.OK).entity(broadcastSegments).build(); - } - catch (DruidException e) { - return ServletResourceUtils.buildErrorResponseFrom(e); - } - catch (Exception e) { - return Response - .serverError() - .entity(ImmutableMap.of("error", "Exception occurred.", "message", Throwables.getRootCause(e).toString())) - .build(); + final Set broadcastSegments = coordinator.getBroadcastSegments(); + if (broadcastSegments == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE) + .entity("No bootstrap segments were found. The coordinator may still be" + + " initializing, please try again later.") + .build(); } + return Response.status(Response.Status.OK).entity(broadcastSegments).build(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index c31a8f8e8970..8c7e0ae14e5f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -39,8 +39,6 @@ import org.apache.druid.curator.CuratorUtils; import org.apache.druid.curator.discovery.LatchableServiceAnnouncer; import org.apache.druid.discovery.DruidLeaderSelector; -import org.apache.druid.error.DruidException; -import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; @@ -76,7 +74,6 @@ import org.apache.druid.server.lookup.cache.LookupCoordinatorManager; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; -import org.hamcrest.MatcherAssert; import org.joda.time.Duration; import org.junit.After; import org.junit.Assert; @@ -270,14 +267,8 @@ public void testCoordinatorRun() throws Exception coordinator.start(); Assert.assertNull(coordinator.getReplicationFactor(dataSegment.getId())); - MatcherAssert.assertThat( - Assert.assertThrows(DruidException.class, () -> coordinator.getBroadcastSegments()), - new DruidExceptionMatcher( - DruidException.Persona.OPERATOR, - DruidException.Category.UNAVAILABLE, - "general" - ).expectMessageIs("bootstrap segments not available yet.") - ); + Assert.assertNull(coordinator.getBroadcastSegments()); + // Wait for this coordinator to become leader leaderAnnouncerLatch.await(); diff --git a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java index 89a06c206291..4d6bbf5929be 100644 --- a/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/MetadataResourceTest.java @@ -507,16 +507,12 @@ public void testEmptyGetBootstrapSegments() } @Test - public void testGetBootstrapSegmentsWhenCoordinatorIsUninitialized() + public void testNullGetBootstrapSegments() { - Mockito.doThrow(DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.UNAVAILABLE) - .build("something went wrong")) - .when(coordinator).getBroadcastSegments(); + Mockito.doReturn(null).when(coordinator).getBroadcastSegments(); Response response = metadataResource.getBootstrapSegments(); Assert.assertEquals(503, response.getStatus()); - Assert.assertEquals("something went wrong", getExceptionMessage(response)); } private List extractResponseList(Response response) From e090da953081f0d20c2a7d1f403bc1535a9db15f Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Tue, 18 Jun 2024 07:40:45 -0700 Subject: [PATCH 12/23] Remove unused logger, add verification of metrics and docs. --- docs/api-reference/legacy-metadata-api.md | 6 +++++- .../coordination/SegmentLoadDropHandler.java | 16 ++++++++-------- .../loading/StrategicSegmentAssigner.java | 2 -- .../SegmentLoadDropHandlerTest.java | 19 ++++++++----------- 4 files changed, 21 insertions(+), 22 deletions(-) diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md index 453159c1a582..5f602e8a4112 100644 --- a/docs/api-reference/legacy-metadata-api.md +++ b/docs/api-reference/legacy-metadata-api.md @@ -176,7 +176,11 @@ Returns a list of all segments, overlapping with any of given intervals, for a d `POST /druid/coordinator/v1/metadata/dataSourceInformation` -Returns information about the specified datasources, including the datasource schema. +Returns information about the specified datasources, including the datasource schema. + +`POST /druid/coordinator/v1/metadata/bootstrapSegments` + +Returns a list of bootstrap segments for all datasources. The returned set will include all broadcast segments if broadcast rules are configured. diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 725426e3a149..e78e7f3bff89 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -228,9 +228,9 @@ public Map getRowCountDistributionPerDataso */ private void loadSegmentsOnStartup() throws IOException { - final List startupSegments = new ArrayList<>(); - startupSegments.addAll(segmentManager.getCachedSegments()); - startupSegments.addAll(getBootstrapSegments()); + final List segmentsOnStartup = new ArrayList<>(); + segmentsOnStartup.addAll(segmentManager.getCachedSegments()); + segmentsOnStartup.addAll(getBootstrapSegments()); final Stopwatch stopwatch = Stopwatch.createStarted(); @@ -244,11 +244,11 @@ private void loadSegmentsOnStartup() throws IOException backgroundSegmentAnnouncer.startAnnouncing(); - final int numSegments = startupSegments.size(); + final int numSegments = segmentsOnStartup.size(); final CountDownLatch latch = new CountDownLatch(numSegments); final AtomicInteger counter = new AtomicInteger(0); final CopyOnWriteArrayList failedSegments = new CopyOnWriteArrayList<>(); - for (final DataSegment segment : startupSegments) { + for (final DataSegment segment : segmentsOnStartup) { loadingExecutor.submit( () -> { try { @@ -302,8 +302,8 @@ private void loadSegmentsOnStartup() throws IOException backgroundSegmentAnnouncer.finishAnnouncing(); } catch (SegmentLoadingException e) { - log.makeAlert(e, "Failed to load startup segments -- likely problem with announcing.") - .addData("numSegments", startupSegments.size()) + log.makeAlert(e, "Failed to load segments on startup -- likely problem with announcing.") + .addData("numSegments", segmentsOnStartup.size()) .emit(); } finally { @@ -312,7 +312,7 @@ private void loadSegmentsOnStartup() throws IOException // At this stage, all tasks have been submitted, send a shutdown command to cleanup any resources alloted // for the bootstrapping function. segmentManager.shutdownBootstrap(); - log.info("Cache load of [%d] startup segments took [%,d]ms.", startupSegments.size(), stopwatch.millisElapsed()); + log.info("Loaded [%d] segments on startup in [%,d]ms.", segmentsOnStartup.size(), stopwatch.millisElapsed()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java index f58e74cdb2aa..9b5d38f198eb 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/StrategicSegmentAssigner.java @@ -22,7 +22,6 @@ import com.google.common.collect.Sets; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.client.DruidServer; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.balancer.BalancerStrategy; @@ -56,7 +55,6 @@ @NotThreadSafe public class StrategicSegmentAssigner implements SegmentActionHandler { - private static final EmittingLogger log = new EmittingLogger(StrategicSegmentAssigner.class); private final SegmentLoadQueueManager loadQueueManager; private final DruidCluster cluster; private final CoordinatorRunStats stats; diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index a935b2bff64e..6a83912df3f9 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentLazyLoadFailCallback; @@ -76,7 +75,7 @@ public class SegmentLoadDropHandlerTest private SegmentLoaderConfig segmentLoaderConfig; private ScheduledExecutorFactory scheduledExecutorFactory; private TestCoordinatorClient coordinatorClient; - private ServiceEmitter emitter; + private StubServiceEmitter serviceEmitter; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -142,8 +141,8 @@ public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) }; coordinatorClient = new TestCoordinatorClient(); - emitter = new StubServiceEmitter(); - EmittingLogger.registerEmitter(emitter); + serviceEmitter = new StubServiceEmitter(); + EmittingLogger.registerEmitter(serviceEmitter); } /** @@ -335,11 +334,10 @@ public void testLoadBootstrapSegments() throws Exception Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + serviceEmitter.verifyValue("bootstrapSegments/fetch/count", expectedBootstrapSegments.size()); + serviceEmitter.verifyEmitted("bootstrapSegments/fetch/time", 1); handler.stop(); - - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); - Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test @@ -360,11 +358,10 @@ public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); + serviceEmitter.verifyValue("bootstrapSegments/fetch/count", 0); + serviceEmitter.verifyEmitted("bootstrapSegments/fetch/time", 1); handler.stop(); - - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); - Assert.assertEquals(1, cacheManager.observedShutdownBootstrapCount.get()); } @Test @@ -642,7 +639,7 @@ private SegmentLoadDropHandler initSegmentLoadDropHandler( scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, - emitter + serviceEmitter ); } From e8e2a21c1d0e663a73ca2e81c5003d801114bbd4 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Thu, 20 Jun 2024 16:46:14 -0700 Subject: [PATCH 13/23] Update error message --- .../java/org/apache/druid/server/http/MetadataResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 806d9467849f..8903d088f9fd 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -484,7 +484,7 @@ public Response getBootstrapSegments() final Set broadcastSegments = coordinator.getBroadcastSegments(); if (broadcastSegments == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity("No bootstrap segments were found. The coordinator may still be" + .entity("Could not determine bootstrap segments. The coordinator is likely still" + " initializing, please try again later.") .build(); } From 05dab3a5813758134f1bea992bf56730443dd55b Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Fri, 21 Jun 2024 08:22:46 -0700 Subject: [PATCH 14/23] Update server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java Co-authored-by: Kashif Faraz --- .../druid/server/coordination/SegmentLoadDropHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index e78e7f3bff89..77f4cb1805e9 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -289,7 +289,7 @@ private void loadSegmentsOnStartup() throws IOException latch.await(); if (failedSegments.size() > 0) { - log.makeAlert("%,d errors seen while loading segments on startup", failedSegments.size()) + log.makeAlert("[%,d] errors seen while loading segments on startup", failedSegments.size()) .addData("failedSegments", failedSegments) .emit(); } From 98e2fa68fa24c5e29b2307b3cbab5c3a25c77302 Mon Sep 17 00:00:00 2001 From: Abhishek Radhakrishnan Date: Fri, 21 Jun 2024 08:25:52 -0700 Subject: [PATCH 15/23] Apply suggestions from code review Co-authored-by: Kashif Faraz --- .../druid/server/coordination/SegmentLoadDropHandler.java | 4 ++-- .../org/apache/druid/server/coordinator/DruidCoordinator.java | 2 +- .../java/org/apache/druid/server/http/MetadataResource.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 77f4cb1805e9..38277e9b6f14 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -338,8 +338,8 @@ private List getBootstrapSegments() finally { stopwatch.stop(); final long fetchRunMillis = stopwatch.millisElapsed(); - emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/time", fetchRunMillis)); - emitter.emit(new ServiceMetricEvent.Builder().setMetric("bootstrapSegments/fetch/count", bootstrapSegments.size())); + emitter.emit(new ServiceMetricEvent.Builder().setMetric("segment/bootstrap/time", fetchRunMillis)); + emitter.emit(new ServiceMetricEvent.Builder().setMetric("segment/bootstrap/count", bootstrapSegments.size())); log.info("Fetched [%d] bootstrap segments in [%d]ms.", bootstrapSegments.size(), fetchRunMillis); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 0e2dbe316ffd..226689ddc47d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -324,7 +324,7 @@ public Map getDatasourceToLoadStatus() /** * @return the set of broadcast segments as determined by the coordinator at a point in time. - * If the coordinator runs haven't triggered or are delayed, this returned information + * If the coordinator runs haven't triggered or are delayed, this information * may be stale. */ @Nullable diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index 8903d088f9fd..e37968db135d 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -484,8 +484,8 @@ public Response getBootstrapSegments() final Set broadcastSegments = coordinator.getBroadcastSegments(); if (broadcastSegments == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity("Could not determine bootstrap segments. The coordinator is likely still" - + " initializing, please try again later.") + .entity("Bootstrap segments are not initialized yet. + + " Please ensure that the Coordinator duties are running and try again.") .build(); } return Response.status(Response.Status.OK).entity(broadcastSegments).build(); From 3a26f0fdae84a2b4b24d4b75468a23d4c85131fd Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 08:37:23 -0700 Subject: [PATCH 16/23] Adjust test metric expectations with the rename. --- .../org/apache/druid/server/http/MetadataResource.java | 2 +- .../server/coordination/SegmentLoadDropHandlerTest.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index e37968db135d..a1cccd2b784a 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -484,7 +484,7 @@ public Response getBootstrapSegments() final Set broadcastSegments = coordinator.getBroadcastSegments(); if (broadcastSegments == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE) - .entity("Bootstrap segments are not initialized yet. + .entity("Bootstrap segments are not initialized yet." + " Please ensure that the Coordinator duties are running and try again.") .build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 6a83912df3f9..9fe04d60d5bf 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -334,8 +334,8 @@ public void testLoadBootstrapSegments() throws Exception Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegments); Assert.assertEquals(expectedBootstrapSegments, cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - serviceEmitter.verifyValue("bootstrapSegments/fetch/count", expectedBootstrapSegments.size()); - serviceEmitter.verifyEmitted("bootstrapSegments/fetch/time", 1); + serviceEmitter.verifyValue("segment/bootstrap/count", expectedBootstrapSegments.size()); + serviceEmitter.verifyEmitted("segment/bootstrap/time", 1); handler.stop(); } @@ -358,8 +358,8 @@ public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegments); Assert.assertEquals(ImmutableList.of(), cacheManager.observedBootstrapSegmentsLoadedIntoPageCache); - serviceEmitter.verifyValue("bootstrapSegments/fetch/count", 0); - serviceEmitter.verifyEmitted("bootstrapSegments/fetch/time", 1); + serviceEmitter.verifyValue("segment/bootstrap/count", 0); + serviceEmitter.verifyEmitted("segment/bootstrap/time", 1); handler.stop(); } From 860d81366ada30519c87efa26867ea7bdce6b9da Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 09:33:36 -0700 Subject: [PATCH 17/23] Add BootstrapSegmentResponse container in the response for future extensibility. --- .../client/coordinator/CoordinatorClient.java | 6 +++--- .../coordinator/CoordinatorClientImpl.java | 18 ++++++++++-------- .../coordination/SegmentLoadDropHandler.java | 7 +++++-- .../coordinator/CoordinatorClientImplTest.java | 12 ++++++++++-- .../coordinator/NoopCoordinatorClient.java | 4 ++-- .../coordination/TestCoordinatorClient.java | 8 +++++--- 6 files changed, 35 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index 3113d7ea4f5a..f7f0803b45be 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -21,9 +21,9 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -60,10 +60,10 @@ public interface CoordinatorClient ListenableFuture> fetchDataSourceInformation(Set datasources); /** - * Fetch bootstrap segments from the coordiantor. The results must be streamed back to the caller as the + * Fetch bootstrap segments from the coordinator. The results must be streamed back to the caller as the * result set can be large. */ - ListenableFuture> fetchBootstrapSegments(); + ListenableFuture fetchBootstrapSegments(); /** * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 85153ff09ae8..9742cda91cee 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -29,13 +29,13 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.server.coordination.LoadableDataSegment; import org.apache.druid.timeline.DataSegment; @@ -164,7 +164,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture> fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { final String path = "/druid/coordinator/v1/metadata/bootstrapSegments"; return FutureUtils.transform( @@ -172,12 +172,14 @@ public ListenableFuture> fetchBootstrapSegments() new RequestBuilder(HttpMethod.POST, path), new InputStreamResponseHandler() ), - in -> new JsonParserIterator<>( - // We specifically use LoadableDataSegment instead of DataSegment so the callers can correctly load the - // returned set of segments, as the load specs are guaranteed not to be pruned. - jsonMapper.getTypeFactory().constructType(LoadableDataSegment.class), - Futures.immediateFuture(in), - jsonMapper + in -> new BootstrapSegmentResponse( + new JsonParserIterator<>( + // We specifically use LoadableDataSegment instead of DataSegment so the callers can correctly load the + // returned set of segments, as the load specs are guaranteed not to be pruned. + jsonMapper.getTypeFactory().constructType(LoadableDataSegment.class), + Futures.immediateFuture(in), + jsonMapper + ) ) ); } diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 38277e9b6f14..e1fc2f62b1eb 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -42,6 +42,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; @@ -326,8 +327,10 @@ private List getBootstrapSegments() List bootstrapSegments = new ArrayList<>(); - try (final CloseableIterator iterator = - FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true)) { + try { + final BootstrapSegmentResponse response = + FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true); + final CloseableIterator iterator = response.getBootstrapSegmentsIterator(); bootstrapSegments = ImmutableList.copyOf(iterator); } catch (Exception e) { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index c8166c3a5fa3..545fb4e5d2a4 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; @@ -31,6 +32,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.DataSourceInformation; @@ -209,9 +211,12 @@ public void test_fetchBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(segments) ); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + Assert.assertNotNull(response); + Assert.assertEquals( segments, - ImmutableList.copyOf(coordinatorClient.fetchBootstrapSegments().get()) + ImmutableList.copyOf(response.get().getBootstrapSegmentsIterator()) ); } @@ -227,9 +232,12 @@ public void test_fetchEmptyBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(segments) ); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + Assert.assertNotNull(response); + Assert.assertEquals( segments, - ImmutableList.copyOf(coordinatorClient.fetchBootstrapSegments().get()) + ImmutableList.copyOf(response.get().getBootstrapSegmentsIterator()) ); } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index bd42b337e4a9..0f648684e850 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -21,9 +21,9 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.ImmutableSegmentLoadInfo; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -64,7 +64,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture> fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { throw new UnsupportedOperationException(); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java index 5389e20840b7..234dbdfdd68c 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java @@ -23,7 +23,7 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.timeline.DataSegment; import java.util.HashSet; @@ -44,8 +44,10 @@ class TestCoordinatorClient extends NoopCoordinatorClient } @Override - public ListenableFuture> fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { - return Futures.immediateFuture(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())); + return Futures.immediateFuture( + new BootstrapSegmentResponse(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())) + ); } } From cf525fa071f011bab51e38ed3caaf84e775c9e79 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 09:50:38 -0700 Subject: [PATCH 18/23] Rename to BootstrapSegmentsInfo for internal consistency. --- .../druid/client/BootstrapSegmentsInfo.java | 39 +++++++++++++++++++ .../client/coordinator/CoordinatorClient.java | 4 +- .../coordinator/CoordinatorClientImpl.java | 6 +-- .../coordination/SegmentLoadDropHandler.java | 6 +-- .../CoordinatorClientImplTest.java | 10 ++--- .../coordinator/NoopCoordinatorClient.java | 4 +- .../coordination/TestCoordinatorClient.java | 6 +-- 7 files changed, 57 insertions(+), 18 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java diff --git a/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java new file mode 100644 index 000000000000..bb2de8a41c3f --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java @@ -0,0 +1,39 @@ +/* + * 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.druid.client; + +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.timeline.DataSegment; + +public class BootstrapSegmentsInfo +{ + private final CloseableIterator iterator; + + public BootstrapSegmentsInfo(final CloseableIterator iteartor) + { + this.iterator = iteartor; + } + + public CloseableIterator getIterator() + { + return iterator; + } + +} diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index f7f0803b45be..e8ad1590b490 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -20,10 +20,10 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -63,7 +63,7 @@ public interface CoordinatorClient * Fetch bootstrap segments from the coordinator. The results must be streamed back to the caller as the * result set can be large. */ - ListenableFuture fetchBootstrapSegments(); + ListenableFuture fetchBootstrapSegments(); /** * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 9742cda91cee..b45b0a179de1 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.common.guava.FutureUtils; @@ -35,7 +36,6 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.server.coordination.LoadableDataSegment; import org.apache.druid.timeline.DataSegment; @@ -164,7 +164,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { final String path = "/druid/coordinator/v1/metadata/bootstrapSegments"; return FutureUtils.transform( @@ -172,7 +172,7 @@ public ListenableFuture fetchBootstrapSegments() new RequestBuilder(HttpMethod.POST, path), new InputStreamResponseHandler() ), - in -> new BootstrapSegmentResponse( + in -> new BootstrapSegmentsInfo( new JsonParserIterator<>( // We specifically use LoadableDataSegment instead of DataSegment so the callers can correctly load the // returned set of segments, as the load specs are guaranteed not to be pruned. diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index e1fc2f62b1eb..ac59848a088a 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.ManageLifecycle; @@ -42,7 +43,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; @@ -328,9 +328,9 @@ private List getBootstrapSegments() List bootstrapSegments = new ArrayList<>(); try { - final BootstrapSegmentResponse response = + final BootstrapSegmentsInfo response = FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true); - final CloseableIterator iterator = response.getBootstrapSegmentsIterator(); + final CloseableIterator iterator = response.getIterator(); bootstrapSegments = ImmutableList.copyOf(iterator); } catch (Exception e) { diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index 545fb4e5d2a4..ed7d5362fba6 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; @@ -32,7 +33,6 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.DataSourceInformation; @@ -211,12 +211,12 @@ public void test_fetchBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(segments) ); - final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); Assert.assertEquals( segments, - ImmutableList.copyOf(response.get().getBootstrapSegmentsIterator()) + ImmutableList.copyOf(response.get().getIterator()) ); } @@ -232,12 +232,12 @@ public void test_fetchEmptyBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(segments) ); - final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); Assert.assertEquals( segments, - ImmutableList.copyOf(response.get().getBootstrapSegmentsIterator()) + ImmutableList.copyOf(response.get().getIterator()) ); } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 0f648684e850..e9c5baaec8d3 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -20,10 +20,10 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.segment.metadata.DataSourceInformation; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -64,7 +64,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { throw new UnsupportedOperationException(); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java index 234dbdfdd68c..9f0d5b970f44 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java @@ -21,9 +21,9 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.java.util.common.CloseableIterators; -import org.apache.druid.segment.BootstrapSegmentResponse; import org.apache.druid.timeline.DataSegment; import java.util.HashSet; @@ -44,10 +44,10 @@ class TestCoordinatorClient extends NoopCoordinatorClient } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { return Futures.immediateFuture( - new BootstrapSegmentResponse(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())) + new BootstrapSegmentsInfo(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())) ); } } From 04793b6f0e8a159d1acb6fecd9ca8fe05b881076 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 09:52:16 -0700 Subject: [PATCH 19/23] Remove unused log. --- .../apache/druid/client/coordinator/CoordinatorClientImpl.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index b45b0a179de1..eb2e75c650c2 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -29,7 +29,6 @@ import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.response.BytesFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamResponseHandler; import org.apache.druid.query.SegmentDescriptor; @@ -48,7 +47,6 @@ public class CoordinatorClientImpl implements CoordinatorClient { - private static final Logger log = new Logger(CoordinatorClientImpl.class); private final ServiceClient client; private final ObjectMapper jsonMapper; From b8680d83137c8f8a3b6e8bac98e2e06640df39ab Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 12:20:03 -0700 Subject: [PATCH 20/23] Use a member variable for broadcast segments instead of segmentAssigner. --- docs/api-reference/legacy-metadata-api.md | 2 +- .../druid/server/coordinator/DruidCoordinator.java | 14 +++++--------- .../coordinator/DruidCoordinatorRuntimeParams.java | 6 ++++++ 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md index 5f602e8a4112..a90087a4daf9 100644 --- a/docs/api-reference/legacy-metadata-api.md +++ b/docs/api-reference/legacy-metadata-api.md @@ -180,7 +180,7 @@ Returns information about the specified datasources, including the datasource sc `POST /druid/coordinator/v1/metadata/bootstrapSegments` -Returns a list of bootstrap segments for all datasources. The returned set will include all broadcast segments if broadcast rules are configured. +Returns information about bootstrap segments for all datasources. The returned set will include all broadcast segments if broadcast rules are configured. diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 226689ddc47d..23526521b271 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -177,9 +177,9 @@ public class DruidCoordinator /** * Used to determine broadcast segments. Similar to {@link #segmentReplicationStatus}, this might contain stale - * information if the coordinator runs are delayed. + * information if the Coordinator duties haven't run or are delayed. */ - private volatile StrategicSegmentAssigner segmentAssigner = null; + private volatile Set broadcastSegments = null; public static final String HISTORICAL_MANAGEMENT_DUTIES_DUTY_GROUP = "HistoricalManagementDuties"; private static final String METADATA_STORE_MANAGEMENT_DUTIES_DUTY_GROUP = "MetadataStoreManagementDuties"; @@ -324,16 +324,12 @@ public Map getDatasourceToLoadStatus() /** * @return the set of broadcast segments as determined by the coordinator at a point in time. - * If the coordinator runs haven't triggered or are delayed, this information - * may be stale. + * If the coordinator runs haven't triggered or are delayed, this information may be stale. */ @Nullable public Set getBroadcastSegments() { - if (segmentAssigner == null) { - return null; - } - return segmentAssigner.getBroadcastSegments(); + return broadcastSegments; } @Nullable @@ -819,7 +815,7 @@ private class UpdateReplicationStatus implements CoordinatorDuty @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - segmentAssigner = params.getSegmentAssigner(); + broadcastSegments = params.getBroadcastSegments(); segmentReplicationStatus = params.getSegmentReplicationStatus(); // Collect stats for unavailable and under-replicated segments diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index 5548636b0000..ebdbd4f500e8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -123,6 +123,12 @@ public SegmentReplicationStatus getSegmentReplicationStatus() return segmentAssigner == null ? null : segmentAssigner.getReplicationStatus(); } + @Nullable + public Set getBroadcastSegments() + { + return segmentAssigner == null ? null : segmentAssigner.getBroadcastSegments(); + } + public StrategicSegmentAssigner getSegmentAssigner() { return segmentAssigner; From 01b81d84db729502e3de21a83ceb949e0b072466 Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 12:41:56 -0700 Subject: [PATCH 21/23] Minor cleanup --- .../druid/server/coordination/SegmentLoadDropHandler.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index ac59848a088a..d04ddc5fdb98 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -330,8 +330,7 @@ private List getBootstrapSegments() try { final BootstrapSegmentsInfo response = FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true); - final CloseableIterator iterator = response.getIterator(); - bootstrapSegments = ImmutableList.copyOf(iterator); + bootstrapSegments = ImmutableList.copyOf(response.getIterator()); } catch (Exception e) { // By default, we "fail open" when there is any error -- finding the coordinator, or if the API endpoint cannot From a6ad9fe2979eff6a66f02a17cbe9e81cd0b5939a Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 16:47:14 -0700 Subject: [PATCH 22/23] Add test for loadable bootstrap segments and clarify comment. --- .../druid/client/BootstrapSegmentsInfo.java | 3 + .../coordinator/CoordinatorClientImpl.java | 5 +- .../coordination/SegmentLoadDropHandler.java | 1 - .../server/coordinator/DruidCoordinator.java | 1 - .../CoordinatorClientImplTest.java | 82 ++++++++++++++----- 5 files changed, 67 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java index bb2de8a41c3f..e9990c28e9ee 100644 --- a/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java +++ b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java @@ -22,6 +22,9 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.timeline.DataSegment; +/** + * A container that holds information about bootstrap segments. + */ public class BootstrapSegmentsInfo { private final CloseableIterator iterator; diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index eb2e75c650c2..8cf5acdf429f 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -172,8 +172,9 @@ public ListenableFuture fetchBootstrapSegments() ), in -> new BootstrapSegmentsInfo( new JsonParserIterator<>( - // We specifically use LoadableDataSegment instead of DataSegment so the callers can correctly load the - // returned set of segments, as the load specs are guaranteed not to be pruned. + // Some servers, like the Broker, may have PruneLoadSpec set to true for optimization reasons. + // We specifically use LoadableDataSegment here instead of DataSegment so the callers can still correctly + // load the bootstrap segments, as the load specs are guaranteed not to be pruned. jsonMapper.getTypeFactory().constructType(LoadableDataSegment.class), Futures.immediateFuture(in), jsonMapper diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d04ddc5fdb98..73a5faa6d5c0 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -39,7 +39,6 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 23526521b271..991dcda86ffa 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -83,7 +83,6 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.SegmentReplicaCount; import org.apache.druid.server.coordinator.loading.SegmentReplicationStatus; -import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index ed7d5362fba6..a254dc4482d7 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -25,8 +25,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; +import com.google.inject.Injector; import org.apache.druid.client.BootstrapSegmentsInfo; import org.apache.druid.client.ImmutableSegmentLoadInfo; +import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.initialization.CoreInjectorBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -39,6 +42,7 @@ import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.PruneLoadSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; @@ -60,6 +64,24 @@ public class CoordinatorClientImplTest private MockServiceClient serviceClient; private CoordinatorClient coordinatorClient; + private static final DataSegment SEGMENT1 = DataSegment.builder() + .dataSource("xyz") + .interval(Intervals.of("1000/2000")) + .version("1") + .loadSpec(ImmutableMap.of("type", "local", "loc", "foo")) + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(); + + private static final DataSegment SEGMENT2 = DataSegment.builder() + .dataSource("xyz") + .interval(Intervals.of("2000/3000")) + .version("1") + .loadSpec(ImmutableMap.of("type", "local", "loc", "bar")) + .shardSpec(new NumberedShardSpec(0, 1)) + .size(1) + .build(); + @Before public void setup() { @@ -186,38 +208,56 @@ public void test_fetchUsedSegments() throws Exception @Test public void test_fetchBootstrapSegments() throws Exception { - final DataSegment segment1 = - DataSegment.builder() - .dataSource("xyz") - .interval(Intervals.of("1000/2000")) - .version("1") - .shardSpec(new NumberedShardSpec(0, 1)) - .size(1) - .build(); - final DataSegment segment2 = - DataSegment.builder() - .dataSource("xyz") - .interval(Intervals.of("2000/3000")) - .version("1") - .shardSpec(new NumberedShardSpec(0, 1)) - .size(1) - .build(); - final List segments = ImmutableList.of(segment1, segment2); + final List expectedSegments = ImmutableList.of(SEGMENT1, SEGMENT2); serviceClient.expectAndRespond( new RequestBuilder(HttpMethod.POST, "/druid/coordinator/v1/metadata/bootstrapSegments"), HttpResponseStatus.OK, ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), - jsonMapper.writeValueAsBytes(segments) + jsonMapper.writeValueAsBytes(expectedSegments) ); final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); - Assert.assertEquals( - segments, - ImmutableList.copyOf(response.get().getIterator()) + final ImmutableList observedDataSegments = ImmutableList.copyOf(response.get().getIterator()); + for (int idx = 0; idx < expectedSegments.size(); idx++) { + Assert.assertEquals(expectedSegments.get(idx).getLoadSpec(), observedDataSegments.get(idx).getLoadSpec()); + } + } + + /** + * Set up a Guice injector with PruneLoadSpec set to true. This test verifies that the bootstrap segments API + * always return segments with load specs present, ensuring they can be loaded anywhere. + */ + @Test + public void test_fetchBootstrapSegmentsAreLoadableWhenPruneLoadSpecIsEnabled() throws Exception + { + final List expectedSegments = ImmutableList.of(SEGMENT1, SEGMENT2); + + // Set up a coordinator client with PruneLoadSpec set to true in the injector + final Injector injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build()) + .addModule(binder -> binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(true)) + .build(); + + final ObjectMapper objectMapper = injector.getInstance(ObjectMapper.class); + final CoordinatorClient coordinatorClient = new CoordinatorClientImpl(serviceClient, objectMapper); + + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.POST, "/druid/coordinator/v1/metadata/bootstrapSegments"), + HttpResponseStatus.OK, + ImmutableMap.of(HttpHeaders.CONTENT_TYPE, MediaType.APPLICATION_JSON), + objectMapper.writeValueAsBytes(expectedSegments) ); + + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + Assert.assertNotNull(response); + + final ImmutableList observedDataSegments = ImmutableList.copyOf(response.get().getIterator()); + Assert.assertEquals(expectedSegments, observedDataSegments); + for (int idx = 0; idx < expectedSegments.size(); idx++) { + Assert.assertEquals(expectedSegments.get(idx).getLoadSpec(), observedDataSegments.get(idx).getLoadSpec()); + } } @Test From 43b8702fa177f19070add6814e3343adacfb3a6d Mon Sep 17 00:00:00 2001 From: Abhishek Balaji Radhakrishnan Date: Fri, 21 Jun 2024 22:10:59 -0700 Subject: [PATCH 23/23] Review suggestions. --- docs/api-reference/legacy-metadata-api.md | 2 +- ...pSegmentsInfo.java => BootstrapSegmentsResponse.java} | 9 +++------ .../druid/client/coordinator/CoordinatorClient.java | 4 ++-- .../druid/client/coordinator/CoordinatorClientImpl.java | 6 +++--- .../server/coordination/SegmentLoadDropHandler.java | 4 ++-- .../druid/server/coordinator/DruidCoordinator.java | 6 +++--- .../client/coordinator/CoordinatorClientImplTest.java | 8 ++++---- .../druid/client/coordinator/NoopCoordinatorClient.java | 4 ++-- .../druid/server/coordination/TestCoordinatorClient.java | 6 +++--- 9 files changed, 23 insertions(+), 26 deletions(-) rename server/src/main/java/org/apache/druid/client/{BootstrapSegmentsInfo.java => BootstrapSegmentsResponse.java} (83%) diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md index a90087a4daf9..d22be18a7ec9 100644 --- a/docs/api-reference/legacy-metadata-api.md +++ b/docs/api-reference/legacy-metadata-api.md @@ -180,7 +180,7 @@ Returns information about the specified datasources, including the datasource sc `POST /druid/coordinator/v1/metadata/bootstrapSegments` -Returns information about bootstrap segments for all datasources. The returned set will include all broadcast segments if broadcast rules are configured. +Returns information about bootstrap segments for all datasources. The returned set includes all broadcast segments if broadcast rules are configured. diff --git a/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsResponse.java similarity index 83% rename from server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java rename to server/src/main/java/org/apache/druid/client/BootstrapSegmentsResponse.java index e9990c28e9ee..7d1569262427 100644 --- a/server/src/main/java/org/apache/druid/client/BootstrapSegmentsInfo.java +++ b/server/src/main/java/org/apache/druid/client/BootstrapSegmentsResponse.java @@ -22,16 +22,13 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.timeline.DataSegment; -/** - * A container that holds information about bootstrap segments. - */ -public class BootstrapSegmentsInfo +public class BootstrapSegmentsResponse { private final CloseableIterator iterator; - public BootstrapSegmentsInfo(final CloseableIterator iteartor) + public BootstrapSegmentsResponse(final CloseableIterator iterator) { - this.iterator = iteartor; + this.iterator = iterator; } public CloseableIterator getIterator() diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index e8ad1590b490..fdf16b2ac505 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -20,7 +20,7 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; @@ -63,7 +63,7 @@ public interface CoordinatorClient * Fetch bootstrap segments from the coordinator. The results must be streamed back to the caller as the * result set can be large. */ - ListenableFuture fetchBootstrapSegments(); + ListenableFuture fetchBootstrapSegments(); /** * Returns a new instance backed by a ServiceClient which follows the provided retryPolicy diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 8cf5acdf429f..4c795c9dbd47 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.client.JsonParserIterator; import org.apache.druid.common.guava.FutureUtils; @@ -162,7 +162,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { final String path = "/druid/coordinator/v1/metadata/bootstrapSegments"; return FutureUtils.transform( @@ -170,7 +170,7 @@ public ListenableFuture fetchBootstrapSegments() new RequestBuilder(HttpMethod.POST, path), new InputStreamResponseHandler() ), - in -> new BootstrapSegmentsInfo( + in -> new BootstrapSegmentsResponse( new JsonParserIterator<>( // Some servers, like the Broker, may have PruneLoadSpec set to true for optimization reasons. // We specifically use LoadableDataSegment here instead of DataSegment so the callers can still correctly diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 73a5faa6d5c0..4a9086ab5727 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -29,7 +29,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.ManageLifecycle; @@ -327,7 +327,7 @@ private List getBootstrapSegments() List bootstrapSegments = new ArrayList<>(); try { - final BootstrapSegmentsInfo response = + final BootstrapSegmentsResponse response = FutureUtils.getUnchecked(coordinatorClient.fetchBootstrapSegments(), true); bootstrapSegments = ImmutableList.copyOf(response.getIterator()); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 991dcda86ffa..36cfac8089c4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -175,8 +175,8 @@ public class DruidCoordinator private volatile SegmentReplicationStatus segmentReplicationStatus = null; /** - * Used to determine broadcast segments. Similar to {@link #segmentReplicationStatus}, this might contain stale - * information if the Coordinator duties haven't run or are delayed. + * Set of broadcast segments determined in the latest coordinator run of the {@link RunRules} duty. + * This might contain stale information if the Coordinator duties haven't run or are delayed. */ private volatile Set broadcastSegments = null; @@ -322,7 +322,7 @@ public Map getDatasourceToLoadStatus() } /** - * @return the set of broadcast segments as determined by the coordinator at a point in time. + * @return Set of broadcast segments determined by the latest run of the {@link RunRules} duty. * If the coordinator runs haven't triggered or are delayed, this information may be stale. */ @Nullable diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index a254dc4482d7..3dde6dda149b 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -26,7 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Injector; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.guice.StartupInjectorBuilder; import org.apache.druid.initialization.CoreInjectorBuilder; @@ -217,7 +217,7 @@ public void test_fetchBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(expectedSegments) ); - final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); final ImmutableList observedDataSegments = ImmutableList.copyOf(response.get().getIterator()); @@ -250,7 +250,7 @@ public void test_fetchBootstrapSegmentsAreLoadableWhenPruneLoadSpecIsEnabled() t objectMapper.writeValueAsBytes(expectedSegments) ); - final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); final ImmutableList observedDataSegments = ImmutableList.copyOf(response.get().getIterator()); @@ -272,7 +272,7 @@ public void test_fetchEmptyBootstrapSegments() throws Exception jsonMapper.writeValueAsBytes(segments) ); - final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); + final ListenableFuture response = coordinatorClient.fetchBootstrapSegments(); Assert.assertNotNull(response); Assert.assertEquals( diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index e9c5baaec8d3..5aee343a851b 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -20,7 +20,7 @@ package org.apache.druid.client.coordinator; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.ImmutableSegmentLoadInfo; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; @@ -64,7 +64,7 @@ public ListenableFuture> fetchDataSourceInformation( } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { throw new UnsupportedOperationException(); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java index 9f0d5b970f44..9f297ddd39eb 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/server/coordination/TestCoordinatorClient.java @@ -21,7 +21,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.client.BootstrapSegmentsInfo; +import org.apache.druid.client.BootstrapSegmentsResponse; import org.apache.druid.client.coordinator.NoopCoordinatorClient; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.timeline.DataSegment; @@ -44,10 +44,10 @@ class TestCoordinatorClient extends NoopCoordinatorClient } @Override - public ListenableFuture fetchBootstrapSegments() + public ListenableFuture fetchBootstrapSegments() { return Futures.immediateFuture( - new BootstrapSegmentsInfo(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())) + new BootstrapSegmentsResponse(CloseableIterators.withEmptyBaggage(bootstrapSegments.iterator())) ); } }