From fb115644593d96183df5cdca97812b87bef287e8 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 9 May 2019 17:01:59 +0200 Subject: [PATCH 01/15] Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource --- .../druid/java/util/common/logger/Logger.java | 5 + .../apache/druid/utils/CollectionUtils.java | 11 + docs/content/ingestion/delete-data.md | 3 +- docs/content/operations/api-reference.md | 26 +- .../MaterializedViewSupervisor.java | 4 +- .../druid/indexing/common/task/KillTask.java | 4 + .../actions/SegmentListActionsTest.java | 2 +- .../apache/druid/client/DruidDataSource.java | 21 +- .../org/apache/druid/client/DruidServer.java | 2 - .../client/ImmutableDruidDataSource.java | 2 +- .../druid/client/ImmutableDruidServer.java | 21 +- .../client/indexing/ClientCompactQuery.java | 6 +- .../ClientCompactQueryTuningConfig.java | 14 +- .../client/indexing/ClientKillQuery.java | 3 + .../druid/client/indexing/ClientQuery.java | 7 +- .../metadata/MetadataSegmentManager.java | 119 +- .../metadata/SQLMetadataRuleManager.java | 2 +- .../metadata/SQLMetadataSegmentManager.java | 1053 +++++++++++------ .../metadata/UnknownSegmentIdException.java | 18 +- .../org/apache/druid/server/JettyUtils.java | 40 + .../coordinator/CoordinatorDynamicConfig.java | 203 ++-- .../coordinator/CostBalancerStrategy.java | 24 +- .../DiskNormalizedCostBalancerStrategy.java | 4 +- .../server/coordinator/DruidCluster.java | 29 +- .../server/coordinator/DruidCoordinator.java | 114 +- ...ruidCoordinatorCleanupPendingSegments.java | 12 +- .../DruidCoordinatorRuntimeParams.java | 140 ++- .../coordinator/ReservoirSegmentSampler.java | 2 +- .../coordinator/SegmentReplicantLookup.java | 2 +- .../helper/DruidCoordinatorBalancer.java | 19 +- .../DruidCoordinatorCleanupOvershadowed.java | 52 +- .../DruidCoordinatorCleanupUnneeded.java | 13 +- .../helper/DruidCoordinatorLogger.java | 13 +- .../helper/DruidCoordinatorRuleRunner.java | 18 +- .../DruidCoordinatorSegmentCompactor.java | 2 +- .../DruidCoordinatorSegmentInfoLoader.java | 32 +- .../helper/DruidCoordinatorSegmentKiller.java | 45 +- .../server/coordinator/rules/DropRule.java | 2 +- .../druid/server/coordinator/rules/Rule.java | 8 +- .../server/http/CoordinatorResource.java | 2 +- .../server/http/DataSourcesResource.java | 435 ++++--- .../druid/server/http/MetadataResource.java | 138 +-- .../client/ImmutableDruidServerTests.java | 37 + .../client/indexing/ClientKillQueryTest.java | 12 +- .../SQLMetadataSegmentManagerTest.java | 997 ++++++---------- .../CoordinatorRuntimeParamsTestHelpers.java | 42 + .../coordinator/CostBalancerStrategyTest.java | 3 +- .../CuratorDruidCoordinatorTest.java | 3 +- ...iskNormalizedCostBalancerStrategyTest.java | 3 +- .../coordinator/DruidClusterBuilder.java | 59 + .../server/coordinator/DruidClusterTest.java | 34 +- .../DruidCoordinatorBalancerProfiler.java | 137 +-- .../DruidCoordinatorBalancerTest.java | 27 +- .../DruidCoordinatorBalancerTester.java | 9 - .../DruidCoordinatorRuleRunnerTest.java | 947 +++++---------- .../coordinator/DruidCoordinatorTest.java | 30 +- .../ReservoirSegmentSamplerTest.java | 9 +- .../cost/CachingCostBalancerStrategyTest.java | 2 +- ...uidCoordinatorCleanupOvershadowedTest.java | 37 +- .../DruidCoordinatorSegmentCompactorTest.java | 5 +- .../DruidCoordinatorSegmentKillerTest.java | 39 +- .../rules/BroadcastDistributionRuleTest.java | 148 ++- .../coordinator/rules/LoadRuleTest.java | 590 +++------ .../http/CoordinatorDynamicConfigTest.java | 34 +- .../server/http/DataSourcesResourceTest.java | 775 +++++------- .../sql/calcite/schema/SystemSchema.java | 2 +- .../sql/calcite/schema/DruidSchemaTest.java | 3 +- .../sql/calcite/schema/SystemSchemaTest.java | 6 +- web-console/README.md | 2 +- 69 files changed, 3051 insertions(+), 3613 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java index d5aa17ada9c5..569537a18a88 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java +++ b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java @@ -123,6 +123,11 @@ public void error(Throwable t, String message, Object... formatArgs) log.error(StringUtils.nonStrictFormat(message, formatArgs), t); } + public void assertionError(String message, Object... formatArgs) + { + log.error("ASSERTION_ERROR: " + message, formatArgs); + } + public void wtf(String message, Object... formatArgs) { log.error(StringUtils.nonStrictFormat("WTF?!: " + message, formatArgs), new Exception()); diff --git a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java index 6b4d3cc3df41..06e6129906e9 100644 --- a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java +++ b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java @@ -19,10 +19,14 @@ package org.apache.druid.utils; +import com.google.common.collect.Iterables; + import java.util.AbstractCollection; import java.util.Collection; +import java.util.Comparator; import java.util.Iterator; import java.util.Spliterator; +import java.util.TreeSet; import java.util.function.Supplier; import java.util.stream.Stream; @@ -68,5 +72,12 @@ public int size() }; } + public static TreeSet newTreeSet(Comparator comparator, Iterable elements) + { + TreeSet set = new TreeSet<>(comparator); + Iterables.addAll(set, elements); + return set; + } + private CollectionUtils() {} } diff --git a/docs/content/ingestion/delete-data.md b/docs/content/ingestion/delete-data.md index 7e21e99bcc95..181fddb0ec63 100644 --- a/docs/content/ingestion/delete-data.md +++ b/docs/content/ingestion/delete-data.md @@ -37,7 +37,8 @@ A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/ ## Kill Task -Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is: +Kill tasks delete all information about a segment and removes it from deep storage. Segments to kill must be unused +(used==0) in the Druid segment table. The available grammar is: ```json { diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 0f5031686ce2..6436a9c51d93 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -113,15 +113,17 @@ Returns the serialized JSON of segments to load and drop for each Historical pro * `/druid/coordinator/v1/metadata/datasources` -Returns a list of the names of enabled datasources in the cluster. +Returns a list of the names of data sources with at least one used segment in the cluster. -* `/druid/coordinator/v1/metadata/datasources?includeDisabled` +* `/druid/coordinator/v1/metadata/datasources?includeUnused` -Returns a list of the names of enabled and disabled datasources in the cluster. +Returns a list of the names of data sources, regardless of whether there are used segments belonging to those data +sources in the cluster or not. * `/druid/coordinator/v1/metadata/datasources?full` -Returns a list of all enabled datasources with all metadata about those datasources as stored in the metadata store. +Returns a list of all data sources with at least one used segment in the cluster. Returns all metadata about those data +sources as stored in the metadata store. * `/druid/coordinator/v1/metadata/datasources/{dataSourceName}` @@ -229,11 +231,15 @@ Caution : Avoid using indexing or kill tasks and these API's at the same time fo * `/druid/coordinator/v1/datasources/{dataSourceName}` -Enables all segments of datasource which are not overshadowed by others. +Marks as used all segments belonging to a data source. Returns a JSON object of the form +`{"numChangedSegments": "}` with the number of segments in the database whose state has been changed (that is, +the segments were marked as used) as the result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` -Enables a segment of a datasource. +Marks as used a segment of a data source. Returns a JSON object of the form `{"segmentStateChanged": "}` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the +result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/markUsed` @@ -259,7 +265,9 @@ JSON Request Payload: * `/druid/coordinator/v1/datasources/{dataSourceName}` -Disables a datasource. +Marks as unused all segments belonging to a data source. Returns a JSON object of the form +`{"numChangedSegments": "}` with the number of segments in the database whose state has been changed (that is, +the segments were marked as unused) as the result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` * `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` @@ -268,7 +276,9 @@ Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` -Disables a segment. +Marks as unused a segment of a data source. Returns a JSON object of the form `{"segmentStateChanged": "}` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the +result of this API call. #### Retention Rules diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index 105afdf8f23f..19fe1832df08 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -365,7 +365,7 @@ Pair, Map>> checkSegment // drop derivative segments which interval equals the interval in toDeleteBaseSegments for (Interval interval : toDropInterval.keySet()) { for (DataSegment segment : derivativeSegments.get(interval)) { - segmentManager.removeSegment(segment.getId()); + segmentManager.markSegmentAsUnused(segment.getId()); } } // data of the latest interval will be built firstly. @@ -473,7 +473,7 @@ private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL); - segmentManager.removeDataSource(dataSource); + segmentManager.markAsUnusedAllSegmentsInDataSource(dataSource); metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index 44c90bc8a47b..ac15e67e74b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.client.indexing.ClientKillQuery; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -37,6 +38,9 @@ import java.util.Map; /** + * The client representation of this task is {@link ClientKillQuery}. + * JSON serialization fields of this class must correspond to those of {@link + * ClientKillQuery}, except for "id" and "context" fields. */ public class KillTask extends AbstractFixedIntervalTask { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java index 7d5c64c2c61c..4e0ab137ad33 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java @@ -73,7 +73,7 @@ public void setup() throws IOException expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); - expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().removeSegment(s.getId())); + expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().markSegmentAsUnused(s.getId())); } private DataSegment createSegment(Interval interval, String version) diff --git a/server/src/main/java/org/apache/druid/client/DruidDataSource.java b/server/src/main/java/org/apache/druid/client/DruidDataSource.java index 7678dad68377..c293523b0797 100644 --- a/server/src/main/java/org/apache/druid/client/DruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/DruidDataSource.java @@ -30,12 +30,13 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Predicate; /** * A mutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source. * - * Concurrency: could be updated concurrently via {@link #addSegment} and {@link #removeSegment}, and accessed - * concurrently (e. g. via {@link #getSegments}) as well. + * Concurrency: could be updated concurrently via {@link #addSegment}, {@link #removeSegment}, and {@link + * #removeSegmentsIf}, and accessed concurrently (e. g. via {@link #getSegments}) as well. * * @see ImmutableDruidDataSource - an immutable counterpart of this class */ @@ -44,10 +45,7 @@ public class DruidDataSource private final String name; private final Map properties; /** - * This map needs to be concurrent because it should be possible to iterate the segments of the data source - * (indirectly via {@link #getSegments} or in {@link #toString}) concurrently updates via {@link #addSegment} or - * {@link #removeSegment}. Concurrent updates are also supported incidentally, though this is not needed for the use - * cases of DruidDataSource. + * This map needs to be concurrent to support concurrent iteration and updates. */ private final ConcurrentMap idToSegmentMap = new ConcurrentHashMap<>(); @@ -80,6 +78,14 @@ public Collection getSegments() return Collections.unmodifiableCollection(idToSegmentMap.values()); } + /** + * Removes segments for which the given filter returns true. + */ + public void removeSegmentsIf(Predicate filter) + { + idToSegmentMap.values().removeIf(filter); + } + public DruidDataSource addSegment(DataSegment dataSegment) { idToSegmentMap.put(dataSegment.getId(), dataSegment); @@ -99,7 +105,7 @@ public boolean addSegmentIfAbsent(DataSegment dataSegment) * Returns the removed segment, or null if there was no segment with the given {@link SegmentId} in this * DruidDataSource. */ - public DataSegment removeSegment(SegmentId segmentId) + public @Nullable DataSegment removeSegment(SegmentId segmentId) { return idToSegmentMap.remove(segmentId); } @@ -126,7 +132,6 @@ public String toString() @Override public boolean equals(Object o) { - //noinspection Contract throw new UnsupportedOperationException("Use ImmutableDruidDataSource instead"); } diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java index ac73d63febf3..d3bcb0dedcee 100644 --- a/server/src/main/java/org/apache/druid/client/DruidServer.java +++ b/server/src/main/java/org/apache/druid/client/DruidServer.java @@ -232,7 +232,6 @@ public DataSegment removeDataSegment(SegmentId segmentId) segmentId ); // Returning null from the lambda here makes the ConcurrentHashMap to not record any entry. - //noinspection ReturnOfNull return null; } DataSegment segment = dataSource.removeSegment(segmentId); @@ -244,7 +243,6 @@ public DataSegment removeDataSegment(SegmentId segmentId) log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), segmentId); } // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - //noinspection ReturnOfNull return dataSource.isEmpty() ? null : dataSource; } ); diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java index 841b7169458e..1f78063cdb79 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java @@ -186,6 +186,6 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(name, properties, idToSegments); + return Objects.hash(name, properties); } } diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java index d01ad961a70c..605b2de877ff 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java @@ -42,19 +42,19 @@ public class ImmutableDruidServer private final DruidServerMetadata metadata; private final long currSize; private final ImmutableMap dataSources; - private final int totalSegments; + private final int numSegments; public ImmutableDruidServer( DruidServerMetadata metadata, long currSize, ImmutableMap dataSources, - int totalSegments + int numSegments ) { this.metadata = Preconditions.checkNotNull(metadata); this.currSize = currSize; this.dataSources = dataSources; - this.totalSegments = totalSegments; + this.numSegments = numSegments; } public String getName() @@ -128,23 +128,26 @@ public ImmutableDruidDataSource getDataSource(String name) } /** - * Returns a lazy collection with all segments in all data sources, stored on this ImmutableDruidServer. The order - * of segments in this collection is unspecified. - * - * Calling {@link Collection#size()} on the returned collection is cheap, O(1). + * Returns a lazy collection with all segments in all data sources stored on this ImmutableDruidServer to be used for + * iteration or {@link Collection#stream()} transformation. The order of segments in this collection is unspecified. * * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only * once rather than several times. */ - public Collection getLazyAllSegments() + public Collection iterateAllSegments() { return CollectionUtils.createLazyCollectionFromStream( () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()), - totalSegments + numSegments ); } + public int getNumSegments() + { + return numSegments; + } + public String getURL() { if (metadata.getHostAndTlsPort() != null) { diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java index b4bbd1a04cd7..4f43eb1768e0 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java @@ -29,6 +29,10 @@ import java.util.Map; import java.util.Objects; +/** + * Client representation of {@link org.apache.druid.indexing.common.task.CompactionTask}. JSON serialization fields of + * this class must correspond to those of {@link org.apache.druid.indexing.common.task.CompactionTask}. + */ public class ClientCompactQuery implements ClientQuery { private final String dataSource; @@ -147,7 +151,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactQuery{" + + return getClass().getSimpleName() + "{" + "dataSource='" + dataSource + '\'' + ", segments=" + segments + ", interval=" + interval + diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java index 068016952b4d..14cf45b55a8e 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java @@ -43,17 +43,17 @@ public class ClientCompactQueryTuningConfig private final Long pushTimeout; public static ClientCompactQueryTuningConfig from( - @Nullable UserCompactTuningConfig userCompactTuningConfig, + @Nullable UserCompactTuningConfig userCompactionTaskQueryTuningConfig, @Nullable Integer maxRowsPerSegment ) { return new ClientCompactQueryTuningConfig( maxRowsPerSegment, - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxRowsInMemory(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxTotalRows(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getIndexSpec(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxPendingPersists(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getPushTimeout() + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxRowsInMemory(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxTotalRows(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getIndexSpec(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxPendingPersists(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getPushTimeout() ); } @@ -150,7 +150,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactQueryTuningConfig{" + + return getClass().getSimpleName() + "{" + "maxRowsPerSegment=" + maxRowsPerSegment + ", maxRowsInMemory=" + maxRowsInMemory + ", maxTotalRows=" + maxTotalRows + diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java index 06d88f9535a3..c704878af948 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java @@ -24,6 +24,9 @@ import org.joda.time.Interval; /** + * Client representation of {@link org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}. JSON searialization + * fields of this class must correspond to those of {@link + * org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}, except for "id" and "context" fields. */ public class ClientKillQuery implements ClientQuery { diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java index aaa8b5c3ce3a..9f0c0e13a534 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java @@ -24,7 +24,12 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** - * org.apache.druid.indexing.common.task.Task representation for clients + * {@link org.apache.druid.indexing.common.task.Task} representations for clients. The magic conversion happens right + * at the moment of making a REST query: {@link HttpIndexingServiceClient#runTask} serializes ClientTaskQuery + * objects and {@link org.apache.druid.indexing.overlord.http.OverlordResource#taskPost} deserializes {@link + * org.apache.druid.indexing.common.task.Task} objects from the same bytes. Therefore JSON serialization fields of + * ClientTaskQuery objects must match with those of the corresponding {@link + * org.apache.druid.indexing.common.task.Task} objects. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index 4fc517718f2a..41523ef84734 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -20,88 +20,137 @@ package org.apache.druid.metadata; import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Collection; import java.util.List; +import java.util.Set; /** + * The difference between this class and {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} is that this + * class resides in Coordinator's memory, while {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} resides + * in Broker's memory. */ public interface MetadataSegmentManager { - void start(); + void startPollingDatabasePeriodically(); - void stop(); + void stopPollingDatabasePeriodically(); + + boolean isPollingDatabasePeriodically(); /** - * Enables all segments for a dataSource which will not be overshadowed. + * Returns the number of segment entries in the database whose state was changed as the result of this call (that is, + * the segments were marked as used). If the call results in a database error, an exception is relayed to the caller. */ - boolean enableDataSource(String dataSource); + int markAsUsedAllNonOvershadowedSegmentsInDataSource(String dataSource); + + int markAsUsedNonOvershadowedSegmentsInInterval(String dataSource, Interval interval); - boolean enableSegment(String segmentId); + int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) + throws UnknownSegmentIdException; /** - * Enables all segments contained in the interval which are not overshadowed by any currently enabled segments. + * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the + * segment was marked as used), false otherwise. If the call results in a database error, an exception is relayed to + * the caller. */ - int enableSegments(String dataSource, Interval interval); + boolean markSegmentAsUsed(String segmentId); /** - * Enables the segments passed which are not overshadowed by any currently enabled segments. + * Returns the number of segment entries in the database whose state was changed as the result of this call (that is, + * the segments were marked as unused). If the call results in a database error, an exception is relayed to the + * caller. */ - int enableSegments(String dataSource, Collection segmentIds); + int markAsUnusedAllSegmentsInDataSource(String dataSource); + + int markAsUnusedSegmentsInInterval(String dataSource, Interval interval); - boolean removeDataSource(String dataSource); + int markSegmentsAsUnused(String dataSource, Set segmentIds) throws UnknownSegmentIdException; /** - * Prefer {@link #removeSegment(SegmentId)} to this method when possible. + * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the + * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to + * the caller. * - * This method is not removed because {@link org.apache.druid.server.http.DataSourcesResource#deleteDatasourceSegment} - * uses it and if it migrates to {@link #removeSegment(SegmentId)} the performance will be worse. + * Prefer {@link #markSegmentAsUnused(SegmentId)} to this method when possible. + * + * This method is not removed because {@link org.apache.druid.server.http.DataSourcesResource#markSegmentAsUnused} + * uses it and if it migrates to {@link #markSegmentAsUnused(SegmentId)} the performance will be worse. */ - boolean removeSegment(String dataSource, String segmentId); - - boolean removeSegment(SegmentId segmentId); + boolean markSegmentAsUnused(String dataSource, String segmentId); - long disableSegments(String dataSource, Collection segmentIds); - - int disableSegments(String dataSource, Interval interval); + /** + * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the + * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to + * the caller. + */ + boolean markSegmentAsUnused(SegmentId segmentId); - boolean isStarted(); + /** + * If there are used segments belonging to the given data source, this method converts this set of segments to an + * {@link ImmutableDruidDataSource} object and returns. If there are no used segments belonging to the given data + * source, this method returns null. + * + * This method's name starts with "prepare" to indicate that it's not cheap (it creates an {@link + * ImmutableDruidDataSource} object). Not used "create" prefix to avoid giving a false impression that this method + * might put something into the database to create a data source with the given name, if absent. + */ + @Nullable ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSource); - @Nullable - ImmutableDruidDataSource getDataSource(String dataSourceName); + /** + * If there are used segments belonging to the given data source, this method returns a {@link DruidDataSource} object + * with a view on those segments. If there are no used segments belonging to the given data source, this method + * returns null. + * + * Note that the returned {@link DruidDataSource} object may be updated concurrently and already be empty by the time + * it is returned. + */ + @Nullable DruidDataSource getDataSourceWithUsedSegments(String dataSource); /** - * Returns a collection of known datasources. + * Prepares a set of {@link ImmutableDruidDataSource} objects containing information about all used segments. {@link + * ImmutableDruidDataSource} objects in the returned collection are unique. If there are no used segments, this method + * returns an empty collection. * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) + * This method's name starts with "prepare" for the same reason as {@link + * #prepareImmutableDataSourceWithUsedSegments}. */ - @Nullable - Collection getDataSources(); + Collection prepareImmutableDataSourcesWithAllUsedSegments(); /** * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is * unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than * several times. - * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) */ - @Nullable - Iterable iterateAllSegments(); + Iterable iterateAllUsedSegments(); - Collection getAllDataSourceNames(); + /** + * Retrieves all data source names for which there are segment in the database, regardless of whether those segments + * are used or not. Data source names in the returned collection are unique. If there are no segments in the database, + * returns an empty collection. + * + * Performance warning: this method makes a query into the database. + * + * This method might return a different set of data source names than may be observed via {@link + * #prepareImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there + * are no used segments belonging to it, while {@link #prepareImmutableDataSourcesWithAllUsedSegments} won't return + * such a data source. + */ + Collection retrieveAllDataSourceNames(); /** - * Returns top N unused segment intervals in given interval when ordered by segment start time, end time. + * Returns top N unused segment intervals with the end time no later than the specified maxEndTime when ordered by + * segment start time, end time. */ - List getUnusedSegmentIntervals(String dataSource, Interval interval, int limit); + List getUnusedSegmentIntervals(String dataSource, DateTime maxEndTime, int limit); @VisibleForTesting void poll(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java index be95669feb25..1f47e44811c7 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java @@ -200,7 +200,7 @@ public void run() { try { // poll() is synchronized together with start() and stop() to ensure that when stop() exits, poll() - // won't actually run anymore after that (it could only enter the syncrhonized section and exit + // won't actually run anymore after that (it could only enter the synchronized section and exit // immediately because the localStartedOrder doesn't match the new currentStartOrder). It's needed // to avoid flakiness in SQLMetadataRuleManagerTest. // See https://github.com/apache/incubator-druid/issues/6028 diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 35843c95b9dd..d63e9c1f6e15 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -20,15 +20,19 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Futures; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.java.util.common.Pair; @@ -40,17 +44,18 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.skife.jdbi.v2.BaseResultSetMapper; import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.TransactionCallback; import org.skife.jdbi.v2.TransactionStatus; -import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.ResultSetMapper; import javax.annotation.Nullable; @@ -58,18 +63,20 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * @@ -80,11 +87,48 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager private static final EmittingLogger log = new EmittingLogger(SQLMetadataSegmentManager.class); /** - * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, and {@link #isStarted()}. These methods - * should be synchronized to prevent from being called at the same time if two different threads are calling them. - * This might be possible if a druid coordinator gets and drops leadership repeatedly in quick succession. + * Marker interface for objects stored in {@link #latestDatabasePoll}. See the comment for that field for details. */ - private final ReentrantReadWriteLock startStopLock = new ReentrantReadWriteLock(); + private interface DatabasePoll + {} + + /** Represents periodic {@link #poll}s happening from {@link #exec}. */ + private static class PeriodicDatabasePoll implements DatabasePoll + { + /** + * This future allows to wait until {@link #dataSources} is initialized in the first {@link #poll()} happening since + * {@link #startPollingDatabasePeriodically()} is called for the first time, or since the last visible (in + * happens-before terms) call to {@link #startPollingDatabasePeriodically()} in case of Coordinator's leadership + * changes. + */ + final CompletableFuture firstPollCompletionFuture = new CompletableFuture<>(); + } + + /** + * Represents on-demand {@link #poll} initiated at periods of time when SqlSegmentsMetadata doesn't poll the database + * periodically. + */ + private static class OnDemandDatabasePoll implements DatabasePoll + { + final long initiationTimeNanos = System.nanoTime(); + final CompletableFuture pollCompletionFuture = new CompletableFuture<>(); + + long nanosElapsedFromInitiation() + { + return System.nanoTime() - initiationTimeNanos; + } + } + + /** + * Use to synchronize {@link #startPollingDatabasePeriodically}, {@link #stopPollingDatabasePeriodically}, {@link + * #poll}, and {@link #isPollingDatabasePeriodically}. These methods should be synchronized to prevent from being + * called at the same time if two different threads are calling them. This might be possible if Coordinator gets and + * drops leadership repeatedly in quick succession. + * + * This lock is also used to synchronize {@link #awaitOrPerformDatabasePoll} for times when SqlSegmentsMetadata + * is not polling the database periodically (in other words, when the Coordinator is not the leader). + */ + private final ReentrantReadWriteLock startStopPollLock = new ReentrantReadWriteLock(); /** * Used to ensure that {@link #poll()} is never run concurrently. It should already be so (at least in production @@ -92,39 +136,72 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager * scheduled in a single-threaded {@link #exec}, so this lock is an additional safety net in case there are bugs in * the code, and for tests, where {@link #poll()} is called from the outside code. * - * Not using {@link #startStopLock}.writeLock() in order to still be able to run {@link #poll()} concurrently with - * {@link #isStarted()}. + * Not using {@link #startStopPollLock}.writeLock() in order to still be able to run {@link #poll()} concurrently + * with {@link #isPollingDatabasePeriodically()}. */ private final Object pollLock = new Object(); private final ObjectMapper jsonMapper; - private final Supplier config; + private final Duration periodicPollDelay; private final Supplier dbTables; private final SQLMetadataConnector connector; - // Volatile since this reference is reassigned in "poll" and then read from in other threads. - // Starts null so we can differentiate "never polled" (null) from "polled, but empty" (empty map). - // Note that this is not simply a lazy-initialized variable: it starts off as null, and may transition between - // null and nonnull multiple times as stop() and start() are called. - @Nullable - private volatile ConcurrentHashMap dataSources = null; + /** + * This field is made volatile to avoid "ghost secondary reads" that may result in NPE, see + * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl (note that dataSources resembles a lazily + * initialized field). Alternative is to always read the field in a snapshot local variable, but it's too easy to + * forget to do. + * + * This field may be updated from {@link #exec}, or from whatever thread calling {@link #doOnDemandPoll} via {@link + * #awaitOrPerformDatabasePoll()} via one of the public methods of SqlSegmentsMetadata. + */ + private volatile @MonotonicNonNull ConcurrentHashMap dataSources = null; /** - * The number of times this SQLMetadataSegmentManager was started. + * The latest {@link DatabasePoll} represent {@link #poll()} calls which update {@link #dataSources}, either + * periodically (see {@link PeriodicDatabasePoll}, {@link #startPollingDatabasePeriodically}, {@link + * #stopPollingDatabasePeriodically}) or "on demand" (see {@link OnDemandDatabasePoll}), when one of the methods that + * accesses {@link #dataSources} state (such as {@link #prepareImmutableDataSourceWithUsedSegments}) is called when + * the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database periodically. + * + * The notion and the complexity of "on demand" database polls was introduced to simplify the interface of {@link + * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods like + * {@link #prepareImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part is + * achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or + * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #awaitOrPerformDatabasePoll} method + * implementation for details. + * + * Note: the overall implementation of periodic/on-demand polls is not completely optimal: for example, when the + * Coordinator just stopped leading, the latest periodic {@link #poll} (which is still "fresh") is not considered + * and a new on-demand poll is always initiated. This is done to simplify the implementation, while the efficiency + * during Coordinator leadership switches is not a priority. + * + * This field is {@code volatile} because it's checked and updated in a double-checked locking manner in {@link + * #awaitOrPerformDatabasePoll()}. */ - private long startCount = 0; + private volatile @Nullable DatabasePoll latestDatabasePoll = null; + + /** Used to cancel periodic poll task in {@link #stopPollingDatabasePeriodically}. */ + @GuardedBy("startStopPollLock") + private @Nullable Future periodicPollTaskFuture = null; + + /** The number of times {@link #startPollingDatabasePeriodically} was called. */ + private long startPollingCount = 0; + /** - * Equal to the current {@link #startCount} value, if the SQLMetadataSegmentManager is currently started; -1 if + * Equal to the current {@link #startPollingCount} value if the SqlSegmentsMetadata is currently started; -1 if * currently stopped. * * This field is used to implement a simple stamp mechanism instead of just a boolean "started" flag to prevent - * the theoretical situation of two or more tasks scheduled in {@link #start()} calling {@link #isStarted()} and - * {@link #poll()} concurrently, if the sequence of {@link #start()} - {@link #stop()} - {@link #start()} actions - * occurs quickly. + * the theoretical situation of two or more tasks scheduled in {@link #startPollingDatabasePeriodically()} calling + * {@link #isPollingDatabasePeriodically()} and {@link #poll()} concurrently, if the sequence of {@link + * #startPollingDatabasePeriodically()} - {@link #stopPollingDatabasePeriodically()} - {@link + * #startPollingDatabasePeriodically()} actions occurs quickly. * - * {@link SQLMetadataRuleManager} also have a similar issue. + * {@link SQLMetadataRuleManager} also has a similar issue. */ - private long currentStartOrder = -1; + private long currentStartPollingOrder = -1; + private ScheduledExecutorService exec = null; @Inject @@ -136,33 +213,70 @@ public SQLMetadataSegmentManager( ) { this.jsonMapper = jsonMapper; - this.config = config; + this.periodicPollDelay = config.get().getPollDuration().toStandardDuration(); this.dbTables = dbTables; this.connector = connector; } - @Override + /** + * Don't confuse this method with {@link #startPollingDatabasePeriodically}. This is a lifecycle starting method to + * be executed just once for an instance of SqlSegmentsMetadata. + */ @LifecycleStart public void start() { - ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock(); + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); + try { + exec = Execs.scheduledSingleThreaded(getClass().getName() + "-Exec--%d"); + } + finally { + lock.unlock(); + } + } + + /** + * Don't confuse this method with {@link #stopPollingDatabasePeriodically}. This is a lifecycle stopping method to + * be executed just once for an instance of SqlSegmentsMetadata. + */ + @LifecycleStop + public void stop() + { + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); + try { + exec.shutdownNow(); + exec = null; + } + finally { + lock.unlock(); + } + } + + @Override + public void startPollingDatabasePeriodically() + { + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (isStarted()) { + if (exec == null) { + throw new IllegalStateException(getClass().getName() + " is not started"); + } + if (isPollingDatabasePeriodically()) { return; } - startCount++; - currentStartOrder = startCount; - final long localStartOrder = currentStartOrder; + PeriodicDatabasePoll periodicPollUpdate = new PeriodicDatabasePoll(); + latestDatabasePoll = periodicPollUpdate; - exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); + startPollingCount++; + currentStartPollingOrder = startPollingCount; + final long localStartOrder = currentStartPollingOrder; - final Duration delay = config.get().getPollDuration().toStandardDuration(); - exec.scheduleWithFixedDelay( - createPollTaskForStartOrder(localStartOrder), + periodicPollTaskFuture = exec.scheduleWithFixedDelay( + createPollTaskForStartOrder(localStartOrder, periodicPollUpdate), 0, - delay.getMillis(), + periodicPollDelay.getMillis(), TimeUnit.MILLISECONDS ); } @@ -171,24 +285,33 @@ public void start() } } - private Runnable createPollTaskForStartOrder(long startOrder) + private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePoll periodicPollUpdate) { return () -> { - // poll() is synchronized together with start(), stop() and isStarted() to ensure that when stop() exits, poll() - // won't actually run anymore after that (it could only enter the syncrhonized section and exit immediately - // because the localStartedOrder doesn't match the new currentStartOrder). It's needed to avoid flakiness in - // SQLMetadataSegmentManagerTest. See https://github.com/apache/incubator-druid/issues/6028 - ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); + // poll() is synchronized together with startPollingDatabasePeriodically(), stopPollingDatabasePeriodically() and + // isPollingDatabasePeriodically() to ensure that when stopPollingDatabasePeriodically() exits, poll() won't + // actually run anymore after that (it could only enter the synchronized section and exit immediately because the + // localStartedOrder doesn't match the new currentStartPollingOrder). It's needed to avoid flakiness in + // SqlSegmentsMetadataTest. See https://github.com/apache/incubator-druid/issues/6028 + ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); lock.lock(); try { - if (startOrder == currentStartOrder) { + if (startOrder == currentStartPollingOrder) { poll(); + periodicPollUpdate.firstPollCompletionFuture.complete(null); } else { - log.debug("startOrder = currentStartOrder = %d, skipping poll()", startOrder); + log.debug("startOrder = currentStartPollingOrder = %d, skipping poll()", startOrder); } } - catch (Exception e) { - log.makeAlert(e, "uncaught exception in segment manager polling thread").emit(); + catch (Throwable t) { + log.makeAlert(t, "Uncaught exception in " + getClass().getName() + "'s polling thread").emit(); + // Swallow the exception, so that scheduled polling goes on. Leave firstPollFutureSinceLastStart uncompleted + // for now, so that it may be completed during the next poll. + if (!(t instanceof Exception)) { + // Don't try to swallow a Throwable which is not an Exception (that is, a Error). + periodicPollUpdate.firstPollCompletionFuture.completeExceptionally(t); + throw t; + } } finally { lock.unlock(); @@ -197,366 +320,488 @@ private Runnable createPollTaskForStartOrder(long startOrder) } @Override - @LifecycleStop - public void stop() + public void stopPollingDatabasePeriodically() { - ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock(); + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (!isStarted()) { + if (!isPollingDatabasePeriodically()) { return; } - dataSources = null; - currentStartOrder = -1; - exec.shutdownNow(); - exec = null; + periodicPollTaskFuture.cancel(false); + latestDatabasePoll = null; + + // NOT nulling dataSources, allowing to query the latest polled data even when this SegmentsMetadata object is + // stopped. + + currentStartPollingOrder = -1; } finally { lock.unlock(); } } - private Pair usedPayloadMapper( - final int index, - final ResultSet resultSet, - final StatementContext context - ) throws SQLException + private void awaitOrPerformDatabasePoll() { + // Double-checked locking with awaitPeriodicOrFreshOnDemandDatabasePoll() call playing the role of the "check". + if (awaitPeriodicOrFreshOnDemandDatabasePoll()) { + return; + } + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); try { - return new Pair<>( - jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class), - resultSet.getBoolean("used") - ); + if (awaitPeriodicOrFreshOnDemandDatabasePoll()) { + return; + } + OnDemandDatabasePoll newOnDemandUpdate = new OnDemandDatabasePoll(); + this.latestDatabasePoll = newOnDemandUpdate; + doOnDemandPoll(newOnDemandUpdate); } - catch (IOException e) { - throw new RuntimeException(e); + finally { + lock.unlock(); } } - /** - * Gets a list of all datasegments that overlap the provided interval along with thier used status. - */ - private List> getDataSegmentsOverlappingInterval( - final String dataSource, - final Interval interval - ) - { - return connector.inReadOnlyTransaction( - (handle, status) -> handle.createQuery( - StringUtils.format( - "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start", - getSegmentsTable(), - connector.getQuoteString() - ) - ) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .map(this::usedPayloadMapper) - .list() - ); - } - - private List> getDataSegments( - final String dataSource, - final Collection segmentIds, - final Handle handle - ) + private boolean awaitPeriodicOrFreshOnDemandDatabasePoll() { - return segmentIds.stream().map( - segmentId -> Optional.ofNullable( - handle.createQuery( - StringUtils.format( - "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id", - getSegmentsTable() - ) - ) - .bind("dataSource", dataSource) - .bind("id", segmentId) - .map(this::usedPayloadMapper) - .first() - ) - .orElseThrow(() -> new UnknownSegmentIdException(StringUtils.format("Cannot find segment id [%s]", segmentId))) - ) - .collect(Collectors.toList()); + DatabasePoll latestDatabasePoll = this.latestDatabasePoll; + if (latestDatabasePoll instanceof PeriodicDatabasePoll) { + Futures.getUnchecked(((PeriodicDatabasePoll) latestDatabasePoll).firstPollCompletionFuture); + return true; + } + if (latestDatabasePoll instanceof OnDemandDatabasePoll) { + long periodicPollDelayNanos = TimeUnit.MILLISECONDS.toNanos(periodicPollDelay.getMillis()); + OnDemandDatabasePoll latestOnDemandUpdate = (OnDemandDatabasePoll) latestDatabasePoll; + boolean latestUpdateIsFresh = latestOnDemandUpdate.nanosElapsedFromInitiation() < periodicPollDelayNanos; + if (latestUpdateIsFresh) { + Futures.getUnchecked(latestOnDemandUpdate.pollCompletionFuture); + return true; + } + // Latest on-demand update is not fresh. Fall through to return false from this method. + } else { + assert latestDatabasePoll == null; + } + return false; } - /** - * Builds a VersionedIntervalTimeline containing used segments that overlap the intervals passed. - */ - private VersionedIntervalTimeline buildVersionedIntervalTimeline( - final String dataSource, - final Collection intervals, - final Handle handle - ) + private void doOnDemandPoll(OnDemandDatabasePoll onDemandPoll) { - return VersionedIntervalTimeline.forSegments(intervals - .stream() - .flatMap(interval -> handle.createQuery( - StringUtils.format( - "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true", - getSegmentsTable(), - connector.getQuoteString() - ) - ) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .map((i, resultSet, context) -> { - try { - return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) - .list() - .stream() - ) - .iterator() - ); + try { + poll(); + onDemandPoll.pollCompletionFuture.complete(null); + } + catch (Throwable t) { + onDemandPoll.pollCompletionFuture.completeExceptionally(t); + throw t; + } } @Override - public boolean enableDataSource(final String dataSource) + public boolean markSegmentAsUsed(final String segmentId) { try { - return enableSegments(dataSource, Intervals.ETERNITY) != 0; + int numUpdatedDatabaseEntries = connector.getDBI().withHandle( + (Handle handle) -> handle + .createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable())) + .bind("id", segmentId) + .execute() + ); + // Unlike bulk markAsUsed methods: markAsUsedAllNonOvershadowedSegmentsInDataSource(), + // markAsUsedNonOvershadowedSegmentsInInterval(), and markAsUsedNonOvershadowedSegments() we don't put the marked + // segment into the respective data source, because we don't have it fetched from the database. It's probably not + // worth complicating the implementation and making two database queries just to add the segment because it will + // be anyway fetched during the next poll(). Segment putting that is done in the bulk markAsUsed methods is a nice + // to have thing, but doesn't formally affects the external guarantees of SegmentsMetadata class. + return numUpdatedDatabaseEntries > 0; } - catch (Exception e) { - log.error(e, "Exception enabling datasource %s", dataSource); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking segment %s as used", segmentId); + throw e; } } @Override - public int enableSegments(final String dataSource, final Interval interval) + public int markAsUsedAllNonOvershadowedSegmentsInDataSource(final String dataSource) { - List> segments = getDataSegmentsOverlappingInterval(dataSource, interval); - List segmentsToEnable = segments.stream() - .filter(segment -> !segment.rhs && interval.contains(segment.lhs.getInterval())) - .map(segment -> segment.lhs) - .collect(Collectors.toList()); - - VersionedIntervalTimeline versionedIntervalTimeline = VersionedIntervalTimeline.forSegments( - segments.stream().filter(segment -> segment.rhs).map(segment -> segment.lhs).iterator() - ); - VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segmentsToEnable.iterator()); - - return enableSegments( - segmentsToEnable, - versionedIntervalTimeline - ); + return doMarkAsUsedNonOvershadowedSegments(dataSource, null); } @Override - public int enableSegments(final String dataSource, final Collection segmentIds) + public int markAsUsedNonOvershadowedSegmentsInInterval(final String dataSource, final Interval interval) { - Pair, VersionedIntervalTimeline> data = connector.inReadOnlyTransaction( - (handle, status) -> { - List segments = getDataSegments(dataSource, segmentIds, handle) - .stream() - .filter(pair -> !pair.rhs) - .map(pair -> pair.lhs) - .collect(Collectors.toList()); - - VersionedIntervalTimeline versionedIntervalTimeline = buildVersionedIntervalTimeline( - dataSource, - JodaUtils.condenseIntervals(segments.stream().map(segment -> segment.getInterval()).collect(Collectors.toList())), - handle - ); - VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segments.iterator()); + Preconditions.checkNotNull(interval); + return doMarkAsUsedNonOvershadowedSegments(dataSource, interval); + } - return new Pair<>( - segments, - versionedIntervalTimeline - ); + /** + * Implementation for both {@link #markAsUsedAllNonOvershadowedSegmentsInDataSource} (if the given interval is null) + * and {@link #markAsUsedNonOvershadowedSegmentsInInterval}. + */ + private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Interval interval) + { + List usedSegmentsOverlappingInterval = new ArrayList<>(); + List unusedSegmentsInInterval = new ArrayList<>(); + connector.inReadOnlyTransaction( + (handle, status) -> { + String queryString = + StringUtils.format("SELECT used, payload FROM %1$s WHERE dataSource = :dataSource", getSegmentsTable()); + if (interval != null) { + queryString += StringUtils.format(" AND start < :end AND %1$send%1$s > :start", connector.getQuoteString()); + } + Query query = handle + .createQuery(queryString) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("dataSource", dataSourceName); + if (interval != null) { + query = query + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + } + query = query + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + DataSegment segment = jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + if (resultSet.getBoolean("used")) { + usedSegmentsOverlappingInterval.add(segment); + } else { + if (interval == null || interval.contains(segment.getInterval())) { + unusedSegmentsInInterval.add(segment); + } + } + return null; + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + // Consume the query results to ensure usedSegmentsOverlappingInterval and unusedSegmentsInInterval are + // populated. + consume(query.iterator()); + return null; } ); - return enableSegments( - data.lhs, - data.rhs + VersionedIntervalTimeline versionedIntervalTimeline = VersionedIntervalTimeline.forSegments( + Iterators.concat(usedSegmentsOverlappingInterval.iterator(), unusedSegmentsInInterval.iterator()) ); + + return markNonOvershadowedSegmentsAsUsed(dataSourceName, unusedSegmentsInInterval, versionedIntervalTimeline); + } + + private static void consume(Iterator iterator) { + while (iterator.hasNext()) { + iterator.next(); + } } - private int enableSegments( - final Collection segments, - final VersionedIntervalTimeline versionedIntervalTimeline + /** Also puts non-overshadowed segments into {@link #dataSources}. */ + private int markNonOvershadowedSegmentsAsUsed( + String dataSourceName, + List unusedSegments, + VersionedIntervalTimeline timeline ) { - if (segments.isEmpty()) { - log.warn("No segments found to update!"); - return 0; + @Nullable + DruidDataSource dataSource = null; + if (dataSources != null) { + dataSource = dataSources.computeIfAbsent( + dataSourceName, + dsName -> new DruidDataSource(dsName, createDefaultDataSourceProperties()) + ); + } + List segmentIdsToMarkAsUsed = new ArrayList<>(); + for (DataSegment segment : unusedSegments) { + if (timeline.isOvershadowed(segment.getInterval(), segment.getVersion())) { + continue; + } + if (dataSource != null) { + dataSource.addSegment(segment); + } + String s = segment.getId().toString(); + segmentIdsToMarkAsUsed.add(s); } - return connector.getDBI().withHandle(handle -> { - Batch batch = handle.createBatch(); - segments - .stream() - .map(segment -> segment.getId()) - .filter(segmentId -> !versionedIntervalTimeline.isOvershadowed( - segmentId.getInterval(), - segmentId.getVersion() - )) - .forEach(segmentId -> batch.add( - StringUtils.format( - "UPDATE %s SET used=true WHERE id = '%s'", - getSegmentsTable(), - segmentId - ) - )); - return batch.execute().length; - }); + return markSegmentsAsUsed(segmentIdsToMarkAsUsed); } @Override - public boolean enableSegment(final String segmentId) + public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set segmentIds) + throws UnknownSegmentIdException { try { - connector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) - { - handle.createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable())) - .bind("id", segmentId) - .execute(); - return null; - } - } - ); + Pair, VersionedIntervalTimeline> unusedSegmentsAndTimeline = connector + .inReadOnlyTransaction( + (handle, status) -> { + List unusedSegments = retreiveUnusedSegments(dataSource, segmentIds, handle); + List unusedSegmentsIntervals = JodaUtils.condenseIntervals( + unusedSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + Iterator usedSegmentsOverlappingUnusedSegmentsIntervals = + retreiveUsedSegmentsOverlappingIntervals(dataSource, unusedSegmentsIntervals, handle); + VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + Iterators.concat(usedSegmentsOverlappingUnusedSegmentsIntervals, unusedSegments.iterator()) + ); + return new Pair<>(unusedSegments, timeline); + } + ); + + List unusedSegments = unusedSegmentsAndTimeline.lhs; + VersionedIntervalTimeline timeline = unusedSegmentsAndTimeline.rhs; + return markNonOvershadowedSegmentsAsUsed(dataSource, unusedSegments, timeline); } catch (Exception e) { - log.error(e, "Exception enabling segment %s", segmentId); - return false; + Throwable rootCause = Throwables.getRootCause(e); + if (rootCause instanceof UnknownSegmentIdException) { + throw (UnknownSegmentIdException) rootCause; + } else { + throw e; + } + } + } + + private List retreiveUnusedSegments( + final String dataSource, + final Set segmentIds, + final Handle handle + ) throws UnknownSegmentIdException + { + List unknownSegmentIds = new ArrayList<>(); + List segments = segmentIds + .stream() + .map( + segmentId -> { + Iterator segmentResultIterator = handle + .createQuery( + StringUtils.format( + "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id", + getSegmentsTable() + ) + ) + .bind("dataSource", dataSource) + .bind("id", segmentId) + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + if (!resultSet.getBoolean("used")) { + return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + } else { + // We emit nulls for used segments. They are filtered out below in this method. + return null; + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + }) + .iterator(); + if (!segmentResultIterator.hasNext()) { + unknownSegmentIds.add(segmentId); + return null; + } else { + @Nullable DataSegment segment = segmentResultIterator.next(); + if (segmentResultIterator.hasNext()) { + log.error( + "There is more than one row corresponding to segment id [%s] in data source [%s] in the database", + segmentId, + dataSource + ); + } + return segment; + } + } + ) + .filter(Objects::nonNull) // Filter nulls corresponding to used segments. + .collect(Collectors.toList()); + if (!unknownSegmentIds.isEmpty()) { + throw new UnknownSegmentIdException(unknownSegmentIds); + } + return segments; + } + + private Iterator retreiveUsedSegmentsOverlappingIntervals( + final String dataSource, + final Collection intervals, + final Handle handle + ) + { + return intervals + .stream() + .flatMap(interval -> { + Iterable segmentResultIterable = () -> handle + .createQuery( + StringUtils.format( + "SELECT payload FROM %1$s " + + "WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true", + getSegmentsTable(), + connector.getQuoteString() + ) + ) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }) + .iterator(); + return StreamSupport.stream(segmentResultIterable.spliterator(), false); + }) + .iterator(); + } + + private int markSegmentsAsUsed(final List segmentIds) + { + if (segmentIds.isEmpty()) { + log.info("No segments found to update!"); + return 0; } - return true; + return connector.getDBI().withHandle(handle -> { + Batch batch = handle.createBatch(); + segmentIds.forEach(segmentId -> batch.add( + StringUtils.format("UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), segmentId) + )); + int[] segmentChanges = batch.execute(); + return computeNumChangedSegments(segmentIds, segmentChanges); + }); } @Override - public boolean removeDataSource(final String dataSource) + public int markAsUnusedAllSegmentsInDataSource(final String dataSource) { try { - final int removed = connector.getDBI().withHandle( - handle -> handle.createStatement( - StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable()) - ).bind("dataSource", dataSource).execute() + final int numUpdatedDatabaseEntries = connector.getDBI().withHandle( + (Handle handle) -> handle + .createStatement( + StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable()) + ) + .bind("dataSource", dataSource) + .execute() ); - Optional.ofNullable(dataSources).ifPresent(m -> m.remove(dataSource)); - - if (removed == 0) { - return false; + if (dataSources != null) { + dataSources.remove(dataSource); } + + return numUpdatedDatabaseEntries; } - catch (Exception e) { - log.error(e, "Error removing datasource %s", dataSource); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking all segments as unused in data source [%s]", dataSource); + throw e; } - - return true; } @Override - public boolean removeSegment(String dataSourceName, final String segmentId) + public boolean markSegmentAsUnused(String dataSourceName, final String segmentId) { try { - final boolean removed = removeSegmentFromTable(segmentId); - - // Call iteratePossibleParsingsWithDataSource() outside of dataSources.computeIfPresent() because the former is a - // potentially expensive operation, while lambda to be passed into computeIfPresent() should preferably run fast. - List possibleSegmentIds = SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId); - Optional.ofNullable(dataSources).ifPresent( - m -> - m.computeIfPresent( - dataSourceName, - (dsName, dataSource) -> { - for (SegmentId possibleSegmentId : possibleSegmentIds) { - if (dataSource.removeSegment(possibleSegmentId) != null) { - break; - } - } - // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - //noinspection ReturnOfNull - return dataSource.isEmpty() ? null : dataSource; - } - ) - ); - - return removed; + boolean segmentStateChanged = markSegmentAsUnusedInDatabase(segmentId); + if (dataSources != null) { + removeSegmentFromPolledDataSources(dataSourceName, segmentId, dataSources); + } + return segmentStateChanged; } - catch (Exception e) { - log.error(e, e.toString()); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking segment [%s] as unused", segmentId); + throw e; } } + private static void removeSegmentFromPolledDataSources( + String dataSourceName, + String segmentId, + ConcurrentHashMap dataSourcesSnapshot + ) + { + // Call iteratePossibleParsingsWithDataSource() outside of dataSources.computeIfPresent() because the former is + // a potentially expensive operation, while lambda to be passed into computeIfPresent() should preferably run + // fast. + List possibleSegmentIds = SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId); + dataSourcesSnapshot.computeIfPresent( + dataSourceName, + (dsName, dataSource) -> { + for (SegmentId possibleSegmentId : possibleSegmentIds) { + if (dataSource.removeSegment(possibleSegmentId) != null) { + break; + } + } + // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. + return dataSource.isEmpty() ? null : dataSource; + } + ); + } + @Override - public boolean removeSegment(SegmentId segmentId) + public boolean markSegmentAsUnused(SegmentId segmentId) { try { - final boolean removed = removeSegmentFromTable(segmentId.toString()); - Optional.ofNullable(dataSources).ifPresent( - m -> - m.computeIfPresent( - segmentId.getDataSource(), - (dsName, dataSource) -> { - dataSource.removeSegment(segmentId); - // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - //noinspection ReturnOfNull - return dataSource.isEmpty() ? null : dataSource; - } - ) - ); - return removed; + final boolean segmentStateChanged = markSegmentAsUnusedInDatabase(segmentId.toString()); + if (dataSources != null) { + dataSources.computeIfPresent( + segmentId.getDataSource(), + (dsName, dataSource) -> { + dataSource.removeSegment(segmentId); + // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. + return dataSource.isEmpty() ? null : dataSource; + } + ); + } + return segmentStateChanged; } - catch (Exception e) { - log.error(e, e.toString()); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking segment [%s] as unused", segmentId); + throw e; } } @Override - public long disableSegments(String dataSource, Collection segmentIds) + public int markSegmentsAsUnused(String dataSourceName, Set segmentIds) { if (segmentIds.isEmpty()) { return 0; } - final long[] result = new long[1]; + final List segmentIdList = new ArrayList<>(segmentIds); try { - connector.getDBI().withHandle(handle -> { + return connector.getDBI().withHandle(handle -> { Batch batch = handle.createBatch(); - segmentIds - .forEach(segmentId -> batch.add( - StringUtils.format( - "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s' ", - getSegmentsTable(), - dataSource, - segmentId - ) - )); - final int[] resultArr = batch.execute(); - result[0] = Arrays.stream(resultArr).filter(x -> x > 0).count(); - return result[0]; + segmentIdList.forEach(segmentId -> batch.add( + StringUtils.format( + "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s'", + getSegmentsTable(), + dataSourceName, + segmentId + ) + )); + final int[] segmentChanges = batch.execute(); + int numChangedSegments = computeNumChangedSegments(segmentIdList, segmentChanges); + + // Also remove segments from polled dataSources. + // Cache dataSourcesSnapshot locally to make sure that we do all updates to a single map, not to two different + // maps if poll() happens concurrently. + @MonotonicNonNull ConcurrentHashMap dataSourcesSnapshot = this.dataSources; + if (dataSourcesSnapshot != null) { + for (String segmentId : segmentIdList) { + removeSegmentFromPolledDataSources(dataSourceName, segmentId, dataSourcesSnapshot); + } + } + return numChangedSegments; }); } catch (Exception e) { throw new RuntimeException(e); } - return result[0]; } @Override - public int disableSegments(String dataSource, Interval interval) + public int markAsUnusedSegmentsInInterval(String dataSourceName, Interval interval) { try { - return connector.getDBI().withHandle( + Integer numUpdatedDatabaseEntries = connector.getDBI().withHandle( handle -> handle .createStatement( StringUtils @@ -566,37 +811,86 @@ public int disableSegments(String dataSource, Interval interval) getSegmentsTable(), connector.getQuoteString() )) - .bind("datasource", dataSource) + .bind("datasource", dataSourceName) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) .execute() ); + if (dataSources != null) { + dataSources.computeIfPresent(dataSourceName, (dsName, dataSource) -> { + dataSource.removeSegmentsIf(segment -> interval.contains(segment.getInterval())); + // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. + return dataSource.isEmpty() ? null : dataSource; + }); + } + return numUpdatedDatabaseEntries; } catch (Exception e) { throw new RuntimeException(e); } } - private boolean removeSegmentFromTable(String segmentId) + private boolean markSegmentAsUnusedInDatabase(String segmentId) { - final int removed = connector.getDBI().withHandle( + final int numUpdatedRows = connector.getDBI().withHandle( handle -> handle .createStatement(StringUtils.format("UPDATE %s SET used=false WHERE id = :segmentID", getSegmentsTable())) .bind("segmentID", segmentId) .execute() ); - return removed > 0; + if (numUpdatedRows < 0) { + log.assertionError( + "Negative number of rows updated for segment id [%s]: %d", + segmentId, + numUpdatedRows + ); + } else if (numUpdatedRows > 1) { + log.error( + "More than one row updated for segment id [%s]: %d, " + + "there may be more than one row for the segment id in the database", + segmentId, + numUpdatedRows + ); + } + return numUpdatedRows > 0; + } + + private static int computeNumChangedSegments(List segmentIds, int[] segmentChanges) + { + int numChangedSegments = 0; + for (int i = 0; i < segmentChanges.length; i++) { + int numUpdatedRows = segmentChanges[i]; + if (numUpdatedRows < 0) { + log.assertionError( + "Negative number of rows updated for segment id [%s]: %d", + segmentIds.get(i), + numUpdatedRows + ); + } else if (numUpdatedRows > 1) { + log.error( + "More than one row updated for segment id [%s]: %d, " + + "there may be more than one row for the segment id in the database", + segmentIds.get(i), + numUpdatedRows + ); + } + if (numUpdatedRows > 0) { + numChangedSegments += 1; + } + } + return numChangedSegments; } @Override - public boolean isStarted() + public boolean isPollingDatabasePeriodically() { - // isStarted() is synchronized together with start(), stop() and poll() to ensure that the latest currentStartOrder - // is always visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. - ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); + // isPollingDatabasePeriodically() is synchronized together with startPollingDatabasePeriodically(), + // stopPollingDatabasePeriodically() and poll() to ensure that the latest currentStartPollingOrder is always + // visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. + ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); lock.lock(); try { - return currentStartOrder >= 0; + return currentStartPollingOrder >= 0; } finally { lock.unlock(); @@ -604,68 +898,58 @@ public boolean isStarted() } @Override - @Nullable - public ImmutableDruidDataSource getDataSource(String dataSourceName) + public @Nullable ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSourceName) { - final DruidDataSource dataSource = Optional.ofNullable(dataSources).map(m -> m.get(dataSourceName)).orElse(null); - return dataSource == null ? null : dataSource.toImmutableDruidDataSource(); + awaitOrPerformDatabasePoll(); + final DruidDataSource dataSource = dataSources.get(dataSourceName); + return dataSource == null || dataSource.isEmpty() ? null : dataSource.toImmutableDruidDataSource(); } @Override - @Nullable - public Collection getDataSources() + public @Nullable DruidDataSource getDataSourceWithUsedSegments(String dataSource) { - return Optional.ofNullable(dataSources) - .map(m -> - m.values() - .stream() - .map(DruidDataSource::toImmutableDruidDataSource) - .collect(Collectors.toList()) - ) - .orElse(null); + awaitOrPerformDatabasePoll(); + return dataSources.get(dataSource); } @Override - @Nullable - public Iterable iterateAllSegments() + public Collection prepareImmutableDataSourcesWithAllUsedSegments() { - final ConcurrentHashMap dataSourcesSnapshot = dataSources; - if (dataSourcesSnapshot == null) { - return null; - } + awaitOrPerformDatabasePoll(); + return dataSources + .values() + .stream() + .map(DruidDataSource::toImmutableDruidDataSource) + .collect(Collectors.toList()); + } - return () -> dataSourcesSnapshot.values() - .stream() - .flatMap(dataSource -> dataSource.getSegments().stream()) - .iterator(); + @Override + public Iterable iterateAllUsedSegments() + { + awaitOrPerformDatabasePoll(); + return () -> dataSources + .values() + .stream() + .flatMap(dataSource -> dataSource.getSegments().stream()) + .iterator(); } @Override - public Collection getAllDataSourceNames() + public Collection retrieveAllDataSourceNames() { return connector.getDBI().withHandle( - handle -> handle.createQuery( - StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable()) - ) - .fold( - new ArrayList<>(), - new Folder3, Map>() - { - @Override - public List fold( - List druidDataSources, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) - { - druidDataSources.add( - MapUtils.getString(stringObjectMap, "datasource") - ); - return druidDataSources; - } - } - ) + handle -> handle + .createQuery(StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable())) + .fold( + new ArrayList<>(), + (List druidDataSources, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext) -> { + druidDataSources.add(MapUtils.getString(stringObjectMap, "datasource")); + return druidDataSources; + } + ) ); } @@ -674,15 +958,12 @@ public void poll() { // See the comment to the pollLock field, explaining this synchronized block synchronized (pollLock) { - try { - doPoll(); - } - catch (Exception e) { - log.makeAlert(e, "Problem polling DB.").emit(); - } + doPoll(); } } + /** This method is extracted from {@link #poll()} solely to reduce code nesting. */ + @GuardedBy("pollLock") private void doPoll() { log.debug("Starting polling of segment table"); @@ -713,6 +994,8 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE } catch (IOException e) { log.makeAlert(e, "Failed to read segment from db.").emit(); + // If one entry is database is corrupted, doPoll() should continue to work overall. See + // .filter(Objects::nonNull) below in this method. return null; } } @@ -724,7 +1007,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE ); if (segments == null || segments.isEmpty()) { - log.warn("No segments found in the database!"); + log.info("No segments found in the database!"); return; } @@ -732,20 +1015,25 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE ConcurrentHashMap newDataSources = new ConcurrentHashMap<>(); - ImmutableMap dataSourceProperties = ImmutableMap.of("created", DateTimes.nowUtc().toString()); + ImmutableMap dataSourceProperties = createDefaultDataSourceProperties(); segments .stream() - .filter(Objects::nonNull) + .filter(Objects::nonNull) // Filter corrupted entries (see above in this method). .forEach(segment -> { newDataSources .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties)) .addSegmentIfAbsent(segment); }); - // Replace "dataSources" atomically. + // Replace dataSources atomically. dataSources = newDataSources; } + private static ImmutableMap createDefaultDataSourceProperties() + { + return ImmutableMap.of("created", DateTimes.nowUtc().toString()); + } + /** * For the garbage collector in Java, it's better to keep new objects short-living, but once they are old enough * (i. e. promoted to old generation), try to keep them alive. In {@link #poll()}, we fetch and deserialize all @@ -757,7 +1045,10 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE */ private DataSegment replaceWithExistingSegmentIfPresent(DataSegment segment) { - DruidDataSource dataSource = Optional.ofNullable(dataSources).map(m -> m.get(segment.getDataSource())).orElse(null); + if (dataSources == null) { + return segment; + } + @Nullable DruidDataSource dataSource = dataSources.get(segment.getDataSource()); if (dataSource == null) { return segment; } @@ -771,11 +1062,7 @@ private String getSegmentsTable() } @Override - public List getUnusedSegmentIntervals( - final String dataSource, - final Interval interval, - final int limit - ) + public List getUnusedSegmentIntervals(final String dataSource, final DateTime maxEndTime, final int limit) { return connector.inReadOnlyTransaction( new TransactionCallback>() @@ -786,7 +1073,8 @@ public List inTransaction(Handle handle, TransactionStatus status) Iterator iter = handle .createQuery( StringUtils.format( - "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource and start >= :start and %2$send%2$s <= :end and used = false ORDER BY start, %2$send%2$s", + "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource AND " + + "%2$send%2$s <= :end AND used = false ORDER BY start, %2$send%2$s", getSegmentsTable(), connector.getQuoteString() ) @@ -794,8 +1082,7 @@ public List inTransaction(Handle handle, TransactionStatus status) .setFetchSize(connector.getStreamingFetchSize()) .setMaxRows(limit) .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) + .bind("end", maxEndTime.toString()) .map( new BaseResultSetMapper() { diff --git a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java index cca37b9318fa..6362077a2010 100644 --- a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java +++ b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java @@ -19,13 +19,23 @@ package org.apache.druid.metadata; +import java.util.Collection; + /** - * Exception thrown by MetadataSegmentManager when an segment id is unknown. + * Exception thrown by {@link MetadataSegmentManager} when a segment id is unknown. */ -public class UnknownSegmentIdException extends RuntimeException +public class UnknownSegmentIdException extends Exception { - public UnknownSegmentIdException(String message) + private final Collection unknownSegmentIds; + + UnknownSegmentIdException(Collection segmentIds) + { + super("Cannot find segment ids " + segmentIds); + this.unknownSegmentIds = segmentIds; + } + + public Collection getUnknownSegmentIds() { - super(message); + return unknownSegmentIds; } } diff --git a/server/src/main/java/org/apache/druid/server/JettyUtils.java b/server/src/main/java/org/apache/druid/server/JettyUtils.java index 9374fb07d931..717eb136b43a 100644 --- a/server/src/main/java/org/apache/druid/server/JettyUtils.java +++ b/server/src/main/java/org/apache/druid/server/JettyUtils.java @@ -19,10 +19,16 @@ package org.apache.druid.server; +import org.apache.druid.java.util.common.logger.Logger; + import javax.annotation.Nullable; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.UriInfo; public class JettyUtils { + private static final Logger log = new Logger(JettyUtils.class); + /** * Concatenate URI parts, in a way that is useful for proxy servlets. * @@ -46,4 +52,38 @@ public static String concatenateForRewrite( return url.toString(); } + + /** + * Returns the value of the query parameter of the given name. If not found, but there is a value corresponding to + * the parameter of the given compatiblityName it is returned instead and a warning is logged suggestion to make + * queries using the new parameter name. + * + * This method is useful for renaming query parameters (from name to compatiblityName) while preserving backward + * compatibility of the REST API. + */ + @Nullable + public static String getQueryParam(UriInfo uriInfo, String name, String compatiblityName) + { + MultivaluedMap queryParameters = uriInfo.getQueryParameters(); + // Returning the first value, according to the @QueryParam spec: + // https://docs.oracle.com/javaee/7/api/javax/ws/rs/QueryParam.html: + // "If the type is not one of the collection types listed in 5 above and the query parameter is represented by + // multiple values then the first value (lexically) of the parameter is used." + String paramValue = queryParameters.getFirst(name); + if (paramValue != null) { + return paramValue; + } + String compatibilityParamValue = queryParameters.getFirst(compatiblityName); + if (compatibilityParamValue != null) { + log.warn( + "Parameter %s in %s query has been renamed to %s. Use the new parameter name.", + compatiblityName, + uriInfo.getPath(), + name + ); + return compatibilityParamValue; + } + // Not found neither name nor compatiblityName + return null; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 14bf3395add3..b26eed745db5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -48,7 +48,7 @@ public class CoordinatorDynamicConfig { public static final String CONFIG_KEY = "coordinator.config"; - private final long millisToWaitBeforeDeleting; + private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; @@ -56,13 +56,25 @@ public class CoordinatorDynamicConfig private final int replicationThrottleLimit; private final int balancerComputeThreads; private final boolean emitBalancingStats; - private final boolean killAllDataSources; - private final Set killableDataSources; + + /** If true, {@link DruidCoordinatorUnusedSegmentsKiller} sends kill tasks for unused segments in all data sources. */ + private final boolean killUnusedSegmentsInAllDataSources; + + /** + * List of specific data sources for which kill tasks are sent in {@link DruidCoordinatorUnusedSegmentsKiller}. + */ + private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; private final int decommissioningMaxPercentOfMaxSegmentsToMove; - // The pending segments of the dataSources in this list are not killed. - private final Set protectedPendingSegmentDatasources; + /** + * Stale pending segments belonging to the data sources in this list are not killed by {@link + * DruidCoordinatorCleanupPendingSegments}. In other words, segments in these data sources are "protected". + * + * Pending segments are considered "stale" when their created_time is older than {@link + * DruidCoordinatorCleanupPendingSegments#KEEP_PENDING_SEGMENTS_OFFSET} from now. + */ + private final Set dataSourcesToNotKillStalePendingSegmentsIn; /** * The maximum number of segments that could be queued for loading to any given server. @@ -74,7 +86,10 @@ public class CoordinatorDynamicConfig @JsonCreator public CoordinatorDynamicConfig( - @JsonProperty("millisToWaitBeforeDeleting") long millisToWaitBeforeDeleting, + // Keeping the legacy 'millisToWaitBeforeDeleting' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("millisToWaitBeforeDeleting") + long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, @@ -82,19 +97,26 @@ public CoordinatorDynamicConfig( @JsonProperty("replicationThrottleLimit") int replicationThrottleLimit, @JsonProperty("balancerComputeThreads") int balancerComputeThreads, @JsonProperty("emitBalancingStats") boolean emitBalancingStats, - - // Type is Object here so that we can support both string and list as - // coordinator console can not send array of strings in the update request. - // See https://github.com/apache/incubator-druid/issues/3055 - @JsonProperty("killDataSourceWhitelist") Object killableDataSources, - @JsonProperty("killAllDataSources") boolean killAllDataSources, - @JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources, + // Type is Object here so that we can support both string and list as Coordinator console can not send array of + // strings in the update request. See https://github.com/apache/incubator-druid/issues/3055. + // Keeping the legacy 'killDataSourceWhitelist' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killDataSourceWhitelist") Object specificDataSourcesToKillUnusedSegmentsIn, + // Keeping the legacy 'killAllDataSources' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killAllDataSources") boolean killUnusedSegmentsInAllDataSources, + // Type is Object here so that we can support both string and list as Coordinator console can not send array of + // strings in the update request, as well as for specificDataSourcesToKillUnusedSegmentsIn. + // Keeping the legacy 'killPendingSegmentsSkipList' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killPendingSegmentsSkipList") Object dataSourcesToNotKillStalePendingSegmentsIn, @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue, @JsonProperty("decommissioningNodes") Object decommissioningNodes, @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") int decommissioningMaxPercentOfMaxSegmentsToMove ) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -102,9 +124,10 @@ public CoordinatorDynamicConfig( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.killableDataSources = parseJsonStringOrArray(killableDataSources); - this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; + this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); + this.dataSourcesToNotKillStalePendingSegmentsIn = + parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); Preconditions.checkArgument( @@ -113,8 +136,10 @@ public CoordinatorDynamicConfig( ); this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; - if (this.killAllDataSources && !this.killableDataSources.isEmpty()) { - throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); + if (this.killUnusedSegmentsInAllDataSources && !this.specificDataSourcesToKillUnusedSegmentsIn.isEmpty()) { + throw new IAE( + "can't have killUnusedSegmentsInAllDataSources and non-empty specificDataSourcesToKillUnusedSegmentsIn" + ); } } @@ -152,10 +177,10 @@ public static CoordinatorDynamicConfig current(final JacksonConfigManager config return Preconditions.checkNotNull(watch(configManager).get(), "Got null config from watcher?!"); } - @JsonProperty - public long getMillisToWaitBeforeDeleting() + @JsonProperty("millisToWaitBeforeDeleting") + public long getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() { - return millisToWaitBeforeDeleting; + return leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; } @JsonProperty @@ -200,30 +225,22 @@ public int getBalancerComputeThreads() return balancerComputeThreads; } - /** - * List of dataSources for which kill tasks are sent in - * {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. - */ @JsonProperty("killDataSourceWhitelist") - public Set getKillableDataSources() + public Set getSpecificDataSourcesToKillUnusedSegmentsIn() { - return killableDataSources; + return specificDataSourcesToKillUnusedSegmentsIn; } - @JsonProperty - public boolean isKillAllDataSources() + @JsonProperty("killAllDataSources") + public boolean isKillUnusedSegmentsInAllDataSources() { - return killAllDataSources; + return killUnusedSegmentsInAllDataSources; } - /** - * List of dataSources for which pendingSegments are NOT cleaned up - * in {@link DruidCoordinatorCleanupPendingSegments}. - */ - @JsonProperty - public Set getProtectedPendingSegmentDatasources() + @JsonProperty("killPendingSegmentsSkipList") + public Set getDataSourcesToNotKillStalePendingSegmentsIn() { - return protectedPendingSegmentDatasources; + return dataSourcesToNotKillStalePendingSegmentsIn; } @JsonProperty @@ -233,9 +250,9 @@ public int getMaxSegmentsInNodeLoadingQueue() } /** - * List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, - * and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by - * {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}. + * List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' + * servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate + * specified by {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}. * * @return list of host:port entries */ @@ -270,7 +287,8 @@ public int getDecommissioningMaxPercentOfMaxSegmentsToMove() public String toString() { return "CoordinatorDynamicConfig{" + - "millisToWaitBeforeDeleting=" + millisToWaitBeforeDeleting + + "leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments=" + + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments + ", mergeBytesLimit=" + mergeBytesLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit + ", maxSegmentsToMove=" + maxSegmentsToMove + @@ -278,9 +296,9 @@ public String toString() ", replicationThrottleLimit=" + replicationThrottleLimit + ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + - ", killAllDataSources=" + killAllDataSources + - ", killDataSourceWhitelist=" + killableDataSources + - ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + + ", killUnusedSegmentsInAllDataSources=" + killUnusedSegmentsInAllDataSources + + ", specificDataSourcesToKillUnusedSegmentsIn=" + specificDataSourcesToKillUnusedSegmentsIn + + ", dataSourcesToNotKillStalePendingSegmentsIn=" + dataSourcesToNotKillStalePendingSegmentsIn + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", decommissioningNodes=" + decommissioningNodes + ", decommissioningMaxPercentOfMaxSegmentsToMove=" + decommissioningMaxPercentOfMaxSegmentsToMove + @@ -299,7 +317,8 @@ public boolean equals(Object o) CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o; - if (millisToWaitBeforeDeleting != that.millisToWaitBeforeDeleting) { + if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments != + that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) { return false; } if (mergeBytesLimit != that.mergeBytesLimit) { @@ -323,16 +342,16 @@ public boolean equals(Object o) if (emitBalancingStats != that.emitBalancingStats) { return false; } - if (killAllDataSources != that.killAllDataSources) { + if (killUnusedSegmentsInAllDataSources != that.killUnusedSegmentsInAllDataSources) { return false; } if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { return false; } - if (!Objects.equals(killableDataSources, that.killableDataSources)) { + if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) { return false; } - if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) { + if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) { return false; } if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) { @@ -345,7 +364,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( - millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, @@ -353,10 +372,10 @@ public int hashCode() replicationThrottleLimit, balancerComputeThreads, emitBalancingStats, - killAllDataSources, + killUnusedSegmentsInAllDataSources, maxSegmentsInNodeLoadingQueue, - killableDataSources, - protectedPendingSegmentDatasources, + specificDataSourcesToKillUnusedSegmentsIn, + dataSourcesToNotKillStalePendingSegmentsIn, decommissioningNodes, decommissioningMaxPercentOfMaxSegmentsToMove ); @@ -369,19 +388,20 @@ public static Builder builder() public static class Builder { - private static final long DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING = TimeUnit.MINUTES.toMillis(15); - private static final long DEFAULT_MERGE_BYTES_LIMIT = 524288000L; + private static final long DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = + TimeUnit.MINUTES.toMillis(15); + private static final long DEFAULT_MERGE_BYTES_LIMIT = 524_288_000L; private static final int DEFAULT_MERGE_SEGMENTS_LIMIT = 100; private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 5; private static final int DEFAULT_REPLICANT_LIFETIME = 15; private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 10; private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1; private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; - private static final boolean DEFAULT_KILL_ALL_DATA_SOURCES = false; + private static final boolean DEFAULT_KILL_UNUSED_SEGMENTS_IN_ALL_DATA_SOURCES = false; private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0; private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; - private Long millisToWaitBeforeDeleting; + private Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; private Long mergeBytesLimit; private Integer mergeSegmentsLimit; private Integer maxSegmentsToMove; @@ -389,9 +409,9 @@ public static class Builder private Integer replicationThrottleLimit; private Boolean emitBalancingStats; private Integer balancerComputeThreads; - private Object killableDataSources; - private Boolean killAllDataSources; - private Object killPendingSegmentsSkipList; + private Object specificDataSourcesToKillUnusedSegmentsIn; + private Boolean killUnusedSegmentsInAllDataSources; + private Object dataSourcesToNotKillStalePendingSegmentsIn; private Integer maxSegmentsInNodeLoadingQueue; private Object decommissioningNodes; private Integer decommissioningMaxPercentOfMaxSegmentsToMove; @@ -402,7 +422,8 @@ public Builder() @JsonCreator public Builder( - @JsonProperty("millisToWaitBeforeDeleting") @Nullable Long millisToWaitBeforeDeleting, + @JsonProperty("millisToWaitBeforeDeleting") + @Nullable Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, @@ -410,15 +431,17 @@ public Builder( @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, - @JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources, - @JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources, - @JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList, + @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKillUnusedSegmentsIn, + @JsonProperty("killAllDataSources") @Nullable Boolean killUnusedSegmentsInAllDataSources, + @JsonProperty("killPendingSegmentsSkipList") @Nullable Object dataSourcesToNotKillStalePendingSegmentsIn, @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, @JsonProperty("decommissioningNodes") @Nullable Object decommissioningNodes, - @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") @Nullable Integer decommissioningMaxPercentOfMaxSegmentsToMove + @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") + @Nullable Integer decommissioningMaxPercentOfMaxSegmentsToMove ) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -426,17 +449,17 @@ public Builder( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = balancerComputeThreads; this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.killableDataSources = killableDataSources; - this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; + this.specificDataSourcesToKillUnusedSegmentsIn = specificDataSourcesToKillUnusedSegmentsIn; + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; + this.dataSourcesToNotKillStalePendingSegmentsIn = dataSourcesToNotKillStalePendingSegmentsIn; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = decommissioningNodes; this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; } - public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting) + public Builder withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(long leadingTimeMillis) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = leadingTimeMillis; return this; } @@ -482,15 +505,15 @@ public Builder withEmitBalancingStats(boolean emitBalancingStats) return this; } - public Builder withKillDataSourceWhitelist(Set killDataSourceWhitelist) + public Builder withSpecificDataSourcesToKillUnusedSegmentsIn(Set dataSources) { - this.killableDataSources = killDataSourceWhitelist; + this.specificDataSourcesToKillUnusedSegmentsIn = dataSources; return this; } - public Builder withKillAllDataSources(boolean killAllDataSources) + public Builder withKillUnusedSegmentsInAllDataSources(boolean killUnusedSegmentsInAllDataSources) { - this.killAllDataSources = killAllDataSources; + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; return this; } @@ -515,7 +538,9 @@ public Builder withDecommissioningMaxPercentOfMaxSegmentsToMove(Integer percent) public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING : millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null + ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit, mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit, maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove, @@ -523,9 +548,11 @@ public CoordinatorDynamicConfig build() replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, - killableDataSources, - killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, - killPendingSegmentsSkipList, + specificDataSourcesToKillUnusedSegmentsIn, + killUnusedSegmentsInAllDataSources == null + ? DEFAULT_KILL_UNUSED_SEGMENTS_IN_ALL_DATA_SOURCES + : killUnusedSegmentsInAllDataSources, + dataSourcesToNotKillStalePendingSegmentsIn, maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE : maxSegmentsInNodeLoadingQueue, @@ -539,7 +566,9 @@ public CoordinatorDynamicConfig build() public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? defaults.getMillisToWaitBeforeDeleting() : millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null + ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit, mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit, maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove, @@ -547,11 +576,15 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, - killableDataSources == null ? defaults.getKillableDataSources() : killableDataSources, - killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, - killPendingSegmentsSkipList == null - ? defaults.getProtectedPendingSegmentDatasources() - : killPendingSegmentsSkipList, + specificDataSourcesToKillUnusedSegmentsIn == null + ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn() + : specificDataSourcesToKillUnusedSegmentsIn, + killUnusedSegmentsInAllDataSources == null + ? defaults.isKillUnusedSegmentsInAllDataSources() + : killUnusedSegmentsInAllDataSources, + dataSourcesToNotKillStalePendingSegmentsIn == null + ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn() + : dataSourcesToNotKillStalePendingSegmentsIn, maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() : maxSegmentsInNodeLoadingQueue, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java index d2d3029b17b3..6f2373b3f8e9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java @@ -117,7 +117,7 @@ public static double intervalCost(double x1, double y0, double y1) // since x_0 <= y_0, Y must overlap X if y_0 < x_1 if (y0 < x1) { - /** + /* * We have two possible cases of overlap: * * X = [ A )[ B )[ C ) or [ A )[ B ) @@ -151,7 +151,7 @@ public static double intervalCost(double x1, double y0, double y1) intervalCost(beta, beta, gamma) + // cost(B, C) 2 * (beta + FastMath.exp(-beta) - 1); // cost(B, B) } else { - /** + /* * In the case where there is no overlap: * * Given that x_0 <= y_0, @@ -258,9 +258,14 @@ public double calculateInitialTotalCost(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { - Iterable segments = server.getServer().getLazyAllSegments(); - for (DataSegment s : segments) { - cost += computeJointSegmentsCost(s, segments); + // segments are dumped into an array because it's probably better than iterating the iterateAllSegments() result + // quadratically in a loop, which can generate garbage in the form of Stream, Spliterator, Iterator, etc. objects + // whose total memory volume exceeds the size of the DataSegment array. + DataSegment[] segments = server.getServer().iterateAllSegments().toArray(new DataSegment[0]); + for (DataSegment s1 : segments) { + for (DataSegment s2 : segments) { + cost += computeJointSegmentsCost(s1, s2); + } } } return cost; @@ -280,7 +285,7 @@ public double calculateNormalization(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { - for (DataSegment segment : server.getServer().getLazyAllSegments()) { + for (DataSegment segment : server.getServer().iterateAllSegments()) { cost += computeJointSegmentsCost(segment, segment); } } @@ -288,10 +293,7 @@ public double calculateNormalization(final List serverHolders) } @Override - public void emitStats( - String tier, - CoordinatorStats stats, List serverHolderList - ) + public void emitStats(String tier, CoordinatorStats stats, List serverHolderList) { final double initialTotalCost = calculateInitialTotalCost(serverHolderList); final double normalization = calculateNormalization(serverHolderList); @@ -334,7 +336,7 @@ protected double computeCost( // the sum of the costs of other (exclusive of the proposalSegment) segments on the server cost += computeJointSegmentsCost( proposalSegment, - Iterables.filter(server.getServer().getLazyAllSegments(), segment -> !proposalSegment.equals(segment)) + Iterables.filter(server.getServer().iterateAllSegments(), segment -> !proposalSegment.equals(segment)) ); // plus the costs of segments that will be loaded diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java index dff28710a006..8e559b4a2741 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java @@ -47,8 +47,8 @@ protected double computeCost( } int nSegments = 1; - if (server.getServer().getLazyAllSegments().size() > 0) { - nSegments = server.getServer().getLazyAllSegments().size(); + if (server.getServer().getNumSegments() > 0) { + nSegments = server.getServer().getNumSegments(); } double normalizedCost = cost / nSegments; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index 586f92b2dd79..2001b3fa4e7f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -22,11 +22,13 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -35,7 +37,6 @@ import java.util.Set; import java.util.TreeSet; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** * Contains a representation of the current state of the cluster by tier. @@ -43,6 +44,16 @@ */ public class DruidCluster { + /** This static factory method must be called only from inside DruidClusterBuilder in tests. */ + @VisibleForTesting + static DruidCluster createDruidClusterFromBuilderInTest( + @Nullable Set realtimes, + Map> historicals + ) + { + return new DruidCluster(realtimes, historicals); + } + private final Set realtimes; private final Map> historicals; @@ -52,8 +63,7 @@ public DruidCluster() this.historicals = new HashMap<>(); } - @VisibleForTesting - public DruidCluster( + private DruidCluster( @Nullable Set realtimes, Map> historicals ) @@ -62,12 +72,13 @@ public DruidCluster( this.historicals = historicals .entrySet() .stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - e -> StreamSupport - .stream(e.getValue().spliterator(), false) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - )); + .collect( + Collectors.toMap( + Map.Entry::getKey, + (Map.Entry> e) -> + CollectionUtils.newTreeSet(Comparator.reverseOrder(), e.getValue()) + ) + ); } public void add(ServerHolder serverHolder) 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 27bee2f0d225..a9ed5110b17b 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 @@ -25,6 +25,9 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntMaps; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.curator.framework.CuratorFramework; @@ -58,12 +61,12 @@ import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnneeded; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; import org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnneeded; import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentInfoLoader; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; @@ -74,7 +77,6 @@ import org.joda.time.DateTime; import org.joda.time.Duration; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -101,7 +103,7 @@ public class DruidCoordinator * It is used in historical nodes' {@link LoadQueuePeon}s to make historicals load more recent segment first. * * It is also used in {@link DruidCoordinatorRuntimeParams} for {@link - * DruidCoordinatorRuntimeParams#getAvailableSegments()} - a collection of segments to be considered during some + * DruidCoordinatorRuntimeParams#getUsedSegments()} - a collection of segments to be considered during some * coordinator run for different {@link DruidCoordinatorHelper}s. The order matters only for {@link * DruidCoordinatorRuleRunner}, which tries to apply the rules while iterating the segments in the order imposed by * this comparator. In {@link LoadRule} the throttling limit may be hit (via {@link ReplicationThrottler}; see @@ -125,7 +127,7 @@ public class DruidCoordinator private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegmentManager segmentsMetadata; private final ServerInventoryView serverInventoryView; private final MetadataRuleManager metadataRuleManager; private final CuratorFramework curator; @@ -151,7 +153,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -171,7 +173,7 @@ public DruidCoordinator( config, zkPaths, configManager, - metadataSegmentManager, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -193,7 +195,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -214,7 +216,7 @@ public DruidCoordinator( this.zkPaths = zkPaths; this.configManager = configManager; - this.metadataSegmentManager = metadataSegmentManager; + this.segmentsMetadata = segmentsMetadata; this.serverInventoryView = serverInventoryView; this.metadataRuleManager = metadataRuleManager; this.curator = curator; @@ -256,11 +258,7 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - final Iterable dataSegments = iterateAvailableDataSegments(); - - if (dataSegments == null) { - return underReplicationCountsPerDataSourcePerTier; - } + final Iterable dataSegments = iterateAllUsedSegments(); final DateTime now = DateTimes.nowUtc(); @@ -288,43 +286,36 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - public Object2LongMap getSegmentAvailability() + public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() { - final Object2LongOpenHashMap retVal = new Object2LongOpenHashMap<>(); - if (segmentReplicantLookup == null) { - return retVal; + return Object2IntMaps.emptyMap(); } - final Iterable dataSegments = iterateAvailableDataSegments(); + final Object2IntOpenHashMap numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>(); - if (dataSegments == null) { - return retVal; - } + final Iterable dataSegments = iterateAllUsedSegments(); for (DataSegment segment : dataSegments) { if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) { - retVal.addTo(segment.getDataSource(), 1); + numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 1); } else { - retVal.addTo(segment.getDataSource(), 0); + numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 0); } } - return retVal; + return numsUnavailableUsedSegmentsPerDataSource; } public Map getLoadStatus() { final Map loadStatus = new HashMap<>(); - final Collection dataSources = metadataSegmentManager.getDataSources(); - - if (dataSources == null) { - return loadStatus; - } + final Collection dataSources = + segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); for (ImmutableDruidDataSource dataSource : dataSources) { final Set segments = Sets.newHashSet(dataSource.getSegments()); - final int availableSegmentSize = segments.size(); + final int numUsedSegments = segments.size(); // remove loaded segments for (DruidServer druidServer : serverInventoryView.getInventory()) { @@ -337,10 +328,10 @@ public Map getLoadStatus() } } } - final int unloadedSegmentSize = segments.size(); + final int numUnloadedSegments = segments.size(); loadStatus.put( dataSource.getName(), - 100 * ((double) (availableSegmentSize - unloadedSegmentSize) / (double) availableSegmentSize) + 100 * ((double) (numUsedSegments - numUnloadedSegments) / (double) numUsedSegments) ); } @@ -362,10 +353,10 @@ public CoordinatorCompactionConfig getCompactionConfig() return CoordinatorCompactionConfig.current(configManager); } - public void removeSegment(DataSegment segment) + public void markSegmentAsUnused(DataSegment segment) { - log.info("Removing Segment[%s]", segment.getId()); - metadataSegmentManager.removeSegment(segment.getId()); + log.info("Marking segment[%s] as unused", segment.getId()); + segmentsMetadata.markSegmentAsUnused(segment.getId()); } public String getCurrentLeader() @@ -393,7 +384,8 @@ public void moveSegment( throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); } - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(segment.getDataSource()); + ImmutableDruidDataSource dataSource = + segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(segment.getDataSource()); if (dataSource == null) { throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); } @@ -472,18 +464,16 @@ public void moveSegment( } /** - * Returns an iterable to go over all available segments in all data sources. The order in which segments are iterated - * is unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try - * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than - * several times. + * Returns an iterable to go over all used segments in all data sources. The order in which segments are iterated + * is unspecified. * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) + * Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try (to some + * reasonable extent) to organize the code so that it iterates the returned iterable only once rather than several + * times. */ - @Nullable - public Iterable iterateAvailableDataSegments() + public Iterable iterateAllUsedSegments() { - return metadataSegmentManager.iterateAllSegments(); + return segmentsMetadata.iterateAllUsedSegments(); } @LifecycleStart @@ -540,7 +530,7 @@ private void becomeLeader() log.info("I am the leader of the coordinators, all must bow!"); log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay()); - metadataSegmentManager.start(); + segmentsMetadata.startPollingDatabasePeriodically(); metadataRuleManager.start(); lookupCoordinatorManager.start(); serviceAnnouncer.announce(self); @@ -608,7 +598,7 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); metadataRuleManager.stop(); - metadataSegmentManager.stop(); + segmentsMetadata.stopPollingDatabasePeriodically(); } } @@ -628,7 +618,7 @@ private List makeIndexingServiceHelpers() public abstract class CoordinatorRunnable implements Runnable { - private final long startTime = System.currentTimeMillis(); + private final long startTimeNanos = System.nanoTime(); private final List helpers; private final int startingLeaderCounter; @@ -652,7 +642,7 @@ public void run() } List allStarted = Arrays.asList( - metadataSegmentManager.isStarted(), + segmentsMetadata.isPollingDatabasePeriodically(), serverInventoryView.isStarted() ); for (Boolean aBoolean : allStarted) { @@ -670,21 +660,19 @@ public void run() BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec); // Do coordinator stuff. - final Collection dataSources = metadataSegmentManager.getDataSources(); - if (dataSources == null) { - log.info("Metadata store not polled yet, skipping this run."); - return; - } + final Collection dataSources = + segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withStartTime(startTime) - .withDataSources(dataSources) - .withDynamicConfigs(getDynamicConfigs()) - .withCompactionConfig(getCompactionConfig()) - .withEmitter(emitter) - .withBalancerStrategy(balancerStrategy) - .build(); + DruidCoordinatorRuntimeParams + .newBuilder() + .withStartTimeNanos(startTimeNanos) + .withDataSourcesWithUsedSegments(dataSources) + .withDynamicConfigs(getDynamicConfigs()) + .withCompactionConfig(getCompactionConfig()) + .withEmitter(emitter) + .withBalancerStrategy(balancerStrategy) + .build(); for (DruidCoordinatorHelper helper : helpers) { // Don't read state and run state in the same helper otherwise racy conditions may exist if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index bf4ffdee80e5..4086dc91d844 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -69,15 +69,15 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // is no running/pending/waiting tasks. Preconditions.checkState(!createdTimes.isEmpty(), "Failed to gather createdTimes of tasks"); - // If there is no running/pending/waiting/complete tasks, pendingSegmentsCleanupEndTime is + // If there is no running/pending/waiting/complete tasks, stalePendingSegmentsCutoffCreationTime is // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). - final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); - for (String dataSource : params.getDataSources().keySet()) { - if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { + final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); + for (String dataSource : params.getDataSourcesWithUsedSegments().keySet()) { + if (!params.getCoordinatorDynamicConfig().getDataSourcesToNotKillStalePendingSegmentsIn().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", - indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime), - pendingSegmentsCleanupEndTime, + indexingServiceClient.killPendingSegments(dataSource, stalePendingSegmentsCutoffCreationTime), + stalePendingSegmentsCutoffCreationTime, dataSource ); } 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 655d6bdf22ff..e56923f52cde 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 @@ -35,29 +35,34 @@ import java.util.HashMap; import java.util.Map; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; /** */ public class DruidCoordinatorRuntimeParams { /** + * Creates a set to be provided to {@link Builder#setUsedSegments(TreeSet)} method from the given {@link + * Iterable} of segments. + * * Creates a TreeSet sorted in {@link DruidCoordinator#SEGMENT_COMPARATOR_RECENT_FIRST} order and populates it with * the segments from the given iterable. The given iterable is iterated exactly once. No special action is taken if * duplicate segments are encountered in the iterable. */ - public static TreeSet createAvailableSegmentsSet(Iterable availableSegments) + public static TreeSet createUsedSegmentsSet(Iterable usedSegments) { TreeSet segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); - availableSegments.forEach(segmentsSet::add); + usedSegments.forEach(segmentsSet::add); return segmentsSet; } - private final long startTime; + private final long startTimeNanos; private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; private final SegmentReplicantLookup segmentReplicantLookup; - private final Map> dataSources; - private final @Nullable TreeSet availableSegments; + /** dataSource -> VersionedIntervalTimeline[version String, DataSegment] */ + private final Map> dataSourcesWithUsedSegments; + private final @Nullable TreeSet usedSegments; private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; @@ -68,12 +73,12 @@ public static TreeSet createAvailableSegmentsSet(Iterable> dataSources, - @Nullable TreeSet availableSegments, + Map> dataSourcesWithUsedSegments, + @Nullable TreeSet usedSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -84,12 +89,12 @@ private DruidCoordinatorRuntimeParams( BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; + this.usedSegments = usedSegments; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -100,9 +105,9 @@ private DruidCoordinatorRuntimeParams( this.balancerStrategy = balancerStrategy; } - public long getStartTime() + public long getStartTimeNanos() { - return startTime; + return startTimeNanos; } public DruidCluster getDruidCluster() @@ -120,15 +125,18 @@ public SegmentReplicantLookup getSegmentReplicantLookup() return segmentReplicantLookup; } - public Map> getDataSources() + /** + * Returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" segments. + */ + public Map> getDataSourcesWithUsedSegments() { - return dataSources; + return dataSourcesWithUsedSegments; } - public TreeSet getAvailableSegments() + public TreeSet getUsedSegments() { - Preconditions.checkState(availableSegments != null, "availableSegments must be set"); - return availableSegments; + Preconditions.checkState(usedSegments != null, "usedSegments must be set"); + return usedSegments; } public Map getLoadManagementPeons() @@ -171,9 +179,13 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } - public boolean hasDeletionWaitTimeElapsed() + public boolean lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements() { - return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting()); + long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); + long lagNanos = TimeUnit.MILLISECONDS.toNanos( + coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + ); + return nanosElapsedSinceCoordinatorStart > lagNanos; } public static Builder newBuilder() @@ -184,12 +196,12 @@ public static Builder newBuilder() public Builder buildFromExisting() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + dataSourcesWithUsedSegments, + usedSegments, loadManagementPeons, replicationManager, emitter, @@ -201,15 +213,15 @@ public Builder buildFromExisting() ); } - public Builder buildFromExistingWithoutAvailableSegments() + public Builder buildFromExistingWithoutUsedSegments() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - null, // availableSegments + dataSourcesWithUsedSegments, + null, // usedSegments loadManagementPeons, replicationManager, emitter, @@ -223,12 +235,12 @@ public Builder buildFromExistingWithoutAvailableSegments() public static class Builder { - private long startTime; + private @Nullable Long startTimeNanos; private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; - private Map> dataSources; - private @Nullable TreeSet availableSegments; + private Map> dataSourcesWithUsedSegments; + private @Nullable TreeSet usedSegments; private final Map loadManagementPeons; private ReplicationThrottler replicationManager; private ServiceEmitter emitter; @@ -238,14 +250,14 @@ public static class Builder private DateTime balancerReferenceTimestamp; private BalancerStrategy balancerStrategy; - Builder() + private Builder() { - this.startTime = 0; + this.startTimeNanos = null; this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; - this.dataSources = new HashMap<>(); - this.availableSegments = null; + this.dataSourcesWithUsedSegments = new HashMap<>(); + this.usedSegments = null; this.loadManagementPeons = new HashMap<>(); this.replicationManager = null; this.emitter = null; @@ -256,12 +268,12 @@ public static class Builder } Builder( - long startTime, + long startTimeNanos, DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSources, - @Nullable TreeSet availableSegments, + Map> dataSourcesWithUsedSegments, + @Nullable TreeSet usedSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -272,12 +284,12 @@ public static class Builder BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; + this.usedSegments = usedSegments; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -290,13 +302,14 @@ public static class Builder public DruidCoordinatorRuntimeParams build() { + Preconditions.checkNotNull(startTimeNanos, "startTime must be set"); return new DruidCoordinatorRuntimeParams( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + dataSourcesWithUsedSegments, + usedSegments, loadManagementPeons, replicationManager, emitter, @@ -308,9 +321,9 @@ public DruidCoordinatorRuntimeParams build() ); } - public Builder withStartTime(long time) + public Builder withStartTimeNanos(long startTimeNanos) { - startTime = time; + this.startTimeNanos = startTimeNanos; return this; } @@ -332,26 +345,29 @@ public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) return this; } - public Builder withDataSources(Map> dataSources) + @VisibleForTesting + public Builder setDataSourcesWithUsedSegments( + Map> dataSources + ) { - this.dataSources = dataSources; + this.dataSourcesWithUsedSegments = dataSources; return this; } - public Builder withDataSources(Collection dataSourcesCollection) + Builder withDataSourcesWithUsedSegments(Collection dataSourcesWithUsedSegments) { - dataSourcesCollection.forEach( + dataSourcesWithUsedSegments.forEach( dataSource -> { - VersionedIntervalTimeline timeline = dataSources.computeIfAbsent( + VersionedIntervalTimeline timeline = this.dataSourcesWithUsedSegments.computeIfAbsent( dataSource.getName(), k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER) ); dataSource.getSegments().forEach( - segment -> timeline.add( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk(segment) + usedSegment -> timeline.add( + usedSegment.getInterval(), + usedSegment.getVersion(), + usedSegment.getShardSpec().createChunk(usedSegment) ) ); } @@ -361,29 +377,29 @@ public Builder withDataSources(Collection dataSourcesC /** This method must be used in test code only. */ @VisibleForTesting - public Builder withAvailableSegmentsInTest(DataSegment... availableSegments) + public Builder withUsedSegmentsInTest(DataSegment... usedSegments) { - return withAvailableSegmentsInTest(Arrays.asList(availableSegments)); + return withUsedSegmentsInTest(Arrays.asList(usedSegments)); } /** This method must be used in test code only. */ @VisibleForTesting - public Builder withAvailableSegmentsInTest(Collection availableSegments) + public Builder withUsedSegmentsInTest(Collection usedSegments) { - return setAvailableSegments(createAvailableSegmentsSet(availableSegments)); + return setUsedSegments(createUsedSegmentsSet(usedSegments)); } /** - * Note: unlike {@link #withAvailableSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the + * Note: unlike {@link #withUsedSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the * provided set. The set passed into this method must not be modified afterwards. */ - public Builder setAvailableSegments(TreeSet availableSegments) + public Builder setUsedSegments(TreeSet usedSegments) { //noinspection ObjectEquality - if (availableSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { + if (usedSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST"); } - this.availableSegments = availableSegments; + this.usedSegments = usedSegments; return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java index e770ef78210c..c2c4a7ad10c4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java @@ -34,7 +34,7 @@ static BalancerSegmentHolder getRandomBalancerSegmentHolder(final List balanceServers( log.info("All servers to move segments from are empty, ending run."); break; } - // DruidCoordinatorRuntimeParams.getAvailableSegments originate from MetadataSegmentManager, i. e. that's a - // "desired" or "theoretical" set of segments. segmentToMoveHolder.getSegment originates from ServerInventoryView, - // i. e. that may be any segment that happens to be loaded on some server, even if it "shouldn't" from the - // "theoretical" point of view (Coordinator closes such discrepancies eventually via - // DruidCoordinatorCleanupUnneeded). Therefore the picked segmentToMoveHolder's segment may not need to be - // balanced. - boolean needToBalancePickedSegment = params.getAvailableSegments().contains(segmentToMoveHolder.getSegment()); + // DruidCoordinatorRuntimeParams.getUsedSegments originate from SegmentsMetadata, i. e. that's a set of segments + // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be + // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such + // discrepancies eventually via DruidCoordinatorUnloadUnusedSegments). Therefore the picked segmentToMoveHolder's + // segment may not need to be balanced. + boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment()); if (needToBalancePickedSegment) { final DataSegment segmentToMove = segmentToMoveHolder.getSegment(); final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 8131e4f62be9..b61d543f6945 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -46,41 +46,41 @@ public DruidCoordinatorCleanupOvershadowed(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - CoordinatorStats stats = new CoordinatorStats(); + // Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data. + if (!params.lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements()) { + return params; + } - // Delete segments that are old - // Unservice old partitions if we've had enough time to make sure we aren't flapping with old data - if (params.hasDeletionWaitTimeElapsed()) { - DruidCluster cluster = params.getDruidCluster(); - Map> timelines = new HashMap<>(); + CoordinatorStats stats = new CoordinatorStats(); - for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serverHolders) { - ImmutableDruidServer server = serverHolder.getServer(); + DruidCluster cluster = params.getDruidCluster(); + Map> timelines = new HashMap<>(); - for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - VersionedIntervalTimeline timeline = timelines.get(dataSource.getName()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); - timelines.put(dataSource.getName(), timeline); - } + for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { + for (ServerHolder serverHolder : serverHolders) { + ImmutableDruidServer server = serverHolder.getServer(); - VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); + for (ImmutableDruidDataSource dataSource : server.getDataSources()) { + VersionedIntervalTimeline timeline = timelines.get(dataSource.getName()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); + timelines.put(dataSource.getName(), timeline); } + + VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); } } + } - //Remove all segments in db that are overshadowed by served segments - for (DataSegment dataSegment : params.getAvailableSegments()) { - VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - coordinator.removeSegment(dataSegment); - stats.addToGlobalStat("overShadowedCount", 1); - } + // Mark all segments as unused in db that are overshadowed by served segments + for (DataSegment dataSegment : params.getUsedSegments()) { + VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); + if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { + coordinator.markSegmentAsUnused(dataSegment); + stats.addToGlobalStat("overShadowedCount", 1); } } - return params.buildFromExisting() - .withCoordinatorStats(stats) - .build(); + + return params.buildFromExisting().withCoordinatorStats(stats).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java index 2e77577b69b0..af7ccc1cd0a1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java @@ -43,22 +43,19 @@ public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { CoordinatorStats stats = new CoordinatorStats(); - Set availableSegments = params.getAvailableSegments(); + Set usedSegments = params.getUsedSegments(); DruidCluster cluster = params.getDruidCluster(); - // Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It's - // also filled atomically, so if there are any segments at all, we should have all of them.) - // - // Note that if the metadata store has not been polled yet, "getAvailableSegments" would throw an error since - // "availableSegments" is null. But this won't happen, since the earlier helper "DruidCoordinatorSegmentInfoLoader" - // would have canceled the run. + // Unload segments that are no longer marked as used from historical servers, *if* the usedSegments collection has + // been populated. Used segments must be already populated because otherwise the earlier helper + // DruidCoordinatorUsedSegmentsLoader would have canceled the Coordinator's run. for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); for (ImmutableDruidDataSource dataSource : server.getDataSources()) { for (DataSegment segment : dataSource.getSegments()) { - if (!availableSegments.contains(segment)) { + if (!usedSegments.contains(segment)) { LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); if (!queuePeon.getSegmentsToDrop().contains(segment)) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java index a271109f6a7c..efd3b62e6a84 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator.helper; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; @@ -214,14 +215,14 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) ); }); - coordinator.getSegmentAvailability().object2LongEntrySet().forEach( - (final Object2LongMap.Entry entry) -> { + coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach( + (final Object2IntMap.Entry entry) -> { final String dataSource = entry.getKey(); - final long count = entry.getLongValue(); + final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue(); emitter.emit( new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/unavailable/count", count + "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource ) ); } @@ -247,7 +248,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) emitter.emit( new ServiceMetricEvent.Builder().build( "compact/task/count", - stats.getGlobalStat("compactTaskCount") + stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT) ) ); @@ -264,7 +265,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Emit segment metrics final Stream allSegments = params - .getDataSources() + .getDataSourcesWithUsedSegments() .values() .stream() .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index bbceaafed273..51714e28f625 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -42,7 +42,7 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper { private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class); - private static int MAX_MISSING_RULES = 10; + private static final int MAX_MISSING_RULES = 10; private final ReplicationThrottler replicatorThrottler; @@ -81,28 +81,28 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - // find available segments which are not overshadowed by other segments in DB - // only those would need to be loaded/dropped - // anything overshadowed by served segments is dropped automatically by DruidCoordinatorCleanupOvershadowed - final Set overshadowed = ImmutableDruidDataSource - .determineOvershadowedSegments(params.getAvailableSegments()); + // Find used segments which are overshadowed by other used segments. Those would not need to be loaded and + // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked + // as unused in DruidCoordinatorMarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to + // Historical nodes to unload such segments in DruidCoordinatorUnloadUnusedSegments. + Set overshadowed = ImmutableDruidDataSource.determineOvershadowedSegments(params.getUsedSegments()); for (String tier : cluster.getTierNames()) { replicatorThrottler.updateReplicationState(tier); } DruidCoordinatorRuntimeParams paramsWithReplicationManager = params - .buildFromExistingWithoutAvailableSegments() + .buildFromExistingWithoutUsedSegments() .withReplicationManager(replicatorThrottler) .build(); - // Run through all matched rules for available segments + // Run through all matched rules for used segments DateTime now = DateTimes.nowUtc(); MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); int missingRules = 0; - for (DataSegment segment : params.getAvailableSegments()) { + for (DataSegment segment : params.getUsedSegments()) { if (overshadowed.contains(segment)) { // Skipping overshadowed segments continue; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index d22f92adb118..dce967c09536 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -76,7 +76,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final CoordinatorStats stats = new CoordinatorStats(); if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { - Map> dataSources = params.getDataSources(); + Map> dataSources = params.getDataSourcesWithUsedSegments(); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList != null && !compactionConfigList.isEmpty()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java index 2353247e3f30..12b262d637f2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java @@ -41,26 +41,22 @@ public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - log.info("Starting coordination. Getting available segments."); + log.info("Starting coordination. Getting used segments."); - final Iterable dataSegments = coordinator.iterateAvailableDataSegments(); - if (dataSegments == null) { - log.info("Metadata store not polled yet, canceling this run."); - return null; - } + final Iterable usedSegments = coordinator.iterateAllUsedSegments(); // The following transform() call doesn't actually transform the iterable. It only checks the sizes of the segments // and emits alerts if segments with negative sizes are encountered. In other words, semantically it's similar to - // Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() (which is called + // Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createUsedSegmentsSet() (which is called // below) guarantees to go over the passed iterable exactly once. // - // An iterable returned from iterateAvailableDataSegments() is not simply iterated (with size checks) before passing - // into DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() because iterateAvailableDataSegments()'s + // An iterable returned from iterateAllUsedSegments() is not simply iterated (with size checks) before passing + // into DruidCoordinatorRuntimeParams.createUsedSegmentsSet() because iterateAllUsedSegments()'s // documentation says to strive to avoid iterating the result more than once. // //noinspection StaticPseudoFunctionalStyleMethod: https://youtrack.jetbrains.com/issue/IDEA-153047 - Iterable availableSegmentsWithSizeChecking = Iterables.transform( - dataSegments, + Iterable usedSegmentsWithSizeChecking = Iterables.transform( + usedSegments, segment -> { if (segment.getSize() < 0) { log.makeAlert("No size on a segment") @@ -70,21 +66,21 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return segment; } ); - final TreeSet availableSegments = - DruidCoordinatorRuntimeParams.createAvailableSegmentsSet(availableSegmentsWithSizeChecking); + final TreeSet usedSegmentSet = + DruidCoordinatorRuntimeParams.createUsedSegmentsSet(usedSegmentsWithSizeChecking); - // Log info about all available segments + // Log info about all used segments if (log.isDebugEnabled()) { - log.debug("Available DataSegments"); - for (DataSegment dataSegment : availableSegments) { + log.debug("Used Segments"); + for (DataSegment dataSegment : usedSegmentSet) { log.debug(" %s", dataSegment); } } - log.info("Found [%,d] available segments.", availableSegments.size()); + log.info("Found [%,d] used segments.", usedSegmentSet.size()); return params.buildFromExisting() - .setAvailableSegments(availableSegments) + .setUsedSegments(usedSegmentSet) .build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java index 3f719d3f3d40..aee7397fcb62 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java @@ -31,10 +31,15 @@ import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collection; import java.util.List; /** + * Completely removes information about unused segments whose end time is older than {@link #retainDuration} from now + * from the metadata store. This action is called "to kill a segment". + * + * @see org.apache.druid.indexing.common.task.KillTask */ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper { @@ -46,12 +51,12 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper private long lastKillTime = 0; - private final MetadataSegmentManager segmentManager; + private final MetadataSegmentManager segmentsMetadata; private final IndexingServiceClient indexingServiceClient; @Inject public DruidCoordinatorSegmentKiller( - MetadataSegmentManager segmentManager, + MetadataSegmentManager segmentsMetadata, IndexingServiceClient indexingServiceClient, DruidCoordinatorConfig config ) @@ -75,30 +80,36 @@ public DruidCoordinatorSegmentKiller( this.maxSegmentsToKill ); - this.segmentManager = segmentManager; + this.segmentsMetadata = segmentsMetadata; this.indexingServiceClient = indexingServiceClient; } @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); - Collection whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources(); - - if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) { - log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled."); + boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources(); + Collection specificDataSourcesToKill = + params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn(); + + if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) { + log.error( + "killAllDataSources can't be true when specificDataSourcesToKill is non-empty. No kill tasks are scheduled." + ); return params; } + Collection dataSourcesToKill = specificDataSourcesToKill; if (killAllDataSources) { - whitelist = segmentManager.getAllDataSourceNames(); + dataSourcesToKill = segmentsMetadata.retrieveAllDataSourceNames(); } - if (whitelist != null && whitelist.size() > 0 && (lastKillTime + period) < System.currentTimeMillis()) { + if (dataSourcesToKill != null && + dataSourcesToKill.size() > 0 && + (lastKillTime + period) < System.currentTimeMillis()) { lastKillTime = System.currentTimeMillis(); - for (String dataSource : whitelist) { - final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); + for (String dataSource : dataSourcesToKill) { + final Interval intervalToKill = findIntervalForKill(dataSource, maxSegmentsToKill); if (intervalToKill != null) { try { indexingServiceClient.killSegments(dataSource, intervalToKill); @@ -117,13 +128,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } @VisibleForTesting - Interval findIntervalForKillTask(String dataSource, int limit) + @Nullable + Interval findIntervalForKill(String dataSource, int limit) { - List unusedSegmentIntervals = segmentManager.getUnusedSegmentIntervals( - dataSource, - new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)), - limit - ); + List unusedSegmentIntervals = + segmentsMetadata.getUnusedSegmentIntervals(dataSource, DateTimes.nowUtc().minus(retainDuration), limit); if (unusedSegmentIntervals != null && unusedSegmentIntervals.size() > 0) { return JodaUtils.umbrellaInterval(unusedSegmentIntervals); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java index fdd017802f52..c565df9b58be 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java @@ -33,7 +33,7 @@ public abstract class DropRule implements Rule public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { CoordinatorStats stats = new CoordinatorStats(); - coordinator.removeSegment(segment); + coordinator.markSegmentAsUnused(segment); stats.addToGlobalStat("deletedCount", 1); return stats; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java index 48aff5129229..14f791e6adb4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java @@ -52,13 +52,13 @@ public interface Rule boolean appliesTo(Interval interval, DateTime referenceTimestamp); /** - * {@link DruidCoordinatorRuntimeParams#getAvailableSegments()} must not be called in Rule's code, because the - * available segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is - * because {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: + * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used + * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because + * {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: * "DruidCoordinatorHelperParams" and "RuleParams" which contain params that only {@link * org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper}s and Rules need, respectively. * For example, {@link org.apache.druid.server.coordinator.ReplicationThrottler} needs to belong only to "RuleParams", - * but not "DruidCoordinatorHelperParams". The opposite for "AvailableSegments". + * but not "DruidCoordinatorHelperParams". The opposite for the collection of used segments. * * See https://github.com/apache/incubator-druid/issues/7228 */ diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java index 779e909c4f60..9522ca87a697 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java @@ -91,7 +91,7 @@ public Response getLoadStatus( ) { if (simple != null) { - return Response.ok(coordinator.getSegmentAvailability()).build(); + return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build(); } if (full != null) { diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 5904d52583bb..d8217e496040 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -22,10 +22,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.commons.lang.StringUtils; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; @@ -50,7 +52,6 @@ import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.http.security.DatasourceResourceFilter; -import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -97,27 +98,24 @@ public class DataSourcesResource private static final Logger log = new Logger(DataSourcesResource.class); private final CoordinatorServerView serverInventoryView; - private final MetadataSegmentManager databaseSegmentManager; - private final MetadataRuleManager databaseRuleManager; + private final MetadataSegmentManager segmentsMetadata; + private final MetadataRuleManager metadataRuleManager; private final IndexingServiceClient indexingServiceClient; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; @Inject public DataSourcesResource( CoordinatorServerView serverInventoryView, - MetadataSegmentManager databaseSegmentManager, - MetadataRuleManager databaseRuleManager, + MetadataSegmentManager segmentsMetadata, + MetadataRuleManager metadataRuleManager, @Nullable IndexingServiceClient indexingServiceClient, - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; - this.databaseSegmentManager = databaseSegmentManager; - this.databaseRuleManager = databaseRuleManager; + this.segmentsMetadata = segmentsMetadata; + this.metadataRuleManager = metadataRuleManager; this.indexingServiceClient = indexingServiceClient; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -130,24 +128,17 @@ public Response getQueryableDataSources( ) { Response.ResponseBuilder builder = Response.ok(); - final Set datasources = InventoryViewUtils.getSecuredDataSources( - req, - serverInventoryView, - authorizerMapper - ); + final Set datasources = + InventoryViewUtils.getSecuredDataSources(req, serverInventoryView, authorizerMapper); final Object entity; if (full != null) { entity = datasources; } else if (simple != null) { - entity = datasources.stream() - .map(this::makeSimpleDatasource) - .collect(Collectors.toList()); + entity = datasources.stream().map(this::makeSimpleDatasource).collect(Collectors.toList()); } else { - entity = datasources.stream() - .map(ImmutableDruidDataSource::getName) - .collect(Collectors.toList()); + entity = datasources.stream().map(ImmutableDruidDataSource::getName).collect(Collectors.toList()); } return builder.entity(entity).build(); @@ -157,7 +148,7 @@ public Response getQueryableDataSources( @Path("/{dataSourceName}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getTheDataSource( + public Response getDataSource( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("full") final String full ) @@ -165,7 +156,7 @@ public Response getTheDataSource( final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } if (full != null) { @@ -175,33 +166,128 @@ public Response getTheDataSource( return Response.ok(getSimpleDatasource(dataSourceName)).build(); } + private interface MarkSegments + { + int markSegments() throws UnknownSegmentIdException; + } + @POST @Path("/{dataSourceName}") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDataSource( - @PathParam("dataSourceName") final String dataSourceName + public Response markAsUsedAllNonOvershadowedSegments(@PathParam("dataSourceName") final String dataSourceName) + { + MarkSegments markSegments = () -> segmentsMetadata.markAsUsedAllNonOvershadowedSegmentsInDataSource(dataSourceName); + return doMarkSegments("markAsUsedAllNonOvershadowedSegments", dataSourceName, markSegments); + } + + @POST + @Path("/{dataSourceName}/markUsed") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response markAsUsedNonOvershadowedSegments( + @PathParam("dataSourceName") String dataSourceName, + MarkDataSourceSegmentsPayload payload + ) + { + MarkSegments markSegments = () -> { + final Interval interval = payload.getInterval(); + if (interval != null) { + return segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName, interval); + } else { + final Set segmentIds = payload.getSegmentIds(); + return segmentsMetadata.markAsUsedNonOvershadowedSegments(dataSourceName, segmentIds); + } + }; + return doMarkSegmentsWithPayload("markAsUsedNonOvershadowedSegments", dataSourceName, payload, markSegments); + } + + @POST + @Path("/{dataSourceName}/markUnused") + @ResourceFilters(DatasourceResourceFilter.class) + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response markSegmentsAsUnused( + @PathParam("dataSourceName") final String dataSourceName, + final MarkDataSourceSegmentsPayload payload + ) + { + MarkSegments markSegments = () -> { + final Interval interval = payload.getInterval(); + if (interval != null) { + return segmentsMetadata.markAsUnusedSegmentsInInterval(dataSourceName, interval); + } else { + final Set segmentIds = payload.getSegmentIds(); + return segmentsMetadata.markSegmentsAsUnused(dataSourceName, segmentIds); + } + }; + return doMarkSegmentsWithPayload("markSegmentsAsUnused", dataSourceName, payload, markSegments); + } + + private Response doMarkSegmentsWithPayload( + String method, + String dataSourceName, + MarkDataSourceSegmentsPayload payload, + MarkSegments markSegments ) { - if (!databaseSegmentManager.enableDataSource(dataSourceName)) { - return Response.noContent().build(); + if (payload == null || !payload.isValid()) { + log.warn("Invalid request payload: [%s]", payload); + return Response + .status(Response.Status.BAD_REQUEST) + .entity("Invalid request payload, either interval or segmentIds array must be specified") + .build(); + } + + final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); + if (dataSource == null) { + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } - return Response.ok().build(); + return doMarkSegments(method, dataSourceName, markSegments); + } + + private static Response logAndCreateDataSourceNotFoundResponse(String dataSourceName) + { + log.warn("datasource not found [%s]", dataSourceName); + return Response.noContent().build(); + } + + private static Response doMarkSegments(String method, String dataSourceName, MarkSegments markSegments) + { + try { + int numChangedSegments = markSegments.markSegments(); + return Response.ok(ImmutableMap.of("numChangedSegments", numChangedSegments)).build(); + } + catch (UnknownSegmentIdException e) { + log.warn("Segment ids %s are not found", e.getUnknownSegmentIds()); + return Response + .status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("message", e.getMessage())) + .build(); + } + catch (Exception e) { + log.error(e, "Error occurred during [%s] call, data source: [%s]", method, dataSourceName); + return Response + .serverError() + .entity(ImmutableMap.of("error", "Exception occurred.", "message", Throwables.getRootCause(e).toString())) + .build(); + } } - /* When this method is removed, a new method needs to be introduced corresponding to - the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters). - Ultimately we want to have no method with kill parameter - - DELETE `{dataSourceName}` will be used to disable datasource and - DELETE `{dataSourceName}/intervals/{interval}` will be used to nuke segments - */ + /** + * When this method is removed, a new method needs to be introduced corresponding to + * the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters). + * Ultimately we want to have no method with kill parameter - + * DELETE `{dataSourceName}` will be used to mark all segments belonging to a data source as unused, and + * DELETE `{dataSourceName}/intervals/{interval}` will be used to kill segments within an interval + */ @DELETE @Deprecated @Path("/{dataSourceName}") @ResourceFilters(DatasourceResourceFilter.class) @Produces(MediaType.APPLICATION_JSON) - public Response deleteDataSource( + public Response markAsUnusedAllSegmentsOrKillSegmentsInInterval( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("kill") final String kill, @QueryParam("interval") final String interval @@ -211,47 +297,20 @@ public Response deleteDataSource( return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } - if (kill != null && Boolean.valueOf(kill)) { - try { - indexingServiceClient.killSegments(dataSourceName, Intervals.of(interval)); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - ImmutableMap.of( - "error", - "Exception occurred. Probably the interval is invalid", - "message", - e.toString() - ) - ) - .build(); - } - catch (Exception e) { - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred. Are you sure you have an indexing service?", - "message", - e.toString() - ) - ) - .build(); - } + boolean killSegments = kill != null && Boolean.valueOf(kill); + if (killSegments) { + return killSegmentsInInterval(dataSourceName, interval); } else { - if (!databaseSegmentManager.removeDataSource(dataSourceName)) { - return Response.noContent().build(); - } + MarkSegments markSegments = () -> segmentsMetadata.markAsUnusedAllSegmentsInDataSource(dataSourceName); + return doMarkSegments("markAsUnusedAllSegments", dataSourceName, markSegments); } - - return Response.ok().build(); } @DELETE @Path("/{dataSourceName}/intervals/{interval}") @ResourceFilters(DatasourceResourceFilter.class) @Produces(MediaType.APPLICATION_JSON) - public Response deleteDataSourceSpecificInterval( + public Response killSegmentsInInterval( @PathParam("dataSourceName") final String dataSourceName, @PathParam("interval") final String interval ) @@ -259,28 +318,32 @@ public Response deleteDataSourceSpecificInterval( if (indexingServiceClient == null) { return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } + if (StringUtils.contains(interval, '_')) { + log.warn("Use interval with '/', not '_': [%s] given", interval); + } final Interval theInterval = Intervals.of(interval.replace('_', '/')); try { indexingServiceClient.killSegments(dataSourceName, theInterval); + return Response.ok().build(); } catch (Exception e) { - return Response.serverError() - .entity(ImmutableMap.of( - "error", - "Exception occurred. Are you sure you have an indexing service?", - "message", - e.toString() - )) - .build(); - } - return Response.ok().build(); + return Response + .serverError() + .entity( + ImmutableMap.of( + "error", "Exception occurred. Are you sure you have an indexing service?", + "message", e.toString() + ) + ) + .build(); + } } @GET @Path("/{dataSourceName}/intervals") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceIntervals( + public Response getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( @PathParam("dataSourceName") String dataSourceName, @QueryParam("simple") String simple, @QueryParam("full") String full @@ -289,21 +352,22 @@ public Response getSegmentDataSourceIntervals( if (simple == null && full == null) { final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); Set intervals = new TreeSet<>(comparator); dataSource.getSegments().forEach(segment -> intervals.add(segment.getInterval())); return Response.ok(intervals).build(); + } else { + return getServedSegmentsInInterval(dataSourceName, full != null, interval -> true); } - return getSegmentDataSourceIntervals(dataSourceName, full != null, interval -> true); } @GET @Path("/{dataSourceName}/intervals/{interval}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSpecificInterval( + public Response getServedSegmentsInInterval( @PathParam("dataSourceName") String dataSourceName, @PathParam("interval") String interval, @QueryParam("simple") String simple, @@ -314,7 +378,7 @@ public Response getSegmentDataSourceSpecificInterval( if (simple == null && full == null) { final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Set segmentIds = new TreeSet<>(); for (DataSegment dataSegment : dataSource.getSegments()) { @@ -324,7 +388,7 @@ public Response getSegmentDataSourceSpecificInterval( } return Response.ok(segmentIds).build(); } - return getSegmentDataSourceIntervals(dataSourceName, full != null, theInterval::contains); + return getServedSegmentsInInterval(dataSourceName, full != null, theInterval::contains); } /** @@ -337,7 +401,7 @@ enum SimpleProperties count } - private Response getSegmentDataSourceIntervals( + private Response getServedSegmentsInInterval( String dataSourceName, boolean full, Predicate intervalFilter @@ -346,7 +410,7 @@ private Response getSegmentDataSourceIntervals( final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); @@ -357,10 +421,13 @@ private Response getSegmentDataSourceIntervals( if (intervalFilter.test(dataSegment.getInterval())) { Map segments = retVal.computeIfAbsent(dataSegment.getInterval(), i -> new HashMap<>()); - Pair> val = getServersWhereSegmentIsServed(dataSegment.getId()); + Pair> segmentAndServers = getServersWhereSegmentIsServed(dataSegment.getId()); - if (val != null) { - segments.put(dataSegment.getId(), ImmutableMap.of("metadata", val.lhs, "servers", val.rhs)); + if (segmentAndServers != null) { + segments.put( + dataSegment.getId(), + ImmutableMap.of("metadata", segmentAndServers.lhs, "servers", segmentAndServers.rhs) + ); } } } @@ -385,14 +452,14 @@ private Response getSegmentDataSourceIntervals( @Path("/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSegments( + public Response getAllServedSegments( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full ) { ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } Response.ResponseBuilder builder = Response.ok(); @@ -407,14 +474,14 @@ public Response getSegmentDataSourceSegments( @Path("/{dataSourceName}/segments/{segmentId}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSegment( + public Response getServedSegment( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } for (SegmentId possibleSegmentId : SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId)) { @@ -423,44 +490,40 @@ public Response getSegmentDataSourceSegment( return Response.ok(ImmutableMap.of("metadata", retVal.lhs, "servers", retVal.rhs)).build(); } } + log.warn("Segment id [%s] is unknown", segmentId); return Response.noContent().build(); } @DELETE @Path("/{dataSourceName}/segments/{segmentId}") @ResourceFilters(DatasourceResourceFilter.class) - public Response deleteDatasourceSegment( + public Response markSegmentAsUnused( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - if (databaseSegmentManager.removeSegment(dataSourceName, segmentId)) { - return Response.ok().build(); - } - return Response.noContent().build(); + boolean segmentStateChanged = segmentsMetadata.markSegmentAsUnused(dataSourceName, segmentId); + return Response.ok(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build(); } @POST @Path("/{dataSourceName}/segments/{segmentId}") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDatasourceSegment( + public Response markSegmentAsUsed( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - if (!databaseSegmentManager.enableSegment(segmentId)) { - return Response.noContent().build(); - } - - return Response.ok().build(); + boolean segmentStateChanged = segmentsMetadata.markSegmentAsUsed(segmentId); + return Response.ok().entity(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build(); } @GET @Path("/{dataSourceName}/tiers") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceTiers(@PathParam("dataSourceName") String dataSourceName) + public Response getTiersWhereSegmentsAreServed(@PathParam("dataSourceName") String dataSourceName) { Set retVal = new HashSet<>(); for (DruidServer druidServer : serverInventoryView.getInventory()) { @@ -475,20 +538,22 @@ public Response getSegmentDataSourceTiers(@PathParam("dataSourceName") String da @Nullable private ImmutableDruidDataSource getDataSource(final String dataSourceName) { - List dataSources = serverInventoryView + List dataSources = serverInventoryView .getInventory() .stream() .map(server -> server.getDataSource(dataSourceName)) .filter(Objects::nonNull) - .map(DruidDataSource::toImmutableDruidDataSource) .collect(Collectors.toList()); if (dataSources.isEmpty()) { return null; } + // Note: this logic doesn't guarantee that the result is a snapshot that ever existed in the cluster because all + // DruidDataSource objects (belonging to different servers) are independently, concurrently mutable objects. + // But this is OK because a "snapshot" hardly even makes sense in a distributed system anyway. final SortedMap segmentMap = new TreeMap<>(); - for (ImmutableDruidDataSource dataSource : dataSources) { + for (DruidDataSource dataSource : dataSources) { Iterable segments = dataSource.getSegments(); for (DataSegment segment : segments) { segmentMap.put(segment.getId(), segment); @@ -589,14 +654,14 @@ private Map> getSimpleDatasource(String dataSourceNa } /** - * Provides serverView for a datasource and Interval which gives details about servers hosting segments for an interval - * Used by the realtime tasks to fetch a view of the interval they are interested in. + * Provides serverView for a datasource and Interval which gives details about servers hosting segments for an + * interval. Used by the realtime tasks to fetch a view of the interval they are interested in. */ @GET @Path("/{dataSourceName}/intervals/{interval}/serverview") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSpecificInterval( + public Response getServedSegmentsInInterval( @PathParam("dataSourceName") String dataSourceName, @PathParam("interval") String interval, @QueryParam("partial") final boolean partial @@ -611,17 +676,25 @@ public Response getSegmentDataSourceSpecificInterval( return Response.ok(new ArrayList()).build(); } - Iterable> lookup = timeline.lookupWithIncompletePartitions(theInterval); - FunctionalIterable retval = FunctionalIterable - .create(lookup).transformCat( + return Response.ok(prepareServedSegmentsInInterval(timeline, theInterval)).build(); + } + + private Iterable prepareServedSegmentsInInterval( + TimelineLookup dataSourceServingTimeline, + Interval interval + ) + { + Iterable> lookup = + dataSourceServingTimeline.lookupWithIncompletePartitions(interval); + return FunctionalIterable + .create(lookup) + .transformCat( (TimelineObjectHolder input) -> Iterables.transform( input.getObject(), - (PartitionChunk chunk) -> - chunk.getObject().toImmutableSegmentLoadInfo() + (PartitionChunk chunk) -> chunk.getObject().toImmutableSegmentLoadInfo() ) ); - return Response.ok(retval).build(); } /** @@ -640,7 +713,7 @@ public Response isHandOffComplete( ) { try { - final List rules = databaseRuleManager.getRulesWithDefault(dataSourceName); + final List rules = metadataRuleManager.getRulesWithDefault(dataSourceName); final Interval theInterval = Intervals.of(interval); final SegmentDescriptor descriptor = new SegmentDescriptor(theInterval, version, partitionNumber); final DateTime now = DateTimes.nowUtc(); @@ -667,18 +740,9 @@ public Response isHandOffComplete( return Response.ok(false).build(); } - Iterable> lookup = timeline.lookupWithIncompletePartitions( - theInterval); - FunctionalIterable loadInfoIterable = FunctionalIterable - .create(lookup).transformCat( - (TimelineObjectHolder input) -> - Iterables.transform( - input.getObject(), - (PartitionChunk chunk) -> - chunk.getObject().toImmutableSegmentLoadInfo() - ) - ); - if (isSegmentLoaded(loadInfoIterable, descriptor)) { + Iterable servedSegmentsInInterval = + prepareServedSegmentsInInterval(timeline, theInterval); + if (isSegmentLoaded(servedSegmentsInInterval, descriptor)) { return Response.ok(true).build(); } @@ -690,58 +754,9 @@ public Response isHandOffComplete( } } - @POST - @Path("/{dataSourceName}/markUnused") - @ResourceFilters(DatasourceResourceFilter.class) - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - public Response markDatasourceUnused( - @PathParam("dataSourceName") final String dataSourceName, - final MarkDatasourceSegmentsPayload payload - ) + static boolean isSegmentLoaded(Iterable servedSegments, SegmentDescriptor descriptor) { - if (payload == null || !payload.isValid()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Invalid request payload, either interval or segmentIds array must be specified") - .build(); - } - - final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); - if (dataSource == null) { - log.warn("datasource not found [%s]", dataSourceName); - return Response.noContent().build(); - } - - long markedSegmentCount = 0; - try { - final Interval interval = payload.getInterval(); - final Set segmentIds = payload.getSegmentIds(); - if (interval != null) { - markedSegmentCount = databaseSegmentManager.disableSegments(dataSourceName, interval); - } else if (segmentIds != null) { - markedSegmentCount = databaseSegmentManager.disableSegments(dataSourceName, segmentIds); - } - } - catch (Exception e) { - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred.", - "message", - e.toString() - ) - ).build(); - - } - if (markedSegmentCount == 0) { - return Response.noContent().build(); - } - return Response.ok().build(); - } - - static boolean isSegmentLoaded(Iterable serverView, SegmentDescriptor descriptor) - { - for (ImmutableSegmentLoadInfo segmentLoadInfo : serverView) { + for (ImmutableSegmentLoadInfo segmentLoadInfo : servedSegments) { if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 @@ -754,68 +769,14 @@ static boolean isSegmentLoaded(Iterable serverView, Se return false; } - @POST - @Path("/{dataSourceName}/markUsed") - @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDatasourceSegments( - @PathParam("dataSourceName") String dataSourceName, - MarkDatasourceSegmentsPayload payload - ) - { - if (payload == null || !payload.isValid()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Invalid request payload, either interval or segmentIds array must be specified") - .build(); - } - - final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); - if (dataSource == null) { - return Response.noContent().build(); - } - - int modified; - try { - if (payload.getInterval() != null) { - modified = databaseSegmentManager.enableSegments(dataSource.getName(), payload.getInterval()); - } else { - modified = databaseSegmentManager.enableSegments(dataSource.getName(), payload.getSegmentIds()); - } - } - catch (Exception e) { - if (e.getCause() instanceof UnknownSegmentIdException) { - return Response.status(Response.Status.NOT_FOUND).entity( - ImmutableMap.of( - "message", - e.getCause().getMessage() - ) - ).build(); - } - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred.", - "message", - e.getMessage() - ) - ).build(); - } - - if (modified == 0) { - return Response.noContent().build(); - } - - return Response.ok().build(); - } - @VisibleForTesting - protected static class MarkDatasourceSegmentsPayload + protected static class MarkDataSourceSegmentsPayload { private final Interval interval; private final Set segmentIds; @JsonCreator - public MarkDatasourceSegmentsPayload( + public MarkDataSourceSegmentsPayload( @JsonProperty("interval") Interval interval, @JsonProperty("segmentIds") Set segmentIds ) 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 3c7e8ac37b83..b250a79fdd28 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 @@ -23,15 +23,15 @@ import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.server.JettyUtils; import org.apache.druid.server.http.security.DatasourceResourceFilter; -import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; @@ -50,13 +50,14 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.TreeSet; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -64,20 +65,19 @@ @Path("/druid/coordinator/v1/metadata") public class MetadataResource { - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegmentManager segmentsMetadata; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; @Inject public MetadataResource( - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, IndexerMetadataStorageCoordinator metadataStorageCoordinator, - AuthConfig authConfig, AuthorizerMapper authorizerMapper, @Json ObjectMapper jsonMapper ) { - this.metadataSegmentManager = metadataSegmentManager; + this.segmentsMetadata = segmentsMetadata; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; } @@ -85,29 +85,28 @@ public MetadataResource( @GET @Path("/datasources") @Produces(MediaType.APPLICATION_JSON) - public Response getDatabaseDataSources( + public Response getDataSources( @QueryParam("full") final String full, - @QueryParam("includeDisabled") final String includeDisabled, + @Context final UriInfo uriInfo, @Context final HttpServletRequest req ) { - // If we haven't polled the metadata store yet, use an empty list of datasources. - final Collection druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources()) - .orElse(Collections.emptyList()); - + final boolean includeUnused = JettyUtils.getQueryParam(uriInfo, "includeUnused", "includeDisabled") != null; + Collection druidDataSources = null; final Set dataSourceNamesPreAuth; - if (includeDisabled != null) { - dataSourceNamesPreAuth = new TreeSet<>(metadataSegmentManager.getAllDataSourceNames()); + if (includeUnused) { + dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames()); } else { - dataSourceNamesPreAuth = Sets.newTreeSet( - Iterables.transform(druidDataSources, ImmutableDruidDataSource::getName) - ); + druidDataSources = segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + dataSourceNamesPreAuth = druidDataSources + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toCollection(TreeSet::new)); } final Set dataSourceNamesPostAuth = new TreeSet<>(); - Function> raGenerator = datasourceName -> { - return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); - }; + Function> raGenerator = datasourceName -> + Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); Iterables.addAll( dataSourceNamesPostAuth, @@ -119,9 +118,9 @@ public Response getDatabaseDataSources( ) ); - // Cannot do both includeDisabled and full, let includeDisabled take priority + // Cannot do both includeUnused and full, let includeUnused take priority // Always use dataSourceNamesPostAuth to determine the set of returned dataSources - if (full != null && includeDisabled == null) { + if (full != null && !includeUnused) { return Response.ok().entity( Collections2.filter(druidDataSources, dataSource -> dataSourceNamesPostAuth.contains(dataSource.getName())) ).build(); @@ -130,44 +129,30 @@ public Response getDatabaseDataSources( } } - @GET - @Path("/datasources/{dataSourceName}") - @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSource(@PathParam("dataSourceName") final String dataSourceName) - { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); - if (dataSource == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - - return Response.status(Response.Status.OK).entity(dataSource).build(); - } - @GET @Path("/segments") @Produces(MediaType.APPLICATION_JSON) - public Response getDatabaseSegments( + public Response getAllUsedSegments( @Context final HttpServletRequest req, - @QueryParam("datasources") final Set datasources, + @QueryParam("datasources") final Set dataSources, @QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus ) { - // If we haven't polled the metadata store yet, use an empty list of datasources. - Collection druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources()) - .orElse(Collections.emptyList()); - Stream dataSourceStream = druidDataSources.stream(); - if (datasources != null && !datasources.isEmpty()) { - dataSourceStream = dataSourceStream.filter(src -> datasources.contains(src.getName())); + Collection dataSourcesWithUsedSegments = + segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); } - final Stream metadataSegments = dataSourceStream.flatMap(t -> t.getSegments().stream()); + final Stream usedSegments = dataSourcesWithUsedSegments + .stream() + .flatMap(t -> t.getSegments().stream()); if (includeOvershadowedStatus != null) { - final Iterable authorizedSegments = findAuthorizedSegmentWithOvershadowedStatus( - req, - druidDataSources, - metadataSegments - ); + final Iterable authorizedSegments = + findAuthorizedSegmentWithOvershadowedStatus(req, dataSourcesWithUsedSegments, usedSegments); Response.ResponseBuilder builder = Response.status(Response.Status.OK); return builder.entity(authorizedSegments).build(); } else { @@ -175,12 +160,8 @@ public Response getDatabaseSegments( final Function> raGenerator = segment -> Collections.singletonList( AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); - final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( - req, - metadataSegments::iterator, - raGenerator, - authorizerMapper - ); + final Iterable authorizedSegments = + AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); Response.ResponseBuilder builder = Response.status(Response.Status.OK); return builder.entity(authorizedSegments).build(); @@ -189,7 +170,7 @@ public Response getDatabaseSegments( private Iterable findAuthorizedSegmentWithOvershadowedStatus( HttpServletRequest req, - Collection druidDataSources, + Collection dataSources, Stream metadataSegments ) { @@ -197,15 +178,12 @@ private Iterable findAuthorizedSegmentWithOversha // a small fraction of the segments in the cluster are expected to be overshadowed, // so building this collection shouldn't generate a lot of garbage. final Set overshadowedSegments = new HashSet<>(); - for (ImmutableDruidDataSource dataSource : druidDataSources) { + for (ImmutableDruidDataSource dataSource : dataSources) { overshadowedSegments.addAll(ImmutableDruidDataSource.determineOvershadowedSegments(dataSource.getSegments())); } final Stream segmentsWithOvershadowedStatus = metadataSegments - .map(segment -> new SegmentWithOvershadowedStatus( - segment, - overshadowedSegments.contains(segment) - )); + .map(segment -> new SegmentWithOvershadowedStatus(segment, overshadowedSegments.contains(segment))); final Function> raGenerator = segment -> Collections .singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource())); @@ -219,16 +197,36 @@ private Iterable findAuthorizedSegmentWithOversha return authorizedSegments; } + /** + * The difference of this method from {@link #getUsedSegmentsInDataSource} is that the latter returns only a list of + * segments, while this method also includes the properties of data source, such as the time when it was created. + */ + @GET + @Path("/datasources/{dataSourceName}") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDataSourceWithUsedSegments(@PathParam("dataSourceName") final String dataSourceName) + { + ImmutableDruidDataSource dataSource = + segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + if (dataSource == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + return Response.status(Response.Status.OK).entity(dataSource).build(); + } + @GET @Path("/datasources/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegments( + public Response getUsedSegmentsInDataSource( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full ) { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); + ImmutableDruidDataSource dataSource = + segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -241,11 +239,15 @@ public Response getDatabaseSegmentDataSourceSegments( return builder.entity(Collections2.transform(dataSource.getSegments(), DataSegment::getId)).build(); } + /** + * This is a {@link POST} method to pass the list of intervals in the body, + * see https://github.com/apache/incubator-druid/pull/2109#issuecomment-182191258 + */ @POST @Path("/datasources/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegments( + public Response getUsedSegmentsInDataSourceForIntervals( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full, List intervals @@ -265,12 +267,12 @@ public Response getDatabaseSegmentDataSourceSegments( @Path("/datasources/{dataSourceName}/segments/{segmentId}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegment( + public Response isSegmentUsed( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); + DruidDataSource dataSource = segmentsMetadata.getDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java new file mode 100644 index 000000000000..8cc5c03c137a --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java @@ -0,0 +1,37 @@ +/* + * 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.timeline.DataSegment; +import org.easymock.EasyMock; + +import java.util.Collection; + +public final class ImmutableDruidServerTests +{ + + public static void expectSegments(ImmutableDruidServer mockServer, Collection segments) + { + EasyMock.expect(mockServer.iterateAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(mockServer.getNumSegments()).andReturn(segments.size()).anyTimes(); + } + + private ImmutableDruidServerTests() {} +} diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java index ebd3b6f4438e..0b6c235b6a7d 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java @@ -32,35 +32,35 @@ public class ClientKillQueryTest private static final String DATA_SOURCE = "data_source"; public static final DateTime START = DateTimes.nowUtc(); private static final Interval INTERVAL = new Interval(START, START.plus(1)); - ClientKillQuery clientKillQuery; + ClientKillQuery clientKillUnusedSegmentsQuery; @Before public void setUp() { - clientKillQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL); + clientKillUnusedSegmentsQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL); } @After public void tearDown() { - clientKillQuery = null; + clientKillUnusedSegmentsQuery = null; } @Test public void testGetType() { - Assert.assertEquals("kill", clientKillQuery.getType()); + Assert.assertEquals("kill", clientKillUnusedSegmentsQuery.getType()); } @Test public void testGetDataSource() { - Assert.assertEquals(DATA_SOURCE, clientKillQuery.getDataSource()); + Assert.assertEquals(DATA_SOURCE, clientKillUnusedSegmentsQuery.getDataSource()); } @Test public void testGetInterval() { - Assert.assertEquals(INTERVAL, clientKillQuery.getInterval()); + Assert.assertEquals(INTERVAL, clientKillUnusedSegmentsQuery.getInterval()); } } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index e7964481688a..522078dcfe34 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -33,14 +34,12 @@ import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; -import org.hamcrest.core.IsInstanceOf; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; import java.util.stream.Collectors; @@ -48,57 +47,64 @@ public class SQLMetadataSegmentManagerTest { - @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private static DataSegment createSegment( + String dataSource, + String interval, + String version, + String bucketKey, + int binaryVersion + ) + { + return new DataSegment( + dataSource, + Intervals.of(interval), + version, + ImmutableMap.of( + "type", "s3_zip", + "bucket", "test", + "key", dataSource + "/" + bucketKey + ), + ImmutableList.of("dim1", "dim2", "dim3"), + ImmutableList.of("count", "value"), + NoneShardSpec.instance(), + binaryVersion, + 1234L + ); + } @Rule - public ExpectedException thrown = ExpectedException.none(); + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private SQLMetadataSegmentManager manager; + private SQLMetadataSegmentManager sqlSegmentsMetadata; private SQLMetadataSegmentPublisher publisher; private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - private final DataSegment segment1 = new DataSegment( + private final DataSegment segment1 = createSegment( "wikipedia", - Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000"), + "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000", "2012-03-16T00:36:30.848Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia/index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip", + 0 ); - private final DataSegment segment2 = new DataSegment( + private final DataSegment segment2 = createSegment( "wikipedia", - Intervals.of("2012-01-05T00:00:00.000/2012-01-06T00:00:00.000"), + "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000", "2012-01-06T22:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip", + 0 ); private void publish(DataSegment segment, boolean used) throws IOException { + boolean partitioned = !(segment.getShardSpec() instanceof NoneShardSpec); publisher.publishSegment( segment.getId().toString(), segment.getDataSource(), DateTimes.nowUtc().toString(), segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), - (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, + partitioned, segment.getVersion(), used, jsonMapper.writeValueAsBytes(segment) @@ -109,12 +115,13 @@ private void publish(DataSegment segment, boolean used) throws IOException public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); - manager = new SQLMetadataSegmentManager( + sqlSegmentsMetadata = new SQLMetadataSegmentManager( jsonMapper, Suppliers.ofInstance(new MetadataSegmentManagerConfig()), derbyConnectorRule.metadataTablesConfigSupplier(), connector ); + sqlSegmentsMetadata.start(); publisher = new SQLMetadataSegmentPublisher( jsonMapper, @@ -131,67 +138,104 @@ public void setUp() throws Exception @After public void teardown() { - if (manager.isStarted()) { - manager.stop(); + if (sqlSegmentsMetadata.isPollingDatabasePeriodically()) { + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); } + sqlSegmentsMetadata.stop(); } @Test public void testPoll() { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + sqlSegmentsMetadata.retrieveAllDataSourceNames() ); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getDataSources().stream().map(d -> d.getName()).collect(Collectors.toList()) + sqlSegmentsMetadata + .prepareImmutableDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.getDataSource("wikipedia").getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testNoPoll() + public void testPrepareImmutableDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException { - manager.start(); - Assert.assertTrue(manager.isStarted()); + DataSegment newSegment = pollThenStopThenStartIntro(); Assert.assertEquals( - ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + ImmutableSet.of(newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) + ); + } + + @Test + public void testGetDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException + { + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.getDataSourceWithUsedSegments("wikipedia2").getSegments()) ); - Assert.assertNull(manager.getDataSources()); - Assert.assertNull(manager.getDataSource("wikipedia")); - Assert.assertNull(manager.iterateAllSegments()); } @Test - public void testPollThenStop() + public void testPrepareImmutableDataSourcesWithAllUsedSegmentsAwaitsPollOnRestart() throws IOException { - manager.start(); - manager.poll(); - manager.stop(); - Assert.assertFalse(manager.isStarted()); + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2, newSegment), + ImmutableSet.copyOf( + sqlSegmentsMetadata + .prepareImmutableDataSourcesWithAllUsedSegments() + .stream() + .flatMap((ImmutableDruidDataSource dataSource) -> dataSource.getSegments().stream()) + .iterator() + ) + ); + } + + @Test + public void testIterateAllUsedSegmentsAwaitsPollOnRestart() throws IOException + { + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2, newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) + ); + } + + private DataSegment pollThenStopThenStartIntro() throws IOException + { + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); + Assert.assertFalse(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + sqlSegmentsMetadata.retrieveAllDataSourceNames() ); - Assert.assertNull(manager.getDataSources()); - Assert.assertNull(manager.getDataSource("wikipedia")); - Assert.assertNull(manager.iterateAllSegments()); + DataSegment newSegment = createNewSegment1("wikipedia2"); + publisher.publishSegment(newSegment); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + return newSegment; } @Test - public void testPollWithCurroptedSegment() + public void testPollWithCorruptedSegment() { //create a corrupted segment entry in segments table, which tests //that overall loading of segments from database continues to work @@ -209,355 +253,227 @@ public void testPollWithCurroptedSegment() ); EmittingLogger.registerEmitter(new NoopServiceEmitter()); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( - "wikipedia", Iterables.getOnlyElement(manager.getDataSources()).getName() + "wikipedia", + Iterables.getOnlyElement(sqlSegmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()).getName() ); } @Test - public void testGetUnusedSegmentsForInterval() + public void testGetUnusedSegmentIntervals() { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - Assert.assertTrue(manager.removeDataSource("wikipedia")); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource("wikipedia"); + Assert.assertEquals(2, numChangedSegments); Assert.assertEquals( ImmutableList.of(segment2.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 1) + sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 1) ); Assert.assertEquals( ImmutableList.of(segment2.getInterval(), segment1.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 5) + sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 5) ); } @Test - public void testRemoveDataSource() throws IOException + public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); final String newDataSource = "wikipedia2"; - final DataSegment newSegment = new DataSegment( + final DataSegment newSegment = createNewSegment1(newDataSource); + + publisher.publishSegment(newSegment); + + Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource(newDataSource); + Assert.assertEquals(1, numChangedSegments); + } + + private static DataSegment createNewSegment1(String newDataSource) + { + return createSegment( newDataSource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), + "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); + } - publisher.publishSegment(newSegment); - - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeDataSource(newDataSource)); + private static DataSegment createNewSegment2(String newDataSource) + { + return createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", + "2017-10-15T20:19:12.565Z", + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 + ); } @Test - public void testRemoveDataSegment() throws IOException + public void testMarkSegmentAsUnused() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); final String newDataSource = "wikipedia2"; - final DataSegment newSegment = new DataSegment( + final DataSegment newSegment = createSegment( newDataSource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), + "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment); - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeSegment(newSegment.getId())); + Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId())); } @Test - public void testEnableSegmentsWithSegmentIds() throws IOException + public void testMarkAsUsedNonOvershadowedSegments() throws Exception { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); // Overshadowed by newSegment2 - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); publish(newSegment1, false); publish(newSegment2, false); publish(newSegment3, false); - final ImmutableList segmentIds = ImmutableList.of( + final ImmutableSet segmentIds = ImmutableSet.of( newSegment1.getId().toString(), newSegment2.getId().toString(), newSegment3.getId().toString() ); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - Assert.assertEquals(2, manager.enableSegments(datasource, segmentIds)); - manager.poll(); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } - @Test - public void testEnableSegmentsWithSegmentIdsInvalidDatasource() throws IOException + @Test(expected = UnknownSegmentIdException.class) + public void testMarkAsUsedNonOvershadowedSegmentsInvalidDataSource() throws Exception { - thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class)); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publish(newSegment1, false); publish(newSegment2, false); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - manager.poll(); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.enableSegments("wrongDataSource", segmentIds)); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments("wrongDataSource", segmentIds)); } - @Test - public void testEnableSegmentsWithInvalidSegmentIds() + @Test(expected = UnknownSegmentIdException.class) + public void testMarkAsUsedNonOvershadowedSegmentsWithInvalidSegmentIds() throws UnknownSegmentIdException { - thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class)); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - manager.poll(); + final DataSegment newSegment2 = createNewSegment1(newDataSource); + + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.enableSegments(datasource, segmentIds)); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds)); } @Test - public void testEnableSegmentsWithInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); // Overshadowed by newSegment2 - final DataSegment newSegment4 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final DataSegment newSegment4 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); @@ -565,141 +481,75 @@ public void testEnableSegmentsWithInterval() throws IOException publish(newSegment4, false); final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); // 2 out of 3 segments match the interval - Assert.assertEquals(2, manager.enableSegments(datasource, theInterval)); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test(expected = IllegalArgumentException.class) - public void testEnableSegmentsWithInvalidInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithInvalidInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); // invalid interval start > end final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000"); - manager.enableSegments(datasource, theInterval); + sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval); } @Test - public void testEnableSegmentsWithOverlappingInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithOverlappingInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); // Overshadowed by newSegment2 - final DataSegment newSegment4 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final DataSegment newSegment4 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); @@ -707,183 +557,90 @@ public void testEnableSegmentsWithOverlappingInterval() throws IOException publish(newSegment4, false); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled - Assert.assertEquals(1, manager.enableSegments(datasource, theInterval)); + // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused + Assert.assertEquals(1, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithSegmentIds() throws IOException + public void testMarkSegmentsAsUnused() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); - final ImmutableList segmentIds = ImmutableList.of(newSegment1.getId().toString(), newSegment1.getId().toString()); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment1.getId().toString()); - Assert.assertEquals(segmentIds.size(), manager.disableSegments(datasource, segmentIds)); - manager.poll(); + Assert.assertEquals(segmentIds.size(), sqlSegmentsMetadata.markSegmentsAsUnused(newDataSource, segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithSegmentIdsInvalidDatasource() throws IOException + public void testMarkSegmentsAsUnusedInvalidDataSource() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.disableSegments("wrongDataSource", segmentIds)); - manager.poll(); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markSegmentsAsUnused("wrongDataSource", segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithInterval() throws IOException + public void testMarkAsUnusedSegmentsInInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment2(newDataSource); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment1); @@ -892,116 +649,58 @@ public void testDisableSegmentsWithInterval() throws IOException final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); // 2 out of 3 segments match the interval - Assert.assertEquals(2, manager.disableSegments(datasource, theInterval)); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment3), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test(expected = IllegalArgumentException.class) - public void testDisableSegmentsWithInvalidInterval() throws IOException + public void testMarkAsUnusedSegmentsInIntervalWithInvalidInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment2(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); // invalid interval start > end final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000"); - manager.disableSegments(datasource, theInterval); + sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval); } @Test - public void testDisableSegmentsWithOverlappingInterval() throws IOException + public void testMarkAsUnusedSegmentsInIntervalWithOverlappingInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"), + final DataSegment newSegment2 = createNewSegment2(newDataSource); + + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment1); @@ -1009,13 +708,13 @@ public void testDisableSegmentsWithOverlappingInterval() throws IOException publisher.publishSegment(newSegment3); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); - // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled - Assert.assertEquals(1, manager.disableSegments(datasource, theInterval)); + // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused + Assert.assertEquals(1, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment3), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @@ -1023,9 +722,9 @@ public void testDisableSegmentsWithOverlappingInterval() throws IOException public void testStopAndStart() { // Simulate successive losing and getting the coordinator leadership - manager.start(); - manager.stop(); - manager.start(); - manager.stop(); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java new file mode 100644 index 000000000000..8bae80ff4c02 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java @@ -0,0 +1,42 @@ +/* + * 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.coordinator; + +import org.apache.druid.java.util.common.DateTimes; + +public class CoordinatorRuntimeParamsTestHelpers +{ + public static DruidCoordinatorRuntimeParams.Builder newBuilder() + { + return DruidCoordinatorRuntimeParams + .newBuilder() + .withStartTimeNanos(System.nanoTime()) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")); + } + + public static DruidCoordinatorRuntimeParams.Builder newBuilder(DruidCluster druidCluster) + { + return newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)); + } + + private CoordinatorRuntimeParamsTestHelpers() {} +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java index e6aa17ebd1ca..3074fb2c0fe1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java @@ -23,6 +23,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -94,7 +95,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.put(segment.getId(), segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments.values()).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments.values()); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 3b2223514f59..940e2bfeee5a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -365,7 +365,8 @@ public void testMoveSegment() throws Exception ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2)); EasyMock.replay(druidDataSource); - EasyMock.expect(databaseSegmentManager.getDataSource(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.expect(databaseSegmentManager.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())) + .andReturn(druidDataSource); EasyMock.replay(databaseSegmentManager); coordinator.moveSegment( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java index 0bbe46d9b85b..bd2aa6eaf14a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java @@ -23,6 +23,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -88,7 +89,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.add(segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java new file mode 100644 index 000000000000..929ea815f995 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java @@ -0,0 +1,59 @@ +/* + * 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.coordinator; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public final class DruidClusterBuilder +{ + public static DruidClusterBuilder newBuilder() + { + return new DruidClusterBuilder(); + } + + private @Nullable Set realtimes = null; + private final Map> historicals = new HashMap<>(); + + private DruidClusterBuilder() {} + + public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) + { + this.realtimes = new HashSet<>(Arrays.asList(realtimes)); + return this; + } + + public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) + { + if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) { + throw new IllegalArgumentException("Duplicate tier: " + tierName); + } + return this; + } + + public DruidCluster build() + { + return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java index c07830e1320c..798c89107b2e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.Intervals; @@ -39,9 +38,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; -import java.util.TreeSet; import java.util.stream.Collectors; -import java.util.stream.Stream; public class DruidClusterTest { @@ -100,8 +97,9 @@ public class DruidClusterTest @Before public void setup() { - cluster = new DruidCluster( - ImmutableSet.of( + cluster = DruidClusterBuilder + .newBuilder() + .withRealtimes( new ServerHolder( new ImmutableDruidServer( new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0), @@ -111,22 +109,20 @@ public void setup() ), new LoadQueuePeonTester() ) - ), - ImmutableMap.of( + ) + .addTier( "tier1", - Stream.of( - new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), - 0L, - ImmutableMap.of("src1", dataSources.get("src1")), - 1 - ), - new LoadQueuePeonTester() - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of("src1", dataSources.get("src1")), + 1 + ), + new LoadQueuePeonTester() + ) ) - ); + .build(); } @Test diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index ed4f600c2cd6..46ec7cf90190 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -42,10 +43,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; +/** + * TODO convert benchmarks to JMH + */ public class DruidCoordinatorBalancerProfiler { private static final int MAX_SEGMENTS_TO_MOVE = 5; @@ -116,9 +117,9 @@ public void bigProfiler() EasyMock.expect(server.getName()).andReturn(Integer.toString(i)).atLeastOnce(); EasyMock.expect(server.getHost()).andReturn(Integer.toString(i)).anyTimes(); if (i == 0) { - EasyMock.expect(server.getLazyAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(server, segments); } else { - EasyMock.expect(server.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + ImmutableDruidServerTests.expectSegments(server, Collections.emptyList()); } EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(server); @@ -128,56 +129,26 @@ public void bigProfiler() serverHolderList.add(new ServerHolder(server, peon)); } - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - serverHolderList.stream().collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - .withLoadManagementPeons( - peonMap - ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).withReplicantLifetime(500) - .withReplicationThrottleLimit(5) - .build() - ) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withEmitter(emitter) - .withDatabaseRuleManager(manager) - .withReplicationManager(new ReplicationThrottler(2, 500)) - .withSegmentReplicantLookup( - SegmentReplicantLookup.make( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - serverHolderList.stream().collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - ) - .build(); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", serverHolderList.toArray(new ServerHolder[0])) + .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder(druidCluster) + .withLoadManagementPeons(peonMap) + .withUsedSegmentsInTest(segments) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) + .withReplicantLifetime(500) + .withReplicationThrottleLimit(5) + .build() + ) + .withEmitter(emitter) + .withDatabaseRuleManager(manager) + .withReplicationManager(new ReplicationThrottler(2, 500)) + .build(); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(coordinator); @@ -197,7 +168,7 @@ public void profileRun() EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getLazyAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer1, segments); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -205,7 +176,7 @@ public void profileRun() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer2, Collections.emptyList()); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -218,42 +189,22 @@ public void profileRun() EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder(druidServer1, fromPeon), - new ServerHolder(druidServer2, toPeon) - ).collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - .withLoadManagementPeons( - ImmutableMap.of( - "from", - fromPeon, - "to", - toPeon - ) - ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).build() - ) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster( + DruidClusterBuilder + .newBuilder() + .addTier( + "normal", + new ServerHolder(druidServer1, fromPeon), + new ServerHolder(druidServer2, toPeon) + ) + .build() + ) + .withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon)) + .withUsedSegmentsInTest(segments) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .build(); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); watch.start(); DruidCoordinatorRuntimeParams balanceParams = tester.run(params); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 0a3089808c6c..ad22c09abce0 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -20,11 +20,11 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; @@ -515,19 +515,19 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( List decommissioning ) { - return DruidCoordinatorRuntimeParams + return CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( + DruidClusterBuilder + .newBuilder() + .addTier( "normal", IntStream .range(0, druidServers.size()) .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i))) - .collect(Collectors.toSet()) + .toArray(ServerHolder[]::new) ) - ) + .build() ) .withLoadManagementPeons( IntStream @@ -535,14 +535,9 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( .boxed() .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get)) ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).build() - ) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")); + .withUsedSegmentsInTest(segments) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .withBalancerStrategy(balancerStrategy); } private static void mockDruidServer( @@ -558,7 +553,7 @@ private static void mockDruidServer( EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce(); - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); if (!segments.isEmpty()) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java index fad85d0effc9..24b8de39de9a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java @@ -25,17 +25,8 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import java.util.Comparator; - public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer { - public static final Comparator percentUsedComparator = (ServerHolder a, ServerHolder b) -> { - int c = Double.compare(a.getPercentUsed(), b.getPercentUsed()); - if (c == 0) { - return a.getServer().getName().compareTo(b.getServer().getName()); - } - return c; - }; public DruidCoordinatorBalancerTester(DruidCoordinator coordinator) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 51a79307c752..bfe5637bb506 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -52,18 +52,18 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Executors; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** */ public class DruidCoordinatorRuleRunnerTest { + public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = + CoordinatorDynamicConfig.builder().withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L).build(); + private DruidCoordinator coordinator; private LoadQueuePeon mockPeon; - private List availableSegments; + private List usedSegments; private DruidCoordinatorRuleRunner ruleRunner; private ServiceEmitter emitter; private MetadataRuleManager databaseRuleManager; @@ -78,9 +78,9 @@ public void setUp() databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); DateTime start = DateTimes.of("2012-01-01"); - availableSegments = new ArrayList<>(); + usedSegments = new ArrayList<>(); for (int i = 0; i < 24; i++) { - availableSegments.add( + usedSegments.add( new DataSegment( "test", new Interval(start, start.plusHours(1)), @@ -111,8 +111,6 @@ public void tearDown() * hot - 1 replicant * normal - 1 replicant * cold - 1 replicant - * - * @throws Exception */ @Test public void testRunThreeTiersOneReplicant() @@ -139,72 +137,40 @@ public void testRunThreeTiersOneReplicant() )).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "cold", - Stream.of( - new ServerHolder( - new DruidServer( - "serverCold", - "hostCold", - null, - 1000, - ServerType.HISTORICAL, - "cold", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -219,12 +185,24 @@ public void testRunThreeTiersOneReplicant() EasyMock.verify(mockPeon); } + private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams( + DruidCluster druidCluster, + BalancerStrategy balancerStrategy + ) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerStrategy(balancerStrategy); + } + /** * Nodes: * hot - 2 replicants * cold - 1 replicant - * - * @throws Exception */ @Test public void testRunTwoTiersTwoReplicants() @@ -248,68 +226,35 @@ public void testRunTwoTiersTwoReplicants() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "cold", - Stream.of( - new ServerHolder( - new DruidServer( - "serverCold", - "hostCold", - null, - 1000, - ServerType.HISTORICAL, - "cold", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -327,8 +272,6 @@ public void testRunTwoTiersTwoReplicants() * Nodes: * hot - 1 replicant * normal - 1 replicant - * - * @throws Exception */ @Test public void testRunTwoTiersWithExistingSegments() @@ -352,63 +295,37 @@ public void testRunTwoTiersWithExistingSegments() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer normServer = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : availableSegments) { - normServer.addDataSegment(availableSegment); + DruidServer normServer = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { + normServer.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of( - new ServerHolder( - normServer.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .addTier("normal", new ServerHolder(normServer.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerStrategy(balancerStrategy) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -448,42 +365,24 @@ public void testRunTwoTiersTierDoesNotExist() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withEmitter(emitter) - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) + .withEmitter(emitter) + .build(); ruleRunner.run(params); @@ -515,35 +414,26 @@ public void testRunRuleDoesNotExist() EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes(); EasyMock.replay(databaseRuleManager, mockPeon); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withEmitter(emitter) - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withEmitter(emitter) + .build(); ruleRunner.run(params); @@ -558,7 +448,7 @@ public void testDropRemove() mockEmptyPeon(); EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); EasyMock.replay(coordinator); @@ -573,47 +463,29 @@ public void testDropRemove() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder( - server.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", new ServerHolder(server.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -644,62 +516,41 @@ public void testDropTooManyInSameTier() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + server1.addDataSegment(usedSegments.get(0)); + + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L) + .build() + ) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -733,64 +584,32 @@ public void testDropTooManyInDifferentTiers() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of( - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -822,54 +641,31 @@ public void testDontDropInDifferentTiers() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of(new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of(new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -899,37 +695,13 @@ public void testDropServerActuallyServesSegment() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server2.addDataSegment(availableSegments.get(1)); - DruidServer server3 = new DruidServer( - "serverNorm3", - "hostNorm3", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server3.addDataSegment(availableSegments.get(1)); - server3.addDataSegment(availableSegments.get(2)); + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "normal", 0); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + server2.addDataSegment(usedSegments.get(1)); + DruidServer server3 = new DruidServer("serverNorm3", "hostNorm3", null, 1000, ServerType.HISTORICAL, "normal", 0); + server3.addDataSegment(usedSegments.get(1)); + server3.addDataSegment(usedSegments.get(2)); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); @@ -941,33 +713,29 @@ public void testDropServerActuallyServesSegment() EasyMock.replay(anotherMockPeon); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), - new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), - new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), + new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), + new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -983,8 +751,6 @@ public void testDropServerActuallyServesSegment() /** * Nodes: * hot - 2 replicants - * - * @throws Exception */ @Test public void testReplicantThrottle() @@ -997,7 +763,8 @@ public void testReplicantThrottle() EasyMock .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) .andReturn( - Collections.singletonList(new IntervalLoadRule( + Collections.singletonList( + new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), ImmutableMap.of("hot", 2) ) @@ -1006,53 +773,27 @@ public void testReplicantThrottle() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -1074,13 +815,13 @@ public void testReplicantThrottle() ); afterParams = ruleRunner.run( - new DruidCoordinatorRuntimeParams.Builder() + CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) .withEmitter(emitter) - .withAvailableSegmentsInTest(Collections.singletonList(overFlowSegment)) + .withUsedSegmentsInTest(overFlowSegment) .withDatabaseRuleManager(databaseRuleManager) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .build() ); @@ -1098,8 +839,6 @@ public void testReplicantThrottle() * Nodes: * hot - nothing loaded * _default_tier - 1 segment loaded - * - * @throws Exception */ @Test public void testReplicantThrottleAcrossTiers() @@ -1114,7 +853,7 @@ public void testReplicantThrottleAcrossTiers() .build() ) .atLeastOnce(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); @@ -1136,56 +875,37 @@ public void testReplicantThrottleAcrossTiers() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverNorm", + "hostNorm", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(new ReplicationThrottler(7, 1), coordinator); DruidCoordinatorRuntimeParams afterParams = runner.run(params); @@ -1231,66 +951,40 @@ public void testDropReplicantThrottle() 1, 0 ); - List longerAvailableSegments = Lists.newArrayList(availableSegments); - longerAvailableSegments.add(overFlowSegment); - - DruidServer server1 = new DruidServer( - "serverNorm1", - "hostNorm1", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : longerAvailableSegments) { - server1.addDataSegment(availableSegment); + List longerUsedSegments = Lists.newArrayList(usedSegments); + longerUsedSegments.add(overFlowSegment); + + DruidServer server1 = new DruidServer("serverNorm1", "hostNorm1", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : longerUsedSegments) { + server1.addDataSegment(segment); } - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : longerAvailableSegments) { - server2.addDataSegment(availableSegment); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : longerUsedSegments) { + server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(longerAvailableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(longerUsedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -1305,7 +999,7 @@ public void testDropReplicantThrottle() @Test public void testRulesRunOnNonOvershadowedSegmentsOnly() { - Set availableSegments = new HashSet<>(); + Set usedSegments = new HashSet<>(); DataSegment v1 = new DataSegment( "test", Intervals.of("2012-01-01/2012-01-02"), @@ -1328,8 +1022,8 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() IndexIO.CURRENT_VERSION_ID, 1 ); - availableSegments.add(v1); - availableSegments.add(v2); + usedSegments.add(v1); + usedSegments.add(v2); mockCoordinator(); mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject()); @@ -1340,42 +1034,37 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverHot", + "hostHot", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerStrategy(balancerStrategy) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -1385,9 +1074,9 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - Assert.assertEquals(2, availableSegments.size()); - Assert.assertEquals(availableSegments, params.getAvailableSegments()); - Assert.assertEquals(availableSegments, afterParams.getAvailableSegments()); + Assert.assertEquals(2, usedSegments.size()); + Assert.assertEquals(usedSegments, params.getUsedSegments()); + Assert.assertEquals(usedSegments, afterParams.getUsedSegments()); EasyMock.verify(mockPeon); exec.shutdown(); @@ -1396,7 +1085,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() private void mockCoordinator() { EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); } @@ -1413,7 +1102,7 @@ private void mockEmptyPeon() private CoordinatorDynamicConfig createCoordinatorDynamicConfig() { return CoordinatorDynamicConfig.builder() - .withMillisToWaitBeforeDeleting(0) + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0) .withMergeBytesLimit(0) .withMergeSegmentsLimit(0) .withMaxSegmentsToMove(0) @@ -1421,8 +1110,8 @@ private CoordinatorDynamicConfig createCoordinatorDynamicConfig() .withReplicationThrottleLimit(24) .withBalancerComputeThreads(0) .withEmitBalancingStats(false) - .withKillDataSourceWhitelist(null) - .withKillAllDataSources(false) + .withSpecificDataSourcesToKillUnusedSegmentsIn(null) + .withKillUnusedSegmentsInAllDataSources(false) .withMaxSegmentsInNodeLoadingQueue(1000) .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 793bd287b950..614b1294485b 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 @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; @@ -85,7 +86,7 @@ public class DruidCoordinatorTest extends CuratorTestBase private static final long COORDINATOR_PERIOD = 100; private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegmentManager segmentsMetadata; private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; @@ -105,7 +106,7 @@ public void setUp() throws Exception { druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -182,7 +183,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -248,8 +249,8 @@ public void testMoveSegment() ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.replay(druidDataSource); - EasyMock.expect(databaseSegmentManager.getDataSource(EasyMock.anyString())).andReturn(druidDataSource); - EasyMock.replay(databaseSegmentManager); + EasyMock.expect(segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.replay(segmentsMetadata); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(scheduledExecutorFactory); EasyMock.replay(metadataRuleManager); @@ -391,9 +392,10 @@ public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getId().toString())); - Map segmentAvailability = coordinator.getSegmentAvailability(); - Assert.assertEquals(1, segmentAvailability.size()); - Assert.assertEquals(0L, segmentAvailability.get(dataSource)); + Object2IntMap numsUnavailableUsedSegmentsPerDataSource = + coordinator.computeNumsUnavailableUsedSegmentsPerDataSource(); + Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size()); + Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource)); Map> underReplicationCountsPerDataSourcePerTier = coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); @@ -527,26 +529,26 @@ public void testCoordinatorTieredRun() throws Exception leaderUnannouncerLatch.await(); EasyMock.verify(serverInventoryView); - EasyMock.verify(databaseSegmentManager); + EasyMock.verify(segmentsMetadata); EasyMock.verify(metadataRuleManager); } private void setupMetadataSegmentManagerMock(DruidDataSource dataSource) { - EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes(); + EasyMock.expect(segmentsMetadata.isPollingDatabasePeriodically()).andReturn(true).anyTimes(); EasyMock - .expect(databaseSegmentManager.iterateAllSegments()) + .expect(segmentsMetadata.iterateAllUsedSegments()) .andReturn(dataSource.getSegments()) .anyTimes(); EasyMock - .expect(databaseSegmentManager.getDataSources()) + .expect(segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .anyTimes(); EasyMock - .expect(databaseSegmentManager.getAllDataSourceNames()) + .expect(segmentsMetadata.retrieveAllDataSourceNames()) .andReturn(Collections.singleton(dataSource.getName())) .anyTimes(); - EasyMock.replay(databaseSegmentManager); + EasyMock.replay(segmentsMetadata); } @Nullable diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 001dc2af24c7..73e829ce0e0c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -138,7 +139,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getLazyAllSegments()).andReturn(segments1).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer1, segments1); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -146,7 +147,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.getLazyAllSegments()).andReturn(segments2).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer2, segments2); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -154,7 +155,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer3.getLazyAllSegments()).andReturn(segments3).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer3, segments3); EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); @@ -162,7 +163,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer4.getLazyAllSegments()).andReturn(segments4).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer4, segments4); EasyMock.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java index b8699fb08666..13942b010eb4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java @@ -115,7 +115,7 @@ private CachingCostBalancerStrategy createCachingCostBalancerStrategy( { ClusterCostCache.Builder builder = ClusterCostCache.builder(); serverHolders.forEach( - s -> s.getServer().getLazyAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment)) + s -> s.getServer().iterateAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment)) ); return new CachingCostBalancerStrategy(builder.build(), listeningExecutorService); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java index 0ae0ee6c07d8..965cf2ee153d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java @@ -20,14 +20,15 @@ package org.apache.druid.server.coordinator.helper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; @@ -38,17 +39,13 @@ import org.joda.time.Interval; import org.junit.Test; -import java.util.Collections; import java.util.List; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class DruidCoordinatorCleanupOvershadowedTest { - DruidCoordinatorCleanupOvershadowed druidCoordinatorCleanupOvershadowed; + DruidCoordinatorCleanupOvershadowed druidCoordinatorMarkAsUnusedOvershadowedSegments; DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); - private List availableSegments; + private List usedSegments; DateTime start = DateTimes.of("2012-01-01"); DruidCluster druidCluster; private LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); @@ -70,8 +67,8 @@ public class DruidCoordinatorCleanupOvershadowedTest @Test public void testRun() { - druidCoordinatorCleanupOvershadowed = new DruidCoordinatorCleanupOvershadowed(coordinator); - availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); + druidCoordinatorMarkAsUnusedOvershadowedSegments = new DruidCoordinatorCleanupOvershadowed(coordinator); + usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); // Dummy values for comparisons in TreeSet EasyMock.expect(mockPeon.getLoadQueueSize()) @@ -103,27 +100,23 @@ public void testRun() .andReturn(ImmutableSet.of(segmentV1, segmentV2)) .anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); - coordinator.removeSegment(segmentV1); - coordinator.removeSegment(segmentV0); + coordinator.markSegmentAsUnused(segmentV1); + coordinator.markSegmentAsUnused(segmentV0); EasyMock.expectLastCall(); EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource); - druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder(druidServer, mockPeon) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - )); + druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", new ServerHolder(druidServer, mockPeon)) + .build(); - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withCoordinatorStats(new CoordinatorStats()) .withDruidCluster(druidCluster) .build(); - druidCoordinatorCleanupOvershadowed.run(params); + druidCoordinatorMarkAsUnusedOvershadowedSegments.run(params); EasyMock.verify(coordinator, druidDataSource, druidServer); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 66b73a789479..7f28bbba44db 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -324,9 +325,9 @@ public String get() private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor, boolean keepSegmentGranularity) { - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() - .withDataSources(dataSources) + .setDataSourcesWithUsedSegments(dataSources) .withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs(keepSegmentGranularity))) .build(); return compactor.run(params).getCoordinatorStats(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 7ee58a2a8f19..d52e032efb1a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -25,6 +25,7 @@ import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.easymock.EasyMock; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; @@ -37,34 +38,34 @@ public class DruidCoordinatorSegmentKillerTest { @Test - public void testFindIntervalForKillTask() + public void testFindIntervalForKill() { - testFindIntervalForKillTask(null, null); - testFindIntervalForKillTask(ImmutableList.of(), null); + testFindIntervalForKill(null, null); + testFindIntervalForKill(ImmutableList.of(), null); - testFindIntervalForKillTask(ImmutableList.of(Intervals.of("2014/2015")), Intervals.of("2014/2015")); + testFindIntervalForKill(ImmutableList.of(Intervals.of("2014/2015")), Intervals.of("2014/2015")); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2016/2017")), Intervals.of("2014/2017") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2015/2016")), Intervals.of("2014/2016") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2015/2016"), Intervals.of("2014/2015")), Intervals.of("2014/2016") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2015/2017"), Intervals.of("2014/2016")), Intervals.of("2014/2017") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of( Intervals.of("2015/2019"), Intervals.of("2014/2016"), @@ -73,7 +74,7 @@ public void testFindIntervalForKillTask() Intervals.of("2014/2020") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of( Intervals.of("2015/2019"), Intervals.of("2014/2016"), @@ -84,23 +85,21 @@ public void testFindIntervalForKillTask() ); } - private void testFindIntervalForKillTask(List segmentManagerResult, Interval expected) + private void testFindIntervalForKill(List segmentIntervals, Interval expected) { - MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect( - segmentManager.getUnusedSegmentIntervals( + segmentsMetadata.getUnusedSegmentIntervals( EasyMock.anyString(), - EasyMock.anyObject(Interval.class), + EasyMock.anyObject(DateTime.class), EasyMock.anyInt() ) - ).andReturn( - segmentManagerResult - ); - EasyMock.replay(segmentManager); + ).andReturn(segmentIntervals); + EasyMock.replay(segmentsMetadata); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller( - segmentManager, + segmentsMetadata, indexingServiceClient, new TestDruidCoordinatorConfig( null, @@ -120,7 +119,7 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In Assert.assertEquals( expected, - coordinatorSegmentKiller.findIntervalForKillTask("test", 10000) + coordinatorSegmentKiller.findIntervalForKill("test", 10000) ); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index adb344792722..7043fa86443e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -20,13 +20,14 @@ package org.apache.druid.server.coordinator.rules; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.SegmentReplicantLookup; @@ -37,12 +38,8 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -244,56 +241,50 @@ public void setUp() true ); - druidCluster = new DruidCluster( - null, - ImmutableMap.of( + druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - holdersOfLargeSegments.get(0), - holderOfSmallSegment, - holdersOfLargeSegments2.get(0) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + holdersOfLargeSegments.get(0), + holderOfSmallSegment, + holdersOfLargeSegments2.get(0) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - holdersOfLargeSegments.get(1), - holdersOfLargeSegments.get(2), - holdersOfLargeSegments2.get(1) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + holdersOfLargeSegments.get(1), + holdersOfLargeSegments.get(2), + holdersOfLargeSegments2.get(1) ) - ); + .build(); - secondCluster = new DruidCluster( - null, - ImmutableMap.of( + secondCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Stream.of( - activeServer, - decommissioningServer1, - decommissioningServer2 - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + activeServer, + decommissioningServer1, + decommissioningServer2 ) - ); + .build(); } @Test public void testBroadcastToSingleDataSource() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + final ForeverBroadcastDistributionRule rule = + new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); @@ -313,6 +304,19 @@ public void testBroadcastToSingleDataSource() assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); } + private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( + DruidCluster druidCluster, + DataSegment... usedSegments + ) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withUsedSegmentsInTest(usedSegments) + .build(); + } + /** * Servers: * name | segments @@ -329,19 +333,17 @@ public void testBroadcastToSingleDataSource() @Test public void testBroadcastDecommissioning() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + final ForeverBroadcastDistributionRule rule = + new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(secondCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1) - ).build(), + makeCoordinartorRuntimeParams( + secondCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1) + ), smallSegment ); @@ -362,18 +364,15 @@ public void testBroadcastToMultipleDataSources() CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); @@ -400,23 +399,20 @@ public void testBroadcastToAllServers() CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); assertEquals(6L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); - assertEquals(false, stats.hasPerTierStats()); + assertFalse(stats.hasPerTierStats()); assertTrue( druidCluster.getAllServers().stream() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 73d47ccc38bf..b624cf30dd53 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -26,6 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -36,9 +37,11 @@ import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.BalancerStrategy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.LoadQueuePeonTester; @@ -56,18 +59,13 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** */ @@ -138,53 +136,34 @@ public void testLoad() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverNorm", + "hostNorm", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER)); @@ -192,6 +171,21 @@ public void testLoad() EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); } + private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams( + DruidCluster druidCluster, + DataSegment... usedSegments + ) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(usedSegments) + .build(); + } + @Test public void testLoadPrimaryAssignDoesNotOverAssign() { @@ -213,47 +207,18 @@ public void testLoadPrimaryAssignDoesNotOverAssign() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ), new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + ImmutableDruidServer server1 = + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); + ImmutableDruidServer server2 = + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon)) + .build(); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + makeCoordinatorRuntimeParams(druidCluster, segment), segment ); @@ -264,50 +229,17 @@ public void testLoadPrimaryAssignDoesNotOverAssign() final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment)); EasyMock.replay(loadingPeon); - DruidCluster afterLoad = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - loadingPeon - ), new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster afterLoad = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1, loadingPeon), new ServerHolder(server2, mockPeon)) + .build(); + CoordinatorStats statsAfterLoadPrimary = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(afterLoad) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(afterLoad)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + makeCoordinatorRuntimeParams(afterLoad, segment), segment ); - Assert.assertEquals(0, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); @@ -330,72 +262,36 @@ public void testLoadPriority() EasyMock.replay(throttler, mockPeon1, mockPeon2, mockBalancerStrategy); - final LoadRule rule = createLoadRule(ImmutableMap.of( - "tier1", 10, - "tier2", 10 - )); + final LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 10, "tier2", 10)); - final DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + final DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Stream.of( - new ServerHolder( - new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "tier1", - 0 - ).toImmutableDruidServer(), - mockPeon1 - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "tier1", 0) + .toImmutableDruidServer(), + mockPeon1 + ) + ) + .addTier( "tier2", - Stream.of( - new ServerHolder( - new DruidServer( - "server2", - "host2", - null, - 1000, - ServerType.HISTORICAL, - "tier2", - 1 - ).toImmutableDruidServer(), - mockPeon2 - ), - new ServerHolder( - new DruidServer( - "server3", - "host3", - null, - 1000, - ServerType.HISTORICAL, - "tier2", - 1 - ).toImmutableDruidServer(), - mockPeon2 - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("server2", "host2", null, 1000, ServerType.HISTORICAL, "tier2", 1) + .toImmutableDruidServer(), + mockPeon2 + ), + new ServerHolder( + new DruidServer("server3", "host3", null, 1000, ServerType.HISTORICAL, "tier2", 1) + .toImmutableDruidServer(), + mockPeon2 + ) ) - ); + .build(); final DataSegment segment = createDataSegment("foo"); - final CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + final CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(0L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); @@ -421,15 +317,7 @@ public void testDrop() final DataSegment segment = createDataSegment("foo"); - DruidServer server1 = new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); + DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); server1.addDataSegment(segment); DruidServer server2 = new DruidServer( "serverNorm", @@ -450,41 +338,17 @@ public void testDrop() DruidServer.DEFAULT_TIER, 0 ); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server3.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server2.toImmutableDruidServer(), mockPeon), + new ServerHolder(server3.toImmutableDruidServer(), mockPeon) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER)); @@ -505,43 +369,32 @@ public void testLoadWithNonExistentTier() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of( - "nonExistentTier", 1, - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); final DataSegment segment = createDataSegment("foo"); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(segment) + .build(), segment ); @@ -561,62 +414,25 @@ public void testDropWithNonExistentTier() .times(2); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of( - "nonExistentTier", 1, - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); final DataSegment segment = createDataSegment("foo"); - DruidServer server1 = new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - DruidServer server2 = new DruidServer( - "serverHo2t", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); + DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server2 = new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0); server1.addDataSegment(segment); server2.addDataSegment(segment); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); @@ -634,46 +450,33 @@ public void testMaxLoadingQueueSize() final LoadQueuePeonTester peon = new LoadQueuePeonTester(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("hot", 1)); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - peon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + peon + ) ) - ); + .build(); DataSegment dataSegment1 = createDataSegment("ds1"); DataSegment dataSegment2 = createDataSegment("ds2"); DataSegment dataSegment3 = createDataSegment("ds3"); - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) + .build(); CoordinatorStats stats1 = rule.run(null, params, dataSegment1); CoordinatorStats stats2 = rule.run(null, params, dataSegment2); @@ -696,10 +499,7 @@ public void testLoadDecommissioning() final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "tier1", 1, - "tier2", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 1, "tier2", 1)); final DataSegment segment = createDataSegment("foo"); @@ -710,27 +510,13 @@ public void testLoadDecommissioning() EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "tier1", - Collections.singleton(createServerHolder("tier1", mockPeon1, true)), - "tier2", - Collections.singleton(createServerHolder("tier2", mockPeon2, false)) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("tier1", createServerHolder("tier1", mockPeon1, true)) + .addTier("tier2", createServerHolder("tier2", mockPeon2, false)) + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); @@ -771,23 +557,13 @@ public void testLoadReplicaDuringDecommissioning() EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("tier1", holder1, holder2) + .addTier("tier2", holder3, holder4) + .build(); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of("tier1", Arrays.asList(holder1, holder2), "tier2", Arrays.asList(holder3, holder4)) - ); - - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); @@ -820,40 +596,21 @@ public void testDropDuringDecommissioning() DruidServer server2 = createServer("tier1"); server2.addDataSegment(segment2); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Arrays.asList( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) - ) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) ) - ); - - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1, segment2) .build(); - CoordinatorStats stats = rule.run( - null, - params, - segment1 - ); + + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, segment1, segment2); + CoordinatorStats stats = rule.run(null, params, segment1); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); - stats = rule.run( - null, - params, - segment2 - ); + stats = rule.run(null, params, segment2); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); - EasyMock.verify(throttler, mockPeon); } @@ -885,32 +642,17 @@ public void testRedundantReplicaDropDuringDecommissioning() DruidServer server3 = createServer("tier1"); server3.addDataSegment(segment1); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Arrays.asList( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), - new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) - ) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), + new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) ) - ); - - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1) .build(); - CoordinatorStats stats = rule.run( - null, - params, - segment1 - ); + + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment1), segment1); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size()); Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size()); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index af97906f9cc7..f678364b38ba 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -223,9 +223,10 @@ public void testBuilderDefaults() @Test public void testUpdate() { - CoordinatorDynamicConfig current = CoordinatorDynamicConfig.builder() - .withKillDataSourceWhitelist(ImmutableSet.of("x")) - .build(); + CoordinatorDynamicConfig current = CoordinatorDynamicConfig + .builder() + .withSpecificDataSourcesToKillUnusedSegmentsIn(ImmutableSet.of("x")) + .build(); Assert.assertEquals( current, @@ -246,7 +247,7 @@ public void testEqualsAndHashCodeSanity() private void assertConfig( CoordinatorDynamicConfig config, - long expectedMillisToWaitBeforeDeleting, + long expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, long expectedMergeBytesLimit, int expectedMergeSegmentsLimit, int expectedMaxSegmentsToMove, @@ -254,14 +255,17 @@ private void assertConfig( int expectedReplicationThrottleLimit, int expectedBalancerComputeThreads, boolean expectedEmitingBalancingStats, - Set expectedKillableDatasources, - boolean expectedKillAllDataSources, + Set expectedSpecificDataSourcesToKillUnusedSegmentsIn, + boolean expectedKillUnusedSegmentsInAllDataSources, int expectedMaxSegmentsInNodeLoadingQueue, - Set decommissioning, + Set decommissioningNodes, int decommissioningMaxPercentOfMaxSegmentsToMove ) { - Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting()); + Assert.assertEquals( + expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, + config.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + ); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove()); @@ -269,10 +273,16 @@ private void assertConfig( Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); - Assert.assertEquals(expectedKillableDatasources, config.getKillableDataSources()); - Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources()); + Assert.assertEquals( + expectedSpecificDataSourcesToKillUnusedSegmentsIn, + config.getSpecificDataSourcesToKillUnusedSegmentsIn() + ); + Assert.assertEquals(expectedKillUnusedSegmentsInAllDataSources, config.isKillUnusedSegmentsInAllDataSources()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); - Assert.assertEquals(decommissioning, config.getDecommissioningNodes()); - Assert.assertEquals(decommissioningMaxPercentOfMaxSegmentsToMove, config.getDecommissioningMaxPercentOfMaxSegmentsToMove()); + Assert.assertEquals(decommissioningNodes, config.getDecommissioningNodes()); + Assert.assertEquals( + decommissioningMaxPercentOfMaxSegmentsToMove, + config.getDecommissioningMaxPercentOfMaxSegmentsToMove() + ); } } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 03b2146b44fa..b3c02a7ccaef 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -20,7 +20,9 @@ package org.apache.druid.server.http; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; @@ -32,6 +34,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.UnknownSegmentIdException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -170,15 +173,9 @@ public void testGetFullQueryableDataSources() EasyMock.expectLastCall().times(1); EasyMock.replay(inventoryView, server, request); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - Response response = DataSourcesResource.getQueryableDataSources("full", null, request); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(2, result.size()); @@ -187,7 +184,7 @@ public void testGetFullQueryableDataSources() new HashSet<>(result) ); - response = DataSourcesResource.getQueryableDataSources(null, null, request); + response = dataSourcesResource.getQueryableDataSources(null, null, request); List result1 = (List) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(2, result1.size()); @@ -213,9 +210,7 @@ public void testSecuredGetFullQueryableDataSources() request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).once(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); // second request EasyMock.expect(server.getDataSources()).andReturn( @@ -230,9 +225,7 @@ public void testSecuredGetFullQueryableDataSources() request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).once(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.replay(inventoryView, server, request); AuthorizerMapper authMapper = new AuthorizerMapper(null) { @@ -255,27 +248,15 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res } }; - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - authMapper - ); - Response response = DataSourcesResource.getQueryableDataSources("full", null, request); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, authMapper); + Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(1, result.size()); - Assert.assertEquals( - listDataSources.subList(0, 1).stream() - .map(DruidDataSource::toImmutableDruidDataSource) - .collect(Collectors.toSet()), - new HashSet<>(result) - ); + Assert.assertEquals(listDataSources.get(0).toImmutableDruidDataSource(), Iterables.getOnlyElement(result)); - response = DataSourcesResource.getQueryableDataSources(null, null, request); + response = dataSourcesResource.getQueryableDataSources(null, null, request); List result1 = (List) response.getEntity(); Assert.assertEquals(200, response.getStatus()); @@ -302,15 +283,9 @@ public void testGetSimpleQueryableDataSources() EasyMock.expectLastCall().times(1); EasyMock.replay(inventoryView, server, request); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); - Response response = DataSourcesResource.getQueryableDataSources(null, "simple", request); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + Response response = dataSourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); List> results = (List>) response.getEntity(); int index = 0; @@ -332,9 +307,9 @@ public void testFullGetTheDataSource() EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).atLeastOnce(); EasyMock.replay(inventoryView, server); - DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", "full"); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getDataSource("datasource1", "full"); ImmutableDruidDataSource result = (ImmutableDruidDataSource) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(dataSource1.toImmutableDruidDataSource(), result); @@ -348,9 +323,9 @@ public void testNullGetTheDataSource() EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).atLeastOnce(); EasyMock.replay(inventoryView, server); - DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Assert.assertEquals(204, DataSourcesResource.getTheDataSource("none", null).getStatus()); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, null); + Assert.assertEquals(204, dataSourcesResource.getDataSource("none", null).getStatus()); EasyMock.verify(inventoryView, server); } @@ -366,9 +341,9 @@ public void testSimpleGetTheDataSource() EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).atLeastOnce(); EasyMock.replay(inventoryView, server); - DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", null); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); Assert.assertEquals(1, ((Map) (result.get("tiers").get(null))).get("segmentCount")); @@ -384,31 +359,22 @@ public void testSimpleGetTheDataSource() @Test public void testSimpleGetTheDataSourceManyTiers() { - EasyMock.expect(server.getDataSource("datasource1")).andReturn( - listDataSources.get(0) - ).atLeastOnce(); + EasyMock.expect(server.getDataSource("datasource1")).andReturn(listDataSources.get(0)).atLeastOnce(); EasyMock.expect(server.getTier()).andReturn("cold").atLeastOnce(); DruidServer server2 = EasyMock.createStrictMock(DruidServer.class); - EasyMock.expect(server2.getDataSource("datasource1")).andReturn( - listDataSources.get(1) - ).atLeastOnce(); + EasyMock.expect(server2.getDataSource("datasource1")).andReturn(listDataSources.get(1)).atLeastOnce(); EasyMock.expect(server2.getTier()).andReturn("hot").atLeastOnce(); DruidServer server3 = EasyMock.createStrictMock(DruidServer.class); - EasyMock.expect(server3.getDataSource("datasource1")).andReturn( - listDataSources.get(1) - ).atLeastOnce(); + EasyMock.expect(server3.getDataSource("datasource1")).andReturn(listDataSources.get(1)).atLeastOnce(); EasyMock.expect(server3.getTier()).andReturn("cold").atLeastOnce(); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server, server2, server3) - ).atLeastOnce(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server, server2, server3)).atLeastOnce(); EasyMock.replay(inventoryView, server, server2, server3); - DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); Assert.assertEquals(2, ((Map) (result.get("tiers").get("cold"))).get("segmentCount")); @@ -430,27 +396,37 @@ public void testGetSegmentDataSourceIntervals() server.addDataSegment(dataSegmentList.get(0)); server.addDataSegment(dataSegmentList.get(1)); server.addDataSegment(dataSegmentList.get(2)); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).atLeastOnce(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).atLeastOnce(); EasyMock.replay(inventoryView); List expectedIntervals = new ArrayList<>(); expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); + new DataSourcesResource(inventoryView, null, null, null, null); - Response response = dataSourcesResource.getSegmentDataSourceIntervals("invalidDataSource", null, null); + Response response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( + "invalidDataSource", + null, + null + ); Assert.assertEquals(response.getEntity(), null); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", null, null); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( + "datasource1", + null, + null + ); TreeSet actualIntervals = (TreeSet) response.getEntity(); Assert.assertEquals(2, actualIntervals.size()); Assert.assertEquals(expectedIntervals.get(0), actualIntervals.first()); Assert.assertEquals(expectedIntervals.get(1), actualIntervals.last()); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", "simple", null); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( + "datasource1", + "simple", + null + ); TreeMap> results = (TreeMap) response.getEntity(); Assert.assertEquals(2, results.size()); Assert.assertEquals(expectedIntervals.get(0), results.firstKey()); @@ -458,7 +434,11 @@ public void testGetSegmentDataSourceIntervals() Assert.assertEquals(1, results.firstEntry().getValue().get(DataSourcesResource.SimpleProperties.count)); Assert.assertEquals(1, results.lastEntry().getValue().get(DataSourcesResource.SimpleProperties.count)); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", null, "full"); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( + "datasource1", + null, + "full" + ); Map> results2 = ((Map>) response.getEntity()); int i = 1; for (Map.Entry> entry : results2.entrySet()) { @@ -473,7 +453,7 @@ public void testGetSegmentDataSourceIntervals() } @Test - public void testGetSegmentDataSourceSpecificInterval() + public void testGetServedSegmentsInIntervalInDataSource() { server = new DruidServer("who", "host", null, 1234, ServerType.HISTORICAL, "tier1", 0); server.addDataSegment(dataSegmentList.get(0)); @@ -483,8 +463,8 @@ public void testGetSegmentDataSourceSpecificInterval() EasyMock.replay(inventoryView); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getServedSegmentsInInterval( "invalidDataSource", "2010-01-01/P1D", null, @@ -492,7 +472,7 @@ public void testGetSegmentDataSourceSpecificInterval() ); Assert.assertEquals(null, response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + response = dataSourcesResource.getServedSegmentsInInterval( "datasource1", "2010-03-01/P1D", null, @@ -500,16 +480,16 @@ public void testGetSegmentDataSourceSpecificInterval() ); // interval not present in the datasource Assert.assertEquals(ImmutableSet.of(), response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1D", null, null); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1D", null, null); Assert.assertEquals(ImmutableSet.of(dataSegmentList.get(0).getId()), response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, null); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1M", null, null); Assert.assertEquals( ImmutableSet.of(dataSegmentList.get(1).getId(), dataSegmentList.get(0).getId()), response.getEntity() ); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + response = dataSourcesResource.getServedSegmentsInInterval( "datasource1", "2010-01-01/P1M", "simple", @@ -527,7 +507,7 @@ public void testGetSegmentDataSourceSpecificInterval() ); } - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, "full"); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1M", null, "full"); Map> results1 = ((Map>) response.getEntity()); i = 1; for (Map.Entry> entry : results1.entrySet()) { @@ -542,7 +522,7 @@ public void testGetSegmentDataSourceSpecificInterval() } @Test - public void testDeleteDataSourceSpecificInterval() + public void testKillSegmentsInIntervalInDataSource() { String interval = "2010-01-01_P1D"; Interval theInterval = Intervals.of(interval.replace('_', '/')); @@ -552,15 +532,9 @@ public void testDeleteDataSourceSpecificInterval() EasyMock.expectLastCall().once(); EasyMock.replay(indexingServiceClient, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - indexingServiceClient, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.deleteDataSourceSpecificInterval("datasource1", interval); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + Response response = dataSourcesResource.killSegmentsInInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(null, response.getEntity()); @@ -568,22 +542,23 @@ public void testDeleteDataSourceSpecificInterval() } @Test - public void testDeleteDataSource() + public void testMarkAsUnusedAllSegmentsInDataSource() { IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); EasyMock.replay(indexingServiceClient, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - indexingServiceClient, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.deleteDataSource("datasource", "true", "???"); - Assert.assertEquals(400, response.getStatus()); - Assert.assertNotNull(response.getEntity()); - Assert.assertTrue(response.getEntity().toString().contains("java.lang.IllegalArgumentException")); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + try { + Response response = + dataSourcesResource.markAsUnusedAllSegmentsOrKillSegmentsInInterval("datasource", "true", "???"); + // 400 (Bad Request) or an IllegalArgumentException is expected. + Assert.assertEquals(400, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertTrue(response.getEntity().toString().contains("java.lang.IllegalArgumentException")); + } + catch (IllegalArgumentException ignore) { + // expected + } EasyMock.verify(indexingServiceClient, server); } @@ -594,14 +569,8 @@ public void testIsHandOffComplete() MetadataRuleManager databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); Rule loadRule = new IntervalLoadRule(Intervals.of("2013-01-02T00:00:00Z/2013-01-03T00:00:00Z"), null); Rule dropRule = new IntervalDropRule(Intervals.of("2013-01-01T00:00:00Z/2013-01-02T00:00:00Z")); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - null, - databaseRuleManager, - null, - new AuthConfig(), - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, databaseRuleManager, null, null); // test dropped EasyMock.expect(databaseRuleManager.getRulesWithDefault("dataSource1")) @@ -610,7 +579,7 @@ public void testIsHandOffComplete() EasyMock.replay(databaseRuleManager); String interval1 = "2013-01-01T01:00:00Z/2013-01-01T02:00:00Z"; - Response response1 = DataSourcesResource.isHandOffComplete("dataSource1", interval1, 1, "v1"); + Response response1 = dataSourcesResource.isHandOffComplete("dataSource1", interval1, 1, "v1"); Assert.assertTrue((boolean) response1.getEntity()); EasyMock.verify(databaseRuleManager); @@ -626,7 +595,7 @@ public void testIsHandOffComplete() EasyMock.replay(inventoryView, databaseRuleManager); String interval2 = "2013-01-02T01:00:00Z/2013-01-02T02:00:00Z"; - Response response2 = DataSourcesResource.isHandOffComplete("dataSource1", interval2, 1, "v1"); + Response response2 = dataSourcesResource.isHandOffComplete("dataSource1", interval2, 1, "v1"); Assert.assertFalse((boolean) response2.getEntity()); EasyMock.verify(inventoryView, databaseRuleManager); @@ -635,17 +604,14 @@ public void testIsHandOffComplete() String interval3 = "2013-01-02T02:00:00Z/2013-01-02T03:00:00Z"; SegmentLoadInfo segmentLoadInfo = new SegmentLoadInfo(createSegment(Intervals.of(interval3), "v1", 1)); segmentLoadInfo.addServer(createHistoricalServerMetadata("test")); - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - null) + VersionedIntervalTimeline timeline = + new VersionedIntervalTimeline(null) { @Override public List> lookupWithIncompletePartitions(Interval interval) { - PartitionHolder partitionHolder = new PartitionHolder<>(new NumberedPartitionChunk<>( - 1, - 1, - segmentLoadInfo - )); + PartitionHolder partitionHolder = + new PartitionHolder<>(new NumberedPartitionChunk<>(1, 1, segmentLoadInfo)); List> ret = new ArrayList<>(); ret.add(new TimelineObjectHolder<>(Intervals.of(interval3), "v1", partitionHolder)); return ret; @@ -660,317 +626,219 @@ public List> lookupWithIncompleteP .once(); EasyMock.replay(inventoryView, databaseRuleManager); - Response response3 = DataSourcesResource.isHandOffComplete("dataSource1", interval3, 1, "v1"); + Response response3 = dataSourcesResource.isHandOffComplete("dataSource1", interval3, 1, "v1"); Assert.assertTrue((boolean) response3.getEntity()); EasyMock.verify(inventoryView, databaseRuleManager); } @Test - public void testEnableDatasourceSegment() + public void testMarkSegmentAsUsed() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); - EasyMock.expect(metadataSegmentManager.enableSegment(dataSegmentList.get(0).getId().toString())) - .andReturn(true) - .once(); - EasyMock.replay(metadataSegmentManager); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSegment segment = dataSegmentList.get(0); + EasyMock.expect(segmentsMetadata.markSegmentAsUsed(segment.getId().toString())).andReturn(true).once(); + EasyMock.replay(segmentsMetadata); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - null, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegment(dataSegmentList.get(0).getDataSource(), dataSegmentList.get(0).getId().toString()); + Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentFailed() + public void testMarkSegmentAsUsedNoChange() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); - EasyMock.expect(metadataSegmentManager.enableSegment(dataSegmentList.get(0).getId().toString())) - .andReturn(false) - .once(); - EasyMock.replay(metadataSegmentManager); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSegment segment = dataSegmentList.get(0); + EasyMock.expect(segmentsMetadata.markSegmentAsUsed(segment.getId().toString())).andReturn(false).once(); + EasyMock.replay(segmentsMetadata); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - null, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegment(dataSegmentList.get(0).getDataSource(), dataSegmentList.get(0).getId().toString()); - Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("segmentStateChanged", false), response.getEntity()); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentsInterval() + public void testMarkAsUsedNonOvershadowedSegmentsInterval() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andReturn(3) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsIntervalNoneUpdated() + public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andReturn(0) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andReturn(0).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); - Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsSet() + public void testMarkAsUsedNonOvershadowedSegmentsSet() throws UnknownSegmentIdException { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Set segmentIds = ImmutableSet.of(dataSegmentList.get(1).getId().toString()); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds))) - .andReturn(3) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds)); + EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds) ); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsIntervalException() + public void testMarkAsUsedNonOvershadowedSegmentsIntervalException() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andThrow(new RuntimeException("Error!")) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andThrow(new RuntimeException("Error!")).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); Assert.assertEquals(500, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentslNoDatasource() + public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(null).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-22/P1D"), - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), null) ); Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadNoArguments() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadNoArguments() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, null) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadBothArguments() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadBothArguments() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-22/P1D"), - ImmutableSet.of() - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), ImmutableSet.of()) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadEmptyArray() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadEmptyArray() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - ImmutableSet.of() - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, ImmutableSet.of()) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsNoPayload() + public void testMarkAsUsedNonOvershadowedSegmentsNoPayload() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.enableDatasourceSegments( - "datasource1", - null - ); + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments("datasource1", null); Assert.assertEquals(400, response.getStatus()); } @Test public void testSegmentLoadChecksForVersion() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertFalse( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1012,9 +880,7 @@ public void testSegmentLoadChecksForVersion() @Test public void testSegmentLoadChecksForAssignableServer() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertTrue( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1043,9 +909,7 @@ public void testSegmentLoadChecksForAssignableServer() @Test public void testSegmentLoadChecksForPartitionNumber() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertTrue( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1102,218 +966,159 @@ public void testSegmentLoadChecksForInterval() } @Test - public void testMarkDatasourceUnusedWithSegments() + public void testMarkSegmentsAsUnused() throws UnknownSegmentIdException { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)).andReturn(1L).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)).andReturn(1).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithSegmentsNoContent() + public void testMarkSegmentsAsUnusedNoChanges() throws UnknownSegmentIdException { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)).andReturn(0L).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)).andReturn(0).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); - Assert.assertEquals(204, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithSegmentsException() + public void testMarkSegmentsAsUnusedException() throws UnknownSegmentIdException { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)) + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)) .andThrow(new RuntimeException("Exception occurred")) .once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithInterval() + public void testMarkAsUnusedSegmentsInInterval() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)).andReturn(1).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(1).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); - EasyMock.verify(segmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithIntervaNoContent() + public void testMarkAsUnusedSegmentsInIntervalNoChanges() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)).andReturn(0).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(0).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); - Assert.assertEquals(204, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithIntervaException() + public void testMarkAsUnusedSegmentsInIntervalException() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)) + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)) .andThrow(new RuntimeException("Exception occurred")) .once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedNullPayload() + public void testMarkSegmentsUnusedNullPayload() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); - - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = null; + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", null); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); Assert.assertEquals( @@ -1323,47 +1128,31 @@ public void testMarkDatasourceUnusedNullPayload() } @Test - public void testMarkDatasourceUnusedInvalidPayload() + public void testMarkSegmentsUnusedInvalidPayload() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, null); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); } @Test - public void testMarkDatasourceUnusedInvalidPayloadBothArguments() + public void testMarkSegmentsUnusedInvalidPayloadBothArguments() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource DataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-01/P1D"), - ImmutableSet.of() - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-01/P1D"), ImmutableSet.of()); - Response response = DataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); } @@ -1380,15 +1169,7 @@ private DruidServerMetadata createHistoricalServerMetadata(String name) private DruidServerMetadata createServerMetadata(String name, ServerType type) { - return new DruidServerMetadata( - name, - name, - null, - 10000, - type, - "tier", - 1 - ); + return new DruidServerMetadata(name, name, null, 10000, type, "tier", 1); } private DataSegment createSegment(Interval interval, String version, int partitionNumber) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index e5cfa911c452..28b87887c14f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -514,7 +514,7 @@ public Enumerable scan(DataContext root) for (ImmutableDruidServer druidServer : druidServers) { final Iterable authorizedServerSegments = AuthorizationUtils.filterAuthorizedResources( authenticationResult, - druidServer.getLazyAllSegments(), + druidServer.iterateAllSegments(), SEGMENT_RA_GENERATOR, authorizerMapper ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index d607587db4af..9826351ce146 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -271,7 +271,8 @@ public void testSegmentMetadataHolderNumRows() final Pair pair = druidServers .stream() .flatMap(druidServer -> druidServer - .getLazyAllSegments().stream() + .iterateAllSegments() + .stream() .filter(segment -> segment.equals(existingSegment)) .map(segment -> Pair.of(druidServer, segment)) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index a942db4b556c..e4cdfd287934 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -97,6 +97,7 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -403,19 +404,18 @@ public void testGetTableMap() @Test public void testSegmentsTable() { - final SystemSchema.SegmentsTable segmentsTable = EasyMock .createMockBuilder(SystemSchema.SegmentsTable.class) .withConstructor(druidSchema, metadataView, mapper, authMapper) .createMock(); EasyMock.replay(segmentsTable); - final Set publishedSegments = Stream.of( + final Set publishedSegments = new HashSet<>(Arrays.asList( new SegmentWithOvershadowedStatus(publishedSegment1, true), new SegmentWithOvershadowedStatus(publishedSegment2, false), new SegmentWithOvershadowedStatus(publishedSegment3, false), new SegmentWithOvershadowedStatus(segment1, true), new SegmentWithOvershadowedStatus(segment2, false) - ).collect(Collectors.toSet()); + )); EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once(); diff --git a/web-console/README.md b/web-console/README.md index ad02c1034316..7617ba7de792 100644 --- a/web-console/README.md +++ b/web-console/README.md @@ -58,7 +58,7 @@ GET /druid/indexer/v1/supervisor?full GET /druid/indexer/v1/workers GET /druid/coordinator/v1/loadqueue?simple GET /druid/coordinator/v1/config -GET /druid/coordinator/v1/metadata/datasources?includeDisabled +GET /druid/coordinator/v1/metadata/datasources?includeUnused GET /druid/coordinator/v1/rules GET /druid/coordinator/v1/config/compaction GET /druid/coordinator/v1/tiers From a0f72bd4beb7024534e0469c9b7c5c1b26c666db Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 14 May 2019 15:30:24 +0200 Subject: [PATCH 02/15] Style fixes --- .../org/apache/druid/metadata/MetadataSegmentManager.java | 2 +- .../apache/druid/metadata/SQLMetadataSegmentManager.java | 3 ++- .../apache/druid/server/coordinator/DruidCoordinator.java | 4 ++-- .../apache/druid/server/http/DataSourcesResourceTest.java | 6 +++--- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index 41523ef84734..cadb2ee19bcc 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -72,7 +72,7 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) int markAsUnusedSegmentsInInterval(String dataSource, Interval interval); - int markSegmentsAsUnused(String dataSource, Set segmentIds) throws UnknownSegmentIdException; + int markSegmentsAsUnused(String dataSource, Set segmentIds); /** * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index d63e9c1f6e15..9d924916c05d 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -489,7 +489,8 @@ private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable return markNonOvershadowedSegmentsAsUsed(dataSourceName, unusedSegmentsInInterval, versionedIntervalTimeline); } - private static void consume(Iterator iterator) { + private static void consume(Iterator iterator) + { while (iterator.hasNext()) { iterator.next(); } 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 a9ed5110b17b..0651b0548fca 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 @@ -61,12 +61,12 @@ import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnneeded; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; import org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnneeded; import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentInfoLoader; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index b3c02a7ccaef..700f12658c0a 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -966,7 +966,7 @@ public void testSegmentLoadChecksForInterval() } @Test - public void testMarkSegmentsAsUnused() throws UnknownSegmentIdException + public void testMarkSegmentsAsUnused() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); final Set segmentIds = @@ -990,7 +990,7 @@ public void testMarkSegmentsAsUnused() throws UnknownSegmentIdException } @Test - public void testMarkSegmentsAsUnusedNoChanges() throws UnknownSegmentIdException + public void testMarkSegmentsAsUnusedNoChanges() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); final Set segmentIds = @@ -1014,7 +1014,7 @@ public void testMarkSegmentsAsUnusedNoChanges() throws UnknownSegmentIdException } @Test - public void testMarkSegmentsAsUnusedException() throws UnknownSegmentIdException + public void testMarkSegmentsAsUnusedException() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); final Set segmentIds = From 2733f8ee95d05010fcf41cf95cc789c86920e87d Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 14 May 2019 18:44:59 +0200 Subject: [PATCH 03/15] Unused imports --- .../org/apache/druid/sql/calcite/schema/SystemSchemaTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index e4cdfd287934..2e384c778f71 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -101,8 +101,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class SystemSchemaTest extends CalciteTestBase { From 32382b199b2d9e2676267e950bba01762bbd7578 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 17 May 2019 17:18:21 +0200 Subject: [PATCH 04/15] Fix tests --- .../MetadataSegmentManagerConfig.java | 5 ++++ .../DruidCoordinatorRuntimeParams.java | 2 +- .../DruidCoordinatorCleanupOvershadowed.java | 2 +- .../SQLMetadataSegmentManagerTest.java | 26 ++++++++++++++----- .../DruidCoordinatorRuleRunnerTest.java | 12 ++++----- ...uidCoordinatorCleanupOvershadowedTest.java | 4 +++ 6 files changed, 36 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java index 3f58cfa70982..d6c881198a89 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java @@ -33,4 +33,9 @@ public Period getPollDuration() { return pollDuration; } + + public void setPollDuration(Period pollDuration) + { + this.pollDuration = pollDuration; + } } 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 e56923f52cde..64d3611af882 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 @@ -179,7 +179,7 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } - public boolean lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements() + public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements() { long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); long lagNanos = TimeUnit.MILLISECONDS.toNanos( diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index b61d543f6945..fc2b954f95f0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -47,7 +47,7 @@ public DruidCoordinatorCleanupOvershadowed(DruidCoordinator coordinator) public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { // Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data. - if (!params.lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements()) { + if (!params.coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()) { return params; } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index 522078dcfe34..944b50fb8faa 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -35,6 +35,7 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -115,9 +116,11 @@ private void publish(DataSegment segment, boolean used) throws IOException public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataSegmentManagerConfig config = new MetadataSegmentManagerConfig(); + config.setPollDuration(Period.seconds(1)); sqlSegmentsMetadata = new SQLMetadataSegmentManager( jsonMapper, - Suppliers.ofInstance(new MetadataSegmentManagerConfig()), + Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), connector ); @@ -282,8 +285,8 @@ public void testGetUnusedSegmentIntervals() ); } - @Test - public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException + @Test(timeout = 60_000) + public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException, InterruptedException { sqlSegmentsMetadata.startPollingDatabasePeriodically(); sqlSegmentsMetadata.poll(); @@ -294,9 +297,10 @@ public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException publisher.publishSegment(newSegment); - Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + awaitPollingNewDataSource(newDataSource); int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource(newDataSource); Assert.assertEquals(1, numChangedSegments); + Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); } private static DataSegment createNewSegment1(String newDataSource) @@ -321,8 +325,8 @@ private static DataSegment createNewSegment2(String newDataSource) ); } - @Test - public void testMarkSegmentAsUnused() throws IOException + @Test(timeout = 60_000) + public void testMarkSegmentAsUnused() throws IOException, InterruptedException { sqlSegmentsMetadata.startPollingDatabasePeriodically(); sqlSegmentsMetadata.poll(); @@ -339,8 +343,16 @@ public void testMarkSegmentAsUnused() throws IOException publisher.publishSegment(newSegment); - Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + awaitPollingNewDataSource(newDataSource); Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId())); + Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + } + + private void awaitPollingNewDataSource(String newDataSource) throws InterruptedException + { + while (sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource) == null) { + Thread.sleep(1000); + } } @Test diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index bfe5637bb506..4b9908baa496 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -58,7 +58,7 @@ */ public class DruidCoordinatorRuleRunnerTest { - public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = + public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = CoordinatorDynamicConfig.builder().withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L).build(); private DruidCoordinator coordinator; @@ -481,7 +481,7 @@ public void testDropRemove() DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -605,7 +605,7 @@ public void testDropTooManyInDifferentTiers() DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -661,7 +661,7 @@ public void testDontDropInDifferentTiers() DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -731,7 +731,7 @@ public void testDropServerActuallyServesSegment() DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -980,7 +980,7 @@ public void testDropReplicantThrottle() DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(longerUsedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java index 965cf2ee153d..f450bf3b051c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java @@ -30,6 +30,7 @@ import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuleRunnerTest; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.ServerHolder; @@ -115,6 +116,9 @@ public void testRun() .withUsedSegmentsInTest(usedSegments) .withCoordinatorStats(new CoordinatorStats()) .withDruidCluster(druidCluster) + .withDynamicConfigs( + DruidCoordinatorRuleRunnerTest.COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + ) .build(); druidCoordinatorMarkAsUnusedOvershadowedSegments.run(params); EasyMock.verify(coordinator, druidDataSource, druidServer); From 366551b87edcfd80f1b34af0fede27363bfa3744 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 17 May 2019 18:19:48 +0200 Subject: [PATCH 05/15] Fix style --- .../org/apache/druid/client/ImmutableDruidServerTests.java | 4 +++- .../coordinator/CoordinatorRuntimeParamsTestHelpers.java | 4 +++- .../apache/druid/server/coordinator/DruidClusterBuilder.java | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java index 8cc5c03c137a..fea252f146f5 100644 --- a/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java +++ b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java @@ -33,5 +33,7 @@ public static void expectSegments(ImmutableDruidServer mockServer, Collection realtimes = null; private final Map> historicals = new HashMap<>(); - private DruidClusterBuilder() {} + private DruidClusterBuilder() + { + } public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) { From f035751bdf7bd3b82080158692e8f842e5c72050 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 24 May 2019 15:20:26 +0200 Subject: [PATCH 06/15] Comments --- docs/content/operations/api-reference.md | 2 +- .../client/indexing/ClientCompactQuery.java | 3 + .../client/indexing/ClientKillQuery.java | 7 +- .../druid/client/indexing/ClientQuery.java | 3 + .../metadata/MetadataSegmentManager.java | 3 + .../metadata/SQLMetadataSegmentManager.java | 84 ++++++++++++------- .../coordinator/CoordinatorDynamicConfig.java | 8 +- .../helper/DruidCoordinatorSegmentKiller.java | 3 + 8 files changed, 79 insertions(+), 34 deletions(-) diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 0cb3eb921409..6cc54da5cd35 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -591,7 +591,7 @@ Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` * `/druid/indexer/v1/worker` -Retreives current overlord dynamic configuration. +Retrieves current overlord dynamic configuration. * `/druid/indexer/v1/worker/history?interval={interval}&counter={count}` diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java index 4f43eb1768e0..020b5dea877e 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java @@ -33,6 +33,9 @@ * Client representation of {@link org.apache.druid.indexing.common.task.CompactionTask}. JSON serialization fields of * this class must correspond to those of {@link org.apache.druid.indexing.common.task.CompactionTask}. */ +// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is +// upgraded to IntelliJ 2018+. +@SuppressWarnings("JavadocResource") public class ClientCompactQuery implements ClientQuery { private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java index c704878af948..b4a62cea335a 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java @@ -24,10 +24,13 @@ import org.joda.time.Interval; /** - * Client representation of {@link org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}. JSON searialization + * Client representation of {@link org.apache.druid.indexing.common.task.KillTask}. JSON searialization * fields of this class must correspond to those of {@link - * org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}, except for "id" and "context" fields. + * org.apache.druid.indexing.common.task.KillTask}, except for "id" and "context" fields. */ +// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is +// upgraded to IntelliJ 2018+. +@SuppressWarnings("JavadocResource") public class ClientKillQuery implements ClientQuery { private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java index 9f0c0e13a534..bac74df9046c 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java @@ -31,6 +31,9 @@ * ClientTaskQuery objects must match with those of the corresponding {@link * org.apache.druid.indexing.common.task.Task} objects. */ +// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is +// upgraded to IntelliJ 2018+. +@SuppressWarnings("JavadocResource") @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "kill", value = ClientKillQuery.class), diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index cadb2ee19bcc..dd88747f148f 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -37,6 +37,9 @@ * class resides in Coordinator's memory, while {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} resides * in Broker's memory. */ +// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is +// upgraded to IntelliJ 2018+. +@SuppressWarnings("JavadocResource") public interface MetadataSegmentManager { void startPollingDatabasePeriodically(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 9d924916c05d..7a5d68f7a6fe 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -164,6 +164,20 @@ long nanosElapsedFromInitiation() * accesses {@link #dataSources} state (such as {@link #prepareImmutableDataSourceWithUsedSegments}) is called when * the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database periodically. * + * Note that if there is a happens-before relationship between a call to {@link #startPollingDatabasePeriodically()} + * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSources}'s state in this + * class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter method + * calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will + * see the latest {@link PeriodicDatabasePoll} value (stored in this field, latestDatabasePoll, in {@link + * #startPollingDatabasePeriodically()}) and to await on its {@link PeriodicDatabasePoll#firstPollCompletionFuture}. + * + * However, the guarantee explained above doesn't make any actual semantic difference, because on both periodic and + * on-demand database polls the same invariant is maintained that the results not older than {@link + * #periodicPollDelay} are used. The main difference is in performance: since on-demand polls are irregular and happen + * in the context of the thread wanting to access the {@link #dataSources}', that may cause a delay in the logic. + * On the other hand, periodic polls are decoupled into {@link #exec} and {@link #dataSources}-accessing methods + * should be generally "wait free" for database polls. + * * The notion and the complexity of "on demand" database polls was introduced to simplify the interface of {@link * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods like * {@link #prepareImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part is @@ -186,6 +200,7 @@ long nanosElapsedFromInitiation() private @Nullable Future periodicPollTaskFuture = null; /** The number of times {@link #startPollingDatabasePeriodically} was called. */ + @GuardedBy("startStopPollLock") private long startPollingCount = 0; /** @@ -200,9 +215,11 @@ long nanosElapsedFromInitiation() * * {@link SQLMetadataRuleManager} also has a similar issue. */ + @GuardedBy("startStopPollLock") private long currentStartPollingOrder = -1; - private ScheduledExecutorService exec = null; + @GuardedBy("startStopPollLock") + private @Nullable ScheduledExecutorService exec = null; @Inject public SQLMetadataSegmentManager( @@ -228,6 +245,9 @@ public void start() ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { + if (exec != null) { + return; // Already started + } exec = Execs.scheduledSingleThreaded(getClass().getName() + "-Exec--%d"); } finally { @@ -304,7 +324,7 @@ private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePo } } catch (Throwable t) { - log.makeAlert(t, "Uncaught exception in " + getClass().getName() + "'s polling thread").emit(); + log.makeAlert(t, "Uncaught exception in %s's polling thread", SQLMetadataSegmentManager.class).emit(); // Swallow the exception, so that scheduled polling goes on. Leave firstPollFutureSinceLastStart uncompleted // for now, so that it may be completed during the next poll. if (!(t instanceof Exception)) { @@ -319,6 +339,22 @@ private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePo }; } + @Override + public boolean isPollingDatabasePeriodically() + { + // isPollingDatabasePeriodically() is synchronized together with startPollingDatabasePeriodically(), + // stopPollingDatabasePeriodically() and poll() to ensure that the latest currentStartPollingOrder is always + // visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. + ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); + lock.lock(); + try { + return currentStartPollingOrder >= 0; + } + finally { + lock.unlock(); + } + } + @Override public void stopPollingDatabasePeriodically() { @@ -344,14 +380,14 @@ public void stopPollingDatabasePeriodically() private void awaitOrPerformDatabasePoll() { - // Double-checked locking with awaitPeriodicOrFreshOnDemandDatabasePoll() call playing the role of the "check". - if (awaitPeriodicOrFreshOnDemandDatabasePoll()) { + // Double-checked locking with awaitLatestDatabasePoll() call playing the role of the "check". + if (awaitLatestDatabasePoll()) { return; } ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (awaitPeriodicOrFreshOnDemandDatabasePoll()) { + if (awaitLatestDatabasePoll()) { return; } OnDemandDatabasePoll newOnDemandUpdate = new OnDemandDatabasePoll(); @@ -363,7 +399,12 @@ private void awaitOrPerformDatabasePoll() } } - private boolean awaitPeriodicOrFreshOnDemandDatabasePoll() + /** + * If the latest {@link DatabasePoll} is a {@link PeriodicDatabasePoll}, or an {@link OnDemandDatabasePoll} that is + * made not longer than {@link #periodicPollDelay} from now, awaits for it and returns true; returns false otherwise, + * meaning that a new on-demand database poll should be initiated. + */ + private boolean awaitLatestDatabasePoll() { DatabasePoll latestDatabasePoll = this.latestDatabasePoll; if (latestDatabasePoll instanceof PeriodicDatabasePoll) { @@ -372,15 +413,16 @@ private boolean awaitPeriodicOrFreshOnDemandDatabasePoll() } if (latestDatabasePoll instanceof OnDemandDatabasePoll) { long periodicPollDelayNanos = TimeUnit.MILLISECONDS.toNanos(periodicPollDelay.getMillis()); - OnDemandDatabasePoll latestOnDemandUpdate = (OnDemandDatabasePoll) latestDatabasePoll; - boolean latestUpdateIsFresh = latestOnDemandUpdate.nanosElapsedFromInitiation() < periodicPollDelayNanos; + OnDemandDatabasePoll latestOnDemandPoll = (OnDemandDatabasePoll) latestDatabasePoll; + boolean latestUpdateIsFresh = latestOnDemandPoll.nanosElapsedFromInitiation() < periodicPollDelayNanos; if (latestUpdateIsFresh) { - Futures.getUnchecked(latestOnDemandUpdate.pollCompletionFuture); + Futures.getUnchecked(latestOnDemandPoll.pollCompletionFuture); return true; } // Latest on-demand update is not fresh. Fall through to return false from this method. } else { assert latestDatabasePoll == null; + // No periodic updates and no on-demand database poll have been done yet, nothing to await for. } return false; } @@ -534,12 +576,12 @@ public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set< Pair, VersionedIntervalTimeline> unusedSegmentsAndTimeline = connector .inReadOnlyTransaction( (handle, status) -> { - List unusedSegments = retreiveUnusedSegments(dataSource, segmentIds, handle); + List unusedSegments = retrieveUnusedSegments(dataSource, segmentIds, handle); List unusedSegmentsIntervals = JodaUtils.condenseIntervals( unusedSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) ); Iterator usedSegmentsOverlappingUnusedSegmentsIntervals = - retreiveUsedSegmentsOverlappingIntervals(dataSource, unusedSegmentsIntervals, handle); + retrieveUsedSegmentsOverlappingIntervals(dataSource, unusedSegmentsIntervals, handle); VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( Iterators.concat(usedSegmentsOverlappingUnusedSegmentsIntervals, unusedSegments.iterator()) ); @@ -561,7 +603,7 @@ public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set< } } - private List retreiveUnusedSegments( + private List retrieveUnusedSegments( final String dataSource, final Set segmentIds, final Handle handle @@ -619,7 +661,7 @@ private List retreiveUnusedSegments( return segments; } - private Iterator retreiveUsedSegmentsOverlappingIntervals( + private Iterator retrieveUsedSegmentsOverlappingIntervals( final String dataSource, final Collection intervals, final Handle handle @@ -882,22 +924,6 @@ private static int computeNumChangedSegments(List segmentIds, int[] segm return numChangedSegments; } - @Override - public boolean isPollingDatabasePeriodically() - { - // isPollingDatabasePeriodically() is synchronized together with startPollingDatabasePeriodically(), - // stopPollingDatabasePeriodically() and poll() to ensure that the latest currentStartPollingOrder is always - // visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. - ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); - lock.lock(); - try { - return currentStartPollingOrder >= 0; - } - finally { - lock.unlock(); - } - } - @Override public @Nullable ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSourceName) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index b26eed745db5..ada41d22bb88 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -57,11 +57,15 @@ public class CoordinatorDynamicConfig private final int balancerComputeThreads; private final boolean emitBalancingStats; - /** If true, {@link DruidCoordinatorUnusedSegmentsKiller} sends kill tasks for unused segments in all data sources. */ + /** + * If true, {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller} sends kill tasks for + * unused segments in all data sources. + */ private final boolean killUnusedSegmentsInAllDataSources; /** - * List of specific data sources for which kill tasks are sent in {@link DruidCoordinatorUnusedSegmentsKiller}. + * List of specific data sources for which kill tasks are sent in {@link + * org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. */ private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java index aee7397fcb62..4dc6936dec28 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java @@ -41,6 +41,9 @@ * * @see org.apache.druid.indexing.common.task.KillTask */ +// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is +// upgraded to IntelliJ 2018+. +@SuppressWarnings("JavadocResource") public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper { private static final Logger log = new Logger(DruidCoordinatorSegmentKiller.class); From 89ba356c2c2bc4e6ea32ceb279bc554d6626b34e Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 24 May 2019 15:25:07 +0200 Subject: [PATCH 07/15] Comment fix --- .../org/apache/druid/metadata/SQLMetadataSegmentManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 7a5d68f7a6fe..decf99946973 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -174,7 +174,7 @@ long nanosElapsedFromInitiation() * However, the guarantee explained above doesn't make any actual semantic difference, because on both periodic and * on-demand database polls the same invariant is maintained that the results not older than {@link * #periodicPollDelay} are used. The main difference is in performance: since on-demand polls are irregular and happen - * in the context of the thread wanting to access the {@link #dataSources}', that may cause a delay in the logic. + * in the context of the thread wanting to access the {@link #dataSources}, that may cause delays in the logic. * On the other hand, periodic polls are decoupled into {@link #exec} and {@link #dataSources}-accessing methods * should be generally "wait free" for database polls. * From f59f5bce73d86dee67717676e493d0633fb8bae2 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 29 May 2019 08:59:04 +0200 Subject: [PATCH 08/15] Remove unresolvable Javadoc references; address comments --- docs/content/operations/api-reference.md | 4 ++-- .../druid/client/indexing/ClientCompactQuery.java | 7 ++----- .../druid/client/indexing/ClientKillQuery.java | 9 +++------ .../apache/druid/client/indexing/ClientQuery.java | 13 +++++-------- .../druid/metadata/MetadataSegmentManager.java | 7 ++----- .../druid/metadata/SQLMetadataSegmentManager.java | 5 ++--- .../helper/DruidCoordinatorSegmentKiller.java | 5 +---- 7 files changed, 17 insertions(+), 33 deletions(-) diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 6cc54da5cd35..63eb5cfb73a9 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -277,8 +277,8 @@ Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` Marks as unused a segment of a data source. Returns a JSON object of the form `{"segmentStateChanged": "}` with -the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the -result of this API call. +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the +result of this API call. #### Retention Rules diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java index 020b5dea877e..fd9579a2053f 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java @@ -30,12 +30,9 @@ import java.util.Objects; /** - * Client representation of {@link org.apache.druid.indexing.common.task.CompactionTask}. JSON serialization fields of - * this class must correspond to those of {@link org.apache.druid.indexing.common.task.CompactionTask}. + * Client representation of org.apache.druid.indexing.common.task.CompactionTask. JSON serialization fields of + * this class must correspond to those of org.apache.druid.indexing.common.task.CompactionTask. */ -// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is -// upgraded to IntelliJ 2018+. -@SuppressWarnings("JavadocResource") public class ClientCompactQuery implements ClientQuery { private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java index b4a62cea335a..583dee506cda 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java @@ -24,13 +24,10 @@ import org.joda.time.Interval; /** - * Client representation of {@link org.apache.druid.indexing.common.task.KillTask}. JSON searialization - * fields of this class must correspond to those of {@link - * org.apache.druid.indexing.common.task.KillTask}, except for "id" and "context" fields. + * Client representation of org.apache.druid.indexing.common.task.KillTask. JSON searialization + * fields of this class must correspond to those of + * org.apache.druid.indexing.common.task.KillTask, except for "id" and "context" fields. */ -// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is -// upgraded to IntelliJ 2018+. -@SuppressWarnings("JavadocResource") public class ClientKillQuery implements ClientQuery { private final String dataSource; diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java index bac74df9046c..306d6e7c0cb2 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java @@ -24,16 +24,13 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** - * {@link org.apache.druid.indexing.common.task.Task} representations for clients. The magic conversion happens right + * org.apache.druid.indexing.common.task.Task representations for clients. The magic conversion happens right * at the moment of making a REST query: {@link HttpIndexingServiceClient#runTask} serializes ClientTaskQuery - * objects and {@link org.apache.druid.indexing.overlord.http.OverlordResource#taskPost} deserializes {@link - * org.apache.druid.indexing.common.task.Task} objects from the same bytes. Therefore JSON serialization fields of - * ClientTaskQuery objects must match with those of the corresponding {@link - * org.apache.druid.indexing.common.task.Task} objects. + * objects and org.apache.druid.indexing.overlord.http.OverlordResource.taskPost() deserializes + * org.apache.druid.indexing.common.task.Task objects from the same bytes. Therefore JSON serialization fields of + * ClientTaskQuery objects must match with those of the corresponding + * org.apache.druid.indexing.common.task.Task objects. */ -// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is -// upgraded to IntelliJ 2018+. -@SuppressWarnings("JavadocResource") @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "kill", value = ClientKillQuery.class), diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index dd88747f148f..4064e010c05d 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -33,13 +33,10 @@ import java.util.Set; /** - * The difference between this class and {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} is that this - * class resides in Coordinator's memory, while {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} resides + * The difference between this class and org.apache.druid.sql.calcite.schema.MetadataSegmentView is that this + * class resides in Coordinator's memory, while org.apache.druid.sql.calcite.schema.MetadataSegmentView resides * in Broker's memory. */ -// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is -// upgraded to IntelliJ 2018+. -@SuppressWarnings("JavadocResource") public interface MetadataSegmentManager { void startPollingDatabasePeriodically(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index decf99946973..f2ad12f3b1df 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -561,8 +561,7 @@ private int markNonOvershadowedSegmentsAsUsed( if (dataSource != null) { dataSource.addSegment(segment); } - String s = segment.getId().toString(); - segmentIdsToMarkAsUsed.add(s); + segmentIdsToMarkAsUsed.add(segment.getId().toString()); } return markSegmentsAsUsed(segmentIdsToMarkAsUsed); @@ -1021,7 +1020,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE } catch (IOException e) { log.makeAlert(e, "Failed to read segment from db.").emit(); - // If one entry is database is corrupted, doPoll() should continue to work overall. See + // If one entry in database is corrupted doPoll() should continue to work overall. See // .filter(Objects::nonNull) below in this method. return null; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java index 4dc6936dec28..302a91d3f08b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java @@ -39,11 +39,8 @@ * Completely removes information about unused segments whose end time is older than {@link #retainDuration} from now * from the metadata store. This action is called "to kill a segment". * - * @see org.apache.druid.indexing.common.task.KillTask + * See org.apache.druid.indexing.common.task.KillTask */ -// IntelliJ 2017 doesn't support forward Javadoc links; the suppression should be removed when TeamCity build is -// upgraded to IntelliJ 2018+. -@SuppressWarnings("JavadocResource") public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper { private static final Logger log = new Logger(DruidCoordinatorSegmentKiller.class); From 8bac8efcf5067a5047900cd40dba54f44bb65656 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 10 Jun 2019 15:14:32 +0200 Subject: [PATCH 09/15] Add comments to ImmutableDruidDataSource --- .../org/apache/druid/client/ImmutableDruidDataSource.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java index 1f78063cdb79..e8798f7a1731 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java @@ -163,6 +163,8 @@ public String toString() @Override public boolean equals(Object o) { + // Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException. + // See https://github.com/apache/incubator-druid/issues/7858. if (this == o) { return true; } @@ -186,6 +188,8 @@ public boolean equals(Object o) @Override public int hashCode() { + // Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException. + // See https://github.com/apache/incubator-druid/issues/7858. return Objects.hash(name, properties); } } From 8915bc095341c33dd3625c653957031daa4ad51e Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 12 Jul 2019 12:26:23 +0300 Subject: [PATCH 10/15] Merge with master --- .../NewestSegmentFirstPolicyBenchmark.java | 14 +- .../java/util/emitter/EmittingLogger.java | 14 +- .../SegmentWithOvershadowedStatus.java | 7 + .../apache/druid/timeline/TimelineLookup.java | 3 +- .../timeline/VersionedIntervalTimeline.java | 65 +- .../partition/ImmutablePartitionHolder.java | 4 +- .../timeline/partition/PartitionHolder.java | 52 +- .../apache/druid/utils/CollectionUtils.java | 14 + .../org/apache/druid/utils/package-info.java | 23 + .../SegmentWithOvershadowedStatusTest.java | 180 ++ .../VersionedIntervalTimelineTest.java | 32 +- docs/content/operations/api-reference.md | 14 + .../MaterializedViewSupervisor.java | 8 +- extensions-core/datasketches/pom.xml | 2 +- .../bloom/BloomFilterAggregatorFactory.java | 149 +- .../bloom/BloomFilterMergeAggregator.java | 2 +- .../bloom/ObjectBloomFilterAggregator.java | 66 + .../bloom/StringBloomFilterAggregator.java | 6 +- .../bloom/BloomFilterGroupByQueryTest.java | 79 +- .../kafka/supervisor/KafkaSupervisor.java | 6 - .../kinesis/supervisor/KinesisSupervisor.java | 6 - .../supervisor/SupervisorManager.java | 6 + .../supervisor/SupervisorResource.java | 44 +- .../supervisor/SeekableStreamSupervisor.java | 13 + .../actions/SegmentListActionsTest.java | 4 +- .../supervisor/SupervisorResourceTest.java | 74 +- .../auth_test_sys_schema_segments.json | 2 +- .../query/lookup/LookupExtractionFn.java | 4 +- .../lookup/RegisteredLookupExtractionFn.java | 12 +- .../druid/client/DataSourcesSnapshot.java | 172 ++ .../client/ImmutableDruidDataSource.java | 40 - .../supervisor/NoopSupervisorSpec.java | 6 + .../overlord/supervisor/Supervisor.java | 2 + .../metadata/MetadataSegmentManager.java | 51 +- .../metadata/SQLMetadataSegmentManager.java | 215 +-- .../server/coordinator/DruidCoordinator.java | 32 +- ...ruidCoordinatorCleanupPendingSegments.java | 2 +- .../DruidCoordinatorRuntimeParams.java | 97 +- .../helper/DruidCoordinatorBalancer.java | 1 + .../DruidCoordinatorCleanupOvershadowed.java | 11 +- .../helper/DruidCoordinatorLogger.java | 2 +- .../helper/DruidCoordinatorRuleRunner.java | 7 +- .../DruidCoordinatorSegmentCompactor.java | 3 +- .../DruidCoordinatorSegmentInfoLoader.java | 45 +- .../server/http/DataSourcesResource.java | 6 +- .../druid/server/http/MetadataResource.java | 76 +- .../client/CachingClusteredClientTest.java | 5 +- .../SQLMetadataSegmentManagerTest.java | 36 +- .../appenderator/TestUsedSegmentChecker.java | 8 +- .../CuratorDruidCoordinatorTest.java | 34 +- .../DruidCoordinatorBalancerProfiler.java | 2 + .../DruidCoordinatorBalancerTest.java | 1 + .../DruidCoordinatorRuleRunnerTest.java | 53 +- .../coordinator/DruidCoordinatorTest.java | 46 +- .../DruidCoordinatorSegmentCompactorTest.java | 39 +- .../helper/NewestSegmentFirstPolicyTest.java | 13 +- .../QueryLookupOperatorConversion.java | 2 +- .../sql/calcite/schema/SystemSchema.java | 13 +- .../druid/sql/calcite/CalciteQueryTest.java | 4 +- .../SpecificSegmentsQuerySegmentWalker.java | 7 +- web-console/script/mkcomp | 27 +- .../react-table-custom-pagination.tsx | 4 +- .../src/bootstrap/react-table-defaults.tsx | 6 +- .../action-cell/action-cell.spec.tsx | 2 +- .../components/action-cell/action-cell.tsx | 4 +- .../action-icon/action-icon.spec.tsx | 2 +- .../components/action-icon/action-icon.tsx | 4 +- .../array-input/array-input.spec.tsx | 2 +- .../components/array-input/array-input.tsx | 6 +- .../components/auto-form/auto-form.spec.tsx | 2 +- .../src/components/auto-form/auto-form.tsx | 15 +- .../center-message/center-message.spec.tsx | 2 +- .../center-message/center-message.tsx | 4 +- .../clearable-input/clearable-input.spec.tsx | 2 +- .../clearable-input/clearable-input.tsx | 4 +- .../external-link/external-link.spec.tsx | 2 +- .../external-link/external-link.tsx | 4 +- .../components/header-bar/header-bar.spec.tsx | 2 +- .../src/components/header-bar/header-bar.tsx | 6 +- web-console/src/components/index.ts | 1 - .../json-collapse/json-collapse.spec.tsx | 2 +- .../json-collapse/json-collapse.tsx | 4 +- .../components/json-input/json-input.spec.tsx | 2 +- .../src/components/json-input/json-input.tsx | 4 +- .../src/components/loader/loader.spec.tsx | 2 +- web-console/src/components/loader/loader.tsx | 4 +- .../menu-checkbox/menu-checkbox.spec.tsx | 2 +- .../menu-checkbox/menu-checkbox.tsx | 4 +- .../rule-editor/rule-editor.spec.tsx | 2 +- .../components/rule-editor/rule-editor.tsx | 4 +- .../components/show-json/show-json.spec.tsx | 2 +- .../src/components/show-json/show-json.tsx | 15 +- .../src/components/show-log/show-log.spec.tsx | 2 +- .../src/components/show-log/show-log.tsx | 6 +- .../__snapshots__/table-cell.spec.tsx.snap | 8 + .../components/table-cell/table-cell.spec.tsx | 13 +- .../src/components/table-cell/table-cell.tsx | 12 +- .../table-column-selector.spec.tsx | 2 +- .../table-column-selector.tsx | 4 +- .../view-control-bar.spec.tsx | 2 +- .../view-control-bar/view-control-bar.tsx | 4 +- web-console/src/console-application.tsx | 6 +- .../about-dialog/about-dialog.spec.tsx | 2 +- .../src/dialogs/about-dialog/about-dialog.tsx | 4 +- .../async-action-dialog.spec.tsx | 2 +- .../async-action-dialog.tsx | 4 +- .../compaction-dialog.spec.tsx.snap | 4 +- .../compaction-dialog.spec.tsx | 2 +- .../compaction-dialog/compaction-dialog.tsx | 4 +- .../coordinator-dynamic-config.spec.tsx | 2 +- .../coordinator-dynamic-config.tsx | 4 +- .../history-dialog/history-dialog.spec.tsx | 2 +- .../dialogs/history-dialog/history-dialog.tsx | 4 +- .../lookup-edit-dialog.spec.tsx | 2 +- .../lookup-edit-dialog/lookup-edit-dialog.tsx | 4 +- .../overload-dynamic-config.spec.tsx | 2 +- .../overlord-dynamic-config.tsx | 4 +- .../query-plan-dialog.spec.tsx | 2 +- .../query-plan-dialog/query-plan-dialog.tsx | 4 +- .../retention-dialog.array.spec.ts | 2 +- .../retention-dialog.spec.tsx | 2 +- .../retention-dialog/retention-dialog.tsx | 4 +- .../snitch-dialog/snitch-dialog.spec.tsx | 2 +- .../dialogs/snitch-dialog/snitch-dialog.tsx | 4 +- .../dialogs/spec-dialog/spec-dialog.spec.tsx | 2 +- .../src/dialogs/spec-dialog/spec-dialog.tsx | 4 +- ...pervisor-table-action-dialog.spec.tsx.snap | 24 +- .../supervisor-table-action-dialog.spec.tsx | 2 +- .../supervisor-table-action-dialog.tsx | 54 +- .../table-action-dialog.spec.tsx | 2 +- .../table-action-dialog.tsx | 4 +- .../task-table-action-dialog.spec.tsx.snap | 24 +- .../task-table-action-dialog.spec.tsx | 2 +- .../task-table-action-dialog.tsx | 57 +- web-console/src/entry.ts | 9 +- web-console/src/setup-tests.ts | 4 +- .../__snapshots__/joda-to-regexp.spec.ts.snap | 15 + web-console/src/utils/basic-action.tsx | 2 +- web-console/src/utils/druid-time.ts | 51 +- .../src/utils/example-ingestion-spec.ts | 97 -- web-console/src/utils/general.tsx | 24 +- web-console/src/utils/ingestion-spec.tsx | 46 +- web-console/src/utils/joda-to-regexp.spec.ts | 37 + web-console/src/utils/joda-to-regexp.ts | 77 + web-console/src/utils/sampler.ts | 64 +- .../datasource-view/datasource-view.spec.tsx | 2 +- .../views/datasource-view/datasource-view.tsx | 4 +- .../src/views/home-view/home-view.spec.tsx | 2 +- web-console/src/views/home-view/home-view.tsx | 4 +- .../__snapshots__/filter-table.spec.tsx.snap | 1014 +++++++++++ .../filter-table/filter-table.scss | 31 + .../filter-table/filter-table.spec.tsx | 48 + .../filter-table/filter-table.tsx | 88 + .../views/load-data-view/load-data-view.scss | 41 - .../load-data-view/load-data-view.spec.tsx | 2 +- .../views/load-data-view/load-data-view.tsx | 515 ++---- .../parse-data-table.spec.tsx.snap | 1476 +++++++++++++++++ .../parse-data-table/parse-data-table.scss | 40 + .../parse-data-table.spec.tsx | 48 + .../parse-data-table/parse-data-table.tsx | 99 ++ .../parse-time-table.spec.tsx.snap | 1014 +++++++++++ .../parse-time-table/parse-time-table.scss | 21 + .../parse-time-table.spec.tsx | 51 + .../parse-time-table/parse-time-table.tsx | 105 ++ .../__snapshots__/schema-table.spec.tsx.snap | 1014 +++++++++++ .../schema-table/schema-table.scss | 45 + .../schema-table/schema-table.spec.tsx | 51 + .../schema-table/schema-table.tsx | 135 ++ .../transform-table.spec.tsx.snap | 1014 +++++++++++ .../transform-table/transform-table.scss | 31 + .../transform-table/transform-table.spec.tsx | 48 + .../transform-table/transform-table.tsx | 92 + .../views/lookups-view/lookups-view.spec.tsx | 2 +- .../src/views/lookups-view/lookups-view.tsx | 6 +- .../segments-view/segments-view.spec.tsx | 2 +- .../src/views/segments-view/segments-view.tsx | 4 +- .../views/servers-view/servers-view.spec.tsx | 2 +- .../src/views/servers-view/servers-view.tsx | 4 +- .../__snapshots__/sql-control.spec.tsx.snap | 2 +- .../sql-view}/sql-control/sql-control.scss | 2 +- .../sql-control/sql-control.spec.tsx | 2 +- .../sql-view}/sql-control/sql-control.tsx | 23 +- .../src/views/sql-view/sql-view.spec.tsx | 2 +- web-console/src/views/sql-view/sql-view.tsx | 10 +- .../src/views/task-view/tasks-view.spec.tsx | 2 +- .../src/views/task-view/tasks-view.tsx | 4 +- web-console/tsconfig.json | 1 + 187 files changed, 8427 insertions(+), 1486 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/utils/package-info.java create mode 100644 core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java create mode 100644 extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java create mode 100644 server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java create mode 100644 web-console/src/utils/__snapshots__/joda-to-regexp.spec.ts.snap delete mode 100644 web-console/src/utils/example-ingestion-spec.ts create mode 100644 web-console/src/utils/joda-to-regexp.spec.ts create mode 100644 web-console/src/utils/joda-to-regexp.ts create mode 100644 web-console/src/views/load-data-view/filter-table/__snapshots__/filter-table.spec.tsx.snap create mode 100644 web-console/src/views/load-data-view/filter-table/filter-table.scss create mode 100644 web-console/src/views/load-data-view/filter-table/filter-table.spec.tsx create mode 100644 web-console/src/views/load-data-view/filter-table/filter-table.tsx create mode 100644 web-console/src/views/load-data-view/parse-data-table/__snapshots__/parse-data-table.spec.tsx.snap create mode 100644 web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss create mode 100644 web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx create mode 100644 web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx create mode 100644 web-console/src/views/load-data-view/parse-time-table/__snapshots__/parse-time-table.spec.tsx.snap create mode 100644 web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss create mode 100644 web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx create mode 100644 web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx create mode 100644 web-console/src/views/load-data-view/schema-table/__snapshots__/schema-table.spec.tsx.snap create mode 100644 web-console/src/views/load-data-view/schema-table/schema-table.scss create mode 100644 web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx create mode 100644 web-console/src/views/load-data-view/schema-table/schema-table.tsx create mode 100644 web-console/src/views/load-data-view/transform-table/__snapshots__/transform-table.spec.tsx.snap create mode 100644 web-console/src/views/load-data-view/transform-table/transform-table.scss create mode 100644 web-console/src/views/load-data-view/transform-table/transform-table.spec.tsx create mode 100644 web-console/src/views/load-data-view/transform-table/transform-table.tsx rename web-console/src/{components => views/sql-view}/sql-control/__snapshots__/sql-control.spec.tsx.snap (98%) rename web-console/src/{components => views/sql-view}/sql-control/sql-control.scss (98%) rename web-console/src/{components => views/sql-view}/sql-control/sql-control.spec.tsx (97%) rename web-console/src/{components => views/sql-view}/sql-control/sql-control.tsx (94%) diff --git a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index d06125ac35df..dfef1283a2a3 100644 --- a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -20,6 +20,8 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator; import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy; @@ -42,6 +44,7 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.infra.Blackhole; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -102,14 +105,10 @@ public void setup() ); } - dataSources = new HashMap<>(); + List segments = new ArrayList<>(); for (int i = 0; i < numDataSources; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - final int startYear = ThreadLocalRandom.current().nextInt(2000, 2040); DateTime date = DateTimes.of(startYear, 1, 1, 0, 0); @@ -127,12 +126,11 @@ public void setup() 0, segmentSizeBytes ); - timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment)); + segments.add(segment); } } - - dataSources.put(dataSource, timeline); } + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()).getUsedSegmentsTimelinesPerDataSource(); } @Benchmark diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java index 8e685c696117..ef5d47a1f4b5 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import javax.annotation.Nullable; import java.io.PrintWriter; import java.io.StringWriter; @@ -61,15 +62,22 @@ public AlertBuilder makeAlert(String message, Object... objects) return makeAlert(null, message, objects); } - public AlertBuilder makeAlert(Throwable t, String message, Object... objects) + public AlertBuilder makeAlert(@Nullable Throwable t, String message, Object... objects) { if (emitter == null) { final String errorMessage = StringUtils.format( - "Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()", this.getClass() + "Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()\n" + + "Message: %s", + this.getClass(), + StringUtils.nonStrictFormat(message, objects) ); error(errorMessage); - throw new ISE(errorMessage); + ISE e = new ISE(errorMessage); + if (t != null) { + e.addSuppressed(t); + } + throw e; } final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter) diff --git a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java index e86daea7d86e..3f2972fd07e9 100644 --- a/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java +++ b/core/src/main/java/org/apache/druid/timeline/SegmentWithOvershadowedStatus.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonUnwrapped; /** * DataSegment object plus the overshadowed status for the segment. An immutable object. @@ -31,6 +32,12 @@ public class SegmentWithOvershadowedStatus implements Comparable { private final boolean overshadowed; + /** + * dataSegment is serialized "unwrapped", i.e. it's properties are included as properties of + * enclosing class. If in future, if {@Code SegmentWithOvershadowedStatus} were to extend {@link DataSegment}, + * there will be no change in the serialized format. + */ + @JsonUnwrapped private final DataSegment dataSegment; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java index af7544083756..6bdab5c723de 100644 --- a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java @@ -22,6 +22,7 @@ import org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; @@ -50,5 +51,5 @@ public interface TimelineLookup */ List> lookupWithIncompletePartitions(Interval interval); - PartitionHolder findEntry(Interval interval, VersionType version); + @Nullable PartitionHolder findEntry(Interval interval, VersionType version); } diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java index 36c177dfb0f0..8deed36fd7e6 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -23,16 +23,18 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; -import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.timeline.partition.ImmutablePartitionHolder; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -44,7 +46,9 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.StreamSupport; /** * VersionedIntervalTimeline is a data structure that manages objects on a specific timeline. @@ -75,12 +79,11 @@ public class VersionedIntervalTimeline implements Timel Comparators.intervalsByStartThenEnd() ); private final Map> allTimelineEntries = new HashMap<>(); + private final AtomicInteger numObjects = new AtomicInteger(); private final Comparator versionComparator; - public VersionedIntervalTimeline( - Comparator versionComparator - ) + public VersionedIntervalTimeline(Comparator versionComparator) { this.versionComparator = versionComparator; } @@ -92,7 +95,8 @@ public static VersionedIntervalTimeline forSegments(Iterabl public static VersionedIntervalTimeline forSegments(Iterator segments) { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + final VersionedIntervalTimeline timeline = + new VersionedIntervalTimeline<>(Comparator.naturalOrder()); addSegments(timeline, segments); return timeline; } @@ -115,6 +119,28 @@ public Map> getAllTimelineEntries( return allTimelineEntries; } + /** + * Returns a lazy collection with all objects (including overshadowed, see {@link #findOvershadowed}) in this + * VersionedIntervalTimeline to be used for iteration or {@link Collection#stream()} transformation. The order of + * objects in this collection is unspecified. + * + * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an + * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only + * once rather than several times. + */ + public Collection iterateAllObjects() + { + return CollectionUtils.createLazyCollectionFromStream( + () -> allTimelineEntries + .values() + .stream() + .flatMap((TreeMap entryMap) -> entryMap.values().stream()) + .flatMap((TimelineEntry entry) -> StreamSupport.stream(entry.getPartitionHolder().spliterator(), false)) + .map(PartitionChunk::getObject), + numObjects.get() + ); + } + public void add(final Interval interval, VersionType version, PartitionChunk object) { addAll(Iterators.singletonIterator(object), o -> interval, o -> version); @@ -143,15 +169,19 @@ private void addAll( TreeMap versionEntry = new TreeMap<>(versionComparator); versionEntry.put(version, entry); allTimelineEntries.put(interval, versionEntry); + numObjects.incrementAndGet(); } else { entry = exists.get(version); if (entry == null) { entry = new TimelineEntry(interval, version, new PartitionHolder<>(object)); exists.put(version, entry); + numObjects.incrementAndGet(); } else { PartitionHolder partitionHolder = entry.getPartitionHolder(); - partitionHolder.add(object); + if (partitionHolder.add(object)) { + numObjects.incrementAndGet(); + } } } @@ -174,6 +204,7 @@ private void addAll( } } + @Nullable public PartitionChunk remove(Interval interval, VersionType version, PartitionChunk chunk) { try { @@ -189,7 +220,11 @@ public PartitionChunk remove(Interval interval, VersionType version, return null; } - PartitionChunk retVal = entry.getPartitionHolder().remove(chunk); + PartitionChunk removedChunk = entry.getPartitionHolder().remove(chunk); + if (removedChunk == null) { + return null; + } + numObjects.decrementAndGet(); if (entry.getPartitionHolder().isEmpty()) { versionEntries.remove(version); if (versionEntries.isEmpty()) { @@ -201,7 +236,7 @@ public PartitionChunk remove(Interval interval, VersionType version, remove(completePartitionsTimeline, interval, entry, false); - return retVal; + return removedChunk; } finally { lock.writeLock().unlock(); @@ -209,7 +244,7 @@ public PartitionChunk remove(Interval interval, VersionType version, } @Override - public PartitionHolder findEntry(Interval interval, VersionType version) + public @Nullable PartitionHolder findEntry(Interval interval, VersionType version) { try { lock.readLock().lock(); @@ -217,9 +252,7 @@ public PartitionHolder findEntry(Interval interval, VersionType vers if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); if (foundEntry != null) { - return new ImmutablePartitionHolder( - foundEntry.getPartitionHolder() - ); + return new ImmutablePartitionHolder<>(foundEntry.getPartitionHolder()); } } } @@ -307,6 +340,10 @@ private TimelineObjectHolder timelineEntryToObjectHolde ); } + /** + * This method should be deduplicated with DataSourcesSnapshot.determineOvershadowedSegments(): see + * https://github.com/apache/incubator-druid/issues/8070. + */ public Set> findOvershadowed() { try { @@ -315,8 +352,8 @@ public Set> findOvershadowed() Map> overShadowed = new HashMap<>(); for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { - Map versionCopy = new HashMap<>(); - versionCopy.putAll(versionEntry.getValue()); + @SuppressWarnings("unchecked") + Map versionCopy = (TreeMap) versionEntry.getValue().clone(); overShadowed.put(versionEntry.getKey(), versionCopy); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java index ec8f7d891a7e..5003f651c121 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java @@ -23,7 +23,7 @@ */ public class ImmutablePartitionHolder extends PartitionHolder { - public ImmutablePartitionHolder(PartitionHolder partitionHolder) + public ImmutablePartitionHolder(PartitionHolder partitionHolder) { super(partitionHolder); } @@ -35,7 +35,7 @@ public PartitionChunk remove(PartitionChunk tPartitionChunk) } @Override - public void add(PartitionChunk tPartitionChunk) + public boolean add(PartitionChunk tPartitionChunk) { throw new UnsupportedOperationException(); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java index 396b4f3bdd63..dcf29aedc488 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java @@ -22,72 +22,62 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import javax.annotation.Nullable; import java.util.Iterator; import java.util.List; -import java.util.SortedSet; import java.util.Spliterator; -import java.util.TreeSet; +import java.util.TreeMap; /** * An object that clumps together multiple other objects which each represent a shard of some space. */ public class PartitionHolder implements Iterable> { - private final TreeSet> holderSet; + private final TreeMap, PartitionChunk> holderMap; public PartitionHolder(PartitionChunk initialChunk) { - this.holderSet = new TreeSet<>(); + this.holderMap = new TreeMap<>(); add(initialChunk); } public PartitionHolder(List> initialChunks) { - this.holderSet = new TreeSet<>(); + this.holderMap = new TreeMap<>(); for (PartitionChunk chunk : initialChunks) { add(chunk); } } - public PartitionHolder(PartitionHolder partitionHolder) + public PartitionHolder(PartitionHolder partitionHolder) { - this.holderSet = new TreeSet<>(); - this.holderSet.addAll(partitionHolder.holderSet); + this.holderMap = new TreeMap<>(); + this.holderMap.putAll(partitionHolder.holderMap); } - public void add(PartitionChunk chunk) + public boolean add(PartitionChunk chunk) { - holderSet.add(chunk); + return holderMap.putIfAbsent(chunk, chunk) == null; } + @Nullable public PartitionChunk remove(PartitionChunk chunk) { - if (!holderSet.isEmpty()) { - // Somewhat funky implementation in order to return the removed object as it exists in the set - SortedSet> tailSet = holderSet.tailSet(chunk, true); - if (!tailSet.isEmpty()) { - PartitionChunk element = tailSet.first(); - if (chunk.equals(element)) { - holderSet.remove(element); - return element; - } - } - } - return null; + return holderMap.remove(chunk); } public boolean isEmpty() { - return holderSet.isEmpty(); + return holderMap.isEmpty(); } public boolean isComplete() { - if (holderSet.isEmpty()) { + if (holderMap.isEmpty()) { return false; } - Iterator> iter = holderSet.iterator(); + Iterator> iter = holderMap.keySet().iterator(); PartitionChunk curr = iter.next(); @@ -117,7 +107,7 @@ public boolean isComplete() public PartitionChunk getChunk(final int partitionNum) { final Iterator> retVal = Iterators.filter( - holderSet.iterator(), + holderMap.keySet().iterator(), input -> input.getChunkNumber() == partitionNum ); @@ -127,13 +117,13 @@ public PartitionChunk getChunk(final int partitionNum) @Override public Iterator> iterator() { - return holderSet.iterator(); + return holderMap.keySet().iterator(); } @Override public Spliterator> spliterator() { - return holderSet.spliterator(); + return holderMap.keySet().spliterator(); } public Iterable payloads() @@ -153,7 +143,7 @@ public boolean equals(Object o) PartitionHolder that = (PartitionHolder) o; - if (!holderSet.equals(that.holderSet)) { + if (!holderMap.equals(that.holderMap)) { return false; } @@ -163,14 +153,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return holderSet.hashCode(); + return holderMap.hashCode(); } @Override public String toString() { return "PartitionHolder{" + - "holderSet=" + holderSet + + "holderMap=" + holderMap + '}'; } } diff --git a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java index c053e041fc46..deb4eafd0466 100644 --- a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java +++ b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java @@ -20,13 +20,16 @@ package org.apache.druid.utils; import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; import java.util.AbstractCollection; import java.util.Collection; import java.util.Comparator; import java.util.Iterator; +import java.util.Map; import java.util.Spliterator; import java.util.TreeSet; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Stream; @@ -79,6 +82,17 @@ public static TreeSet newTreeSet(Comparator comparator, Iterab return set; } + /** + * Returns a transformed map from the given input map where the value is modified based on the given valueMapper + * function. + */ + public static Map mapValues(Map map, Function valueMapper) + { + final Map result = Maps.newHashMapWithExpectedSize(map.size()); + map.forEach((k, v) -> result.put(k, valueMapper.apply(v))); + return result; + } + private CollectionUtils() { } diff --git a/core/src/main/java/org/apache/druid/utils/package-info.java b/core/src/main/java/org/apache/druid/utils/package-info.java new file mode 100644 index 000000000000..f41e226c8160 --- /dev/null +++ b/core/src/main/java/org/apache/druid/utils/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +@EverythingIsNonnullByDefault +package org.apache.druid.utils; + +import org.apache.druid.annotations.EverythingIsNonnullByDefault; diff --git a/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java new file mode 100644 index 000000000000..050f9e04934b --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/SegmentWithOvershadowedStatusTest.java @@ -0,0 +1,180 @@ +/* + * 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.timeline; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.TestObjectMapper; +import org.apache.druid.jackson.CommaListJoinDeserializer; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class SegmentWithOvershadowedStatusTest +{ + private static final ObjectMapper mapper = new TestObjectMapper(); + private static final int TEST_VERSION = 0x9; + + @Before + public void setUp() + { + InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT); + mapper.setInjectableValues(injectableValues); + } + + @Test + public void testUnwrappedSegmentWithOvershadowedStatusDeserialization() throws Exception + { + final Interval interval = Intervals.of("2011-10-01/2011-10-02"); + final ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + + final DataSegment dataSegment = new DataSegment( + "something", + interval, + "1", + loadSpec, + Arrays.asList("dim1", "dim2"), + Arrays.asList("met1", "met2"), + NoneShardSpec.instance(), + TEST_VERSION, + 1 + ); + + final SegmentWithOvershadowedStatus segment = new SegmentWithOvershadowedStatus(dataSegment, false); + + final Map objectMap = mapper.readValue( + mapper.writeValueAsString(segment), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + Assert.assertEquals(11, objectMap.size()); + Assert.assertEquals("something", objectMap.get("dataSource")); + Assert.assertEquals(interval.toString(), objectMap.get("interval")); + Assert.assertEquals("1", objectMap.get("version")); + Assert.assertEquals(loadSpec, objectMap.get("loadSpec")); + Assert.assertEquals("dim1,dim2", objectMap.get("dimensions")); + Assert.assertEquals("met1,met2", objectMap.get("metrics")); + Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec")); + Assert.assertEquals(TEST_VERSION, objectMap.get("binaryVersion")); + Assert.assertEquals(1, objectMap.get("size")); + Assert.assertEquals(false, objectMap.get("overshadowed")); + + final String json = mapper.writeValueAsString(segment); + + final TestSegmentWithOvershadowedStatus deserializedSegment = mapper.readValue( + json, + TestSegmentWithOvershadowedStatus.class + ); + + Assert.assertEquals(segment.getDataSegment().getDataSource(), deserializedSegment.getDataSource()); + Assert.assertEquals(segment.getDataSegment().getInterval(), deserializedSegment.getInterval()); + Assert.assertEquals(segment.getDataSegment().getVersion(), deserializedSegment.getVersion()); + Assert.assertEquals(segment.getDataSegment().getLoadSpec(), deserializedSegment.getLoadSpec()); + Assert.assertEquals(segment.getDataSegment().getDimensions(), deserializedSegment.getDimensions()); + Assert.assertEquals(segment.getDataSegment().getMetrics(), deserializedSegment.getMetrics()); + Assert.assertEquals(segment.getDataSegment().getShardSpec(), deserializedSegment.getShardSpec()); + Assert.assertEquals(segment.getDataSegment().getSize(), deserializedSegment.getSize()); + Assert.assertEquals(segment.getDataSegment().getId(), deserializedSegment.getId()); + + } +} + +/** + * Subclass of DataSegment with overshadowed status + */ +class TestSegmentWithOvershadowedStatus extends DataSegment +{ + private final boolean overshadowed; + + @JsonCreator + public TestSegmentWithOvershadowedStatus( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + @JsonProperty("loadSpec") @Nullable Map loadSpec, + @JsonProperty("dimensions") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List dimensions, + @JsonProperty("metrics") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List metrics, + @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, + @JsonProperty("binaryVersion") Integer binaryVersion, + @JsonProperty("size") long size, + @JsonProperty("overshadowed") boolean overshadowed + ) + { + super( + dataSource, + interval, + version, + loadSpec, + dimensions, + metrics, + shardSpec, + binaryVersion, + size + ); + this.overshadowed = overshadowed; + } + + @JsonProperty + public boolean isOvershadowed() + { + return overshadowed; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof TestSegmentWithOvershadowedStatus)) { + return false; + } + if (!super.equals(o)) { + return false; + } + final TestSegmentWithOvershadowedStatus that = (TestSegmentWithOvershadowedStatus) o; + if (overshadowed != (that.overshadowed)) { + return false; + } + return true; + } + +} diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java index 681f6a605c5c..3e66bf53de79 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -240,22 +240,22 @@ public void testRemove() public void testFindEntry() { Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") ); @@ -279,7 +279,7 @@ public void testFindEntryWithOverlap() add("2011-01-02/2011-01-05", "2", 1); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1") ); } @@ -1564,11 +1564,11 @@ public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk(1)); - add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk(1)); + add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk<>(1)); + add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk<>(1)); - add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk(1)); - add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk(1)); + add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<>(1)); + add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<>(1)); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0")); @@ -1629,11 +1629,11 @@ public void testIsOvershadowedWithOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk(1)); - add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk(1)); + add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk<>(1)); + add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk<>(1)); - add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk(1)); - add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk(1)); + add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk<>(1)); + add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<>(1)); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0")); @@ -1730,13 +1730,13 @@ private Pair>> createExpected( { return Pair.of( Intervals.of(intervalString), - Pair.of(version, new PartitionHolder(values)) + Pair.of(version, new PartitionHolder<>(values)) ); } private SingleElementPartitionChunk makeSingle(Integer value) { - return new SingleElementPartitionChunk(value); + return new SingleElementPartitionChunk<>(value); } private void add(String interval, String version, Integer value) @@ -1808,7 +1808,7 @@ public Pair>> apply( private VersionedIntervalTimeline makeStringIntegerTimeline() { - return new VersionedIntervalTimeline(Ordering.natural()); + return new VersionedIntervalTimeline<>(Ordering.natural()); } } diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 63eb5cfb73a9..80af3f819ca0 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -520,8 +520,22 @@ Returns a list of objects of the currently active supervisors. |Field|Type|Description| |---|---|---| |`id`|String|supervisor unique identifier| +|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`| +|`detailedState`|String|supervisor specific state. (See documentation of specific supervisor for details)| +|`healthy`|Boolean|true or false indicator of overall supervisor health| |`spec`|SupervisorSpec|json specification of supervisor (See Supervisor Configuration for details)| +* `/druid/indexer/v1/supervisor?state=true` + +Returns a list of objects of the currently active supervisors and their current state. + +|Field|Type|Description| +|---|---|---| +|`id`|String|supervisor unique identifier| +|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`| +|`detailedState`|String|supervisor specific state. (See documentation of specific supervisor for details)| +|`healthy`|Boolean|true or false indicator of overall supervisor health| + * `/druid/indexer/v1/supervisor/` Returns the current spec for the supervisor with the provided ID. diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index 5635c7ecb40b..1ed80ca83c9a 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -240,6 +240,12 @@ public SupervisorReport getStatus() ); } + @Override + public SupervisorStateManager.State getState() + { + return stateManager.getSupervisorState(); + } + @Override public Boolean isHealthy() { @@ -384,7 +390,7 @@ Pair, Map>> checkSegment // drop derivative segments which interval equals the interval in toDeleteBaseSegments for (Interval interval : toDropInterval.keySet()) { for (DataSegment segment : derivativeSegments.get(interval)) { - segmentManager.markSegmentAsUnused(segment.getId()); + segmentManager.markSegmentAsUnused(segment.getId().toString()); } } // data of the latest interval will be built firstly. diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 77ebc276a893..db91883f1cf3 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -38,7 +38,7 @@ com.yahoo.datasketches sketches-core - 0.13.3 + 0.13.4 com.google.code.findbugs diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java index 2c633f2dde51..53839e82f7ad 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java @@ -82,95 +82,13 @@ public BloomFilterAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory columnFactory) { - ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension()); - - if (capabilities == null) { - BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension()); - if (selector instanceof NilColumnValueSelector) { - // BloomKFilter must be the same size so we cannot use a constant for the empty agg - return new NoopBloomFilterAggregator(maxNumEntries, true); - } - throw new IAE( - "Cannot create bloom filter buffer aggregator for column selector type [%s]", - selector.getClass().getName() - ); - } - ValueType type = capabilities.getType(); - switch (type) { - case STRING: - return new StringBloomFilterAggregator( - columnFactory.makeDimensionSelector(field), - maxNumEntries, - true - ); - case LONG: - return new LongBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - true - ); - case FLOAT: - return new FloatBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - true - ); - case DOUBLE: - return new DoubleBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - true - ); - default: - throw new IAE("Cannot create bloom filter aggregator for invalid column type [%s]", type); - } + return factorizeInternal(columnFactory, true); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnFactory) { - ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension()); - - if (capabilities == null) { - BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension()); - if (selector instanceof NilColumnValueSelector) { - return new NoopBloomFilterAggregator(maxNumEntries, false); - } - throw new IAE( - "Cannot create bloom filter buffer aggregator for column selector type [%s]", - selector.getClass().getName() - ); - } - - ValueType type = capabilities.getType(); - switch (type) { - case STRING: - return new StringBloomFilterAggregator( - columnFactory.makeDimensionSelector(field), - maxNumEntries, - false - ); - case LONG: - return new LongBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - false - ); - case FLOAT: - return new FloatBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - false - ); - case DOUBLE: - return new DoubleBloomFilterAggregator( - columnFactory.makeColumnValueSelector(field.getDimension()), - maxNumEntries, - false - ); - default: - throw new IAE("Cannot create bloom filter buffer aggregator for invalid column type [%s]", type); - } + return factorizeInternal(columnFactory, false); } @Override @@ -310,4 +228,67 @@ public String toString() ", maxNumEntries=" + maxNumEntries + '}'; } + + private BaseBloomFilterAggregator factorizeInternal(ColumnSelectorFactory columnFactory, boolean onHeap) + { + if (field == null || field.getDimension() == null) { + return new NoopBloomFilterAggregator(maxNumEntries, onHeap); + } + + ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension()); + + if (capabilities != null) { + ValueType type = capabilities.getType(); + switch (type) { + case STRING: + return new StringBloomFilterAggregator( + columnFactory.makeDimensionSelector(field), + maxNumEntries, + onHeap + ); + case LONG: + return new LongBloomFilterAggregator( + columnFactory.makeColumnValueSelector(field.getDimension()), + maxNumEntries, + onHeap + ); + case FLOAT: + return new FloatBloomFilterAggregator( + columnFactory.makeColumnValueSelector(field.getDimension()), + maxNumEntries, + onHeap + ); + case DOUBLE: + return new DoubleBloomFilterAggregator( + columnFactory.makeColumnValueSelector(field.getDimension()), + maxNumEntries, + onHeap + ); + case COMPLEX: + // in an ideal world, we would check complex type, but until then assume it's a bloom filter + return new BloomFilterMergeAggregator( + columnFactory.makeColumnValueSelector(field.getDimension()), + maxNumEntries, + onHeap + ); + default: + throw new IAE( + "Cannot create bloom filter %s for invalid column type [%s]", + onHeap ? "aggregator" : "buffer aggregator", + type + ); + } + } else { + BaseNullableColumnValueSelector selector = columnFactory.makeColumnValueSelector(field.getDimension()); + if (selector instanceof NilColumnValueSelector) { + return new NoopBloomFilterAggregator(maxNumEntries, onHeap); + } + // no column capabilities, use fallback 'object' aggregator + return new ObjectBloomFilterAggregator( + columnFactory.makeColumnValueSelector(field.getDimension()), + maxNumEntries, + onHeap + ); + } + } } diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java index 011f2f6c563d..6855d83a35ef 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterMergeAggregator.java @@ -27,7 +27,7 @@ public final class BloomFilterMergeAggregator extends BaseBloomFilterAggregator> { - public BloomFilterMergeAggregator(ColumnValueSelector selector, int maxNumEntries, boolean onHeap) + BloomFilterMergeAggregator(ColumnValueSelector selector, int maxNumEntries, boolean onHeap) { super(selector, maxNumEntries, onHeap); } diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java new file mode 100644 index 000000000000..bc97fab800ac --- /dev/null +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/ObjectBloomFilterAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.query.aggregation.bloom; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.filter.BloomKFilter; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; + +import java.nio.ByteBuffer; + +/** + * Handles "unknown" columns by examining what comes out of the selector + */ +class ObjectBloomFilterAggregator extends BaseBloomFilterAggregator +{ + ObjectBloomFilterAggregator( + ColumnValueSelector selector, + int maxNumEntries, + boolean onHeap + ) + { + super(selector, maxNumEntries, onHeap); + } + + @Override + void bufferAdd(ByteBuffer buf) + { + final Object object = selector.getObject(); + if (object instanceof ByteBuffer) { + final ByteBuffer other = (ByteBuffer) object; + BloomKFilter.mergeBloomFilterByteBuffers(buf, buf.position(), other, other.position()); + } else { + if (NullHandling.replaceWithDefault() || !selector.isNull()) { + if (object instanceof Long) { + BloomKFilter.addLong(buf, selector.getLong()); + } else if (object instanceof Double) { + BloomKFilter.addDouble(buf, selector.getDouble()); + } else if (object instanceof Float) { + BloomKFilter.addFloat(buf, selector.getFloat()); + } else { + StringBloomFilterAggregator.stringBufferAdd(buf, (DimensionSelector) selector); + } + } else { + BloomKFilter.addBytes(buf, null, 0, 0); + } + } + } +} diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java index f3f6daec75ef..db65ca5b2a69 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/StringBloomFilterAggregator.java @@ -26,7 +26,6 @@ public final class StringBloomFilterAggregator extends BaseBloomFilterAggregator { - StringBloomFilterAggregator(DimensionSelector selector, int maxNumEntries, boolean onHeap) { super(selector, maxNumEntries, onHeap); @@ -34,6 +33,11 @@ public final class StringBloomFilterAggregator extends BaseBloomFilterAggregator @Override public void bufferAdd(ByteBuffer buf) + { + stringBufferAdd(buf, selector); + } + + static void stringBufferAdd(ByteBuffer buf, DimensionSelector selector) { if (selector.getRow().size() > 1) { selector.getRow().forEach(v -> { diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java index ce3b9322d743..5661e7b9e47a 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterGroupByQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.filter.BloomKFilter; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.segment.TestHelper; import org.junit.After; import org.junit.Assert; @@ -61,6 +62,7 @@ public class BloomFilterGroupByQueryTest } private AggregationTestHelper helper; + private boolean isV2; @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -72,6 +74,7 @@ public BloomFilterGroupByQueryTest(final GroupByQueryConfig config) config, tempFolder ); + isV2 = config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V2); } @Parameterized.Parameters(name = "{0}") @@ -93,7 +96,6 @@ public void teardown() throws IOException @Test public void testQuery() throws Exception { - String query = "{" + "\"queryType\": \"groupBy\"," + "\"dataSource\": \"test_datasource\"," @@ -115,6 +117,81 @@ public void testQuery() throws Exception Assert.assertFalse(filter.testString("entertainment")); } + @Test + public void testNestedQuery() throws Exception + { + if (!isV2) { + return; + } + + String query = "{" + + "\"queryType\": \"groupBy\"," + + "\"dataSource\": {" + + "\"type\": \"query\"," + + "\"query\": {" + + "\"queryType\":\"groupBy\"," + + "\"dataSource\": \"test_datasource\"," + + "\"intervals\": [ \"1970/2050\" ]," + + "\"granularity\":\"ALL\"," + + "\"dimensions\":[]," + + "\"aggregations\": [{ \"type\":\"longSum\", \"name\":\"innerSum\", \"fieldName\":\"count\"}]" + + "}" + + "}," + + "\"granularity\": \"ALL\"," + + "\"dimensions\": []," + + "\"aggregations\": [" + + " { \"type\": \"bloom\", \"name\": \"bloom\", \"field\": \"innerSum\" }" + + "]," + + "\"intervals\": [ \"1970/2050\" ]" + + "}"; + + MapBasedRow row = ingestAndQuery(query); + + + BloomKFilter filter = BloomKFilter.deserialize((ByteBuffer) row.getRaw("bloom")); + Assert.assertTrue(filter.testLong(13L)); + Assert.assertFalse(filter.testLong(5L)); + } + + + @Test + public void testNestedQueryComplex() throws Exception + { + if (!isV2) { + return; + } + + String query = "{" + + "\"queryType\": \"groupBy\"," + + "\"dataSource\": {" + + "\"type\": \"query\"," + + "\"query\": {" + + "\"queryType\":\"groupBy\"," + + "\"dataSource\": \"test_datasource\"," + + "\"intervals\": [ \"1970/2050\" ]," + + "\"granularity\":\"ALL\"," + + "\"dimensions\":[]," + + "\"filter\":{ \"type\":\"selector\", \"dimension\":\"market\", \"value\":\"upfront\"}," + + "\"aggregations\": [{ \"type\":\"bloom\", \"name\":\"innerBloom\", \"field\":\"quality\"}]" + + "}" + + "}," + + "\"granularity\": \"ALL\"," + + "\"dimensions\": []," + + "\"aggregations\": [" + + " { \"type\": \"bloom\", \"name\": \"innerBloom\", \"field\": \"innerBloom\" }" + + "]," + + "\"intervals\": [ \"1970/2050\" ]" + + "}"; + + MapBasedRow row = ingestAndQuery(query); + + + BloomKFilter filter = BloomKFilter.deserialize((ByteBuffer) row.getRaw("innerBloom")); + Assert.assertTrue(filter.testString("mezzanine")); + Assert.assertTrue(filter.testString("premium")); + Assert.assertFalse(filter.testString("entertainment")); + } + @Test public void testQueryFakeDimension() throws Exception { diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index 5d419a4497f3..cdf133677bff 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -385,10 +385,4 @@ public KafkaSupervisorIOConfig getIoConfig() { return spec.getIoConfig(); } - - @Override - public Boolean isHealthy() - { - return stateManager.isHealthy(); - } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 39619a268630..5a0c8614b805 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -316,10 +316,4 @@ protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() { return true; } - - @Override - public Boolean isHealthy() - { - return stateManager.isHealthy(); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java index 56112d15732d..5727c4ed70bb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorManager.java @@ -65,6 +65,12 @@ public Optional getSupervisorSpec(String id) return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.rhs); } + public Optional getSupervisorState(String id) + { + Pair supervisor = supervisors.get(id); + return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getState()); + } + public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec) { Preconditions.checkState(started, "SupervisorManager not started"); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java index 9d97a80aca7b..e7e9daf74184 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResource.java @@ -52,6 +52,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -118,6 +119,7 @@ public Response specPost(final SupervisorSpec spec, @Context final HttpServletRe @Produces(MediaType.APPLICATION_JSON) public Response specGetAll( @QueryParam("full") String full, + @QueryParam("state") Boolean state, @Context final HttpServletRequest req ) { @@ -128,20 +130,36 @@ public Response specGetAll( manager, manager.getSupervisorIds() ); - - if (full == null) { - return Response.ok(authorizedSupervisorIds).build(); - } else { - List> all = - authorizedSupervisorIds.stream() - .map(x -> ImmutableMap.builder() - .put("id", x) - .put("spec", manager.getSupervisorSpec(x).get()) - .build() - ) - .collect(Collectors.toList()); - return Response.ok(all).build(); + final boolean includeFull = full != null; + final boolean includeState = state != null && state; + + if (includeFull || includeState) { + List> allStates = authorizedSupervisorIds + .stream() + .map(x -> { + Optional theState = + manager.getSupervisorState(x); + ImmutableMap.Builder theBuilder = ImmutableMap.builder(); + theBuilder.put("id", x); + if (theState.isPresent()) { + theBuilder.put("state", theState.get().getBasicState()); + theBuilder.put("detailedState", theState.get()); + theBuilder.put("healthy", theState.get().isHealthy()); + } + if (includeFull) { + Optional theSpec = manager.getSupervisorSpec(x); + if (theSpec.isPresent()) { + theBuilder.put("spec", theSpec.get()); + } + } + return theBuilder.build(); + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + return Response.ok(allStates).build(); } + + return Response.ok(authorizedSupervisorIds).build(); } ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java index 5e7c693aa0c6..64112d6c5fde 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java @@ -807,6 +807,19 @@ public SupervisorReport getStatus() return generateReport(true); } + + @Override + public SupervisorStateManager.State getState() + { + return stateManager.getSupervisorState(); + } + + @Override + public Boolean isHealthy() + { + return stateManager.isHealthy(); + } + private SupervisorReport> generateReport( boolean includeOffsets ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java index 4e0ab137ad33..c5f677121edf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java @@ -73,7 +73,9 @@ public void setup() throws IOException expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); - expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().markSegmentAsUnused(s.getId())); + expectedUnusedSegments.forEach( + s -> actionTestKit.getMetadataSegmentManager().markSegmentAsUnused(s.getId().toString()) + ); } private DataSegment createSegment(Interval interval, String version) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java index 96afde5ed1d5..f6cb0080a966 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/SupervisorResourceTest.java @@ -166,7 +166,7 @@ public List getDataSources() EasyMock.expectLastCall().anyTimes(); replayAll(); - Response response = supervisorResource.specGetAll(null, request); + Response response = supervisorResource.specGetAll(null, null, request); verifyAll(); Assert.assertEquals(200, response.getStatus()); @@ -176,7 +176,7 @@ public List getDataSources() EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.absent()); replayAll(); - response = supervisorResource.specGetAll(null, request); + response = supervisorResource.specGetAll(null, null, request); verifyAll(); Assert.assertEquals(503, response.getStatus()); @@ -205,11 +205,15 @@ public List getDataSources() return Collections.singletonList("datasource2"); } }; + SupervisorStateManager.State state1 = SupervisorStateManager.BasicState.RUNNING; + SupervisorStateManager.State state2 = SupervisorStateManager.BasicState.SUSPENDED; EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds).atLeastOnce(); EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).times(2); EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).times(2); + EasyMock.expect(supervisorManager.getSupervisorState("id1")).andReturn(Optional.of(state1)).times(1); + EasyMock.expect(supervisorManager.getSupervisorState("id2")).andReturn(Optional.of(state2)).times(1); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce(); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( @@ -219,7 +223,7 @@ public List getDataSources() EasyMock.expectLastCall().anyTimes(); replayAll(); - Response response = supervisorResource.specGetAll("", request); + Response response = supervisorResource.specGetAll("", null, request); verifyAll(); Assert.assertEquals(200, response.getStatus()); @@ -233,6 +237,70 @@ public List getDataSources() ); } + @Test + public void testSpecGetState() + { + Set supervisorIds = ImmutableSet.of("id1", "id2"); + SupervisorSpec spec1 = new TestSupervisorSpec("id1", null, null) + { + + @Override + public List getDataSources() + { + return Collections.singletonList("datasource1"); + } + }; + SupervisorSpec spec2 = new TestSupervisorSpec("id2", null, null) + { + + @Override + public List getDataSources() + { + return Collections.singletonList("datasource2"); + } + }; + + SupervisorStateManager.State state1 = SupervisorStateManager.BasicState.RUNNING; + SupervisorStateManager.State state2 = SupervisorStateManager.BasicState.SUSPENDED; + + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)); + EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds).atLeastOnce(); + EasyMock.expect(supervisorManager.getSupervisorSpec("id1")).andReturn(Optional.of(spec1)).times(1); + EasyMock.expect(supervisorManager.getSupervisorSpec("id2")).andReturn(Optional.of(spec2)).times(1); + EasyMock.expect(supervisorManager.getSupervisorState("id1")).andReturn(Optional.of(state1)).times(1); + EasyMock.expect(supervisorManager.getSupervisorState("id2")).andReturn(Optional.of(state2)).times(1); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); + EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)).andReturn( + new AuthenticationResult("druid", "druid", null, null) + ).atLeastOnce(); + request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + replayAll(); + + Response response = supervisorResource.specGetAll(null, true, request); + verifyAll(); + + Assert.assertEquals(200, response.getStatus()); + List> states = (List>) response.getEntity(); + Assert.assertTrue( + states.stream() + .allMatch(state -> { + final String id = (String) state.get("id"); + if ("id1".equals(id)) { + return state1.equals(state.get("state")) + && state1.equals(state.get("detailedState")) + && (Boolean) state.get("healthy") == state1.isHealthy(); + } else if ("id2".equals(id)) { + return state2.equals(state.get("state")) + && state2.equals(state.get("detailedState")) + && (Boolean) state.get("healthy") == state2.isHealthy(); + } + return false; + }) + ); + } + @Test public void testSpecGet() { diff --git a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json index f2046dedf3a6..4437e725e28b 100644 --- a/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json +++ b/integration-tests/src/test/resources/results/auth_test_sys_schema_segments.json @@ -13,6 +13,6 @@ "is_available": 1, "is_realtime": 0, "is_overshadowed": 0, - "payload": "{\"dataSegment\":{\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\"},\"overshadowed\":false}" + "payload": "{\"dataSource\":\"auth_test\",\"interval\":\"2012-12-29T00:00:00.000Z/2013-01-10T08:00:00.000Z\",\"version\":\"2013-01-10T08:13:47.830Z_v9\",\"loadSpec\":{\"load spec is pruned, because it's not needed on Brokers, but eats a lot of heap space\":\"\"},\"dimensions\":\"anonymous,area_code,city,continent_code,country_name,dma_code,geo,language,namespace,network,newpage,page,postal_code,region_lookup,robot,unpatrolled,user\",\"metrics\":\"added,count,deleted,delta,delta_hist,unique_users,variation\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":446027801,\"identifier\":\"auth_test_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\",\"overshadowed\":false}" } ] diff --git a/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractionFn.java b/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractionFn.java index a774aee3baf9..9266cf2da621 100644 --- a/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractionFn.java +++ b/processing/src/main/java/org/apache/druid/query/lookup/LookupExtractionFn.java @@ -41,8 +41,8 @@ public LookupExtractionFn( @JsonProperty("lookup") final LookupExtractor lookup, @JsonProperty("retainMissingValue") final boolean retainMissingValue, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, - @JsonProperty("injective") final Boolean injective, - @JsonProperty("optimize") final Boolean optimize + @Nullable @JsonProperty("injective") final Boolean injective, + @Nullable @JsonProperty("optimize") final Boolean optimize ) { super( diff --git a/processing/src/main/java/org/apache/druid/query/lookup/RegisteredLookupExtractionFn.java b/processing/src/main/java/org/apache/druid/query/lookup/RegisteredLookupExtractionFn.java index 6595d6cdd18d..e8280b88db0b 100644 --- a/processing/src/main/java/org/apache/druid/query/lookup/RegisteredLookupExtractionFn.java +++ b/processing/src/main/java/org/apache/druid/query/lookup/RegisteredLookupExtractionFn.java @@ -48,7 +48,7 @@ public RegisteredLookupExtractionFn( @JsonProperty("lookup") String lookup, @JsonProperty("retainMissingValue") final boolean retainMissingValue, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, - @JsonProperty("injective") final Boolean injective, + @Nullable @JsonProperty("injective") final Boolean injective, @JsonProperty("optimize") Boolean optimize ) { @@ -73,12 +73,14 @@ public boolean isRetainMissingValue() return retainMissingValue; } + @Nullable @JsonProperty("replaceMissingValueWith") public String getReplaceMissingValueWith() { return replaceMissingValueWith; } + @Nullable @JsonProperty("injective") public Boolean isInjective() { @@ -151,10 +153,10 @@ private LookupExtractionFn ensureDelegate() delegate = new LookupExtractionFn( factory, - isRetainMissingValue(), - getReplaceMissingValueWith(), - injective == null ? factory.isOneToOne() : injective, - isOptimize() + retainMissingValue, + replaceMissingValueWith, + injective, + optimize ); } } diff --git a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java new file mode 100644 index 000000000000..865b8bd0363a --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java @@ -0,0 +1,172 @@ +/* + * 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 com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * An immutable snapshot information about used segments and overshadowed segments for + * {@link org.apache.druid.metadata.SQLMetadataSegmentManager}. + */ +public class DataSourcesSnapshot +{ + public static DataSourcesSnapshot fromUsedSegments( + Iterable segments, + ImmutableMap dataSourceProperties + ) + { + Map dataSources = new HashMap<>(); + segments.forEach(segment -> { + dataSources + .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties)) + .addSegmentIfAbsent(segment); + }); + return new DataSourcesSnapshot(CollectionUtils.mapValues(dataSources, DruidDataSource::toImmutableDruidDataSource)); + } + + public static DataSourcesSnapshot fromUsedSegmentsTimelines( + Map> usedSegmentsTimelinesPerDataSource, + ImmutableMap dataSourceProperties + ) + { + Map dataSourcesWithAllUsedSegments = + Maps.newHashMapWithExpectedSize(usedSegmentsTimelinesPerDataSource.size()); + usedSegmentsTimelinesPerDataSource.forEach( + (dataSourceName, usedSegmentsTimeline) -> { + DruidDataSource dataSource = new DruidDataSource(dataSourceName, dataSourceProperties); + usedSegmentsTimeline.iterateAllObjects().forEach(dataSource::addSegment); + dataSourcesWithAllUsedSegments.put(dataSourceName, dataSource.toImmutableDruidDataSource()); + } + ); + return new DataSourcesSnapshot(dataSourcesWithAllUsedSegments, usedSegmentsTimelinesPerDataSource); + } + + private final Map dataSourcesWithAllUsedSegments; + private final Map> usedSegmentsTimelinesPerDataSource; + private final ImmutableSet overshadowedSegments; + + public DataSourcesSnapshot(Map dataSourcesWithAllUsedSegments) + { + this( + dataSourcesWithAllUsedSegments, + CollectionUtils.mapValues( + dataSourcesWithAllUsedSegments, + dataSource -> VersionedIntervalTimeline.forSegments(dataSource.getSegments()) + ) + ); + } + + private DataSourcesSnapshot( + Map dataSourcesWithAllUsedSegments, + Map> usedSegmentsTimelinesPerDataSource + ) + { + this.dataSourcesWithAllUsedSegments = dataSourcesWithAllUsedSegments; + this.usedSegmentsTimelinesPerDataSource = usedSegmentsTimelinesPerDataSource; + this.overshadowedSegments = ImmutableSet.copyOf(determineOvershadowedSegments()); + } + + public Collection getDataSourcesWithAllUsedSegments() + { + return dataSourcesWithAllUsedSegments.values(); + } + + public Map getDataSourcesMap() + { + return dataSourcesWithAllUsedSegments; + } + + @Nullable + public ImmutableDruidDataSource getDataSource(String dataSourceName) + { + return dataSourcesWithAllUsedSegments.get(dataSourceName); + } + + public Map> getUsedSegmentsTimelinesPerDataSource() + { + return usedSegmentsTimelinesPerDataSource; + } + + public ImmutableSet getOvershadowedSegments() + { + return overshadowedSegments; + } + + /** + * Returns an iterable to go over all used segments in all data sources. The order in which segments are iterated + * is unspecified. + * + * Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try (to some + * reasonable extent) to organize the code so that it iterates the returned iterable only once rather than several + * times. + * + * This method's name starts with "iterate" because the result is expected to be consumed immediately in a for-each + * statement or a stream pipeline, like + * for (DataSegment segment : snapshot.iterateAllUsedSegmentsInSnapshot()) {...} + */ + public Iterable iterateAllUsedSegmentsInSnapshot() + { + return () -> dataSourcesWithAllUsedSegments + .values() + .stream() + .flatMap(dataSource -> dataSource.getSegments().stream()) + .iterator(); + } + + /** + * This method builds timelines from all data sources and finds the overshadowed segments list + * + * This method should be deduplicated with {@link VersionedIntervalTimeline#findOvershadowed()}: see + * https://github.com/apache/incubator-druid/issues/8070. + * + * @return overshadowed segment Ids list + */ + private List determineOvershadowedSegments() + { + // It's fine to add all overshadowed segments to a single collection because only + // a small fraction of the segments in the cluster are expected to be overshadowed, + // so building this collection shouldn't generate a lot of garbage. + final List overshadowedSegments = new ArrayList<>(); + for (ImmutableDruidDataSource dataSource : dataSourcesWithAllUsedSegments.values()) { + VersionedIntervalTimeline usedSegmentsTimeline = + usedSegmentsTimelinesPerDataSource.get(dataSource.getName()); + for (DataSegment segment : dataSource.getSegments()) { + if (usedSegmentsTimeline.isOvershadowed(segment.getInterval(), segment.getVersion())) { + overshadowedSegments.add(segment.getId()); + } + } + } + return overshadowedSegments; + } + +} diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java index e8798f7a1731..7ce3fd6dffb9 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java @@ -25,17 +25,12 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSortedMap; -import com.google.common.collect.Ordering; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.VersionedIntervalTimeline; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Objects; -import java.util.Set; /** * An immutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source. @@ -114,41 +109,6 @@ public long getTotalSizeOfSegments() return totalSizeOfSegments; } - /** - * This method finds the overshadowed segments from the given segments - * - * @return set of overshadowed segments - */ - public static Set determineOvershadowedSegments(Iterable segments) - { - final Map> timelines = buildTimelines(segments); - - final Set overshadowedSegments = new HashSet<>(); - for (DataSegment dataSegment : segments) { - final VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - overshadowedSegments.add(dataSegment); - } - } - return overshadowedSegments; - } - - /** - * Builds a timeline from given segments - * - * @return map of datasource to VersionedIntervalTimeline of segments - */ - private static Map> buildTimelines( - Iterable segments - ) - { - final Map> timelines = new HashMap<>(); - segments.forEach(segment -> timelines - .computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural())) - .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment))); - return timelines; - } - @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java index d8b726602525..3a904b9e0f74 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/NoopSupervisorSpec.java @@ -113,6 +113,12 @@ public SupervisorReport getStatus() return null; } + @Override + public SupervisorStateManager.State getState() + { + return SupervisorStateManager.BasicState.RUNNING; + } + @Override public void reset(DataSourceMetadata dataSourceMetadata) { diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java index cf3f4d5fa2e2..c0ecf44c29c0 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/supervisor/Supervisor.java @@ -39,6 +39,8 @@ public interface Supervisor SupervisorReport getStatus(); + SupervisorStateManager.State getState(); + default Map> getStats() { return ImmutableMap.of(); diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index 4064e010c05d..550dd49b0bf9 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -20,7 +20,7 @@ package org.apache.druid.metadata; import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.client.DruidDataSource; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -78,51 +78,32 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to * the caller. - * - * Prefer {@link #markSegmentAsUnused(SegmentId)} to this method when possible. - * - * This method is not removed because {@link org.apache.druid.server.http.DataSourcesResource#markSegmentAsUnused} - * uses it and if it migrates to {@link #markSegmentAsUnused(SegmentId)} the performance will be worse. */ - boolean markSegmentAsUnused(String dataSource, String segmentId); + boolean markSegmentAsUnused(String segmentId); /** - * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the - * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to - * the caller. + * If there are used segments belonging to the given data source this method returns them as an {@link + * ImmutableDruidDataSource} object. If there are no used segments belonging to the given data source this method + * returns null. */ - boolean markSegmentAsUnused(SegmentId segmentId); + @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource); /** - * If there are used segments belonging to the given data source, this method converts this set of segments to an - * {@link ImmutableDruidDataSource} object and returns. If there are no used segments belonging to the given data - * source, this method returns null. - * - * This method's name starts with "prepare" to indicate that it's not cheap (it creates an {@link - * ImmutableDruidDataSource} object). Not used "create" prefix to avoid giving a false impression that this method - * might put something into the database to create a data source with the given name, if absent. + * Returns a set of {@link ImmutableDruidDataSource} objects containing information about all used segments. {@link + * ImmutableDruidDataSource} objects in the returned collection are unique. If there are no used segments, this method + * returns an empty collection. */ - @Nullable ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSource); + Collection getImmutableDataSourcesWithAllUsedSegments(); /** - * If there are used segments belonging to the given data source, this method returns a {@link DruidDataSource} object - * with a view on those segments. If there are no used segments belonging to the given data source, this method - * returns null. - * - * Note that the returned {@link DruidDataSource} object may be updated concurrently and already be empty by the time - * it is returned. + * Returns a set of overshadowed segment ids. */ - @Nullable DruidDataSource getDataSourceWithUsedSegments(String dataSource); + Set getOvershadowedSegments(); /** - * Prepares a set of {@link ImmutableDruidDataSource} objects containing information about all used segments. {@link - * ImmutableDruidDataSource} objects in the returned collection are unique. If there are no used segments, this method - * returns an empty collection. - * - * This method's name starts with "prepare" for the same reason as {@link - * #prepareImmutableDataSourceWithUsedSegments}. + * Returns a snapshot of DruidDataSources and overshadowed segments */ - Collection prepareImmutableDataSourcesWithAllUsedSegments(); + DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments(); /** * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is @@ -140,8 +121,8 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) * Performance warning: this method makes a query into the database. * * This method might return a different set of data source names than may be observed via {@link - * #prepareImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there - * are no used segments belonging to it, while {@link #prepareImmutableDataSourcesWithAllUsedSegments} won't return + * #getImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there + * are no used segments belonging to it, while {@link #getImmutableDataSourcesWithAllUsedSegments} won't return * such a data source. */ Collection retrieveAllDataSourceNames(); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index f2ad12f3b1df..db1d19e49ae9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -24,12 +24,13 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.util.concurrent.Futures; import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; -import org.apache.druid.client.DruidDataSource; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; @@ -70,7 +71,6 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -96,10 +96,10 @@ private interface DatabasePoll private static class PeriodicDatabasePoll implements DatabasePoll { /** - * This future allows to wait until {@link #dataSources} is initialized in the first {@link #poll()} happening since - * {@link #startPollingDatabasePeriodically()} is called for the first time, or since the last visible (in - * happens-before terms) call to {@link #startPollingDatabasePeriodically()} in case of Coordinator's leadership - * changes. + * This future allows to wait until {@link #dataSourcesSnapshot} is initialized in the first {@link #poll()} + * happening since {@link #startPollingDatabasePeriodically()} is called for the first time, or since the last + * visible (in happens-before terms) call to {@link #startPollingDatabasePeriodically()} in case of Coordinator's + * leadership changes. */ final CompletableFuture firstPollCompletionFuture = new CompletableFuture<>(); } @@ -148,40 +148,41 @@ long nanosElapsedFromInitiation() /** * This field is made volatile to avoid "ghost secondary reads" that may result in NPE, see - * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl (note that dataSources resembles a lazily - * initialized field). Alternative is to always read the field in a snapshot local variable, but it's too easy to - * forget to do. + * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl (note that dataSourcesSnapshot resembles + * a lazily initialized field). Alternative is to always read the field in a snapshot local variable, but it's too + * easy to forget to do. * * This field may be updated from {@link #exec}, or from whatever thread calling {@link #doOnDemandPoll} via {@link * #awaitOrPerformDatabasePoll()} via one of the public methods of SqlSegmentsMetadata. */ - private volatile @MonotonicNonNull ConcurrentHashMap dataSources = null; + private volatile @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = null; /** - * The latest {@link DatabasePoll} represent {@link #poll()} calls which update {@link #dataSources}, either + * The latest {@link DatabasePoll} represent {@link #poll()} calls which update {@link #dataSourcesSnapshot}, either * periodically (see {@link PeriodicDatabasePoll}, {@link #startPollingDatabasePeriodically}, {@link * #stopPollingDatabasePeriodically}) or "on demand" (see {@link OnDemandDatabasePoll}), when one of the methods that - * accesses {@link #dataSources} state (such as {@link #prepareImmutableDataSourceWithUsedSegments}) is called when - * the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database periodically. + * accesses {@link #dataSourcesSnapshot}'s state (such as {@link #getImmutableDataSourceWithUsedSegments}) is + * called when the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database + * periodically. * * Note that if there is a happens-before relationship between a call to {@link #startPollingDatabasePeriodically()} - * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSources}'s state in this - * class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter method - * calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will + * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSourcesSnapshot}'s state in + * this class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter + * method calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will * see the latest {@link PeriodicDatabasePoll} value (stored in this field, latestDatabasePoll, in {@link * #startPollingDatabasePeriodically()}) and to await on its {@link PeriodicDatabasePoll#firstPollCompletionFuture}. * * However, the guarantee explained above doesn't make any actual semantic difference, because on both periodic and * on-demand database polls the same invariant is maintained that the results not older than {@link * #periodicPollDelay} are used. The main difference is in performance: since on-demand polls are irregular and happen - * in the context of the thread wanting to access the {@link #dataSources}, that may cause delays in the logic. - * On the other hand, periodic polls are decoupled into {@link #exec} and {@link #dataSources}-accessing methods - * should be generally "wait free" for database polls. + * in the context of the thread wanting to access the {@link #dataSourcesSnapshot}, that may cause delays in the + * logic. On the other hand, periodic polls are decoupled into {@link #exec} and {@link + * #dataSourcesSnapshot}-accessing methods should be generally "wait free" for database polls. * * The notion and the complexity of "on demand" database polls was introduced to simplify the interface of {@link - * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods like - * {@link #prepareImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part is - * achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or + * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods + * like {@link #getImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part + * is achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #awaitOrPerformDatabasePoll} method * implementation for details. * @@ -368,8 +369,8 @@ public void stopPollingDatabasePeriodically() periodicPollTaskFuture.cancel(false); latestDatabasePoll = null; - // NOT nulling dataSources, allowing to query the latest polled data even when this SegmentsMetadata object is - // stopped. + // NOT nulling dataSourcesSnapshot, allowing to query the latest polled data even when this SegmentsMetadata + // object is stopped. currentStartPollingOrder = -1; } @@ -528,7 +529,7 @@ private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Iterators.concat(usedSegmentsOverlappingInterval.iterator(), unusedSegmentsInInterval.iterator()) ); - return markNonOvershadowedSegmentsAsUsed(dataSourceName, unusedSegmentsInInterval, versionedIntervalTimeline); + return markNonOvershadowedSegmentsAsUsed(unusedSegmentsInInterval, versionedIntervalTimeline); } private static void consume(Iterator iterator) @@ -538,29 +539,16 @@ private static void consume(Iterator iterator) } } - /** Also puts non-overshadowed segments into {@link #dataSources}. */ private int markNonOvershadowedSegmentsAsUsed( - String dataSourceName, List unusedSegments, VersionedIntervalTimeline timeline ) { - @Nullable - DruidDataSource dataSource = null; - if (dataSources != null) { - dataSource = dataSources.computeIfAbsent( - dataSourceName, - dsName -> new DruidDataSource(dsName, createDefaultDataSourceProperties()) - ); - } List segmentIdsToMarkAsUsed = new ArrayList<>(); for (DataSegment segment : unusedSegments) { if (timeline.isOvershadowed(segment.getInterval(), segment.getVersion())) { continue; } - if (dataSource != null) { - dataSource.addSegment(segment); - } segmentIdsToMarkAsUsed.add(segment.getId().toString()); } @@ -590,7 +578,7 @@ public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set< List unusedSegments = unusedSegmentsAndTimeline.lhs; VersionedIntervalTimeline timeline = unusedSegmentsAndTimeline.rhs; - return markNonOvershadowedSegmentsAsUsed(dataSource, unusedSegments, timeline); + return markNonOvershadowedSegmentsAsUsed(unusedSegments, timeline); } catch (Exception e) { Throwable rootCause = Throwables.getRootCause(e); @@ -726,10 +714,6 @@ public int markAsUnusedAllSegmentsInDataSource(final String dataSource) .execute() ); - if (dataSources != null) { - dataSources.remove(dataSource); - } - return numUpdatedDatabaseEntries; } catch (RuntimeException e) { @@ -738,62 +722,15 @@ public int markAsUnusedAllSegmentsInDataSource(final String dataSource) } } + /** + * This method does not update {@link #dataSourcesSnapshot}, see the comments in {@link #doPoll()} about + * snapshot update. The update of the segment's state will be reflected after the next {@link DatabasePoll}. + */ @Override - public boolean markSegmentAsUnused(String dataSourceName, final String segmentId) - { - try { - boolean segmentStateChanged = markSegmentAsUnusedInDatabase(segmentId); - if (dataSources != null) { - removeSegmentFromPolledDataSources(dataSourceName, segmentId, dataSources); - } - return segmentStateChanged; - } - catch (RuntimeException e) { - log.error(e, "Exception marking segment [%s] as unused", segmentId); - throw e; - } - } - - private static void removeSegmentFromPolledDataSources( - String dataSourceName, - String segmentId, - ConcurrentHashMap dataSourcesSnapshot - ) - { - // Call iteratePossibleParsingsWithDataSource() outside of dataSources.computeIfPresent() because the former is - // a potentially expensive operation, while lambda to be passed into computeIfPresent() should preferably run - // fast. - List possibleSegmentIds = SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId); - dataSourcesSnapshot.computeIfPresent( - dataSourceName, - (dsName, dataSource) -> { - for (SegmentId possibleSegmentId : possibleSegmentIds) { - if (dataSource.removeSegment(possibleSegmentId) != null) { - break; - } - } - // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - return dataSource.isEmpty() ? null : dataSource; - } - ); - } - - @Override - public boolean markSegmentAsUnused(SegmentId segmentId) + public boolean markSegmentAsUnused(final String segmentId) { try { - final boolean segmentStateChanged = markSegmentAsUnusedInDatabase(segmentId.toString()); - if (dataSources != null) { - dataSources.computeIfPresent( - segmentId.getDataSource(), - (dsName, dataSource) -> { - dataSource.removeSegment(segmentId); - // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - return dataSource.isEmpty() ? null : dataSource; - } - ); - } - return segmentStateChanged; + return markSegmentAsUnusedInDatabase(segmentId); } catch (RuntimeException e) { log.error(e, "Exception marking segment [%s] as unused", segmentId); @@ -820,18 +757,7 @@ public int markSegmentsAsUnused(String dataSourceName, Set segmentIds) ) )); final int[] segmentChanges = batch.execute(); - int numChangedSegments = computeNumChangedSegments(segmentIdList, segmentChanges); - - // Also remove segments from polled dataSources. - // Cache dataSourcesSnapshot locally to make sure that we do all updates to a single map, not to two different - // maps if poll() happens concurrently. - @MonotonicNonNull ConcurrentHashMap dataSourcesSnapshot = this.dataSources; - if (dataSourcesSnapshot != null) { - for (String segmentId : segmentIdList) { - removeSegmentFromPolledDataSources(dataSourceName, segmentId, dataSourcesSnapshot); - } - } - return numChangedSegments; + return computeNumChangedSegments(segmentIdList, segmentChanges); }); } catch (Exception e) { @@ -858,13 +784,6 @@ public int markAsUnusedSegmentsInInterval(String dataSourceName, Interval interv .bind("end", interval.getEnd().toString()) .execute() ); - if (dataSources != null) { - dataSources.computeIfPresent(dataSourceName, (dsName, dataSource) -> { - dataSource.removeSegmentsIf(segment -> interval.contains(segment.getInterval())); - // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - return dataSource.isEmpty() ? null : dataSource; - }); - } return numUpdatedDatabaseEntries; } catch (Exception e) { @@ -924,37 +843,36 @@ private static int computeNumChangedSegments(List segmentIds, int[] segm } @Override - public @Nullable ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSourceName) + public @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSourceName) { - awaitOrPerformDatabasePoll(); - final DruidDataSource dataSource = dataSources.get(dataSourceName); - return dataSource == null || dataSource.isEmpty() ? null : dataSource.toImmutableDruidDataSource(); + return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSource(dataSourceName); } @Override - public @Nullable DruidDataSource getDataSourceWithUsedSegments(String dataSource) + public Collection getImmutableDataSourcesWithAllUsedSegments() { - awaitOrPerformDatabasePoll(); - return dataSources.get(dataSource); + return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSourcesWithAllUsedSegments(); + } + + @Override + public Set getOvershadowedSegments() + { + return getSnapshotOfDataSourcesWithAllUsedSegments().getOvershadowedSegments(); } @Override - public Collection prepareImmutableDataSourcesWithAllUsedSegments() + public DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments() { awaitOrPerformDatabasePoll(); - return dataSources - .values() - .stream() - .map(DruidDataSource::toImmutableDruidDataSource) - .collect(Collectors.toList()); + return dataSourcesSnapshot; } @Override public Iterable iterateAllUsedSegments() { awaitOrPerformDatabasePoll(); - return () -> dataSources - .values() + return () -> dataSourcesSnapshot + .getDataSourcesWithAllUsedSegments() .stream() .flatMap(dataSource -> dataSource.getSegments().stream()) .iterator(); @@ -1021,7 +939,7 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE catch (IOException e) { log.makeAlert(e, "Failed to read segment from db.").emit(); // If one entry in database is corrupted doPoll() should continue to work overall. See - // .filter(Objects::nonNull) below in this method. + // filter by `Objects::nonNull` below in this method. return null; } } @@ -1039,20 +957,20 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE log.info("Polled and found %,d segments in the database", segments.size()); - ConcurrentHashMap newDataSources = new ConcurrentHashMap<>(); - ImmutableMap dataSourceProperties = createDefaultDataSourceProperties(); - segments - .stream() - .filter(Objects::nonNull) // Filter corrupted entries (see above in this method). - .forEach(segment -> { - newDataSources - .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties)) - .addSegmentIfAbsent(segment); - }); - - // Replace dataSources atomically. - dataSources = newDataSources; + + // dataSourcesSnapshot is updated only here and the DataSourcesSnapshot object is immutable. If data sources or + // segments are marked as used or unused directly (via markAs...() methods in MetadataSegmentManager), the + // dataSourcesSnapshot can become invalid until the next database poll. + // DataSourcesSnapshot computes the overshadowed segments, which makes it an expensive operation if the + // snapshot was invalidated on each segment mark as unused or used, especially if a user issues a lot of single + // segment mark calls in rapid succession. So the snapshot update is not done outside of database poll at this time. + // Updates outside of database polls were primarily for the user experience, so users would immediately see the + // effect of a segment mark call reflected in MetadataResource API calls. + dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments( + Iterables.filter(segments, Objects::nonNull), // Filter corrupted entries (see above in this method). + dataSourceProperties + ); } private static ImmutableMap createDefaultDataSourceProperties() @@ -1063,18 +981,19 @@ private static ImmutableMap createDefaultDataSourceProperties() /** * For the garbage collector in Java, it's better to keep new objects short-living, but once they are old enough * (i. e. promoted to old generation), try to keep them alive. In {@link #poll()}, we fetch and deserialize all - * existing segments each time, and then replace them in {@link #dataSources}. This method allows to use already - * existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link + * existing segments each time, and then replace them in {@link #dataSourcesSnapshot}. This method allows to use + * already existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link * com.google.common.collect.Interner}, aiming to make the majority of {@link DataSegment} objects garbage soon after * they are deserialized and to die in young generation. It allows to avoid fragmentation of the old generation and * full GCs. */ private DataSegment replaceWithExistingSegmentIfPresent(DataSegment segment) { - if (dataSources == null) { + @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = this.dataSourcesSnapshot; + if (dataSourcesSnapshot == null) { return segment; } - @Nullable DruidDataSource dataSource = dataSources.get(segment.getDataSource()); + @Nullable ImmutableDruidDataSource dataSource = dataSourcesSnapshot.getDataSource(segment.getDataSource()); if (dataSource == null) { return segment; } 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 0651b0548fca..431cb81ca5a7 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 @@ -32,6 +32,7 @@ import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.utils.ZKPaths; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; @@ -258,7 +259,7 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - final Iterable dataSegments = iterateAllUsedSegments(); + final Iterable dataSegments = segmentsMetadata.iterateAllUsedSegments(); final DateTime now = DateTimes.nowUtc(); @@ -294,7 +295,7 @@ public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() final Object2IntOpenHashMap numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>(); - final Iterable dataSegments = iterateAllUsedSegments(); + final Iterable dataSegments = segmentsMetadata.iterateAllUsedSegments(); for (DataSegment segment : dataSegments) { if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) { @@ -311,7 +312,7 @@ public Map getLoadStatus() { final Map loadStatus = new HashMap<>(); final Collection dataSources = - segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); for (ImmutableDruidDataSource dataSource : dataSources) { final Set segments = Sets.newHashSet(dataSource.getSegments()); @@ -356,7 +357,7 @@ public CoordinatorCompactionConfig getCompactionConfig() public void markSegmentAsUnused(DataSegment segment) { log.info("Marking segment[%s] as unused", segment.getId()); - segmentsMetadata.markSegmentAsUnused(segment.getId()); + segmentsMetadata.markSegmentAsUnused(segment.getId().toString()); } public String getCurrentLeader() @@ -365,6 +366,7 @@ public String getCurrentLeader() } public void moveSegment( + DruidCoordinatorRuntimeParams params, ImmutableDruidServer fromServer, ImmutableDruidServer toServer, DataSegment segment, @@ -384,8 +386,7 @@ public void moveSegment( throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); } - ImmutableDruidDataSource dataSource = - segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(segment.getDataSource()); + ImmutableDruidDataSource dataSource = params.getDataSourcesSnapshot().getDataSource(segment.getDataSource()); if (dataSource == null) { throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); } @@ -463,19 +464,6 @@ public void moveSegment( } } - /** - * Returns an iterable to go over all used segments in all data sources. The order in which segments are iterated - * is unspecified. - * - * Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try (to some - * reasonable extent) to organize the code so that it iterates the returned iterable only once rather than several - * times. - */ - public Iterable iterateAllUsedSegments() - { - return segmentsMetadata.iterateAllUsedSegments(); - } - @LifecycleStart public void start() { @@ -660,19 +648,19 @@ public void run() BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec); // Do coordinator stuff. - final Collection dataSources = - segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments(); DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder() .withStartTimeNanos(startTimeNanos) - .withDataSourcesWithUsedSegments(dataSources) + .withSnapshotOfDataSourcesWithAllUsedSegments(dataSourcesSnapshot) .withDynamicConfigs(getDynamicConfigs()) .withCompactionConfig(getCompactionConfig()) .withEmitter(emitter) .withBalancerStrategy(balancerStrategy) .build(); + for (DruidCoordinatorHelper helper : helpers) { // Don't read state and run state in the same helper otherwise racy conditions may exist if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index 4086dc91d844..07cb037f6f0c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -72,7 +72,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // If there is no running/pending/waiting/complete tasks, stalePendingSegmentsCutoffCreationTime is // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); - for (String dataSource : params.getDataSourcesWithUsedSegments().keySet()) { + for (String dataSource : params.getUsedSegmentsTimelinesPerDataSource().keySet()) { if (!params.getCoordinatorDynamicConfig().getDataSourcesToNotKillStalePendingSegmentsIn().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", 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 64d3611af882..600caaffeec9 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 @@ -21,7 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.druid.client.ImmutableDruidDataSource; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; @@ -42,14 +43,11 @@ public class DruidCoordinatorRuntimeParams { /** - * Creates a set to be provided to {@link Builder#setUsedSegments(TreeSet)} method from the given {@link - * Iterable} of segments. - * * Creates a TreeSet sorted in {@link DruidCoordinator#SEGMENT_COMPARATOR_RECENT_FIRST} order and populates it with * the segments from the given iterable. The given iterable is iterated exactly once. No special action is taken if * duplicate segments are encountered in the iterable. */ - public static TreeSet createUsedSegmentsSet(Iterable usedSegments) + private static TreeSet createUsedSegmentsSet(Iterable usedSegments) { TreeSet segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); usedSegments.forEach(segmentsSet::add); @@ -60,9 +58,8 @@ public static TreeSet createUsedSegmentsSet(Iterable u private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; private final SegmentReplicantLookup segmentReplicantLookup; - /** dataSource -> VersionedIntervalTimeline[version String, DataSegment] */ - private final Map> dataSourcesWithUsedSegments; private final @Nullable TreeSet usedSegments; + private final @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; @@ -77,8 +74,8 @@ private DruidCoordinatorRuntimeParams( DruidCluster druidCluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSourcesWithUsedSegments, @Nullable TreeSet usedSegments, + @Nullable DataSourcesSnapshot dataSourcesSnapshot, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -93,8 +90,8 @@ private DruidCoordinatorRuntimeParams( this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; this.usedSegments = usedSegments; + this.dataSourcesSnapshot = dataSourcesSnapshot; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -126,16 +123,18 @@ public SegmentReplicantLookup getSegmentReplicantLookup() } /** - * Returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" segments. + * Creates and returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" + * segments. */ - public Map> getDataSourcesWithUsedSegments() + public Map> getUsedSegmentsTimelinesPerDataSource() { - return dataSourcesWithUsedSegments; + Preconditions.checkState(dataSourcesSnapshot != null, "dataSourcesSnapshot or usedSegments must be set"); + return dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource(); } public TreeSet getUsedSegments() { - Preconditions.checkState(usedSegments != null, "usedSegments must be set"); + Preconditions.checkState(usedSegments != null, "usedSegments or dataSourcesSnapshot must be set"); return usedSegments; } @@ -188,6 +187,11 @@ public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements return nanosElapsedSinceCoordinatorStart > lagNanos; } + public DataSourcesSnapshot getDataSourcesSnapshot() + { + return dataSourcesSnapshot; + } + public static Builder newBuilder() { return new Builder(); @@ -200,8 +204,8 @@ public Builder buildFromExisting() druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSourcesWithUsedSegments, usedSegments, + dataSourcesSnapshot, loadManagementPeons, replicationManager, emitter, @@ -220,8 +224,8 @@ public Builder buildFromExistingWithoutUsedSegments() druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSourcesWithUsedSegments, null, // usedSegments + null, // dataSourcesSnapshot loadManagementPeons, replicationManager, emitter, @@ -239,8 +243,8 @@ public static class Builder private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; - private Map> dataSourcesWithUsedSegments; private @Nullable TreeSet usedSegments; + private @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final Map loadManagementPeons; private ReplicationThrottler replicationManager; private ServiceEmitter emitter; @@ -256,8 +260,8 @@ private Builder() this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; - this.dataSourcesWithUsedSegments = new HashMap<>(); this.usedSegments = null; + this.dataSourcesSnapshot = null; this.loadManagementPeons = new HashMap<>(); this.replicationManager = null; this.emitter = null; @@ -272,8 +276,8 @@ private Builder() DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSourcesWithUsedSegments, @Nullable TreeSet usedSegments, + @Nullable DataSourcesSnapshot dataSourcesSnapshot, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -288,8 +292,8 @@ private Builder() this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; this.usedSegments = usedSegments; + this.dataSourcesSnapshot = dataSourcesSnapshot; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -308,8 +312,8 @@ public DruidCoordinatorRuntimeParams build() druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSourcesWithUsedSegments, usedSegments, + dataSourcesSnapshot, loadManagementPeons, replicationManager, emitter, @@ -345,33 +349,10 @@ public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) return this; } - @VisibleForTesting - public Builder setDataSourcesWithUsedSegments( - Map> dataSources - ) - { - this.dataSourcesWithUsedSegments = dataSources; - return this; - } - - Builder withDataSourcesWithUsedSegments(Collection dataSourcesWithUsedSegments) + public Builder withSnapshotOfDataSourcesWithAllUsedSegments(DataSourcesSnapshot snapshot) { - dataSourcesWithUsedSegments.forEach( - dataSource -> { - VersionedIntervalTimeline timeline = this.dataSourcesWithUsedSegments.computeIfAbsent( - dataSource.getName(), - k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER) - ); - - dataSource.getSegments().forEach( - usedSegment -> timeline.add( - usedSegment.getInterval(), - usedSegment.getVersion(), - usedSegment.getShardSpec().createChunk(usedSegment) - ) - ); - } - ); + this.usedSegments = createUsedSegmentsSet(snapshot.iterateAllUsedSegmentsInSnapshot()); + this.dataSourcesSnapshot = snapshot; return this; } @@ -386,20 +367,22 @@ public Builder withUsedSegmentsInTest(DataSegment... usedSegments) @VisibleForTesting public Builder withUsedSegmentsInTest(Collection usedSegments) { - return setUsedSegments(createUsedSegmentsSet(usedSegments)); + this.usedSegments = createUsedSegmentsSet(usedSegments); + this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments(usedSegments, ImmutableMap.of()); + return this; } - /** - * Note: unlike {@link #withUsedSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the - * provided set. The set passed into this method must not be modified afterwards. - */ - public Builder setUsedSegments(TreeSet usedSegments) + /** This method must be used in test code only. */ + @VisibleForTesting + public Builder withUsedSegmentsTimelinesPerDataSourceInTest( + Map> usedSegmentsTimelinesPerDataSource + ) { - //noinspection ObjectEquality - if (usedSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { - throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST"); - } - this.usedSegments = usedSegments; + this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines( + usedSegmentsTimelinesPerDataSource, + ImmutableMap.of() + ); + usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()); return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index caab9fa73a31..dc164c890eef 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -268,6 +268,7 @@ protected boolean moveSegment( movingSegments.put(segmentId, segment); callback = () -> movingSegments.remove(segmentId); coordinator.moveSegment( + params, fromServer, toServer, segmentToMove, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index fc2b954f95f0..bde7c15810d2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -61,12 +61,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) ImmutableDruidServer server = serverHolder.getServer(); for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - VersionedIntervalTimeline timeline = timelines.get(dataSource.getName()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); - timelines.put(dataSource.getName(), timeline); - } - + VersionedIntervalTimeline timeline = timelines + .computeIfAbsent( + dataSource.getName(), + dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder()) + ); VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java index efd3b62e6a84..69363639f221 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -265,7 +265,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Emit segment metrics final Stream allSegments = params - .getDataSourcesWithUsedSegments() + .getUsedSegmentsTimelinesPerDataSource() .values() .stream() .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index 51714e28f625..680825a9ff39 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -20,7 +20,6 @@ package org.apache.druid.server.coordinator.helper; import com.google.common.collect.Lists; -import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.MetadataRuleManager; @@ -81,11 +80,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - // Find used segments which are overshadowed by other used segments. Those would not need to be loaded and + // Get used segments which are overshadowed by other used segments. Those would not need to be loaded and // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked // as unused in DruidCoordinatorMarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to // Historical nodes to unload such segments in DruidCoordinatorUnloadUnusedSegments. - Set overshadowed = ImmutableDruidDataSource.determineOvershadowedSegments(params.getUsedSegments()); + Set overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments(); for (String tier : cluster.getTierNames()) { replicatorThrottler.updateReplicationState(tier); @@ -103,7 +102,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); int missingRules = 0; for (DataSegment segment : params.getUsedSegments()) { - if (overshadowed.contains(segment)) { + if (overshadowed.contains(segment.getId())) { // Skipping overshadowed segments continue; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index 9574240a3e3e..f2a1a9fa51fd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -76,7 +76,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final CoordinatorStats stats = new CoordinatorStats(); if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { - Map> dataSources = params.getDataSourcesWithUsedSegments(); + Map> dataSources = + params.getUsedSegmentsTimelinesPerDataSource(); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList != null && !compactionConfigList.isEmpty()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java index 12b262d637f2..1b7beab2a960 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java @@ -19,14 +19,12 @@ package org.apache.druid.server.coordinator.helper; -import com.google.common.collect.Iterables; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.timeline.DataSegment; -import java.util.TreeSet; - public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper { private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class); @@ -43,44 +41,25 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { log.info("Starting coordination. Getting used segments."); - final Iterable usedSegments = coordinator.iterateAllUsedSegments(); - - // The following transform() call doesn't actually transform the iterable. It only checks the sizes of the segments - // and emits alerts if segments with negative sizes are encountered. In other words, semantically it's similar to - // Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createUsedSegmentsSet() (which is called - // below) guarantees to go over the passed iterable exactly once. - // - // An iterable returned from iterateAllUsedSegments() is not simply iterated (with size checks) before passing - // into DruidCoordinatorRuntimeParams.createUsedSegmentsSet() because iterateAllUsedSegments()'s - // documentation says to strive to avoid iterating the result more than once. - // - //noinspection StaticPseudoFunctionalStyleMethod: https://youtrack.jetbrains.com/issue/IDEA-153047 - Iterable usedSegmentsWithSizeChecking = Iterables.transform( - usedSegments, - segment -> { - if (segment.getSize() < 0) { - log.makeAlert("No size on a segment") - .addData("segment", segment) - .emit(); - } - return segment; - } - ); - final TreeSet usedSegmentSet = - DruidCoordinatorRuntimeParams.createUsedSegmentsSet(usedSegmentsWithSizeChecking); + DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); + for (DataSegment segment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { + if (segment.getSize() < 0) { + log.makeAlert("No size on a segment") + .addData("segment", segment) + .emit(); + } + } // Log info about all used segments if (log.isDebugEnabled()) { log.debug("Used Segments"); - for (DataSegment dataSegment : usedSegmentSet) { + for (DataSegment dataSegment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { log.debug(" %s", dataSegment); } } - log.info("Found [%,d] used segments.", usedSegmentSet.size()); + log.info("Found [%,d] used segments.", params.getUsedSegments().size()); - return params.buildFromExisting() - .setUsedSegments(usedSegmentSet) - .build(); + return params; } } diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index d8217e496040..67e788f9846a 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -502,7 +502,7 @@ public Response markSegmentAsUnused( @PathParam("segmentId") String segmentId ) { - boolean segmentStateChanged = segmentsMetadata.markSegmentAsUnused(dataSourceName, segmentId); + boolean segmentStateChanged = segmentsMetadata.markSegmentAsUnused(segmentId); return Response.ok(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build(); } @@ -614,9 +614,7 @@ private Map> getSimpleDatasource(String dataSourceNa continue; } - if (!tierDistinctSegments.containsKey(tier)) { - tierDistinctSegments.put(tier, new HashSet<>()); - } + tierDistinctSegments.computeIfAbsent(tier, t -> new HashSet<>()); long dataSourceSegmentSize = 0; for (DataSegment dataSegment : druidDataSource.getSegments()) { 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 b250a79fdd28..f4cbcfa3088d 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 @@ -25,7 +25,7 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.client.DruidDataSource; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; @@ -40,6 +40,7 @@ import org.apache.druid.timeline.SegmentWithOvershadowedStatus; import org.joda.time.Interval; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.GET; import javax.ws.rs.POST; @@ -53,7 +54,6 @@ import javax.ws.rs.core.UriInfo; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.TreeSet; @@ -97,7 +97,7 @@ public Response getDataSources( if (includeUnused) { dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames()); } else { - druidDataSources = segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + druidDataSources = segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); dataSourceNamesPreAuth = druidDataSources .stream() .map(ImmutableDruidDataSource::getName) @@ -134,12 +134,16 @@ public Response getDataSources( @Produces(MediaType.APPLICATION_JSON) public Response getAllUsedSegments( @Context final HttpServletRequest req, - @QueryParam("datasources") final Set dataSources, + @QueryParam("datasources") final @Nullable Set dataSources, @QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus ) { + if (includeOvershadowedStatus != null) { + return getAllUsedSegmentsWithOvershadowedStatus(req, dataSources); + } + Collection dataSourcesWithUsedSegments = - segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); + segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); if (dataSources != null && !dataSources.isEmpty()) { dataSourcesWithUsedSegments = dataSourcesWithUsedSegments .stream() @@ -150,51 +154,53 @@ public Response getAllUsedSegments( .stream() .flatMap(t -> t.getSegments().stream()); - if (includeOvershadowedStatus != null) { - final Iterable authorizedSegments = - findAuthorizedSegmentWithOvershadowedStatus(req, dataSourcesWithUsedSegments, usedSegments); - Response.ResponseBuilder builder = Response.status(Response.Status.OK); - return builder.entity(authorizedSegments).build(); - } else { - - final Function> raGenerator = segment -> Collections.singletonList( - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); + final Function> raGenerator = segment -> Collections.singletonList( + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); - final Iterable authorizedSegments = - AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); + final Iterable authorizedSegments = + AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); - Response.ResponseBuilder builder = Response.status(Response.Status.OK); - return builder.entity(authorizedSegments).build(); - } + Response.ResponseBuilder builder = Response.status(Response.Status.OK); + return builder.entity(authorizedSegments).build(); } - private Iterable findAuthorizedSegmentWithOvershadowedStatus( + private Response getAllUsedSegmentsWithOvershadowedStatus( HttpServletRequest req, - Collection dataSources, - Stream metadataSegments + @Nullable Set dataSources ) { - // It's fine to add all overshadowed segments to a single collection because only - // a small fraction of the segments in the cluster are expected to be overshadowed, - // so building this collection shouldn't generate a lot of garbage. - final Set overshadowedSegments = new HashSet<>(); - for (ImmutableDruidDataSource dataSource : dataSources) { - overshadowedSegments.addAll(ImmutableDruidDataSource.determineOvershadowedSegments(dataSource.getSegments())); + DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments(); + Collection dataSourcesWithUsedSegments = + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); } + final Stream usedSegments = dataSourcesWithUsedSegments + .stream() + .flatMap(t -> t.getSegments().stream()); + final Set overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments(); - final Stream segmentsWithOvershadowedStatus = metadataSegments - .map(segment -> new SegmentWithOvershadowedStatus(segment, overshadowedSegments.contains(segment))); + final Stream usedSegmentsWithOvershadowedStatus = usedSegments + .map(segment -> new SegmentWithOvershadowedStatus( + segment, + overshadowedSegments.contains(segment.getId()) + )); final Function> raGenerator = segment -> Collections .singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSegment().getDataSource())); final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( req, - segmentsWithOvershadowedStatus::iterator, + usedSegmentsWithOvershadowedStatus::iterator, raGenerator, authorizerMapper ); - return authorizedSegments; + + Response.ResponseBuilder builder = Response.status(Response.Status.OK); + return builder.entity(authorizedSegments).build(); } /** @@ -208,7 +214,7 @@ private Iterable findAuthorizedSegmentWithOversha public Response getDataSourceWithUsedSegments(@PathParam("dataSourceName") final String dataSourceName) { ImmutableDruidDataSource dataSource = - segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -226,7 +232,7 @@ public Response getUsedSegmentsInDataSource( ) { ImmutableDruidDataSource dataSource = - segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -272,7 +278,7 @@ public Response isSegmentUsed( @PathParam("segmentId") String segmentId ) { - DruidDataSource dataSource = segmentsMetadata.getDataSourceWithUsedSegments(dataSourceName); + ImmutableDruidDataSource dataSource = segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index de24926a3ab9..2a8e755cdad1 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2198,9 +2198,8 @@ private List> populateTimeline( serverExpectationList.add(serverExpectations); for (int j = 0; j < numChunks; ++j) { DruidServer lastServer = servers[random.nextInt(servers.length)]; - if (!serverExpectations.containsKey(lastServer)) { - serverExpectations.put(lastServer, new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); - } + serverExpectations + .computeIfAbsent(lastServer, server -> new ServerExpectations(server, makeMock(mocks, QueryRunner.class))); DataSegment mockSegment = makeMock(mocks, DataSegment.class); ServerExpectation expectation = new ServerExpectation<>( diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index 944b50fb8faa..a1852be765e3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -160,14 +160,14 @@ public void testPoll() Assert.assertEquals( ImmutableList.of("wikipedia"), sqlSegmentsMetadata - .prepareImmutableDataSourcesWithAllUsedSegments() + .getImmutableDataSourcesWithAllUsedSegments() .stream() .map(ImmutableDruidDataSource::getName) .collect(Collectors.toList()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), @@ -181,7 +181,7 @@ public void testPrepareImmutableDataSourceWithUsedSegmentsAwaitsPollOnRestart() DataSegment newSegment = pollThenStopThenStartIntro(); Assert.assertEquals( ImmutableSet.of(newSegment), - ImmutableSet.copyOf(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) ); } @@ -191,7 +191,7 @@ public void testGetDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOExce DataSegment newSegment = pollThenStopThenStartIntro(); Assert.assertEquals( ImmutableSet.of(newSegment), - ImmutableSet.copyOf(sqlSegmentsMetadata.getDataSourceWithUsedSegments("wikipedia2").getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) ); } @@ -203,7 +203,7 @@ public void testPrepareImmutableDataSourcesWithAllUsedSegmentsAwaitsPollOnRestar ImmutableSet.of(segment1, segment2, newSegment), ImmutableSet.copyOf( sqlSegmentsMetadata - .prepareImmutableDataSourcesWithAllUsedSegments() + .getImmutableDataSourcesWithAllUsedSegments() .stream() .flatMap((ImmutableDruidDataSource dataSource) -> dataSource.getSegments().stream()) .iterator() @@ -261,7 +261,7 @@ public void testPollWithCorruptedSegment() Assert.assertEquals( "wikipedia", - Iterables.getOnlyElement(sqlSegmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()).getName() + Iterables.getOnlyElement(sqlSegmentsMetadata.getImmutableDataSourcesWithAllUsedSegments()).getName() ); } @@ -297,10 +297,11 @@ public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException, Interr publisher.publishSegment(newSegment); - awaitPollingNewDataSource(newDataSource); + awaitDataSourceAppeared(newDataSource); int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource(newDataSource); Assert.assertEquals(1, numChangedSegments); - Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + awaitDataSourceDisappeared(newDataSource); + Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); } private static DataSegment createNewSegment1(String newDataSource) @@ -342,15 +343,24 @@ public void testMarkSegmentAsUnused() throws IOException, InterruptedException ); publisher.publishSegment(newSegment); + awaitDataSourceAppeared(newDataSource); + Assert.assertNotNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); - awaitPollingNewDataSource(newDataSource); - Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId())); - Assert.assertNull(sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId().toString())); + awaitDataSourceDisappeared(newDataSource); + Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); } - private void awaitPollingNewDataSource(String newDataSource) throws InterruptedException + private void awaitDataSourceAppeared(String newDataSource) throws InterruptedException { - while (sqlSegmentsMetadata.prepareImmutableDataSourceWithUsedSegments(newDataSource) == null) { + while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource) == null) { + Thread.sleep(1000); + } + } + + private void awaitDataSourceDisappeared(String dataSource) throws InterruptedException + { + while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSource) != null) { Thread.sleep(1000); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 4d7d1108dd91..9db997be895c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -41,13 +41,7 @@ public TestUsedSegmentChecker(AppenderatorTester appenderatorTester) public Set findUsedSegments(Set identifiers) { final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { - timeline.add( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(dataSegment) - ); - } + VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator()); final Set retVal = new HashSet<>(); for (SegmentIdWithShardSpec identifier : identifiers) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index ec53c94b1448..eb3b25b6899f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -32,6 +32,7 @@ import org.apache.druid.client.BatchServerInventoryView; import org.apache.druid.client.CoordinatorSegmentWatcherConfig; import org.apache.druid.client.CoordinatorServerView; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.common.config.JacksonConfigManager; @@ -82,7 +83,10 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase { private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegmentManager segmentsMetadata; + private DataSourcesSnapshot dataSourcesSnapshot; + private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; + private ScheduledExecutorFactory scheduledExecutorFactory; private ConcurrentMap loadManagementPeons; private LoadQueuePeon sourceLoadQueuePeon; @@ -124,7 +128,10 @@ public CuratorDruidCoordinatorTest() @Before public void setUp() throws Exception { - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); + dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); + coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); + metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -133,7 +140,7 @@ public void setUp() throws Exception EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes(); + ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), @@ -209,7 +216,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, baseView, metadataRuleManager, curator, @@ -362,11 +369,15 @@ public void testMoveSegment() throws Exception ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2)); EasyMock.replay(druidDataSource); - EasyMock.expect(databaseSegmentManager.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())) + EasyMock.expect(segmentsMetadata.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) .andReturn(druidDataSource); - EasyMock.replay(databaseSegmentManager); + EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(dataSourcesSnapshot).anyTimes(); + EasyMock.replay(segmentsMetadata, coordinatorRuntimeParams); + EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); + EasyMock.replay(dataSourcesSnapshot); coordinator.moveSegment( + coordinatorRuntimeParams, source.toImmutableDruidServer(), dest.toImmutableDruidServer(), sourceSegments.get(2), @@ -493,7 +504,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, baseView, metadataRuleManager, curator, @@ -530,14 +541,7 @@ private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String return DataSegment.builder() .dataSource("test_curator_druid_coordinator") .interval(Intervals.of(intervalStr)) - .loadSpec( - ImmutableMap.of( - "type", - "local", - "path", - "somewhere" - ) - ) + .loadSpec(ImmutableMap.of("type", "local", "path", "somewhere")) .version(version) .dimensions(ImmutableList.of()) .metrics(ImmutableList.of()) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index 46ec7cf90190..fa0954fac0dc 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -84,6 +84,7 @@ public void bigProfiler() EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); @@ -184,6 +185,7 @@ public void profileRun() EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java index ad22c09abce0..6a9d7f4405ba 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -571,6 +571,7 @@ private static void mockCoordinator(DruidCoordinator coordinator) EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 4b9908baa496..2b7975380f7e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; import org.apache.druid.metadata.MetadataRuleManager; +import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; @@ -67,6 +68,7 @@ public class DruidCoordinatorRuleRunnerTest private DruidCoordinatorRuleRunner ruleRunner; private ServiceEmitter emitter; private MetadataRuleManager databaseRuleManager; + private MetadataSegmentManager segmentsMetadata; @Before public void setUp() @@ -76,6 +78,7 @@ public void setUp() emitter = EasyMock.createMock(ServiceEmitter.class); EmittingLogger.registerEmitter(emitter); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); DateTime start = DateTimes.of("2012-01-01"); usedSegments = new ArrayList<>(); @@ -189,14 +192,19 @@ private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams( DruidCluster druidCluster, BalancerStrategy balancerStrategy ) + { + return createCoordinatorRuntimeParams(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerStrategy(balancerStrategy); + } + + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster) { return CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster(druidCluster) .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy); + .withDatabaseRuleManager(databaseRuleManager); } /** @@ -318,11 +326,7 @@ public void testRunTwoTiersWithExistingSegments() ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) .build(); @@ -426,11 +430,7 @@ public void testRunRuleDoesNotExist() ) .build(); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withEmitter(emitter) .build(); @@ -478,12 +478,8 @@ public void testDropRemove() ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) .build(); @@ -538,17 +534,13 @@ public void testDropTooManyInSameTier() ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withDynamicConfigs( CoordinatorDynamicConfig .builder() .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L) .build() ) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) .build(); @@ -602,12 +594,8 @@ public void testDropTooManyInDifferentTiers() ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) .build(); @@ -658,12 +646,8 @@ public void testDontDropInDifferentTiers() ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers - .newBuilder() - .withDruidCluster(druidCluster) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) - .withUsedSegmentsInTest(usedSegments) - .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) .build(); @@ -1063,6 +1047,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) .build(); @@ -1087,7 +1072,7 @@ private void mockCoordinator() EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); + EasyMock.replay(coordinator, segmentsMetadata); } private void mockEmptyPeon() 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 af0c0c350463..b587842d797d 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 @@ -30,6 +30,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; import org.apache.druid.client.ImmutableDruidDataSource; @@ -87,6 +88,9 @@ public class DruidCoordinatorTest extends CuratorTestBase private DruidCoordinator coordinator; private MetadataSegmentManager segmentsMetadata; + private DataSourcesSnapshot dataSourcesSnapshot; + private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; + private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; @@ -107,6 +111,8 @@ public void setUp() throws Exception druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); + dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); + coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -246,8 +252,12 @@ public void testMoveSegment() ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.replay(druidDataSource); - EasyMock.expect(segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock + .expect(segmentsMetadata.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) + .andReturn(druidDataSource); EasyMock.replay(segmentsMetadata); + EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); + EasyMock.replay(dataSourcesSnapshot); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(scheduledExecutorFactory); EasyMock.replay(metadataRuleManager); @@ -282,7 +292,10 @@ public void testMoveSegment() EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once(); EasyMock.replay(serverInventoryView); + mockCoordinatorRuntimeParams(); + coordinator.moveSegment( + coordinatorRuntimeParams, druidServer.toImmutableDruidServer(), druidServer2.toImmutableDruidServer(), segment, @@ -295,11 +308,14 @@ public void testMoveSegment() LoadPeonCallback dropCallback = dropCallbackCapture.getValue(); dropCallback.execute(); - EasyMock.verify(druidServer); - EasyMock.verify(druidServer2); - EasyMock.verify(loadQueuePeon); - EasyMock.verify(serverInventoryView); - EasyMock.verify(metadataRuleManager); + EasyMock.verify(druidServer, druidServer2, loadQueuePeon, serverInventoryView, metadataRuleManager); + EasyMock.verify(coordinatorRuntimeParams); + } + + private void mockCoordinatorRuntimeParams() + { + EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(this.dataSourcesSnapshot).anyTimes(); + EasyMock.replay(coordinatorRuntimeParams); } @Test(timeout = 60_000L) @@ -538,14 +554,30 @@ private void setupMetadataSegmentManagerMock(DruidDataSource dataSource) .andReturn(dataSource.getSegments()) .anyTimes(); EasyMock - .expect(segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()) + .expect(segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments()) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .anyTimes(); + DataSourcesSnapshot dataSourcesSnapshot = + new DataSourcesSnapshot(ImmutableMap.of(dataSource.getName(), dataSource.toImmutableDruidDataSource())); + EasyMock + .expect(segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments()) + .andReturn(dataSourcesSnapshot) + .anyTimes(); EasyMock .expect(segmentsMetadata.retrieveAllDataSourceNames()) .andReturn(Collections.singleton(dataSource.getName())) .anyTimes(); EasyMock.replay(segmentsMetadata); + + EasyMock + .expect(this.dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) + .andReturn(dataSource.getSegments()) + .anyTimes(); + EasyMock + .expect(this.dataSourcesSnapshot.getDataSourcesWithAllUsedSegments()) + .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) + .anyTimes(); + EasyMock.replay(this.dataSourcesSnapshot); } @Nullable diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 8dc9d112136e..0eb8b39ccc0b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -21,7 +21,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; @@ -49,7 +51,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; @@ -124,34 +125,19 @@ public int getTotalWorkerCapacity() @Before public void setup() { - dataSources = new HashMap<>(); + List segments = new ArrayList<>(); for (int i = 0; i < 3; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - - for (int j = 0; j < 4; j++) { - for (int k = 0; k < 2; k++) { - DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - } - } - - for (int j = 7; j < 9; j++) { + for (int j : new int[] {0, 1, 2, 3, 7, 8}) { for (int k = 0; k < 2; k++) { - DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + segments.add(createSegment(dataSource, j, true, k)); + segments.add(createSegment(dataSource, j, false, k)); } } - - dataSources.put(dataSource, timeline); } + dataSources = DataSourcesSnapshot + .fromUsedSegments(segments, ImmutableMap.of()) + .getUsedSegmentsTimelinesPerDataSource(); } private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) @@ -264,7 +250,7 @@ private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor { DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() - .setDataSourcesWithUsedSegments(dataSources) + .withUsedSegmentsTimelinesPerDataSourceInTest(dataSources) .withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs())) .build(); return compactor.run(params).getCoordinatorStats(); @@ -292,10 +278,7 @@ private void assertCompactSegments( long numDataSourceOfExpectedRemainingSegments = stats .getDataSources(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT) .stream() - .mapToLong(dataSource -> stats.getDataSourceStat( - DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT, - dataSource) - ) + .mapToLong(ds -> stats.getDataSourceStat(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT, ds)) .filter(stat -> stat == expectedRemainingSegments) .count(); Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java index 9c5b9b4e2759..b28bd7faf98f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java @@ -625,10 +625,7 @@ private static VersionedIntervalTimeline createTimeline( SegmentGenerateSpec... specs ) { - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - + List segments = new ArrayList<>(); final String version = DateTimes.nowUtc().toString(); final List orderedSpecs = Arrays.asList(specs); @@ -659,18 +656,14 @@ private static VersionedIntervalTimeline createTimeline( 0, spec.segmentSize ); - timeline.add( - segmentInterval, - version, - shardSpec.createChunk(segment) - ); + segments.add(segment); } remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval); } } - return timeline; + return VersionedIntervalTimeline.forSegments(segments); } private DataSourceCompactionConfig createCompactionConfig( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java index c4a2c6900b7f..3b6607d208e2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java @@ -81,7 +81,7 @@ public DruidExpression toDruidExpression( (String) lookupNameExpr.getLiteralValue(), false, null, - false, + null, true ) ); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 0303d582d2df..0b171bdc6feb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -364,12 +364,13 @@ private Iterator getAuthorizedPublishedSegments( final AuthenticationResult authenticationResult = (AuthenticationResult) root.get(PlannerContext.DATA_CTX_AUTHENTICATION_RESULT); - final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( - authenticationResult, - () -> it, - SEGMENT_WITH_OVERSHADOWED_STATUS_RA_GENERATOR, - authorizerMapper - ); + final Iterable authorizedSegments = AuthorizationUtils + .filterAuthorizedResources( + authenticationResult, + () -> it, + SEGMENT_WITH_OVERSHADOWED_STATUS_RA_GENERATOR, + authorizerMapper + ); return authorizedSegments.iterator(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index b130511268ca..1f1e0b4dd318 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -5887,7 +5887,7 @@ public void testFilterAndGroupByLookup() throws Exception "lookyloo", false, null, - false, + null, true ); @@ -5940,7 +5940,7 @@ public void testCountDistinctOfLookup() throws Exception "lookyloo", false, null, - false, + null, true ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index af0e89454328..6aac2ce2eab1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -78,11 +78,8 @@ public SpecificSegmentsQuerySegmentWalker add( ) { final Segment segment = new QueryableIndexSegment(index, descriptor.getId()); - if (!timelines.containsKey(descriptor.getDataSource())) { - timelines.put(descriptor.getDataSource(), new VersionedIntervalTimeline<>(Ordering.natural())); - } - - final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); + final VersionedIntervalTimeline timeline = timelines + .computeIfAbsent(descriptor.getDataSource(), dsName -> new VersionedIntervalTimeline<>(Ordering.natural())); timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment)); segments.add(descriptor); closeables.add(index); diff --git a/web-console/script/mkcomp b/web-console/script/mkcomp index a2b9bb59182a..5c60a80af239 100755 --- a/web-console/script/mkcomp +++ b/web-console/script/mkcomp @@ -17,24 +17,20 @@ * limitations under the License. */ -let fs = require('fs-extra'); +const fs = require('fs-extra'); if (!(process.argv.length === 3 || process.argv.length === 4)) { - console.log('Usage: mkcomp '); + console.log('Usage: mkcomp '); process.exit(); } let name; -let what; +let where; if (process.argv.length === 4) { - what = process.argv[2]; + where = process.argv[2]; name = process.argv[3]; - if (!(what === 'component' || what === 'dialog' || what === 'singleton')) { - console.log(`Bad what, should be on of: component, dialog, singleton`); - process.exit(); - } } else { - what = 'component'; + where = 'components'; name = process.argv[2]; } @@ -43,7 +39,7 @@ if (!/^([a-z-])+$/.test(name)) { process.exit(); } -let path = `./src/${what}s/${name}/`; +const path = `./src/${where}/${name}/`; fs.ensureDirSync(path); console.log('Making path:', path); @@ -85,7 +81,7 @@ writeFile(path + name + '.tsx', import { Button, InputGroup } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; import './${name}.scss'; @@ -95,7 +91,7 @@ export interface ${camelName}Props extends React.Props { export interface ${camelName}State { } -export class ${camelName} extends React.Component<${camelName}Props, ${camelName}State> { +export class ${camelName} extends React.PureComponent<${camelName}Props, ${camelName}State> { constructor(props: ${camelName}Props, context: any) { super(props, context); // this.state = {}; @@ -155,16 +151,17 @@ writeFile(path + name + '.spec.tsx', * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ${camelName} } from './${name}'; describe('${spaceName}', () => { - it('action cell snapshot', () => { + it('matches snapshot', () => { const ${snakeName} = <${camelName} />; - const { container, getByText } = render(${snakeName}); + + const { container } = render(${snakeName}); expect(container.firstChild).toMatchSnapshot(); }); }); diff --git a/web-console/src/bootstrap/react-table-custom-pagination.tsx b/web-console/src/bootstrap/react-table-custom-pagination.tsx index ee5f12f76373..a209989c7090 100644 --- a/web-console/src/bootstrap/react-table-custom-pagination.tsx +++ b/web-console/src/bootstrap/react-table-custom-pagination.tsx @@ -17,7 +17,7 @@ */ import { Button, Classes, HTMLSelect } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import './react-table-custom-pagination.scss'; @@ -44,7 +44,7 @@ interface ReactTableCustomPaginationState { page: string | number; } -export class ReactTableCustomPagination extends React.Component { +export class ReactTableCustomPagination extends React.PureComponent { constructor(props: ReactTableCustomPaginationProps) { super(props); diff --git a/web-console/src/bootstrap/react-table-defaults.tsx b/web-console/src/bootstrap/react-table-defaults.tsx index d68912b44c09..b39d6ec73676 100644 --- a/web-console/src/bootstrap/react-table-defaults.tsx +++ b/web-console/src/bootstrap/react-table-defaults.tsx @@ -16,17 +16,17 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { Filter, ReactTableDefaults } from 'react-table'; -import { Loader } from '../components/loader/loader'; +import { Loader } from '../components'; import { booleanCustomTableFilter, countBy, makeTextFilter } from '../utils'; import { ReactTableCustomPagination } from './react-table-custom-pagination'; /* tslint:disable:max-classes-per-file */ -class NoData extends React.Component { +class NoData extends React.PureComponent { render() { const { children } = this.props; if (!children) return null; diff --git a/web-console/src/components/action-cell/action-cell.spec.tsx b/web-console/src/components/action-cell/action-cell.spec.tsx index 35368942c63c..71639d296989 100644 --- a/web-console/src/components/action-cell/action-cell.spec.tsx +++ b/web-console/src/components/action-cell/action-cell.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ActionCell } from './action-cell'; diff --git a/web-console/src/components/action-cell/action-cell.tsx b/web-console/src/components/action-cell/action-cell.tsx index b4448d2da68a..1daa5df904db 100644 --- a/web-console/src/components/action-cell/action-cell.tsx +++ b/web-console/src/components/action-cell/action-cell.tsx @@ -18,7 +18,7 @@ import { Popover, Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import * as React from 'react'; +import React from 'react'; import { BasicAction, basicActionsToMenu } from '../../utils/basic-action'; import { ActionIcon } from '../action-icon/action-icon'; @@ -30,7 +30,7 @@ export interface ActionCellProps extends React.Props { actions?: BasicAction[]; } -export class ActionCell extends React.Component { +export class ActionCell extends React.PureComponent { static COLUMN_ID = 'actions'; static COLUMN_LABEL = 'Actions'; static COLUMN_WIDTH = 70; diff --git a/web-console/src/components/action-icon/action-icon.spec.tsx b/web-console/src/components/action-icon/action-icon.spec.tsx index 4e74d8be04be..3e07523b5c14 100644 --- a/web-console/src/components/action-icon/action-icon.spec.tsx +++ b/web-console/src/components/action-icon/action-icon.spec.tsx @@ -17,7 +17,7 @@ */ import { IconNames } from '@blueprintjs/icons'; -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ActionIcon } from './action-icon'; diff --git a/web-console/src/components/action-icon/action-icon.tsx b/web-console/src/components/action-icon/action-icon.tsx index 9742f44851eb..eb67bfa49ab8 100644 --- a/web-console/src/components/action-icon/action-icon.tsx +++ b/web-console/src/components/action-icon/action-icon.tsx @@ -18,7 +18,7 @@ import { Icon, IconName } from '@blueprintjs/core'; import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; import './action-icon.scss'; @@ -28,7 +28,7 @@ export interface ActionIconProps extends React.Props { onClick?: () => void; } -export class ActionIcon extends React.Component { +export class ActionIcon extends React.PureComponent { render() { const { className, icon, onClick } = this.props; diff --git a/web-console/src/components/array-input/array-input.spec.tsx b/web-console/src/components/array-input/array-input.spec.tsx index c2a39afed8aa..ca84d0e04b7c 100644 --- a/web-console/src/components/array-input/array-input.spec.tsx +++ b/web-console/src/components/array-input/array-input.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ArrayInput } from './array-input'; diff --git a/web-console/src/components/array-input/array-input.tsx b/web-console/src/components/array-input/array-input.tsx index 6b2bab96030e..6e9f7762b215 100644 --- a/web-console/src/components/array-input/array-input.tsx +++ b/web-console/src/components/array-input/array-input.tsx @@ -16,8 +16,8 @@ * limitations under the License. */ -import { ITagInputProps, TextArea } from '@blueprintjs/core'; -import * as React from 'react'; +import { TextArea } from '@blueprintjs/core'; +import React from 'react'; export interface ArrayInputProps { className?: string; @@ -28,7 +28,7 @@ export interface ArrayInputProps { disabled?: boolean; } -export class ArrayInput extends React.Component { +export class ArrayInput extends React.PureComponent { constructor(props: ArrayInputProps) { super(props); this.state = { diff --git a/web-console/src/components/auto-form/auto-form.spec.tsx b/web-console/src/components/auto-form/auto-form.spec.tsx index cee3d8debcc7..7fb96f369e2b 100644 --- a/web-console/src/components/auto-form/auto-form.spec.tsx +++ b/web-console/src/components/auto-form/auto-form.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { AutoForm } from './auto-form'; diff --git a/web-console/src/components/auto-form/auto-form.tsx b/web-console/src/components/auto-form/auto-form.tsx index 6a54af721a2f..248c1230a292 100644 --- a/web-console/src/components/auto-form/auto-form.tsx +++ b/web-console/src/components/auto-form/auto-form.tsx @@ -29,7 +29,7 @@ import { Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import * as React from 'react'; +import React from 'react'; import { deepDelete, deepGet, deepSet } from '../../utils/object-change'; import { ArrayInput } from '../array-input/array-input'; @@ -68,7 +68,7 @@ export interface AutoFormState { jsonInputsValidity: any; } -export class AutoForm> extends React.Component, AutoFormState> { +export class AutoForm> extends React.PureComponent, AutoFormState> { static makeLabelName(label: string): string { let newLabel = label.split(/(?=[A-Z])/).join(' ').toLowerCase().replace(/\./g, ' '); newLabel = newLabel[0].toUpperCase() + newLabel.slice(1); @@ -173,11 +173,16 @@ export class AutoForm> extends React.Component : undefined; + const modalValue = deepGet(model as any, field.name); return { - const v = e.target.value; - this.fieldChange(field, v === '' ? undefined : (sanitize ? sanitize(v) : v)); + let v = e.target.value; + if (sanitize) v = sanitize(v); + this.fieldChange(field, v); + }} + onBlur={() => { + if (modalValue === '') this.fieldChange(field, undefined); }} placeholder={field.placeholder} rightElement={ diff --git a/web-console/src/components/center-message/center-message.spec.tsx b/web-console/src/components/center-message/center-message.spec.tsx index 9cbcebbedee7..f8e039f842ff 100644 --- a/web-console/src/components/center-message/center-message.spec.tsx +++ b/web-console/src/components/center-message/center-message.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { CenterMessage } from './center-message'; diff --git a/web-console/src/components/center-message/center-message.tsx b/web-console/src/components/center-message/center-message.tsx index 89a83f8c3209..bae0d89dfe94 100644 --- a/web-console/src/components/center-message/center-message.tsx +++ b/web-console/src/components/center-message/center-message.tsx @@ -16,14 +16,14 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import './center-message.scss'; export interface CenterMessageProps extends React.Props { } -export class CenterMessage extends React.Component { +export class CenterMessage extends React.PureComponent { render() { return
diff --git a/web-console/src/components/clearable-input/clearable-input.spec.tsx b/web-console/src/components/clearable-input/clearable-input.spec.tsx index ca26f6e49a15..2f1a26795b57 100644 --- a/web-console/src/components/clearable-input/clearable-input.spec.tsx +++ b/web-console/src/components/clearable-input/clearable-input.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ClearableInput } from './clearable-input'; diff --git a/web-console/src/components/clearable-input/clearable-input.tsx b/web-console/src/components/clearable-input/clearable-input.tsx index da4df344ed4d..b695bbbe6393 100644 --- a/web-console/src/components/clearable-input/clearable-input.tsx +++ b/web-console/src/components/clearable-input/clearable-input.tsx @@ -19,7 +19,7 @@ import { Button, InputGroup } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; export interface ClearableInputProps extends React.Props { className?: string; @@ -28,7 +28,7 @@ export interface ClearableInputProps extends React.Props { placeholder: string; } -export class ClearableInput extends React.Component { +export class ClearableInput extends React.PureComponent { render() { const { className, value, onChange, placeholder } = this.props; diff --git a/web-console/src/components/external-link/external-link.spec.tsx b/web-console/src/components/external-link/external-link.spec.tsx index 235e7ada1d48..c6e9be0d1124 100644 --- a/web-console/src/components/external-link/external-link.spec.tsx +++ b/web-console/src/components/external-link/external-link.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ExternalLink } from './external-link'; diff --git a/web-console/src/components/external-link/external-link.tsx b/web-console/src/components/external-link/external-link.tsx index 0a9c10200fed..9dc0562f65bf 100644 --- a/web-console/src/components/external-link/external-link.tsx +++ b/web-console/src/components/external-link/external-link.tsx @@ -16,13 +16,13 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; export interface ExternalLinkProps extends React.Props { href: string; } -export class ExternalLink extends React.Component { +export class ExternalLink extends React.PureComponent { render() { const { href, children } = this.props; diff --git a/web-console/src/components/header-bar/header-bar.spec.tsx b/web-console/src/components/header-bar/header-bar.spec.tsx index 0cb4ef0b0f0d..d93ec8fa2129 100644 --- a/web-console/src/components/header-bar/header-bar.spec.tsx +++ b/web-console/src/components/header-bar/header-bar.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { HeaderBar } from './header-bar'; diff --git a/web-console/src/components/header-bar/header-bar.tsx b/web-console/src/components/header-bar/header-bar.tsx index 6792db2e5d0a..f48fab3503f2 100644 --- a/web-console/src/components/header-bar/header-bar.tsx +++ b/web-console/src/components/header-bar/header-bar.tsx @@ -30,13 +30,11 @@ import { Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; import { AboutDialog } from '../../dialogs/about-dialog/about-dialog'; import { CoordinatorDynamicConfigDialog } from '../../dialogs/coordinator-dynamic-config/coordinator-dynamic-config'; import { OverlordDynamicConfigDialog } from '../../dialogs/overlord-dynamic-config/overlord-dynamic-config'; -import { getWikipediaSpec } from '../../utils/example-ingestion-spec'; import { DRUID_DOCS, DRUID_GITHUB, @@ -60,7 +58,7 @@ export interface HeaderBarState { overlordDynamicConfigDialogOpen: boolean; } -export class HeaderBar extends React.Component { +export class HeaderBar extends React.PureComponent { constructor(props: HeaderBarProps) { super(props); this.state = { diff --git a/web-console/src/components/index.ts b/web-console/src/components/index.ts index 6a706a7305db..063875a14954 100644 --- a/web-console/src/components/index.ts +++ b/web-console/src/components/index.ts @@ -30,7 +30,6 @@ export * from './table-cell/table-cell'; export * from './rule-editor/rule-editor'; export * from './show-json/show-json'; export * from './show-log/show-log'; -export * from './sql-control/sql-control'; export * from './table-column-selector/table-column-selector'; export * from './view-control-bar/view-control-bar'; export * from './clearable-input/clearable-input'; diff --git a/web-console/src/components/json-collapse/json-collapse.spec.tsx b/web-console/src/components/json-collapse/json-collapse.spec.tsx index 680b8a43c94d..4fbdfa66d15b 100644 --- a/web-console/src/components/json-collapse/json-collapse.spec.tsx +++ b/web-console/src/components/json-collapse/json-collapse.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { JSONCollapse } from './json-collapse'; diff --git a/web-console/src/components/json-collapse/json-collapse.tsx b/web-console/src/components/json-collapse/json-collapse.tsx index d80035246207..b0700d03c92b 100644 --- a/web-console/src/components/json-collapse/json-collapse.tsx +++ b/web-console/src/components/json-collapse/json-collapse.tsx @@ -18,7 +18,7 @@ import { Button, Collapse, TextArea } from '@blueprintjs/core'; import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; interface JSONCollapseProps extends React.Props { stringValue: string; @@ -29,7 +29,7 @@ interface JSONCollapseState { isOpen: boolean; } -export class JSONCollapse extends React.Component { +export class JSONCollapse extends React.PureComponent { constructor(props: any) { super(props); this.state = { diff --git a/web-console/src/components/json-input/json-input.spec.tsx b/web-console/src/components/json-input/json-input.spec.tsx index 134ca8314ec2..8d6e4f425f5a 100644 --- a/web-console/src/components/json-input/json-input.spec.tsx +++ b/web-console/src/components/json-input/json-input.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { JSONInput } from './json-input'; diff --git a/web-console/src/components/json-input/json-input.tsx b/web-console/src/components/json-input/json-input.tsx index 8fa8163b6184..489b971a4368 100644 --- a/web-console/src/components/json-input/json-input.tsx +++ b/web-console/src/components/json-input/json-input.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import AceEditor from 'react-ace'; import { parseStringToJSON, stringifyJSON, validJson } from '../../utils'; @@ -34,7 +34,7 @@ interface JSONInputState { stringValue: string; } -export class JSONInput extends React.Component { +export class JSONInput extends React.PureComponent { constructor(props: JSONInputProps) { super(props); this.state = { diff --git a/web-console/src/components/loader/loader.spec.tsx b/web-console/src/components/loader/loader.spec.tsx index 85354aa662ca..022db9b75680 100644 --- a/web-console/src/components/loader/loader.spec.tsx +++ b/web-console/src/components/loader/loader.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { Loader } from './loader'; diff --git a/web-console/src/components/loader/loader.tsx b/web-console/src/components/loader/loader.tsx index 3a122b9df3b9..d536337eba0c 100644 --- a/web-console/src/components/loader/loader.tsx +++ b/web-console/src/components/loader/loader.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import './loader.scss'; @@ -28,7 +28,7 @@ export interface LoaderProps extends React.Props { export interface LoaderState { } -export class Loader extends React.Component { +export class Loader extends React.PureComponent { render() { const { loadingText, loading } = this.props; diff --git a/web-console/src/components/menu-checkbox/menu-checkbox.spec.tsx b/web-console/src/components/menu-checkbox/menu-checkbox.spec.tsx index f228ee7daf12..c25cb7cc7d75 100644 --- a/web-console/src/components/menu-checkbox/menu-checkbox.spec.tsx +++ b/web-console/src/components/menu-checkbox/menu-checkbox.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { MenuCheckbox } from './menu-checkbox'; diff --git a/web-console/src/components/menu-checkbox/menu-checkbox.tsx b/web-console/src/components/menu-checkbox/menu-checkbox.tsx index 21b438b8a3c2..f5ea252755d5 100644 --- a/web-console/src/components/menu-checkbox/menu-checkbox.tsx +++ b/web-console/src/components/menu-checkbox/menu-checkbox.tsx @@ -17,11 +17,11 @@ */ import { Checkbox, ICheckboxProps } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import './menu-checkbox.scss'; -export class MenuCheckbox extends React.Component { +export class MenuCheckbox extends React.PureComponent { render() { return
  • diff --git a/web-console/src/components/rule-editor/rule-editor.spec.tsx b/web-console/src/components/rule-editor/rule-editor.spec.tsx index 54330c177f60..acdade53b8ea 100644 --- a/web-console/src/components/rule-editor/rule-editor.spec.tsx +++ b/web-console/src/components/rule-editor/rule-editor.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { Rule, RuleEditor } from './rule-editor'; diff --git a/web-console/src/components/rule-editor/rule-editor.tsx b/web-console/src/components/rule-editor/rule-editor.tsx index 7ba88366fbf9..dfffb0aae004 100644 --- a/web-console/src/components/rule-editor/rule-editor.tsx +++ b/web-console/src/components/rule-editor/rule-editor.tsx @@ -19,7 +19,7 @@ import { Button, Card, Collapse, ControlGroup, FormGroup, HTMLSelect, InputGroup, NumericInput, TagInput } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import './rule-editor.scss'; @@ -47,7 +47,7 @@ export interface RuleEditorState { isOpen: boolean; } -export class RuleEditor extends React.Component { +export class RuleEditor extends React.PureComponent { static ruleToString(rule: Rule): string { return rule.type + (rule.period ? `(${rule.period})` : '') + (rule.interval ? `(${rule.interval})` : ''); } diff --git a/web-console/src/components/show-json/show-json.spec.tsx b/web-console/src/components/show-json/show-json.spec.tsx index 8e002e0f49f4..d2451bdf87a8 100644 --- a/web-console/src/components/show-json/show-json.spec.tsx +++ b/web-console/src/components/show-json/show-json.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ShowJson } from './show-json'; diff --git a/web-console/src/components/show-json/show-json.tsx b/web-console/src/components/show-json/show-json.tsx index 477588a66e49..16454259bc64 100644 --- a/web-console/src/components/show-json/show-json.tsx +++ b/web-console/src/components/show-json/show-json.tsx @@ -18,8 +18,8 @@ import { Button, ButtonGroup, InputGroup, Intent, TextArea } from '@blueprintjs/core'; import axios from 'axios'; -import * as copy from 'copy-to-clipboard'; -import * as React from 'react'; +import copy from 'copy-to-clipboard'; +import React from 'react'; import { AppToaster } from '../../singletons/toaster'; import { UrlBaser } from '../../singletons/url-baser'; @@ -29,6 +29,7 @@ import './show-json.scss'; export interface ShowJsonProps extends React.Props { endpoint: string; + transform?: (x: any) => any; downloadFilename?: string; } @@ -36,7 +37,7 @@ export interface ShowJsonState { jsonValue: string; } -export class ShowJson extends React.Component { +export class ShowJson extends React.PureComponent { constructor(props: ShowJsonProps, context: any) { super(props, context); this.state = { @@ -47,12 +48,14 @@ export class ShowJson extends React.Component { } private getJsonInfo = async (): Promise => { - const { endpoint } = this.props; + const { endpoint, transform } = this.props; + try { const resp = await axios.get(endpoint); - const data = resp.data; + let data = resp.data; + if (transform) data = transform(data); this.setState({ - jsonValue: typeof (data) === 'string' ? data : JSON.stringify(data, undefined, 2) + jsonValue: typeof data === 'string' ? data : JSON.stringify(data, undefined, 2) }); } catch (e) { this.setState({ diff --git a/web-console/src/components/show-log/show-log.spec.tsx b/web-console/src/components/show-log/show-log.spec.tsx index 878125212f62..d4ed5aa97838 100644 --- a/web-console/src/components/show-log/show-log.spec.tsx +++ b/web-console/src/components/show-log/show-log.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ShowLog } from './show-log'; diff --git a/web-console/src/components/show-log/show-log.tsx b/web-console/src/components/show-log/show-log.tsx index 3549a2f5e190..1db9c8b29fa2 100644 --- a/web-console/src/components/show-log/show-log.tsx +++ b/web-console/src/components/show-log/show-log.tsx @@ -18,8 +18,8 @@ import { Button, ButtonGroup, Checkbox, InputGroup, Intent, TextArea } from '@blueprintjs/core'; import axios from 'axios'; -import * as copy from 'copy-to-clipboard'; -import * as React from 'react'; +import copy from 'copy-to-clipboard'; +import React from 'react'; import { AppToaster } from '../../singletons/toaster'; import { UrlBaser } from '../../singletons/url-baser'; @@ -46,7 +46,7 @@ export interface ShowLogState { tail: boolean; } -export class ShowLog extends React.Component { +export class ShowLog extends React.PureComponent { public log = React.createRef(); constructor(props: ShowLogProps, context: any) { diff --git a/web-console/src/components/table-cell/__snapshots__/table-cell.spec.tsx.snap b/web-console/src/components/table-cell/__snapshots__/table-cell.spec.tsx.snap index 48d579dab941..d1cf2ffaabfe 100644 --- a/web-console/src/components/table-cell/__snapshots__/table-cell.spec.tsx.snap +++ b/web-console/src/components/table-cell/__snapshots__/table-cell.spec.tsx.snap @@ -43,6 +43,14 @@ exports[`table cell matches snapshot null 1`] = ` `; +exports[`table cell matches snapshot null timestamp 1`] = ` + + unparseable timestamp + +`; + exports[`table cell matches snapshot simple 1`] = `Hello World`; exports[`table cell matches snapshot truncate 1`] = ` diff --git a/web-console/src/components/table-cell/table-cell.spec.tsx b/web-console/src/components/table-cell/table-cell.spec.tsx index 821119a458cf..2d7e598d1891 100644 --- a/web-console/src/components/table-cell/table-cell.spec.tsx +++ b/web-console/src/components/table-cell/table-cell.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { TableCell } from './table-cell'; @@ -33,6 +33,17 @@ describe('table cell', () => { expect(container.firstChild).toMatchSnapshot(); }); + it('matches snapshot null timestamp', () => { + const tableCell = ; + + const { container } = render(tableCell); + expect(container.firstChild).toMatchSnapshot(); + }); + it('matches snapshot simple', () => { const tableCell = { +export class TableCell extends React.PureComponent { static MAX_CHARS_TO_SHOW = 50; static possiblyTruncate(str: string): React.ReactNode { @@ -89,7 +89,11 @@ export class TableCell extends React.Component { return TableCell.possiblyTruncate(String(value)); } } else { - return null; + if (timestamp) { + return unparseable timestamp; + } else { + return null; + } } } } diff --git a/web-console/src/components/table-column-selector/table-column-selector.spec.tsx b/web-console/src/components/table-column-selector/table-column-selector.spec.tsx index e6e7c154cd72..8084701088ad 100644 --- a/web-console/src/components/table-column-selector/table-column-selector.spec.tsx +++ b/web-console/src/components/table-column-selector/table-column-selector.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { TableColumnSelector } from './table-column-selector'; diff --git a/web-console/src/components/table-column-selector/table-column-selector.tsx b/web-console/src/components/table-column-selector/table-column-selector.tsx index 718f90dae553..fcbdb23a6162 100644 --- a/web-console/src/components/table-column-selector/table-column-selector.tsx +++ b/web-console/src/components/table-column-selector/table-column-selector.tsx @@ -18,7 +18,7 @@ import { Button, Checkbox, FormGroup, Menu, Popover, Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import * as React from 'react'; +import React from 'react'; import { MenuCheckbox } from '../menu-checkbox/menu-checkbox'; @@ -34,7 +34,7 @@ interface TableColumnSelectorState { } -export class TableColumnSelector extends React.Component { +export class TableColumnSelector extends React.PureComponent { constructor(props: TableColumnSelectorProps) { super(props); diff --git a/web-console/src/components/view-control-bar/view-control-bar.spec.tsx b/web-console/src/components/view-control-bar/view-control-bar.spec.tsx index 410868036efb..506366b9ebef 100644 --- a/web-console/src/components/view-control-bar/view-control-bar.spec.tsx +++ b/web-console/src/components/view-control-bar/view-control-bar.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { ViewControlBar } from './view-control-bar'; diff --git a/web-console/src/components/view-control-bar/view-control-bar.tsx b/web-console/src/components/view-control-bar/view-control-bar.tsx index 679c671c21d8..247004cc3835 100644 --- a/web-console/src/components/view-control-bar/view-control-bar.tsx +++ b/web-console/src/components/view-control-bar/view-control-bar.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import './view-control-bar.scss'; @@ -24,7 +24,7 @@ export interface ViewControlBarProps { label: string; } -export class ViewControlBar extends React.Component { +export class ViewControlBar extends React.PureComponent { constructor(props: ViewControlBarProps) { super(props); } diff --git a/web-console/src/console-application.tsx b/web-console/src/console-application.tsx index 582dc8e41341..084fd44c8478 100644 --- a/web-console/src/console-application.tsx +++ b/web-console/src/console-application.tsx @@ -19,8 +19,8 @@ import { Intent } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as classNames from 'classnames'; -import * as React from 'react'; +import classNames from 'classnames'; +import React from 'react'; import { HashRouter, Route, Switch } from 'react-router-dom'; import { ExternalLink } from './components/external-link/external-link'; @@ -54,7 +54,7 @@ export interface ConsoleApplicationState { capabilitiesLoading: boolean; } -export class ConsoleApplication extends React.Component { +export class ConsoleApplication extends React.PureComponent { static MESSAGE_KEY = 'druid-console-message'; static MESSAGE_DISMISSED = 'dismissed'; private capabilitiesQueryManager: QueryManager; diff --git a/web-console/src/dialogs/about-dialog/about-dialog.spec.tsx b/web-console/src/dialogs/about-dialog/about-dialog.spec.tsx index 0df9793dc9b6..46c33f8bddf1 100644 --- a/web-console/src/dialogs/about-dialog/about-dialog.spec.tsx +++ b/web-console/src/dialogs/about-dialog/about-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { AboutDialog } from './about-dialog'; diff --git a/web-console/src/dialogs/about-dialog/about-dialog.tsx b/web-console/src/dialogs/about-dialog/about-dialog.tsx index e2b957747f38..e8c87962c099 100644 --- a/web-console/src/dialogs/about-dialog/about-dialog.tsx +++ b/web-console/src/dialogs/about-dialog/about-dialog.tsx @@ -18,7 +18,7 @@ import { AnchorButton, Button, Classes, Dialog, Intent } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; -import * as React from 'react'; +import React from 'react'; import { ExternalLink } from '../../components'; import { DRUID_COMMUNITY, DRUID_DEVELOPER_GROUP, DRUID_USER_GROUP, DRUID_WEBSITE } from '../../variables'; @@ -30,7 +30,7 @@ export interface AboutDialogProps extends React.Props { export interface AboutDialogState { } -export class AboutDialog extends React.Component { +export class AboutDialog extends React.PureComponent { constructor(props: AboutDialogProps) { super(props); this.state = {}; diff --git a/web-console/src/dialogs/async-action-dialog/async-action-dialog.spec.tsx b/web-console/src/dialogs/async-action-dialog/async-action-dialog.spec.tsx index 63452cce5645..27e0dea697e9 100644 --- a/web-console/src/dialogs/async-action-dialog/async-action-dialog.spec.tsx +++ b/web-console/src/dialogs/async-action-dialog/async-action-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { AsyncActionDialog } from './async-action-dialog'; diff --git a/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx b/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx index 9a00ff53abc3..003adea34be4 100644 --- a/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx +++ b/web-console/src/dialogs/async-action-dialog/async-action-dialog.tsx @@ -26,7 +26,7 @@ import { } from '@blueprintjs/core'; import { IconName } from '@blueprintjs/icons'; import classNames from 'classnames'; -import * as React from 'react'; +import React from 'react'; import { AppToaster } from '../../singletons/toaster'; @@ -47,7 +47,7 @@ export interface AsyncAlertDialogState { working: boolean; } -export class AsyncActionDialog extends React.Component { +export class AsyncActionDialog extends React.PureComponent { constructor(props: AsyncAlertDialogProps) { super(props); this.state = { diff --git a/web-console/src/dialogs/compaction-dialog/__snapshots__/compaction-dialog.spec.tsx.snap b/web-console/src/dialogs/compaction-dialog/__snapshots__/compaction-dialog.spec.tsx.snap index 13832308057c..5e8951ccc379 100644 --- a/web-console/src/dialogs/compaction-dialog/__snapshots__/compaction-dialog.spec.tsx.snap +++ b/web-console/src/dialogs/compaction-dialog/__snapshots__/compaction-dialog.spec.tsx.snap @@ -352,7 +352,7 @@ exports[`compaction dialog matches snapshot 1`] = ` class="bp3-form-content" >
    @@ -543,7 +543,7 @@ exports[`compaction dialog matches snapshot 1`] = ` class="bp3-form-content" >
    diff --git a/web-console/src/dialogs/compaction-dialog/compaction-dialog.spec.tsx b/web-console/src/dialogs/compaction-dialog/compaction-dialog.spec.tsx index b17e2f6750f9..187a07f26e37 100644 --- a/web-console/src/dialogs/compaction-dialog/compaction-dialog.spec.tsx +++ b/web-console/src/dialogs/compaction-dialog/compaction-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { CompactionDialog } from './compaction-dialog'; diff --git a/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx b/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx index ac39ce2bb11a..a19ef0b6ad04 100644 --- a/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx +++ b/web-console/src/dialogs/compaction-dialog/compaction-dialog.tsx @@ -17,7 +17,7 @@ */ import { Button, Classes, Dialog, Intent } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import { AutoForm } from '../../components'; @@ -36,7 +36,7 @@ export interface CompactionDialogState { allJSONValid: boolean; } -export class CompactionDialog extends React.Component { +export class CompactionDialog extends React.PureComponent { constructor(props: CompactionDialogProps) { super(props); this.state = { diff --git a/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.spec.tsx b/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.spec.tsx index 273eeb49d0a6..97a23b73f2fa 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.spec.tsx +++ b/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { CoordinatorDynamicConfigDialog } from './coordinator-dynamic-config'; diff --git a/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.tsx b/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.tsx index 17d3b4f0ba5a..ab11e09f7087 100644 --- a/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.tsx +++ b/web-console/src/dialogs/coordinator-dynamic-config/coordinator-dynamic-config.tsx @@ -19,7 +19,7 @@ import { Intent } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import { AutoForm, ExternalLink } from '../../components'; import { AppToaster } from '../../singletons/toaster'; @@ -37,7 +37,7 @@ export interface CoordinatorDynamicConfigDialogState { historyRecords: any[]; } -export class CoordinatorDynamicConfigDialog extends React.Component { +export class CoordinatorDynamicConfigDialog extends React.PureComponent { private historyQueryManager: QueryManager; constructor(props: CoordinatorDynamicConfigDialogProps) { diff --git a/web-console/src/dialogs/history-dialog/history-dialog.spec.tsx b/web-console/src/dialogs/history-dialog/history-dialog.spec.tsx index f9e5d9f6bd10..73b54bb9f068 100644 --- a/web-console/src/dialogs/history-dialog/history-dialog.spec.tsx +++ b/web-console/src/dialogs/history-dialog/history-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { HistoryDialog } from './history-dialog'; diff --git a/web-console/src/dialogs/history-dialog/history-dialog.tsx b/web-console/src/dialogs/history-dialog/history-dialog.tsx index c20907e47651..7e2f460a339a 100644 --- a/web-console/src/dialogs/history-dialog/history-dialog.tsx +++ b/web-console/src/dialogs/history-dialog/history-dialog.tsx @@ -17,7 +17,7 @@ */ import { Card, Dialog, Divider, IDialogProps } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import { JSONCollapse } from '../../components'; @@ -31,7 +31,7 @@ interface HistoryDialogState { } -export class HistoryDialog extends React.Component { +export class HistoryDialog extends React.PureComponent { constructor(props: HistoryDialogProps) { super(props); this.state = { diff --git a/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.spec.tsx b/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.spec.tsx index e31b652cd506..66bc05127f44 100644 --- a/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.spec.tsx +++ b/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { LookupEditDialog } from './lookup-edit-dialog'; diff --git a/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.tsx b/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.tsx index af32fa217b0e..6f59cdff4174 100644 --- a/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.tsx +++ b/web-console/src/dialogs/lookup-edit-dialog/lookup-edit-dialog.tsx @@ -17,7 +17,7 @@ */ import { Button, Classes, Dialog, FormGroup, HTMLSelect, InputGroup, Intent } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import AceEditor from 'react-ace'; import { validJson } from '../../utils'; @@ -40,7 +40,7 @@ export interface LookupEditDialogProps extends React.Props { export interface LookupEditDialogState { } -export class LookupEditDialog extends React.Component { +export class LookupEditDialog extends React.PureComponent { constructor(props: LookupEditDialogProps) { super(props); diff --git a/web-console/src/dialogs/overlord-dynamic-config/overload-dynamic-config.spec.tsx b/web-console/src/dialogs/overlord-dynamic-config/overload-dynamic-config.spec.tsx index ad61b05ec2d6..196aa5500d24 100644 --- a/web-console/src/dialogs/overlord-dynamic-config/overload-dynamic-config.spec.tsx +++ b/web-console/src/dialogs/overlord-dynamic-config/overload-dynamic-config.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { LookupEditDialog } from '../lookup-edit-dialog/lookup-edit-dialog'; diff --git a/web-console/src/dialogs/overlord-dynamic-config/overlord-dynamic-config.tsx b/web-console/src/dialogs/overlord-dynamic-config/overlord-dynamic-config.tsx index 112b0d49d64d..1e229a05bbc6 100644 --- a/web-console/src/dialogs/overlord-dynamic-config/overlord-dynamic-config.tsx +++ b/web-console/src/dialogs/overlord-dynamic-config/overlord-dynamic-config.tsx @@ -19,7 +19,7 @@ import { Intent } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import { AutoForm, ExternalLink } from '../../components'; import { AppToaster } from '../../singletons/toaster'; @@ -38,7 +38,7 @@ export interface OverlordDynamicConfigDialogState { historyRecords: any[]; } -export class OverlordDynamicConfigDialog extends React.Component { +export class OverlordDynamicConfigDialog extends React.PureComponent { private historyQueryManager: QueryManager; constructor(props: OverlordDynamicConfigDialogProps) { diff --git a/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.spec.tsx b/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.spec.tsx index 6fbdd55d7b93..088383f60733 100644 --- a/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.spec.tsx +++ b/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { QueryPlanDialog } from './query-plan-dialog'; diff --git a/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.tsx b/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.tsx index 57f9c7047dbb..aebfa5a95aa1 100644 --- a/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.tsx +++ b/web-console/src/dialogs/query-plan-dialog/query-plan-dialog.tsx @@ -17,7 +17,7 @@ */ import { Button, Classes, Dialog, FormGroup, InputGroup, TextArea } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import { BasicQueryExplanation, SemiJoinQueryExplanation } from '../../utils'; @@ -33,7 +33,7 @@ export interface QueryPlanDialogState { } -export class QueryPlanDialog extends React.Component { +export class QueryPlanDialog extends React.PureComponent { constructor(props: QueryPlanDialogProps) { super(props); diff --git a/web-console/src/dialogs/retention-dialog/retention-dialog.array.spec.ts b/web-console/src/dialogs/retention-dialog/retention-dialog.array.spec.ts index 7dd280b064ef..4fd3280f4dfd 100644 --- a/web-console/src/dialogs/retention-dialog/retention-dialog.array.spec.ts +++ b/web-console/src/dialogs/retention-dialog/retention-dialog.array.spec.ts @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { reorderArray } from './retention-dialog'; diff --git a/web-console/src/dialogs/retention-dialog/retention-dialog.spec.tsx b/web-console/src/dialogs/retention-dialog/retention-dialog.spec.tsx index beb60c4f646f..2e4aeaf5ff08 100644 --- a/web-console/src/dialogs/retention-dialog/retention-dialog.spec.tsx +++ b/web-console/src/dialogs/retention-dialog/retention-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { RetentionDialog } from './retention-dialog'; diff --git a/web-console/src/dialogs/retention-dialog/retention-dialog.tsx b/web-console/src/dialogs/retention-dialog/retention-dialog.tsx index bf78d21bb24d..564872889258 100644 --- a/web-console/src/dialogs/retention-dialog/retention-dialog.tsx +++ b/web-console/src/dialogs/retention-dialog/retention-dialog.tsx @@ -19,7 +19,7 @@ import { Button, FormGroup } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import { Rule, RuleEditor } from '../../components'; import { QueryManager } from '../../utils'; @@ -51,7 +51,7 @@ export interface RetentionDialogState { historyRecords: any[]; } -export class RetentionDialog extends React.Component { +export class RetentionDialog extends React.PureComponent { private historyQueryManager: QueryManager; constructor(props: RetentionDialogProps) { diff --git a/web-console/src/dialogs/snitch-dialog/snitch-dialog.spec.tsx b/web-console/src/dialogs/snitch-dialog/snitch-dialog.spec.tsx index 5ffef875ce4d..bb01327c3b45 100644 --- a/web-console/src/dialogs/snitch-dialog/snitch-dialog.spec.tsx +++ b/web-console/src/dialogs/snitch-dialog/snitch-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { SnitchDialog } from './snitch-dialog'; diff --git a/web-console/src/dialogs/snitch-dialog/snitch-dialog.tsx b/web-console/src/dialogs/snitch-dialog/snitch-dialog.tsx index cb906977f10b..cf6240a2e11c 100644 --- a/web-console/src/dialogs/snitch-dialog/snitch-dialog.tsx +++ b/web-console/src/dialogs/snitch-dialog/snitch-dialog.tsx @@ -27,7 +27,7 @@ import { } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import classNames = require('classnames'); -import * as React from 'react'; +import React from 'react'; import { HistoryDialog } from '../history-dialog/history-dialog'; @@ -49,7 +49,7 @@ export interface SnitchDialogState { showHistory?: boolean; } -export class SnitchDialog extends React.Component { +export class SnitchDialog extends React.PureComponent { constructor(props: SnitchDialogProps) { super(props); diff --git a/web-console/src/dialogs/spec-dialog/spec-dialog.spec.tsx b/web-console/src/dialogs/spec-dialog/spec-dialog.spec.tsx index 316b583218a9..5073ac6eb06e 100644 --- a/web-console/src/dialogs/spec-dialog/spec-dialog.spec.tsx +++ b/web-console/src/dialogs/spec-dialog/spec-dialog.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { SpecDialog } from './spec-dialog'; diff --git a/web-console/src/dialogs/spec-dialog/spec-dialog.tsx b/web-console/src/dialogs/spec-dialog/spec-dialog.tsx index 50adebe743d8..2150ccc16a81 100644 --- a/web-console/src/dialogs/spec-dialog/spec-dialog.tsx +++ b/web-console/src/dialogs/spec-dialog/spec-dialog.tsx @@ -17,7 +17,7 @@ */ import { Button, Classes, Dialog, Intent } from '@blueprintjs/core'; -import * as React from 'react'; +import React from 'react'; import AceEditor from 'react-ace'; import './spec-dialog.scss'; @@ -33,7 +33,7 @@ export interface SpecDialogState { spec: string; } -export class SpecDialog extends React.Component { +export class SpecDialog extends React.PureComponent { static validJson(json: string): boolean { try { JSON.parse(json); diff --git a/web-console/src/dialogs/supervisor-table-action-dialog/__snapshots__/supervisor-table-action-dialog.spec.tsx.snap b/web-console/src/dialogs/supervisor-table-action-dialog/__snapshots__/supervisor-table-action-dialog.spec.tsx.snap index f216fcf38871..84017de7f811 100644 --- a/web-console/src/dialogs/supervisor-table-action-dialog/__snapshots__/supervisor-table-action-dialog.spec.tsx.snap +++ b/web-console/src/dialogs/supervisor-table-action-dialog/__snapshots__/supervisor-table-action-dialog.spec.tsx.snap @@ -63,20 +63,20 @@ exports[`supervisor table action dialog matches snapshot 1`] = ` type="button" > - align-left + dashboard @@ -84,7 +84,7 @@ exports[`supervisor table action dialog matches snapshot 1`] = ` - Payload + Status
    ; } @@ -854,9 +811,16 @@ export class LoadDataView extends React.Component; } + private onFlattenFieldSelect = (field: FlattenField, index: number) => { + this.setState({ + selectedFlattenFieldIndex: index, + selectedFlattenField: field + }); + } + renderFlattenControls() { const { spec, selectedFlattenField, selectedFlattenFieldIndex } = this.state; - const parseSpec: ParseSpec = deepGet(spec, 'dataSchema.parser.parseSpec') || {}; + const parseSpec: ParseSpec = deepGet(spec, 'dataSchema.parser.parseSpec') || EMPTY_OBJECT; if (!parseSpecHasFlatten(parseSpec)) return null; const close = () => { @@ -929,8 +893,9 @@ export class LoadDataView extends React.Component; } else if (timestampQueryState.data) { - const timestampData = timestampQueryState.data; mainFill =
    this.setState({ specialColumnsOnly: !specialColumnsOnly })} />
    - { - const timestamp = columnName === '__time'; - if (!timestamp && !filterMatch(columnName, columnFilter)) return null; - const selected = timestampSpec.column === columnName; - const possibleFormat = timestamp ? null : possibleDruidFormatForValues(filterMap(timestampData.rows, d => d.parsed ? d.parsed[columnName] : null)); - if (specialColumnsOnly && !timestamp && !possibleFormat) return null; - - const columnClassName = classNames({ - timestamp, - selected - }); - return { - Header: ( -
    { - const newTimestampSpec = { - column: columnName, - format: possibleFormat || '!!! Could not auto detect a format !!!' - }; - this.updateSpec(deepSet(spec, 'dataSchema.parser.parseSpec.timestampSpec', newTimestampSpec)); - }} - > -
    {columnName}
    -
    - { - timestamp ? - (timestampSpecFromColumn ? `from: '${timestampSpecColumn}'` : `mv: ${timestampSpec.missingValue}`) : - (possibleFormat || '') - }  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, - Cell: row => { - if (columnName === '__error__') { - return ; - } - if (row.original.unparseable) { - return ; - } - return ; - }, - minWidth: timestamp ? 200 : 100, - resizable: !timestamp - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} +
    ; } @@ -1126,12 +1042,17 @@ export class LoadDataView extends React.Component; } + private onTimestampColumnSelect = (newTimestampSpec: TimestampSpec) => { + const { spec } = this.state; + this.updateSpec(deepSet(spec, 'dataSchema.parser.parseSpec.timestampSpec', newTimestampSpec)); + } + // ================================================================== async queryForTransform(initRun = false) { const { spec, sampleStrategy, cacheKey } = this.state; - const ioConfig: IoConfig = deepGet(spec, 'ioConfig') || {}; - const parser: Parser = deepGet(spec, 'dataSchema.parser') || {}; + const ioConfig: IoConfig = deepGet(spec, 'ioConfig') || EMPTY_OBJECT; + const parser: Parser = deepGet(spec, 'dataSchema.parser') || EMPTY_OBJECT; let issue: string | null = null; if (issueWithIoConfig(ioConfig)) { @@ -1171,7 +1092,7 @@ export class LoadDataView extends React.Component
    - { - if (!filterMatch(columnName, columnFilter)) return null; - const timestamp = columnName === '__time'; - const transformIndex = transforms.findIndex(f => f.name === columnName); - if (transformIndex === -1 && specialColumnsOnly) return null; - const transform = transforms[transformIndex]; - - const columnClassName = classNames({ - transformed: transform, - selected: transform && transformIndex === selectedTransformIndex - }); - return { - Header: ( -
    { - if (transform) { - this.setState({ - selectedTransformIndex: transformIndex, - selectedTransform: transform - }); - } else { - this.setState({ - selectedTransformIndex: -1, - selectedTransform: { - type: 'expression', - name: columnName, - expression: escapeColumnName(columnName) - } - }); - } - }} - > -
    {columnName}
    -
    - {transform ? `= ${transform.expression}` : ''}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} +
  • ; } @@ -1303,6 +1180,13 @@ export class LoadDataView extends React.Component; } + private onTransformSelect = (transform: Transform, index: number) => { + this.setState({ + selectedTransformIndex: index, + selectedTransform: transform + }); + } + renderTransformControls() { const { spec, selectedTransform, selectedTransformIndex } = this.state; @@ -1371,8 +1255,8 @@ export class LoadDataView extends React.Component((spec) => { + const { dimensionFilters } = splitFilter(deepGet(spec, 'dataSchema.transformSpec.filter')); + return dimensionFilters; + }); + renderFilterStage() { const { spec, columnFilter, filterQueryState, selectedFilter, selectedFilterIndex, showGlobalFilter } = this.state; - const parseSpec: ParseSpec = deepGet(spec, 'dataSchema.parser.parseSpec') || {}; - const { dimensionFilters } = splitFilter(deepGet(spec, 'dataSchema.transformSpec.filter')); + const parseSpec: ParseSpec = deepGet(spec, 'dataSchema.parser.parseSpec') || EMPTY_OBJECT; + const dimensionFilters = this.getMemoizedDimensionFiltersFromSpec(spec); const isBlank = !parseSpec.format; @@ -1440,56 +1329,13 @@ export class LoadDataView extends React.Component
    - { - if (!filterMatch(columnName, columnFilter)) return null; - const timestamp = columnName === '__time'; - const filterIndex = dimensionFilters.findIndex(f => f.dimension === columnName); - const filter = dimensionFilters[filterIndex]; - - const columnClassName = classNames({ - filtered: filter, - selected: filter && filterIndex === selectedFilterIndex - }); - return { - Header: ( -
    { - if (timestamp) { - this.setState({ - showGlobalFilter: true - }); - } else if (filter) { - this.setState({ - selectedFilterIndex: filterIndex, - selectedFilter: filter - }); - } else { - this.setState({ - selectedFilterIndex: -1, - selectedFilter: { type: 'selector', dimension: columnName, value: '' } - }); - } - }} - > -
    {columnName}
    -
    - {filter ? `(filtered)` : ''}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} + ; } @@ -1523,6 +1369,17 @@ export class LoadDataView extends React.Component; } + private onShowGlobalFilter = () => { + this.setState({ showGlobalFilter: true }); + } + + private onFilterSelect = (filter: DruidFilter, index: number) => { + this.setState({ + selectedFilterIndex: index, + selectedFilter: filter + }); + } + renderColumnFilterControls() { const { spec, selectedFilter, selectedFilterIndex } = this.state; @@ -1653,8 +1510,10 @@ export class LoadDataView extends React.Component; } else if (schemaQueryState.data) { - const dimensionMetricSortedHeader = sortWithPrefixSuffix(schemaQueryState.data.header, ['__time'], metricsSpec.map(getMetricSpecName)); mainFill =
    - { - if (!filterMatch(columnName, columnFilter)) return null; - - const metricSpecIndex = metricsSpec.findIndex(m => getMetricSpecName(m) === columnName); - const metricSpec = metricsSpec[metricSpecIndex]; - - if (metricSpec) { - const columnClassName = classNames('metric', { - selected: metricSpec && metricSpecIndex === selectedMetricSpecIndex - }); - return { - Header: ( -
    { - this.setState({ - selectedMetricSpecIndex: metricSpecIndex, - selectedMetricSpec: metricSpec, - selectedDimensionSpecIndex: -1, - selectedDimensionSpec: null - }); - }} - > -
    {columnName}
    -
    - {metricSpec.type}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - } else { - const timestamp = columnName === '__time'; - const dimensionSpecIndex = dimensionsSpec.dimensions ? dimensionsSpec.dimensions.findIndex(d => getDimensionSpecName(d) === columnName) : -1; - const dimensionSpec = dimensionsSpec.dimensions ? dimensionsSpec.dimensions[dimensionSpecIndex] : null; - const dimensionSpecType = dimensionSpec ? getDimensionSpecType(dimensionSpec) : null; - - const columnClassName = classNames(timestamp ? 'timestamp' : 'dimension', dimensionSpecType || 'string', { - selected: dimensionSpec && dimensionSpecIndex === selectedDimensionSpecIndex - }); - return { - Header: ( -
    { - if (timestamp) { - this.setState({ - selectedDimensionSpecIndex: -1, - selectedDimensionSpec: null, - selectedMetricSpecIndex: -1, - selectedMetricSpec: null - }); - return; - } - - if (!dimensionSpec) return; - this.setState({ - selectedDimensionSpecIndex: dimensionSpecIndex, - selectedDimensionSpec: inflateDimensionSpec(dimensionSpec), - selectedMetricSpecIndex: -1, - selectedMetricSpec: null - }); - }} - > -
    {columnName}
    -
    - {timestamp ? 'long (time column)' : (dimensionSpecType || 'string (auto)')}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - } - })} - defaultPageSize={50} - showPagination={false} - sortable={false} +
    ; } @@ -1921,6 +1700,20 @@ export class LoadDataView extends React.Component; } + private onDimensionOrMetricSelect = ( + selectedDimensionSpec: DimensionSpec | null, + selectedDimensionSpecIndex: number, + selectedMetricSpec: MetricSpec | null, + selectedMetricSpecIndex: number + ) => { + this.setState({ + selectedDimensionSpec, + selectedDimensionSpecIndex, + selectedMetricSpec, + selectedMetricSpecIndex + }); + } + renderChangeRollupAction() { const { newRollup, spec, sampleStrategy, cacheKey } = this.state; if (newRollup === null) return; @@ -2020,7 +1813,7 @@ export class LoadDataView extends React.Component { - const curDimensions = deepGet(spec, `dataSchema.parser.parseSpec.dimensionsSpec.dimensions`) || []; + const curDimensions = deepGet(spec, `dataSchema.parser.parseSpec.dimensionsSpec.dimensions`) || EMPTY_ARRAY; if (curDimensions.length <= 1) return; // Guard against removing the last dimension, ToDo: some better feedback here would be good this.updateSpec(deepDelete(spec, `dataSchema.parser.parseSpec.dimensionsSpec.dimensions.${selectedDimensionSpecIndex}`)); @@ -2123,8 +1916,8 @@ export class LoadDataView extends React.Component
    @@ -2182,8 +1975,8 @@ export class LoadDataView extends React.Component => { + const { initSupervisorId } = this.props; + try { - const resp = await axios.get(`/druid/indexer/v1/supervisor/${this.props.initSupervisorId}`); - this.updateSpec(resp.data); + const resp = await axios.get(`/druid/indexer/v1/supervisor/${initSupervisorId}`); + this.updateSpec(normalizeSpecType(resp.data)); this.updateStage('json-spec'); } catch (e) { AppToaster.show({ @@ -2362,9 +2157,11 @@ export class LoadDataView extends React.Component => { + const { initTaskId } = this.props; + try { - const resp = await axios.get(`/druid/indexer/v1/task/${this.props.initTaskId}`); - this.updateSpec(resp.data.payload.spec); + const resp = await axios.get(`/druid/indexer/v1/task/${initTaskId}`); + this.updateSpec(normalizeSpecType(resp.data.payload.spec)); this.updateStage('json-spec'); } catch (e) { AppToaster.show({ @@ -2388,7 +2185,7 @@ export class LoadDataView extends React.Component { if (!s) return; - this.updateSpec(s); + this.updateSpec(normalizeSpecType(s)); }} height="100%" /> diff --git a/web-console/src/views/load-data-view/parse-data-table/__snapshots__/parse-data-table.spec.tsx.snap b/web-console/src/views/load-data-view/parse-data-table/__snapshots__/parse-data-table.spec.tsx.snap new file mode 100644 index 000000000000..8a047e5a2391 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-data-table/__snapshots__/parse-data-table.spec.tsx.snap @@ -0,0 +1,1476 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`parse data table matches snapshot 1`] = ` +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    + c1 +
    +
    + +   +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    + • +
    +
    +
    + hello +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    + +   + +
    +
    +
    +
    +
    +
    +
    + Loading... +
    +
    +
    +`; diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss new file mode 100644 index 000000000000..306b3d3b9d8d --- /dev/null +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss @@ -0,0 +1,40 @@ +/* + * 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. + */ + +.parse-data-table { + .rt-th { + &.flattened { + background: rgba(201, 128, 22, 0.2); + } + } + + .rt-td { + &.flattened { + background: rgba(201, 128, 22, 0.05); + } + } + + .parse-detail { + padding: 10px; + + .parse-error { + color: #9E2B0E; + margin-bottom: 12px; + } + } +} diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx new file mode 100644 index 000000000000..6a83e73a6a19 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import React from 'react'; +import { render } from 'react-testing-library'; + +import { ParseDataTable } from './parse-data-table'; + +describe('parse data table', () => { + it('matches snapshot', () => { + const sampleData = { + header: ['c1'], + rows: [ + { + raw: `{"c1":"hello"}`, + parsed: { c1: 'hello' } + } + ] + }; + + const parseDataTable = null} + />; + + const { container } = render(parseDataTable); + expect(container.firstChild).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx new file mode 100644 index 000000000000..c58f6070f682 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx @@ -0,0 +1,99 @@ +/* + * 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. + */ + +import classNames from 'classnames'; +import React from 'react'; +import ReactTable from 'react-table'; + +import { TableCell } from '../../../components'; +import { caseInsensitiveContains, filterMap, parseJson } from '../../../utils'; +import { FlattenField } from '../../../utils/ingestion-spec'; +import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; + +import './parse-data-table.scss'; + +export interface ParseDataTableProps extends React.Props { + sampleData: HeaderAndRows; + columnFilter: string; + canFlatten: boolean; + flattenedColumnsOnly: boolean; + flattenFields: FlattenField[]; + onFlattenFieldSelect: (field: FlattenField, index: number) => void; +} + +export class ParseDataTable extends React.PureComponent { + render() { + const { sampleData, columnFilter, canFlatten, flattenedColumnsOnly, flattenFields, onFlattenFieldSelect } = this.props; + + return { + if (!caseInsensitiveContains(columnName, columnFilter)) return null; + const flattenFieldIndex = flattenFields.findIndex(f => f.name === columnName); + if (flattenFieldIndex === -1 && flattenedColumnsOnly) return null; + const flattenField = flattenFields[flattenFieldIndex]; + return { + Header: ( +
    { + if (!flattenField) return; + onFlattenFieldSelect(flattenField, flattenFieldIndex); + }} + > +
    {columnName}
    +
    + {flattenField ? `${flattenField.type}: ${flattenField.expr}` : ''}  +
    +
    + ), + id: String(i), + accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, + Cell: row => { + if (row.original.unparseable) { + return ; + } + return ; + }, + headerClassName: classNames({ + flattened: flattenField + }) + }; + })} + SubComponent={rowInfo => { + const { raw, error } = rowInfo.original; + const parsedJson: any = parseJson(raw); + + if (!error && parsedJson && canFlatten) { + return
    +            {'Original row: ' + JSON.stringify(parsedJson, null, 2)}
    +          
    ; + } else { + return
    + {error &&
    {error}
    } +
    {'Original row: ' + rowInfo.original.raw}
    +
    ; + } + }} + defaultPageSize={50} + showPagination={false} + sortable={false} + />; + } +} diff --git a/web-console/src/views/load-data-view/parse-time-table/__snapshots__/parse-time-table.spec.tsx.snap b/web-console/src/views/load-data-view/parse-time-table/__snapshots__/parse-time-table.spec.tsx.snap new file mode 100644 index 000000000000..8809a061eff1 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-time-table/__snapshots__/parse-time-table.spec.tsx.snap @@ -0,0 +1,1014 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`parse time table matches snapshot 1`] = ` +
    +
    +
    +
    +
    +
    +
    +
    + c1 +
    +
    + +   +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    + hello +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    +
    + Loading... +
    +
    +
    +`; diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss new file mode 100644 index 000000000000..a107a178c643 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +.parse-time-table { + +} diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx new file mode 100644 index 000000000000..ce0be3d4e76f --- /dev/null +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx @@ -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. + */ + +import React from 'react'; +import { render } from 'react-testing-library'; + +import { getEmptyTimestampSpec } from '../../../utils/ingestion-spec'; + +import { ParseTimeTable } from './parse-time-table'; + +describe('parse time table', () => { + it('matches snapshot', () => { + const sampleData = { + header: ['c1'], + rows: [ + { + raw: `{"c1":"hello"}`, + parsed: { c1: 'hello' } + } + ] + }; + + const parseTimeTable = null} + />; + + const { container } = render(parseTimeTable); + expect(container.firstChild).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx new file mode 100644 index 000000000000..b6f1a155c190 --- /dev/null +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx @@ -0,0 +1,105 @@ +/* + * 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. + */ + +import classNames from 'classnames'; +import React from 'react'; +import ReactTable from 'react-table'; + +import { TableCell } from '../../../components'; +import { caseInsensitiveContains, filterMap } from '../../../utils'; +import { possibleDruidFormatForValues } from '../../../utils/druid-time'; +import { getTimestampSpecColumn, isColumnTimestampSpec, TimestampSpec } from '../../../utils/ingestion-spec'; +import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; + +import './parse-time-table.scss'; + +export interface ParseTimeTableProps extends React.Props { + sampleBundle: { + headerAndRows: HeaderAndRows; + timestampSpec: TimestampSpec; + }; + columnFilter: string; + possibleTimestampColumnsOnly: boolean; + onTimestampColumnSelect: (newTimestampSpec: TimestampSpec) => void; +} + +export class ParseTimeTable extends React.PureComponent { + render() { + const { sampleBundle, columnFilter, possibleTimestampColumnsOnly, onTimestampColumnSelect } = this.props; + const { headerAndRows, timestampSpec } = sampleBundle; + const timestampSpecColumn = getTimestampSpecColumn(timestampSpec); + const timestampSpecFromColumn = isColumnTimestampSpec(timestampSpec); + + return { + const timestamp = columnName === '__time'; + if (!timestamp && !caseInsensitiveContains(columnName, columnFilter)) return null; + const selected = timestampSpec.column === columnName; + const possibleFormat = timestamp ? null : possibleDruidFormatForValues(filterMap(headerAndRows.rows, d => d.parsed ? d.parsed[columnName] : null)); + if (possibleTimestampColumnsOnly && !timestamp && !possibleFormat) return null; + + const columnClassName = classNames({ + timestamp, + selected + }); + return { + Header: ( +
    { + onTimestampColumnSelect({ + column: columnName, + format: possibleFormat || '!!! Could not auto detect a format !!!' + }); + }} + > +
    {columnName}
    +
    + { + timestamp ? + (timestampSpecFromColumn ? `from: '${timestampSpecColumn}'` : `mv: ${timestampSpec.missingValue}`) : + (possibleFormat || '') + }  +
    +
    + ), + headerClassName: columnClassName, + className: columnClassName, + id: String(i), + accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, + Cell: row => { + if (columnName === '__error__') { + return ; + } + if (row.original.unparseable) { + return ; + } + return ; + }, + minWidth: timestamp ? 200 : 100, + resizable: !timestamp + }; + })} + defaultPageSize={50} + showPagination={false} + sortable={false} + />; + } +} diff --git a/web-console/src/views/load-data-view/schema-table/__snapshots__/schema-table.spec.tsx.snap b/web-console/src/views/load-data-view/schema-table/__snapshots__/schema-table.spec.tsx.snap new file mode 100644 index 000000000000..f5f1e2e9fd4d --- /dev/null +++ b/web-console/src/views/load-data-view/schema-table/__snapshots__/schema-table.spec.tsx.snap @@ -0,0 +1,1014 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`schema table matches snapshot 1`] = ` +
    +
    +
    +
    +
    +
    +
    +
    + c1 +
    +
    + string (auto) +   +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    + hello +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    +
    + Loading... +
    +
    +
    +`; diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.scss b/web-console/src/views/load-data-view/schema-table/schema-table.scss new file mode 100644 index 000000000000..d8e6b37e8534 --- /dev/null +++ b/web-console/src/views/load-data-view/schema-table/schema-table.scss @@ -0,0 +1,45 @@ +/* + * 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. + */ + +.schema-table { + .rt-th { + &.dimension { + background: rgba(38, 170, 201, 0.5); + + &.long { background: rgba(19, 129, 201, 0.5); } + &.float { background: rgba(25, 145, 201, 0.5); } + } + + &.metric { + background: rgba(201, 191, 55, 0.5); + } + } + + .rt-td { + &.dimension { + background: rgba(38, 170, 201, 0.1); + + &.long { background: rgba(19, 129, 201, 0.1); } + &.float { background: rgba(25, 145, 201, 0.1); } + } + + &.metric { + background: rgba(201, 191, 55, 0.1); + } + } +} diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx b/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx new file mode 100644 index 000000000000..e7c2047d5007 --- /dev/null +++ b/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx @@ -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. + */ + +import React from 'react'; +import { render } from 'react-testing-library'; + +import { SchemaTable } from './schema-table'; + +describe('schema table', () => { + it('matches snapshot', () => { + const sampleData = { + header: ['c1'], + rows: [ + { + raw: `{"c1":"hello"}`, + parsed: { c1: 'hello' } + } + ] + }; + + const schemaTable = null} + />; + + const { container } = render(schemaTable); + expect(container.firstChild).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.tsx b/web-console/src/views/load-data-view/schema-table/schema-table.tsx new file mode 100644 index 000000000000..1b6f96f8b90b --- /dev/null +++ b/web-console/src/views/load-data-view/schema-table/schema-table.tsx @@ -0,0 +1,135 @@ +/* + * 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. + */ + +import classNames from 'classnames'; +import React from 'react'; +import ReactTable from 'react-table'; + +import { TableCell } from '../../../components'; +import { caseInsensitiveContains, filterMap, sortWithPrefixSuffix } from '../../../utils'; +import { + DimensionSpec, + DimensionsSpec, + getDimensionSpecName, + getDimensionSpecType, + getMetricSpecName, + inflateDimensionSpec, + MetricSpec, TimestampSpec +} from '../../../utils/ingestion-spec'; +import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; + +import './schema-table.scss'; + +export interface SchemaTableProps extends React.Props { + sampleBundle: { + headerAndRows: HeaderAndRows; + dimensionsSpec: DimensionsSpec; + metricsSpec: MetricSpec[]; + }; + columnFilter: string; + selectedDimensionSpecIndex: number; + selectedMetricSpecIndex: number; + onDimensionOrMetricSelect: ( + selectedDimensionSpec: DimensionSpec | null, + selectedDimensionSpecIndex: number, + selectedMetricSpec: MetricSpec | null, + selectedMetricSpecIndex: number + ) => void; +} + +export class SchemaTable extends React.PureComponent { + render() { + const { sampleBundle, columnFilter, selectedDimensionSpecIndex, selectedMetricSpecIndex, onDimensionOrMetricSelect } = this.props; + const { headerAndRows, dimensionsSpec, metricsSpec } = sampleBundle; + + const dimensionMetricSortedHeader = sortWithPrefixSuffix(headerAndRows.header, ['__time'], metricsSpec.map(getMetricSpecName)); + + return { + if (!caseInsensitiveContains(columnName, columnFilter)) return null; + + const metricSpecIndex = metricsSpec.findIndex(m => getMetricSpecName(m) === columnName); + const metricSpec = metricsSpec[metricSpecIndex]; + + if (metricSpec) { + const columnClassName = classNames('metric', { + selected: metricSpec && metricSpecIndex === selectedMetricSpecIndex + }); + return { + Header: ( +
    onDimensionOrMetricSelect(null, -1, metricSpec, metricSpecIndex)} + > +
    {columnName}
    +
    + {metricSpec.type}  +
    +
    + ), + headerClassName: columnClassName, + className: columnClassName, + id: String(i), + accessor: row => row.parsed ? row.parsed[columnName] : null, + Cell: row => + }; + } else { + const timestamp = columnName === '__time'; + const dimensionSpecIndex = dimensionsSpec.dimensions ? dimensionsSpec.dimensions.findIndex(d => getDimensionSpecName(d) === columnName) : -1; + const dimensionSpec = dimensionsSpec.dimensions ? dimensionsSpec.dimensions[dimensionSpecIndex] : null; + const dimensionSpecType = dimensionSpec ? getDimensionSpecType(dimensionSpec) : null; + + const columnClassName = classNames(timestamp ? 'timestamp' : 'dimension', dimensionSpecType || 'string', { + selected: dimensionSpec && dimensionSpecIndex === selectedDimensionSpecIndex + }); + return { + Header: ( +
    { + if (timestamp) { + onDimensionOrMetricSelect(null, -1, null, -1); + return; + } + + if (!dimensionSpec) return; + onDimensionOrMetricSelect(inflateDimensionSpec(dimensionSpec), dimensionSpecIndex, null, -1); + }} + > +
    {columnName}
    +
    + {timestamp ? 'long (time column)' : (dimensionSpecType || 'string (auto)')}  +
    +
    + ), + headerClassName: columnClassName, + className: columnClassName, + id: String(i), + accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, + Cell: row => + }; + } + })} + defaultPageSize={50} + showPagination={false} + sortable={false} + />; + } +} diff --git a/web-console/src/views/load-data-view/transform-table/__snapshots__/transform-table.spec.tsx.snap b/web-console/src/views/load-data-view/transform-table/__snapshots__/transform-table.spec.tsx.snap new file mode 100644 index 000000000000..2febec32dc22 --- /dev/null +++ b/web-console/src/views/load-data-view/transform-table/__snapshots__/transform-table.spec.tsx.snap @@ -0,0 +1,1014 @@ +// Jest Snapshot v1, https://goo.gl/fbAQLP + +exports[`transform table matches snapshot 1`] = ` +
    +
    +
    +
    +
    +
    +
    +
    + c1 +
    +
    + +   +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    +
    + hello +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    + +   + +
    +
    +
    +
    +
    +
    +
    + Loading... +
    +
    +
    +`; diff --git a/web-console/src/views/load-data-view/transform-table/transform-table.scss b/web-console/src/views/load-data-view/transform-table/transform-table.scss new file mode 100644 index 000000000000..0b85f72b94da --- /dev/null +++ b/web-console/src/views/load-data-view/transform-table/transform-table.scss @@ -0,0 +1,31 @@ +/* + * 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. + */ + +.transform-table { + .rt-th { + &.transformed { + background: rgba(201, 128, 22, 0.2); + } + } + + .rt-td { + &.transformed { + background: rgba(201, 128, 22, 0.05); + } + } +} diff --git a/web-console/src/views/load-data-view/transform-table/transform-table.spec.tsx b/web-console/src/views/load-data-view/transform-table/transform-table.spec.tsx new file mode 100644 index 000000000000..b750f2002381 --- /dev/null +++ b/web-console/src/views/load-data-view/transform-table/transform-table.spec.tsx @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import React from 'react'; +import { render } from 'react-testing-library'; + +import { TransformTable } from './transform-table'; + +describe('transform table', () => { + it('matches snapshot', () => { + const sampleData = { + header: ['c1'], + rows: [ + { + raw: `{"c1":"hello"}`, + parsed: { c1: 'hello' } + } + ] + }; + + const transformTable = null} + />; + + const { container } = render(transformTable); + expect(container.firstChild).toMatchSnapshot(); + }); +}); diff --git a/web-console/src/views/load-data-view/transform-table/transform-table.tsx b/web-console/src/views/load-data-view/transform-table/transform-table.tsx new file mode 100644 index 000000000000..5a389239525d --- /dev/null +++ b/web-console/src/views/load-data-view/transform-table/transform-table.tsx @@ -0,0 +1,92 @@ +/* + * 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. + */ + +import classNames from 'classnames'; +import React from 'react'; +import ReactTable from 'react-table'; + +import { TableCell } from '../../../components'; +import { caseInsensitiveContains, filterMap } from '../../../utils'; +import { escapeColumnName } from '../../../utils/druid-expression'; +import { Transform } from '../../../utils/ingestion-spec'; +import { HeaderAndRows } from '../../../utils/sampler'; + +import './transform-table.scss'; + +export interface TransformTableProps extends React.Props { + sampleData: HeaderAndRows; + columnFilter: string; + transformedColumnsOnly: boolean; + transforms: Transform[]; + selectedTransformIndex: number; + onTransformSelect: (transform: Transform, index: number) => void; +} + +export class TransformTable extends React.PureComponent { + render() { + const { sampleData, columnFilter, transformedColumnsOnly, transforms, selectedTransformIndex, onTransformSelect } = this.props; + + return { + if (!caseInsensitiveContains(columnName, columnFilter)) return null; + const timestamp = columnName === '__time'; + const transformIndex = transforms.findIndex(f => f.name === columnName); + if (transformIndex === -1 && transformedColumnsOnly) return null; + const transform = transforms[transformIndex]; + + const columnClassName = classNames({ + transformed: transform, + selected: transform && transformIndex === selectedTransformIndex + }); + return { + Header: ( +
    { + if (transform) { + onTransformSelect(transform, transformIndex); + } else { + onTransformSelect({ + type: 'expression', + name: columnName, + expression: escapeColumnName(columnName) + }, transformIndex); + } + }} + > +
    {columnName}
    +
    + {transform ? `= ${transform.expression}` : ''}  +
    +
    + ), + headerClassName: columnClassName, + className: columnClassName, + id: String(i), + accessor: row => row.parsed ? row.parsed[columnName] : null, + Cell: row => + }; + })} + defaultPageSize={50} + showPagination={false} + sortable={false} + />; + } +} diff --git a/web-console/src/views/lookups-view/lookups-view.spec.tsx b/web-console/src/views/lookups-view/lookups-view.spec.tsx index a10e8c6ba12f..a2d1cf0e0865 100644 --- a/web-console/src/views/lookups-view/lookups-view.spec.tsx +++ b/web-console/src/views/lookups-view/lookups-view.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { LookupsView } from './lookups-view'; diff --git a/web-console/src/views/lookups-view/lookups-view.tsx b/web-console/src/views/lookups-view/lookups-view.tsx index b94c2beb9974..a9dd83bc5f5a 100644 --- a/web-console/src/views/lookups-view/lookups-view.tsx +++ b/web-console/src/views/lookups-view/lookups-view.tsx @@ -19,8 +19,8 @@ import { Button, Icon, Intent, Popover, Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as classNames from 'classnames'; -import * as React from 'react'; +import classNames from 'classnames'; +import React from 'react'; import ReactTable from 'react-table'; import { ActionCell, TableColumnSelector, ViewControlBar } from '../../components'; @@ -61,7 +61,7 @@ export interface LookupsViewState { deleteLookupTier: string | null; } -export class LookupsView extends React.Component { +export class LookupsView extends React.PureComponent { private lookupsGetQueryManager: QueryManager; private tableColumnSelectionHandler: TableColumnSelectionHandler; diff --git a/web-console/src/views/segments-view/segments-view.spec.tsx b/web-console/src/views/segments-view/segments-view.spec.tsx index f41e0e13d54d..b835cc68b22f 100644 --- a/web-console/src/views/segments-view/segments-view.spec.tsx +++ b/web-console/src/views/segments-view/segments-view.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { SegmentsView } from '../segments-view/segments-view'; diff --git a/web-console/src/views/segments-view/segments-view.tsx b/web-console/src/views/segments-view/segments-view.tsx index c81feed31db4..9e3f895e7c71 100644 --- a/web-console/src/views/segments-view/segments-view.tsx +++ b/web-console/src/views/segments-view/segments-view.tsx @@ -20,7 +20,7 @@ import { Button, Intent } from '@blueprintjs/core'; import { H5 } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import ReactTable from 'react-table'; import { Filter } from 'react-table'; @@ -80,7 +80,7 @@ interface SegmentQueryResultRow { is_overshadowed: number; } -export class SegmentsView extends React.Component { +export class SegmentsView extends React.PureComponent { private segmentsSqlQueryManager: QueryManager; private segmentsJsonQueryManager: QueryManager; private tableColumnSelectionHandler: TableColumnSelectionHandler; diff --git a/web-console/src/views/servers-view/servers-view.spec.tsx b/web-console/src/views/servers-view/servers-view.spec.tsx index b9e5ca555ca1..a8f01a7a4dd1 100644 --- a/web-console/src/views/servers-view/servers-view.spec.tsx +++ b/web-console/src/views/servers-view/servers-view.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { ServersView } from './servers-view'; diff --git a/web-console/src/views/servers-view/servers-view.tsx b/web-console/src/views/servers-view/servers-view.tsx index 4eca97bd96b5..aa0789afc7cc 100644 --- a/web-console/src/views/servers-view/servers-view.tsx +++ b/web-console/src/views/servers-view/servers-view.tsx @@ -20,7 +20,7 @@ import { Button, ButtonGroup, Intent, Label } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; import { sum } from 'd3-array'; -import * as React from 'react'; +import React from 'react'; import ReactTable from 'react-table'; import { Filter } from 'react-table'; @@ -108,7 +108,7 @@ interface MiddleManagerQueryResultRow { interface ServerResultRow extends ServerQueryResultRow, Partial, Partial {} -export class ServersView extends React.Component { +export class ServersView extends React.PureComponent { private serverQueryManager: QueryManager; private serverTableColumnSelectionHandler: TableColumnSelectionHandler; diff --git a/web-console/src/components/sql-control/__snapshots__/sql-control.spec.tsx.snap b/web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap similarity index 98% rename from web-console/src/components/sql-control/__snapshots__/sql-control.spec.tsx.snap rename to web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap index fcb5b8fcedcc..80ab5c401f89 100644 --- a/web-console/src/components/sql-control/__snapshots__/sql-control.spec.tsx.snap +++ b/web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap @@ -8,7 +8,7 @@ exports[`sql control matches snapshot 1`] = ` class="ace-container" >
    diff --git a/web-console/src/components/sql-control/sql-control.scss b/web-console/src/views/sql-view/sql-control/sql-control.scss similarity index 98% rename from web-console/src/components/sql-control/sql-control.scss rename to web-console/src/views/sql-view/sql-control/sql-control.scss index 694de3618d22..95c798b92585 100644 --- a/web-console/src/components/sql-control/sql-control.scss +++ b/web-console/src/views/sql-view/sql-control/sql-control.scss @@ -16,7 +16,7 @@ * limitations under the License. */ -@import "../../variables"; +@import "../../../variables"; .sql-control { .ace-container { diff --git a/web-console/src/components/sql-control/sql-control.spec.tsx b/web-console/src/views/sql-view/sql-control/sql-control.spec.tsx similarity index 97% rename from web-console/src/components/sql-control/sql-control.spec.tsx rename to web-console/src/views/sql-view/sql-control/sql-control.spec.tsx index 8f5482a219d4..1ac07bee1a9c 100644 --- a/web-console/src/components/sql-control/sql-control.spec.tsx +++ b/web-console/src/views/sql-view/sql-control/sql-control.spec.tsx @@ -16,7 +16,7 @@ * limitations under the License. */ -import * as React from 'react'; +import React from 'react'; import { render } from 'react-testing-library'; import { SqlControl } from './sql-control'; diff --git a/web-console/src/components/sql-control/sql-control.tsx b/web-console/src/views/sql-view/sql-control/sql-control.tsx similarity index 94% rename from web-console/src/components/sql-control/sql-control.tsx rename to web-console/src/views/sql-view/sql-control/sql-control.tsx index a3acae819f2b..44730b9c9d30 100644 --- a/web-console/src/components/sql-control/sql-control.tsx +++ b/web-console/src/views/sql-view/sql-control/sql-control.tsx @@ -28,21 +28,16 @@ import { import { Hotkey, Hotkeys, HotkeysTarget } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as ace from 'brace'; -import 'brace/ext/language_tools'; -import 'brace/mode/hjson'; -import 'brace/mode/sql'; -import 'brace/theme/solarized_dark'; -import * as Hjson from 'hjson'; -import * as React from 'react'; +import ace from 'brace'; +import Hjson from 'hjson'; +import React from 'react'; import AceEditor from 'react-ace'; -import * as ReactDOMServer from 'react-dom/server'; +import ReactDOMServer from 'react-dom/server'; -import { SQLFunctionDoc } from '../../../lib/sql-function-doc'; -import { AppToaster } from '../../singletons/toaster'; -import { DRUID_DOCS_RUNE, DRUID_DOCS_SQL } from '../../variables'; - -import { MenuCheckbox } from './../menu-checkbox/menu-checkbox'; +import { SQLFunctionDoc } from '../../../../lib/sql-function-doc'; +import { MenuCheckbox } from '../../../components'; +import { AppToaster } from '../../../singletons/toaster'; +import { DRUID_DOCS_RUNE, DRUID_DOCS_SQL } from '../../../variables'; import './sql-control.scss'; @@ -79,7 +74,7 @@ export interface SqlControlState { } @HotkeysTarget -export class SqlControl extends React.Component { +export class SqlControl extends React.PureComponent { constructor(props: SqlControlProps, context: any) { super(props, context); this.state = { diff --git a/web-console/src/views/sql-view/sql-view.spec.tsx b/web-console/src/views/sql-view/sql-view.spec.tsx index 23edd3846978..c3801d360b92 100644 --- a/web-console/src/views/sql-view/sql-view.spec.tsx +++ b/web-console/src/views/sql-view/sql-view.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { SqlView } from './sql-view'; diff --git a/web-console/src/views/sql-view/sql-view.tsx b/web-console/src/views/sql-view/sql-view.tsx index a7e31f39ff0d..fedd2439356c 100644 --- a/web-console/src/views/sql-view/sql-view.tsx +++ b/web-console/src/views/sql-view/sql-view.tsx @@ -16,12 +16,12 @@ * limitations under the License. */ -import * as Hjson from 'hjson'; -import * as React from 'react'; +import Hjson from 'hjson'; +import React from 'react'; import SplitterLayout from 'react-splitter-layout'; import ReactTable from 'react-table'; -import { SqlControl, TableCell } from '../../components'; +import { TableCell } from '../../components'; import { QueryPlanDialog } from '../../dialogs'; import { BasicQueryExplanation, @@ -34,6 +34,8 @@ import { SemiJoinQueryExplanation } from '../../utils'; +import { SqlControl } from './sql-control/sql-control'; + import './sql-view.scss'; interface QueryWithContext { @@ -62,7 +64,7 @@ interface SqlQueryResult { queryElapsed: number; } -export class SqlView extends React.Component { +export class SqlView extends React.PureComponent { static trimSemicolon(query: string): string { // Trims out a trailing semicolon while preserving space (https://bit.ly/1n1yfkJ) return query.replace(/;+(\s*)$/, '$1'); diff --git a/web-console/src/views/task-view/tasks-view.spec.tsx b/web-console/src/views/task-view/tasks-view.spec.tsx index 3ac80294a23d..2e962f4e993a 100644 --- a/web-console/src/views/task-view/tasks-view.spec.tsx +++ b/web-console/src/views/task-view/tasks-view.spec.tsx @@ -17,7 +17,7 @@ */ import { shallow } from 'enzyme'; -import * as React from 'react'; +import React from 'react'; import { TasksView } from './tasks-view'; diff --git a/web-console/src/views/task-view/tasks-view.tsx b/web-console/src/views/task-view/tasks-view.tsx index e58424d0a37c..d8fa4eb08618 100644 --- a/web-console/src/views/task-view/tasks-view.tsx +++ b/web-console/src/views/task-view/tasks-view.tsx @@ -19,7 +19,7 @@ import { Alert, Button, ButtonGroup, Intent, Label, Menu, MenuItem, Popover, Position } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -import * as React from 'react'; +import React from 'react'; import SplitterLayout from 'react-splitter-layout'; import ReactTable from 'react-table'; import { Filter } from 'react-table'; @@ -109,7 +109,7 @@ function statusToColor(status: string): string { } } -export class TasksView extends React.Component { +export class TasksView extends React.PureComponent { private supervisorQueryManager: QueryManager; private taskQueryManager: QueryManager; private supervisorTableColumnSelectionHandler: TableColumnSelectionHandler; diff --git a/web-console/tsconfig.json b/web-console/tsconfig.json index b13473cbe434..510a2ea9c171 100644 --- a/web-console/tsconfig.json +++ b/web-console/tsconfig.json @@ -11,6 +11,7 @@ "strictFunctionTypes": false, "skipLibCheck": true, "importHelpers": true, + "esModuleInterop": true, "target": "es5", "module": "commonjs", "moduleResolution": "node", From bd97056b300f5e037d7a9ad50a28871f1bfd24c8 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 12 Jul 2019 18:07:32 +0300 Subject: [PATCH 11/15] Fix bad web-console merge --- .../supervisor-table-action-dialog.tsx | 6 - .../task-table-action-dialog.tsx | 53 --- web-console/src/entry.ts | 3 - web-console/src/utils/druid-time.ts | 33 -- web-console/src/utils/general.tsx | 18 - web-console/src/utils/ingestion-spec.tsx | 26 -- web-console/src/utils/joda-to-regexp.spec.ts | 11 - web-console/src/utils/joda-to-regexp.ts | 4 - web-console/src/utils/sampler.ts | 5 - web-console/src/views/home-view/home-view.tsx | 3 - .../filter-table/filter-table.spec.tsx | 16 - .../filter-table/filter-table.tsx | 58 ---- .../views/load-data-view/load-data-view.tsx | 328 ------------------ .../parse-data-table/parse-data-table.scss | 4 - .../parse-data-table.spec.tsx | 16 - .../parse-data-table/parse-data-table.tsx | 67 ---- .../parse-time-table/parse-time-table.scss | 4 - .../parse-time-table.spec.tsx | 17 - .../parse-time-table/parse-time-table.tsx | 71 ---- .../schema-table/schema-table.scss | 10 - .../schema-table/schema-table.spec.tsx | 19 - .../schema-table/schema-table.tsx | 92 ----- .../transform-table/transform-table.tsx | 60 ---- .../src/views/lookups-view/lookups-view.tsx | 4 - .../__snapshots__/sql-control.spec.tsx.snap | 174 ---------- web-console/tsconfig.json | 3 - 26 files changed, 1105 deletions(-) delete mode 100644 web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap diff --git a/web-console/src/dialogs/supervisor-table-action-dialog/supervisor-table-action-dialog.tsx b/web-console/src/dialogs/supervisor-table-action-dialog/supervisor-table-action-dialog.tsx index 4504e1b4d92c..49d6af46f22f 100644 --- a/web-console/src/dialogs/supervisor-table-action-dialog/supervisor-table-action-dialog.tsx +++ b/web-console/src/dialogs/supervisor-table-action-dialog/supervisor-table-action-dialog.tsx @@ -61,12 +61,6 @@ export class SupervisorTableActionDialog extends React.PureComponent< active: activeTab === 'payload', onClick: () => this.setState({ activeTab: 'payload' }), }, - { - icon: 'align-left', - text: 'Payload', - active: activeTab === 'payload', - onClick: () => this.setState({ activeTab: 'payload' }) - }, { icon: 'chart', text: 'Statistics', diff --git a/web-console/src/dialogs/task-table-action-dialog/task-table-action-dialog.tsx b/web-console/src/dialogs/task-table-action-dialog/task-table-action-dialog.tsx index 35b0b2d228b3..c6c5540e5794 100644 --- a/web-console/src/dialogs/task-table-action-dialog/task-table-action-dialog.tsx +++ b/web-console/src/dialogs/task-table-action-dialog/task-table-action-dialog.tsx @@ -42,11 +42,7 @@ export class TaskTableActionDialog extends React.PureComponent< constructor(props: TaskTableActionDialogProps) { super(props); this.state = { -<<<<<<< HEAD - activeTab: 'status' -======= activeTab: 'status', ->>>>>>> upstream/master }; } @@ -67,12 +63,6 @@ export class TaskTableActionDialog extends React.PureComponent< active: activeTab === 'payload', onClick: () => this.setState({ activeTab: 'payload' }), }, - { - icon: 'align-left', - text: 'Payload', - active: activeTab === 'payload', - onClick: () => this.setState({ activeTab: 'payload' }) - }, { icon: 'comparison', text: 'Reports', @@ -87,48 +77,6 @@ export class TaskTableActionDialog extends React.PureComponent< }, ]; -<<<<<<< HEAD - return - { - activeTab === 'status' && - deepGet(x, 'status')} - downloadFilename={`task-status-${taskId}.json`} - /> - } - { - activeTab === 'payload' && - deepGet(x, 'payload')} - downloadFilename={`task-payload-${taskId}.json`} - /> - } - { - activeTab === 'reports' && - deepGet(x, 'ingestionStatsAndErrors.payload')} - downloadFilename={`task-reports-${taskId}.json`} - /> - } - { - activeTab === 'log' && - - } - ; -======= return ( ); ->>>>>>> upstream/master } } diff --git a/web-console/src/entry.ts b/web-console/src/entry.ts index fc5483c940d7..c1096ac0c7b4 100644 --- a/web-console/src/entry.ts +++ b/web-console/src/entry.ts @@ -18,11 +18,8 @@ import 'brace'; // Import Ace editor and all the sub components used in the app import 'brace/ext/language_tools'; -<<<<<<< HEAD import 'brace/mode/hjson'; import 'brace/mode/sql'; -======= ->>>>>>> upstream/master import 'brace/theme/solarized_dark'; import 'es6-shim/es6-shim'; import 'es7-shim'; // Webpack with automatically pick browser.js which does the shim() diff --git a/web-console/src/utils/druid-time.ts b/web-console/src/utils/druid-time.ts index 1bb2e19e2885..08cba49a1256 100644 --- a/web-console/src/utils/druid-time.ts +++ b/web-console/src/utils/druid-time.ts @@ -18,32 +18,6 @@ import { jodaFormatToRegExp } from './joda-to-regexp'; -<<<<<<< HEAD -export const BASIC_FORMAT_VALUES: string[] = [ - 'iso', - 'millis', - 'posix' -]; - -export const DATE_FORMAT_VALUES: string[] = [ - 'dd/MM/yyyy', - 'MM/dd/yyyy', - 'd/M/yy', - 'M/d/yy', - 'd/M/yyyy', - 'M/d/yyyy' -]; - -export const DATE_TIME_FORMAT_VALUES: string[] = [ - 'd/M/yyyy H:mm:ss', - 'M/d/yyyy H:mm:ss', - 'MM/dd/yyyy hh:mm:ss a', - 'yyyy-MM-dd HH:mm:ss', - 'yyyy-MM-dd HH:mm:ss.S' -]; - -const ALL_FORMAT_VALUES: string[] = BASIC_FORMAT_VALUES.concat(DATE_FORMAT_VALUES, DATE_TIME_FORMAT_VALUES); -======= export const BASIC_FORMAT_VALUES: string[] = ['iso', 'millis', 'posix']; export const DATE_FORMAT_VALUES: string[] = [ @@ -67,7 +41,6 @@ const ALL_FORMAT_VALUES: string[] = BASIC_FORMAT_VALUES.concat( DATE_FORMAT_VALUES, DATE_TIME_FORMAT_VALUES, ); ->>>>>>> upstream/master const EXAMPLE_DATE_ISO = '2015-10-29T23:00:00.000Z'; const EXAMPLE_DATE_VALUE = Date.parse(EXAMPLE_DATE_ISO); @@ -99,15 +72,9 @@ export function timeFormatMatches(format: string, value: string | number): boole } export function possibleDruidFormatForValues(values: any[]): string | null { -<<<<<<< HEAD - return ALL_FORMAT_VALUES.filter(format => { - return values.every(value => timeFormatMatches(format, value)); - })[0] || null; -======= return ( ALL_FORMAT_VALUES.filter(format => { return values.every(value => timeFormatMatches(format, value)); })[0] || null ); ->>>>>>> upstream/master } diff --git a/web-console/src/utils/general.tsx b/web-console/src/utils/general.tsx index 6a582cba846b..6f834e8b25b4 100644 --- a/web-console/src/utils/general.tsx +++ b/web-console/src/utils/general.tsx @@ -19,10 +19,7 @@ import { Button, HTMLSelect, InputGroup } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import FileSaver from 'file-saver'; -<<<<<<< HEAD -======= import hasOwnProp from 'has-own-prop'; ->>>>>>> upstream/master import numeral from 'numeral'; import React from 'react'; import { Filter, FilterRender } from 'react-table'; @@ -137,14 +134,10 @@ export function caseInsensitiveContains(testString: string, searchString: string // ---------------------------- -<<<<<<< HEAD -export function countBy(array: T[], fn: (x: T, index: number) => string = String): Record { -======= export function countBy( array: T[], fn: (x: T, index: number) => string = String, ): Record { ->>>>>>> upstream/master const counts: Record = {}; for (let i = 0; i < array.length; i++) { const key = fn(array[i], i); @@ -248,17 +241,6 @@ export function pluralIfNeeded(n: number, singular: string, plural?: string): st // ---------------------------- -export function memoize(fn: (x: T) => U): (x: T) => U { - let lastInput: T; - let lastOutput: U; - return (x: T) => { - if (x === lastInput) return lastOutput; - lastInput = x; - lastOutput = fn(lastInput); - return lastOutput; - }; -} - export function parseJson(json: string): any { try { return JSON.parse(json); diff --git a/web-console/src/utils/ingestion-spec.tsx b/web-console/src/utils/ingestion-spec.tsx index a5ba8c7c2d25..c31e12b84321 100644 --- a/web-console/src/utils/ingestion-spec.tsx +++ b/web-console/src/utils/ingestion-spec.tsx @@ -17,10 +17,6 @@ */ import { Code } from '@blueprintjs/core'; -<<<<<<< HEAD -import { number } from 'prop-types'; -======= ->>>>>>> upstream/master import React from 'react'; import { Field } from '../components/auto-form/auto-form'; @@ -198,12 +194,8 @@ export function isParallel(spec: IngestionSpec): boolean { export type DimensionMode = 'specific' | 'auto-detect'; export function getDimensionMode(spec: IngestionSpec): DimensionMode { -<<<<<<< HEAD - const dimensions = deepGet(spec, 'dataSchema.parser.parseSpec.dimensionsSpec.dimensions') || EMPTY_ARRAY; -======= const dimensions = deepGet(spec, 'dataSchema.parser.parseSpec.dimensionsSpec.dimensions') || EMPTY_ARRAY; ->>>>>>> upstream/master return Array.isArray(dimensions) && dimensions.length === 0 ? 'auto-detect' : 'specific'; } @@ -380,21 +372,12 @@ const TIMESTAMP_SPEC_FORM_FIELDS: Field[] = [ ...BASIC_FORMAT_VALUES, { group: 'Date and time formats', -<<<<<<< HEAD - suggestions: DATE_TIME_FORMAT_VALUES - }, - { - group: 'Date only formats', - suggestions: DATE_FORMAT_VALUES - } -======= suggestions: DATE_TIME_FORMAT_VALUES, }, { group: 'Date only formats', suggestions: DATE_FORMAT_VALUES, }, ->>>>>>> upstream/master ], isDefined: (timestampSpec: TimestampSpec) => isColumnTimestampSpec(timestampSpec), info: ( @@ -1335,13 +1318,9 @@ export function guessDataSourceName(ioConfig: IoConfig): string | null { return filenameFromPath(firehose.baseDir); case 'static-s3': -<<<<<<< HEAD - return filenameFromPath((firehose.uris || EMPTY_ARRAY)[0] || (firehose.prefixes || EMPTY_ARRAY)[0]); -======= return filenameFromPath( (firehose.uris || EMPTY_ARRAY)[0] || (firehose.prefixes || EMPTY_ARRAY)[0], ); ->>>>>>> upstream/master case 'http': return filenameFromPath(firehose.uris ? firehose.uris[0] : undefined); @@ -1880,10 +1859,6 @@ export interface DimensionFiltersWithRest { } export function splitFilter(filter: DruidFilter | null): DimensionFiltersWithRest { -<<<<<<< HEAD - const inputAndFilters: DruidFilter[] = filter ? ((filter.type === 'and' && Array.isArray(filter.fields)) ? filter.fields : [filter]) : EMPTY_ARRAY; - const dimensionFilters: DruidFilter[] = inputAndFilters.filter(f => typeof f.dimension === 'string'); -======= const inputAndFilters: DruidFilter[] = filter ? filter.type === 'and' && Array.isArray(filter.fields) ? filter.fields @@ -1892,7 +1867,6 @@ export function splitFilter(filter: DruidFilter | null): DimensionFiltersWithRes const dimensionFilters: DruidFilter[] = inputAndFilters.filter( f => typeof f.dimension === 'string', ); ->>>>>>> upstream/master const restFilters: DruidFilter[] = inputAndFilters.filter(f => typeof f.dimension !== 'string'); return { diff --git a/web-console/src/utils/joda-to-regexp.spec.ts b/web-console/src/utils/joda-to-regexp.spec.ts index 20f9c481a62e..c88da0d6c13e 100644 --- a/web-console/src/utils/joda-to-regexp.spec.ts +++ b/web-console/src/utils/joda-to-regexp.spec.ts @@ -24,26 +24,15 @@ describe('jodaFormatToRegExp', () => { expect(jodaFormatToRegExp('MM/dd/YYYY').toString()).toMatchSnapshot(); expect(jodaFormatToRegExp('M/d/YY').toString()).toMatchSnapshot(); expect(jodaFormatToRegExp('d-M-yyyy hh:mm:ss a').toString()).toMatchSnapshot(); -<<<<<<< HEAD - expect(jodaFormatToRegExp('MM/dd/YYYY hh:mm:ss a' ).toString()).toMatchSnapshot(); - expect(jodaFormatToRegExp('YYYY-MM-dd HH:mm:ss' ).toString()).toMatchSnapshot(); -======= expect(jodaFormatToRegExp('MM/dd/YYYY hh:mm:ss a').toString()).toMatchSnapshot(); expect(jodaFormatToRegExp('YYYY-MM-dd HH:mm:ss').toString()).toMatchSnapshot(); ->>>>>>> upstream/master expect(jodaFormatToRegExp('YYYY-MM-dd HH:mm:ss.S').toString()).toMatchSnapshot(); }); it('matches dates when needed', () => { expect(jodaFormatToRegExp('d-M-yyyy hh:mm:ss a').test('26-4-1986 01:23:40 am')).toEqual(true); -<<<<<<< HEAD - expect(jodaFormatToRegExp('YYYY-MM-dd HH:mm:ss.S').test('26-4-1986 01:23:40 am')).toEqual(false); - }); - -======= expect(jodaFormatToRegExp('YYYY-MM-dd HH:mm:ss.S').test('26-4-1986 01:23:40 am')).toEqual( false, ); }); ->>>>>>> upstream/master }); diff --git a/web-console/src/utils/joda-to-regexp.ts b/web-console/src/utils/joda-to-regexp.ts index 1c885b4a7212..10ca006986a3 100644 --- a/web-console/src/utils/joda-to-regexp.ts +++ b/web-console/src/utils/joda-to-regexp.ts @@ -64,11 +64,7 @@ const JODA_FRAGMENT_TO_REG_EXP_STRING: Record = { SS: '[0-9]{2,3}', SSS: '[0-9]{3}', z: TEXT, -<<<<<<< HEAD - Z: TEXT -======= Z: TEXT, ->>>>>>> upstream/master }; export function jodaFormatToRegExp(jodaFormat: string): RegExp { diff --git a/web-console/src/utils/sampler.ts b/web-console/src/utils/sampler.ts index 81b249d5947d..1009bb0780ae 100644 --- a/web-console/src/utils/sampler.ts +++ b/web-console/src/utils/sampler.ts @@ -304,13 +304,8 @@ export async function sampleForTimestamp( }, }, samplerConfig: Object.assign({}, BASE_SAMPLER_CONFIG, { -<<<<<<< HEAD - cacheKey: sampleColumns.cacheKey || cacheKey - }) -======= cacheKey: sampleColumns.cacheKey || cacheKey, }), ->>>>>>> upstream/master }; const sampleTime = await postToSampler(sampleSpec, 'timestamp-time'); diff --git a/web-console/src/views/home-view/home-view.tsx b/web-console/src/views/home-view/home-view.tsx index 91eab13c607e..1babeb305a0d 100644 --- a/web-console/src/views/home-view/home-view.tsx +++ b/web-console/src/views/home-view/home-view.tsx @@ -19,10 +19,7 @@ import { Card, H5, Icon } from '@blueprintjs/core'; import { IconName, IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -<<<<<<< HEAD -======= import { sum } from 'd3-array'; ->>>>>>> upstream/master import React from 'react'; import { UrlBaser } from '../../singletons/url-baser'; diff --git a/web-console/src/views/load-data-view/filter-table/filter-table.spec.tsx b/web-console/src/views/load-data-view/filter-table/filter-table.spec.tsx index 8678a9e4dc5f..effc22b4fd19 100644 --- a/web-console/src/views/load-data-view/filter-table/filter-table.spec.tsx +++ b/web-console/src/views/load-data-view/filter-table/filter-table.spec.tsx @@ -28,21 +28,6 @@ describe('filter table', () => { rows: [ { raw: `{"c1":"hello"}`, -<<<<<<< HEAD - parsed: { c1: 'hello' } - } - ] - }; - - const filterTable = null} - onFilterSelect={() => null} - />; -======= parsed: { c1: 'hello' }, }, ], @@ -58,7 +43,6 @@ describe('filter table', () => { onFilterSelect={() => null} /> ); ->>>>>>> upstream/master const { container } = render(filterTable); expect(container.firstChild).toMatchSnapshot(); diff --git a/web-console/src/views/load-data-view/filter-table/filter-table.tsx b/web-console/src/views/load-data-view/filter-table/filter-table.tsx index ec4166cf83ad..25756f7f0a93 100644 --- a/web-console/src/views/load-data-view/filter-table/filter-table.tsx +++ b/web-console/src/views/load-data-view/filter-table/filter-table.tsx @@ -22,21 +22,12 @@ import ReactTable from 'react-table'; import { TableCell } from '../../../components'; import { caseInsensitiveContains, filterMap } from '../../../utils'; -<<<<<<< HEAD -import { DruidFilter, Transform } from '../../../utils/ingestion-spec'; -import { HeaderAndRows } from '../../../utils/sampler'; - -import './filter-table.scss'; - -export interface FilterTableProps extends React.Props { -======= import { DruidFilter } from '../../../utils/ingestion-spec'; import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; import './filter-table.scss'; export interface FilterTableProps { ->>>>>>> upstream/master sampleData: HeaderAndRows; columnFilter: string; dimensionFilters: DruidFilter[]; @@ -47,54 +38,6 @@ export interface FilterTableProps { export class FilterTable extends React.PureComponent { render() { -<<<<<<< HEAD - const { sampleData, columnFilter, dimensionFilters, selectedFilterIndex, onShowGlobalFilter, onFilterSelect } = this.props; - - return { - if (!caseInsensitiveContains(columnName, columnFilter)) return null; - const timestamp = columnName === '__time'; - const filterIndex = dimensionFilters.findIndex(f => f.dimension === columnName); - const filter = dimensionFilters[filterIndex]; - - const columnClassName = classNames({ - filtered: filter, - selected: filter && filterIndex === selectedFilterIndex - }); - return { - Header: ( -
    { - if (timestamp) { - onShowGlobalFilter(); - } else if (filter) { - onFilterSelect(filter, filterIndex); - } else { - onFilterSelect({ type: 'selector', dimension: columnName, value: '' }, -1); - } - }} - > -
    {columnName}
    -
    - {filter ? `(filtered)` : ''}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} - />; -======= const { sampleData, columnFilter, @@ -148,6 +91,5 @@ export class FilterTable extends React.PureComponent { sortable={false} /> ); ->>>>>>> upstream/master } } diff --git a/web-console/src/views/load-data-view/load-data-view.tsx b/web-console/src/views/load-data-view/load-data-view.tsx index fd737e7dd20a..85200b7f61ed 100644 --- a/web-console/src/views/load-data-view/load-data-view.tsx +++ b/web-console/src/views/load-data-view/load-data-view.tsx @@ -37,10 +37,7 @@ import { import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; import classNames from 'classnames'; -<<<<<<< HEAD -======= import memoize from 'memoize-one'; ->>>>>>> upstream/master import React from 'react'; import { @@ -60,14 +57,9 @@ import { getDruidErrorMessage, localStorageGet, LocalStorageKeys, -<<<<<<< HEAD - localStorageSet, memoize, parseJson, - QueryState -======= localStorageSet, parseJson, QueryState, ->>>>>>> upstream/master } from '../../utils'; import { possibleDruidFormatForValues } from '../../utils/druid-time'; import { updateSchemaWithSample } from '../../utils/druid-type'; @@ -76,13 +68,9 @@ import { DimensionMode, DimensionSpec, DimensionsSpec, -<<<<<<< HEAD - DruidFilter, EMPTY_ARRAY, EMPTY_OBJECT, -======= DruidFilter, EMPTY_ARRAY, EMPTY_OBJECT, ->>>>>>> upstream/master fillDataSourceName, fillParser, FlattenField, @@ -107,11 +95,7 @@ import { getTuningSpecFormFields, GranularitySpec, hasParallelAbility, -<<<<<<< HEAD - IngestionComboType, -======= IngestionComboTypeWithExtra, ->>>>>>> upstream/master IngestionSpec, IoConfig, isColumnTimestampSpec, @@ -916,10 +900,7 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master let issue: string | null = null; if (issueWithIoConfig(ioConfig)) { @@ -957,12 +938,6 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master const isBlank = !parseSpec.format; const canFlatten = parseSpec.format === 'json'; @@ -1018,19 +992,7 @@ export class LoadDataView extends React.PureComponent
    -<<<<<<< HEAD - -
    ; -======= ); ->>>>>>> upstream/master } let sugestedFlattenFields: FlattenField[] | null = null; @@ -1118,15 +1080,9 @@ export class LoadDataView extends React.PureComponent { this.setState({ selectedFlattenFieldIndex: index, -<<<<<<< HEAD - selectedFlattenField: field - }); - } -======= selectedFlattenField: field, }); }; ->>>>>>> upstream/master renderFlattenControls() { const { spec, selectedFlattenField, selectedFlattenFieldIndex } = this.state; @@ -1219,13 +1175,9 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master let issue: string | null = null; if (issueWithIoConfig(ioConfig)) { @@ -1259,12 +1211,6 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master }); } renderTimestampStep() { const { spec, columnFilter, specialColumnsOnly, timestampQueryState } = this.state; const parseSpec: ParseSpec = deepGet(spec, 'dataSchema.parser.parseSpec') || EMPTY_OBJECT; -<<<<<<< HEAD - const timestampSpec: TimestampSpec = deepGet(spec, 'dataSchema.parser.parseSpec.timestampSpec') || EMPTY_OBJECT; -======= const timestampSpec: TimestampSpec = deepGet(spec, 'dataSchema.parser.parseSpec.timestampSpec') || EMPTY_OBJECT; ->>>>>>> upstream/master const timestampSpecFromColumn = isColumnTimestampSpec(timestampSpec); const isBlank = !parseSpec.format; @@ -1303,59 +1244,6 @@ export class LoadDataView extends React.PureComponent{`Error: ${timestampQueryState.error}`}; } else if (timestampQueryState.data) { -<<<<<<< HEAD - mainFill =
    -
    - this.setState({ columnFilter })} - placeholder="Search columns" - /> - this.setState({ specialColumnsOnly: !specialColumnsOnly })} - /> -
    - -
    ; - } - - return <> -
    {mainFill}
    -
    - -

    - Druid partitions data based on the primary time column of your data. - This column is stored internally in Druid as __time. - Please specify the primary time column. - If you do not have any time columns, you can choose "Constant Value" to create a default one. -

    -

    - Click "Preview" to check if Druid can properly parse your time values. -

    -
    - - -
    -<<<<<<< HEAD - -
    ; -======= ); ->>>>>>> upstream/master } return ( @@ -1886,24 +1718,14 @@ export class LoadDataView extends React.PureComponent { this.setState({ showGlobalFilter: true }); -<<<<<<< HEAD - } -======= }; ->>>>>>> upstream/master private onFilterSelect = (filter: DruidFilter, index: number) => { this.setState({ selectedFilterIndex: index, -<<<<<<< HEAD - selectedFilter: filter - }); - } -======= selectedFilter: filter, }); }; ->>>>>>> upstream/master renderColumnFilterControls() { const { spec, selectedFilter, selectedFilterIndex } = this.state; @@ -2050,15 +1872,10 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master let issue: string | null = null; if (issueWithIoConfig(ioConfig)) { @@ -2092,18 +1909,6 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master const rollup: boolean = Boolean(deepGet(spec, 'dataSchema.granularitySpec.rollup')); const somethingSelected = Boolean(selectedDimensionSpec || selectedMetricSpec); const dimensionMode = getDimensionMode(spec); @@ -2139,65 +1943,6 @@ export class LoadDataView extends React.PureComponent{`Error: ${schemaQueryState.error}`}; } else if (schemaQueryState.data) { -<<<<<<< HEAD - mainFill =
    -
    - this.setState({ columnFilter })} - placeholder="Search columns" - /> -
    - -
    ; - } - - return <> -
    {mainFill}
    -
    - -

    - Each column in Druid must have an assigned type (string, long, float, complex, etc). - Default primitive types have been automatically assigned to your columns. - If you want to change the type, click on the column header. -

    -

    - Select whether or not you want to roll-up your data. -

    -
    - { - !somethingSelected && - <> - - this.setState({ newDimensionMode: dimensionMode === 'specific' ? 'auto-detect' : 'specific' })} - label="Explicitly specify dimension list" - /> - -

    - Select whether or not you want to set an explicit list of dimensions and metrics. - Explicitly setting dimensions and metrics can lead to better compression and performance. - If you disable this option, Druid will try to auto-detect fields in your data and treat them as individual columns. -

    -
    - } - position="left-bottom" - > - - - - { - dimensionMode === 'auto-detect' && -======= mainFill = (
    @@ -2323,7 +2068,6 @@ export class LoadDataView extends React.PureComponent ->>>>>>> upstream/master >>>>>> upstream/master ) => { this.setState({ selectedDimensionSpec, selectedDimensionSpecIndex, selectedMetricSpec, -<<<<<<< HEAD - selectedMetricSpecIndex - }); - } -======= selectedMetricSpecIndex, }); }; ->>>>>>> upstream/master renderChangeRollupAction() { const { newRollup, spec, sampleStrategy, cacheKey } = this.state; @@ -2483,12 +2217,6 @@ export class LoadDataView extends React.PureComponent { -<<<<<<< HEAD - const curDimensions = deepGet(spec, `dataSchema.parser.parseSpec.dimensionsSpec.dimensions`) || EMPTY_ARRAY; - if (curDimensions.length <= 1) return; // Guard against removing the last dimension, ToDo: some better feedback here would be good - - this.updateSpec(deepDelete(spec, `dataSchema.parser.parseSpec.dimensionsSpec.dimensions.${selectedDimensionSpecIndex}`)); -======= this.updateSpec( deepSet( spec, @@ -2496,7 +2224,6 @@ export class LoadDataView extends React.PureComponent>>>>>> upstream/master closeAndQuery(); }} /> @@ -2628,60 +2355,6 @@ export class LoadDataView extends React.PureComponent -
    -
    Primary partitioning (by time)
    - - This spec is used to generated segments with uniform intervals. - - }, - { - name: 'segmentGranularity', - type: 'string', - suggestions: ['HOUR', 'DAY', 'WEEK', 'MONTH', 'YEAR'], - isDefined: (g: GranularitySpec) => g.type === 'uniform', - info: <> - The granularity to create time chunks at. - Multiple segments can be created per time chunk. - For example, with 'DAY' segmentGranularity, the events of the same day fall into the same time chunk which can be optionally further partitioned into multiple segments based on other configurations and input size. - - } - ]} - model={granularitySpec} - onChange={g => this.updateSpec(deepSet(spec, 'dataSchema.granularitySpec', g))} - /> -
    -
    -
    Secondary partitioning
    - this.updateSpec(deepSet(spec, 'tuningConfig', t))} - /> -
    -
    - -

    - Optional -

    -

    - Configure how Druid will partition data. -

    -
    - {this.renderParallelPickerIfNeeded()} -
    - {this.renderNextBar({})} - ; -======= const granularitySpec: GranularitySpec = deepGet(spec, 'dataSchema.granularitySpec') || EMPTY_OBJECT; @@ -2734,7 +2407,6 @@ export class LoadDataView extends React.PureComponent ); ->>>>>>> upstream/master } // ================================================================== diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss index 654eb04eb2bf..294617e77776 100644 --- a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.scss @@ -33,11 +33,7 @@ padding: 10px; .parse-error { -<<<<<<< HEAD - color: #9E2B0E; -======= color: #9e2b0e; ->>>>>>> upstream/master margin-bottom: 12px; } } diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx index 953dddbb9156..497795545454 100644 --- a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.spec.tsx @@ -28,21 +28,6 @@ describe('parse data table', () => { rows: [ { raw: `{"c1":"hello"}`, -<<<<<<< HEAD - parsed: { c1: 'hello' } - } - ] - }; - - const parseDataTable = null} - />; -======= parsed: { c1: 'hello' }, }, ], @@ -59,7 +44,6 @@ describe('parse data table', () => { onFlattenFieldSelect={() => null} /> ); ->>>>>>> upstream/master const { container } = render(parseDataTable); expect(container.firstChild).toMatchSnapshot(); diff --git a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx index 23e5efe2b2fb..b30d08b477e4 100644 --- a/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx +++ b/web-console/src/views/load-data-view/parse-data-table/parse-data-table.tsx @@ -27,84 +27,18 @@ import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; import './parse-data-table.scss'; -<<<<<<< HEAD -export interface ParseDataTableProps extends React.Props { -======= export interface ParseDataTableProps { ->>>>>>> upstream/master sampleData: HeaderAndRows; columnFilter: string; canFlatten: boolean; flattenedColumnsOnly: boolean; flattenFields: FlattenField[]; onFlattenFieldSelect: (field: FlattenField, index: number) => void; -<<<<<<< HEAD -======= openModal: (str: string) => void; ->>>>>>> upstream/master } export class ParseDataTable extends React.PureComponent { render() { -<<<<<<< HEAD - const { sampleData, columnFilter, canFlatten, flattenedColumnsOnly, flattenFields, onFlattenFieldSelect } = this.props; - - return { - if (!caseInsensitiveContains(columnName, columnFilter)) return null; - const flattenFieldIndex = flattenFields.findIndex(f => f.name === columnName); - if (flattenFieldIndex === -1 && flattenedColumnsOnly) return null; - const flattenField = flattenFields[flattenFieldIndex]; - return { - Header: ( -
    { - if (!flattenField) return; - onFlattenFieldSelect(flattenField, flattenFieldIndex); - }} - > -
    {columnName}
    -
    - {flattenField ? `${flattenField.type}: ${flattenField.expr}` : ''}  -
    -
    - ), - id: String(i), - accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, - Cell: row => { - if (row.original.unparseable) { - return ; - } - return ; - }, - headerClassName: classNames({ - flattened: flattenField - }) - }; - })} - SubComponent={rowInfo => { - const { raw, error } = rowInfo.original; - const parsedJson: any = parseJson(raw); - - if (!error && parsedJson && canFlatten) { - return
    -            {'Original row: ' + JSON.stringify(parsedJson, null, 2)}
    -          
    ; - } else { - return
    - {error &&
    {error}
    } -
    {'Original row: ' + rowInfo.original.raw}
    -
    ; - } - }} - defaultPageSize={50} - showPagination={false} - sortable={false} - />; -======= const { sampleData, columnFilter, @@ -175,6 +109,5 @@ export class ParseDataTable extends React.PureComponent { sortable={false} /> ); ->>>>>>> upstream/master } } diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss index 7df19b8ca5b8..3122c374057c 100644 --- a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.scss @@ -17,9 +17,5 @@ */ .parse-time-table { -<<<<<<< HEAD - -======= position: relative; ->>>>>>> upstream/master } diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx index 7235760a5b1d..7ce9709197e8 100644 --- a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.spec.tsx @@ -30,22 +30,6 @@ describe('parse time table', () => { rows: [ { raw: `{"c1":"hello"}`, -<<<<<<< HEAD - parsed: { c1: 'hello' } - } - ] - }; - - const parseTimeTable = null} - />; -======= parsed: { c1: 'hello' }, }, ], @@ -62,7 +46,6 @@ describe('parse time table', () => { onTimestampColumnSelect={() => null} /> ); ->>>>>>> upstream/master const { container } = render(parseTimeTable); expect(container.firstChild).toMatchSnapshot(); diff --git a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx index c6827078f3c0..4a8c3e1a0c90 100644 --- a/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx +++ b/web-console/src/views/load-data-view/parse-time-table/parse-time-table.tsx @@ -23,24 +23,16 @@ import ReactTable from 'react-table'; import { TableCell } from '../../../components'; import { caseInsensitiveContains, filterMap } from '../../../utils'; import { possibleDruidFormatForValues } from '../../../utils/druid-time'; -<<<<<<< HEAD -import { getTimestampSpecColumn, isColumnTimestampSpec, TimestampSpec } from '../../../utils/ingestion-spec'; -======= import { getTimestampSpecColumn, isColumnTimestampSpec, TimestampSpec, } from '../../../utils/ingestion-spec'; ->>>>>>> upstream/master import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; import './parse-time-table.scss'; -<<<<<<< HEAD -export interface ParseTimeTableProps extends React.Props { -======= export interface ParseTimeTableProps { ->>>>>>> upstream/master sampleBundle: { headerAndRows: HeaderAndRows; timestampSpec: TimestampSpec; @@ -52,78 +44,16 @@ export interface ParseTimeTableProps { export class ParseTimeTable extends React.PureComponent { render() { -<<<<<<< HEAD - const { sampleBundle, columnFilter, possibleTimestampColumnsOnly, onTimestampColumnSelect } = this.props; -======= const { sampleBundle, columnFilter, possibleTimestampColumnsOnly, onTimestampColumnSelect, } = this.props; ->>>>>>> upstream/master const { headerAndRows, timestampSpec } = sampleBundle; const timestampSpecColumn = getTimestampSpecColumn(timestampSpec); const timestampSpecFromColumn = isColumnTimestampSpec(timestampSpec); -<<<<<<< HEAD - return { - const timestamp = columnName === '__time'; - if (!timestamp && !caseInsensitiveContains(columnName, columnFilter)) return null; - const selected = timestampSpec.column === columnName; - const possibleFormat = timestamp ? null : possibleDruidFormatForValues(filterMap(headerAndRows.rows, d => d.parsed ? d.parsed[columnName] : null)); - if (possibleTimestampColumnsOnly && !timestamp && !possibleFormat) return null; - - const columnClassName = classNames({ - timestamp, - selected - }); - return { - Header: ( -
    { - onTimestampColumnSelect({ - column: columnName, - format: possibleFormat || '!!! Could not auto detect a format !!!' - }); - }} - > -
    {columnName}
    -
    - { - timestamp ? - (timestampSpecFromColumn ? `from: '${timestampSpecColumn}'` : `mv: ${timestampSpec.missingValue}`) : - (possibleFormat || '') - }  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, - Cell: row => { - if (columnName === '__error__') { - return ; - } - if (row.original.unparseable) { - return ; - } - return ; - }, - minWidth: timestamp ? 200 : 100, - resizable: !timestamp - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} - />; -======= return ( { sortable={false} /> ); ->>>>>>> upstream/master } } diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.scss b/web-console/src/views/load-data-view/schema-table/schema-table.scss index 5a688a8294a0..ca0df900f5e2 100644 --- a/web-console/src/views/load-data-view/schema-table/schema-table.scss +++ b/web-console/src/views/load-data-view/schema-table/schema-table.scss @@ -21,17 +21,12 @@ &.dimension { background: rgba(38, 170, 201, 0.5); -<<<<<<< HEAD - &.long { background: rgba(19, 129, 201, 0.5); } - &.float { background: rgba(25, 145, 201, 0.5); } -======= &.long { background: rgba(19, 129, 201, 0.5); } &.float { background: rgba(25, 145, 201, 0.5); } ->>>>>>> upstream/master } &.metric { @@ -43,17 +38,12 @@ &.dimension { background: rgba(38, 170, 201, 0.1); -<<<<<<< HEAD - &.long { background: rgba(19, 129, 201, 0.1); } - &.float { background: rgba(25, 145, 201, 0.1); } -======= &.long { background: rgba(19, 129, 201, 0.1); } &.float { background: rgba(25, 145, 201, 0.1); } ->>>>>>> upstream/master } &.metric { diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx b/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx index ba618acc304e..dd1f5d4ceeb0 100644 --- a/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx +++ b/web-console/src/views/load-data-view/schema-table/schema-table.spec.tsx @@ -28,24 +28,6 @@ describe('schema table', () => { rows: [ { raw: `{"c1":"hello"}`, -<<<<<<< HEAD - parsed: { c1: 'hello' } - } - ] - }; - - const schemaTable = null} - />; -======= parsed: { c1: 'hello' }, }, ], @@ -64,7 +46,6 @@ describe('schema table', () => { onDimensionOrMetricSelect={() => null} /> ); ->>>>>>> upstream/master const { container } = render(schemaTable); expect(container.firstChild).toMatchSnapshot(); diff --git a/web-console/src/views/load-data-view/schema-table/schema-table.tsx b/web-console/src/views/load-data-view/schema-table/schema-table.tsx index 33577ddbc908..aac4e56601d4 100644 --- a/web-console/src/views/load-data-view/schema-table/schema-table.tsx +++ b/web-console/src/views/load-data-view/schema-table/schema-table.tsx @@ -29,21 +29,13 @@ import { getDimensionSpecType, getMetricSpecName, inflateDimensionSpec, -<<<<<<< HEAD - MetricSpec, TimestampSpec -======= MetricSpec, ->>>>>>> upstream/master } from '../../../utils/ingestion-spec'; import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; import './schema-table.scss'; -<<<<<<< HEAD -export interface SchemaTableProps extends React.Props { -======= export interface SchemaTableProps { ->>>>>>> upstream/master sampleBundle: { headerAndRows: HeaderAndRows; dimensionsSpec: DimensionsSpec; @@ -56,95 +48,12 @@ export interface SchemaTableProps { selectedDimensionSpec: DimensionSpec | null, selectedDimensionSpecIndex: number, selectedMetricSpec: MetricSpec | null, -<<<<<<< HEAD - selectedMetricSpecIndex: number -======= selectedMetricSpecIndex: number, ->>>>>>> upstream/master ) => void; } export class SchemaTable extends React.PureComponent { render() { -<<<<<<< HEAD - const { sampleBundle, columnFilter, selectedDimensionSpecIndex, selectedMetricSpecIndex, onDimensionOrMetricSelect } = this.props; - const { headerAndRows, dimensionsSpec, metricsSpec } = sampleBundle; - - const dimensionMetricSortedHeader = sortWithPrefixSuffix(headerAndRows.header, ['__time'], metricsSpec.map(getMetricSpecName)); - - return { - if (!caseInsensitiveContains(columnName, columnFilter)) return null; - - const metricSpecIndex = metricsSpec.findIndex(m => getMetricSpecName(m) === columnName); - const metricSpec = metricsSpec[metricSpecIndex]; - - if (metricSpec) { - const columnClassName = classNames('metric', { - selected: metricSpec && metricSpecIndex === selectedMetricSpecIndex - }); - return { - Header: ( -
    onDimensionOrMetricSelect(null, -1, metricSpec, metricSpecIndex)} - > -
    {columnName}
    -
    - {metricSpec.type}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - } else { - const timestamp = columnName === '__time'; - const dimensionSpecIndex = dimensionsSpec.dimensions ? dimensionsSpec.dimensions.findIndex(d => getDimensionSpecName(d) === columnName) : -1; - const dimensionSpec = dimensionsSpec.dimensions ? dimensionsSpec.dimensions[dimensionSpecIndex] : null; - const dimensionSpecType = dimensionSpec ? getDimensionSpecType(dimensionSpec) : null; - - const columnClassName = classNames(timestamp ? 'timestamp' : 'dimension', dimensionSpecType || 'string', { - selected: dimensionSpec && dimensionSpecIndex === selectedDimensionSpecIndex - }); - return { - Header: ( -
    { - if (timestamp) { - onDimensionOrMetricSelect(null, -1, null, -1); - return; - } - - if (!dimensionSpec) return; - onDimensionOrMetricSelect(inflateDimensionSpec(dimensionSpec), dimensionSpecIndex, null, -1); - }} - > -
    {columnName}
    -
    - {timestamp ? 'long (time column)' : (dimensionSpecType || 'string (auto)')}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: (row: SampleEntry) => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - } - })} - defaultPageSize={50} - showPagination={false} - sortable={false} - />; -======= const { sampleBundle, columnFilter, @@ -246,6 +155,5 @@ export class SchemaTable extends React.PureComponent { sortable={false} /> ); ->>>>>>> upstream/master } } diff --git a/web-console/src/views/load-data-view/transform-table/transform-table.tsx b/web-console/src/views/load-data-view/transform-table/transform-table.tsx index 0787c4b87dee..36e721e8366c 100644 --- a/web-console/src/views/load-data-view/transform-table/transform-table.tsx +++ b/web-console/src/views/load-data-view/transform-table/transform-table.tsx @@ -24,19 +24,11 @@ import { TableCell } from '../../../components'; import { caseInsensitiveContains, filterMap } from '../../../utils'; import { escapeColumnName } from '../../../utils/druid-expression'; import { Transform } from '../../../utils/ingestion-spec'; -<<<<<<< HEAD -import { HeaderAndRows } from '../../../utils/sampler'; - -import './transform-table.scss'; - -export interface TransformTableProps extends React.Props { -======= import { HeaderAndRows, SampleEntry } from '../../../utils/sampler'; import './transform-table.scss'; export interface TransformTableProps { ->>>>>>> upstream/master sampleData: HeaderAndRows; columnFilter: string; transformedColumnsOnly: boolean; @@ -47,57 +39,6 @@ export interface TransformTableProps { export class TransformTable extends React.PureComponent { render() { -<<<<<<< HEAD - const { sampleData, columnFilter, transformedColumnsOnly, transforms, selectedTransformIndex, onTransformSelect } = this.props; - - return { - if (!caseInsensitiveContains(columnName, columnFilter)) return null; - const timestamp = columnName === '__time'; - const transformIndex = transforms.findIndex(f => f.name === columnName); - if (transformIndex === -1 && transformedColumnsOnly) return null; - const transform = transforms[transformIndex]; - - const columnClassName = classNames({ - transformed: transform, - selected: transform && transformIndex === selectedTransformIndex - }); - return { - Header: ( -
    { - if (transform) { - onTransformSelect(transform, transformIndex); - } else { - onTransformSelect({ - type: 'expression', - name: columnName, - expression: escapeColumnName(columnName) - }, transformIndex); - } - }} - > -
    {columnName}
    -
    - {transform ? `= ${transform.expression}` : ''}  -
    -
    - ), - headerClassName: columnClassName, - className: columnClassName, - id: String(i), - accessor: row => row.parsed ? row.parsed[columnName] : null, - Cell: row => - }; - })} - defaultPageSize={50} - showPagination={false} - sortable={false} - />; -======= const { sampleData, columnFilter, @@ -159,6 +100,5 @@ export class TransformTable extends React.PureComponent { sortable={false} /> ); ->>>>>>> upstream/master } } diff --git a/web-console/src/views/lookups-view/lookups-view.tsx b/web-console/src/views/lookups-view/lookups-view.tsx index 85e61a12bd77..2b6bb19bbbca 100644 --- a/web-console/src/views/lookups-view/lookups-view.tsx +++ b/web-console/src/views/lookups-view/lookups-view.tsx @@ -19,10 +19,6 @@ import { Button, Intent } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import axios from 'axios'; -<<<<<<< HEAD -import classNames from 'classnames'; -======= ->>>>>>> upstream/master import React from 'react'; import ReactTable from 'react-table'; diff --git a/web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap b/web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap deleted file mode 100644 index 80ab5c401f89..000000000000 --- a/web-console/src/views/sql-view/sql-control/__snapshots__/sql-control.spec.tsx.snap +++ /dev/null @@ -1,174 +0,0 @@ -// Jest Snapshot v1, https://goo.gl/fbAQLP - -exports[`sql control matches snapshot 1`] = ` -
    -
    -
    -