From ebe647cecf4266fdaa085832649990d8574e77f2 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 24 Apr 2026 16:43:34 -0700 Subject: [PATCH 1/4] refactor: remove zookeeper segment announcement and discovery --- docs/configuration/index.md | 25 - docs/design/broker.md | 4 +- docs/design/coordinator.md | 14 +- docs/design/historical.md | 6 +- docs/design/zookeeper.md | 38 +- .../druid/indexing/common/TaskToolbox.java | 18 - .../indexing/common/TaskToolboxFactory.java | 5 - .../SeekableStreamIndexTaskRunner.java | 2 - .../indexing/common/TaskToolboxTest.java | 2 - .../SingleTaskBackgroundRunnerTest.java | 1 - .../indexing/overlord/TaskLifecycleTest.java | 2 - .../overlord/TestTaskToolboxFactory.java | 9 - .../SeekableStreamIndexTaskRunnerTest.java | 2 - .../SeekableStreamIndexTaskTestBase.java | 2 - .../worker/WorkerTaskManagerTest.java | 1 - .../worker/WorkerTaskMonitorTest.java | 1 - .../client/BatchServerInventoryView.java | 424 --------- .../BatchServerInventoryViewProvider.java | 57 -- ...teredBatchServerInventoryViewProvider.java | 55 -- .../FilteredServerInventoryViewProvider.java | 1 - .../client/ServerInventoryViewProvider.java | 1 - .../discovery/NoopServiceAnnouncer.java | 39 - .../apache/druid/guice/AnnouncerModule.java | 26 +- .../apache/druid/guice/ServerViewModule.java | 22 +- .../BatchDataSegmentAnnouncer.java | 421 +-------- .../BatchDataSegmentAnnouncerProvider.java | 39 - .../CuratorDataSegmentServerAnnouncer.java | 103 --- .../DataSegmentAnnouncerProvider.java | 34 - .../DataSegmentServerAnnouncer.java | 50 - .../SegmentCacheBootstrapper.java | 31 - .../server/coordination/ZkCoordinator.java | 113 --- .../BatchDataSegmentAnnouncerConfig.java | 75 -- .../server/initialization/ZkPathsConfig.java | 40 +- .../druid/client/BrokerServerViewTest.java | 871 ------------------ .../client/CoordinatorServerViewTest.java | 502 ---------- .../client/BatchServerInventoryViewTest.java | 517 ----------- .../apache/druid/curator/CuratorTestBase.java | 85 -- .../initialization/ZkPathsConfigTest.java | 3 - .../SegmentCacheBootstrapperCacheTest.java | 15 - .../SegmentCacheBootstrapperTest.java | 23 - .../SegmentLoadDropHandlerTest.java | 4 +- .../coordination/TestDataServerAnnouncer.java | 53 -- .../coordination/ZkCoordinatorTest.java | 102 -- .../BatchDataSegmentAnnouncerTest.java | 624 +------------ .../java/org/apache/druid/cli/CliBroker.java | 15 - .../org/apache/druid/cli/CliHistorical.java | 17 - .../java/org/apache/druid/cli/CliIndexer.java | 6 - .../java/org/apache/druid/cli/CliPeon.java | 9 - .../org/apache/druid/cli/CliBrokerTest.java | 1 - 49 files changed, 138 insertions(+), 4372 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java delete mode 100644 server/src/main/java/org/apache/druid/client/BatchServerInventoryViewProvider.java delete mode 100644 server/src/main/java/org/apache/druid/client/FilteredBatchServerInventoryViewProvider.java delete mode 100644 server/src/main/java/org/apache/druid/curator/discovery/NoopServiceAnnouncer.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncerProvider.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordination/DataSegmentServerAnnouncer.java delete mode 100644 server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java delete mode 100644 server/src/main/java/org/apache/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java delete mode 100644 server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java delete mode 100644 server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java delete mode 100644 server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java delete mode 100644 server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 504997ea9bf9..4ca3c56bb8e9 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -154,9 +154,7 @@ Druid interacts with ZooKeeper through a set of standard path configurations. We |Property|Description|Default| |--------|-----------|-------| |`druid.zk.paths.base`|Base ZooKeeper path.|`/druid`| -|`druid.zk.paths.propertiesPath`|ZooKeeper properties path.|`${druid.zk.paths.base}/properties`| |`druid.zk.paths.announcementsPath`|Druid service announcement path.|`${druid.zk.paths.base}/announcements`| -|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid services announce their segments.|`${druid.zk.paths.base}/segments`| |`druid.zk.paths.coordinatorPath`|Used by the Coordinator for leader election.|`${druid.zk.paths.base}/coordinator`| The indexing service also uses its own set of paths. These configs can be included in the common configuration. @@ -648,27 +646,6 @@ This config is used to find the [Coordinator](../design/coordinator.md) using Cu |--------|-----------|-------| |`druid.selectors.coordinator.serviceName`|The druid.service name of the Coordinator service. To start the Coordinator with a different name, set it with this property. |druid/coordinator| -### Announcing segments - -You can configure how to announce and unannounce Znodes in ZooKeeper (using Curator). For normal operations you do not need to override any of these configs. - -#### Batch data segment announcer - -In current Druid, multiple data segments may be announced under the same Znode. - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50| -|`druid.announcer.maxBytesPerNode`|Max byte size for Znode. Allowed range is [1024, 1048576].|524288| -|`druid.announcer.skipDimensionsAndMetrics`|Skip Dimensions and Metrics list from segment announcements. NOTE: Enabling this will also remove the dimensions and metrics list from Coordinator and Broker endpoints.|false| -|`druid.announcer.skipLoadSpec`|Skip segment LoadSpec from segment announcements. NOTE: Enabling this will also remove the loadspec from Coordinator and Broker endpoints.|false| - -If you want to turn off the batch data segment announcer, you can add a property to skip announcing segments. **You do not want to enable this config if you have any services using `batch` for `druid.serverview.type`** - -|Property|Description|Default| -|--------|-----------|-------| -|`druid.announcer.skipSegmentAnnouncementOnZk`|Skip announcing segments to ZooKeeper. Note that the batch server view will not work if this is set to true.|false| - ### JavaScript Druid supports dynamic runtime extension through JavaScript functions. This functionality can be configured through @@ -795,7 +772,6 @@ These Coordinator static configurations can be defined in the `coordinator/runti |Property|Possible values|Description|Default| |--------|---------------|-----------|-------| -|`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of ZooKeeper.|http| |`druid.coordinator.segment.awaitInitializationOnStart`|true or false|Whether the Coordinator will wait for its view of segments to fully initialize before starting up. If set to 'true', the Coordinator's HTTP server will not start up, and the Coordinator will not announce itself as available, until the server view is initialized.|true| ##### Metadata retrieval @@ -1928,7 +1904,6 @@ See [cache configuration](#cache-configuration) for how to configure cache setti |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| -|`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of ZooKeeper.|http| |`druid.broker.segment.watchedTiers`|List of strings|The Broker watches segment announcements from processes that serve segments to build a cache to relate each process to the segments it serves. This configuration allows the Broker to only consider segments being served from a list of tiers. By default, Broker considers all tiers. This can be used to partition your dataSources in specific Historical tiers and configure brokers in partitions so that they are only queryable for specific dataSources. This config is mutually exclusive from `druid.broker.segment.ignoredTiers` and at most one of these can be configured on a Broker.|none| |`druid.broker.segment.ignoredTiers`|List of strings|The Broker watches segment announcements from processes that serve segments to build a cache to relate each process to the segments it serves. This configuration allows the Broker to ignore the segments being served from a list of tiers. By default, Broker considers all tiers. This config is mutually exclusive from `druid.broker.segment.watchedTiers` and at most one of these can be configured on a Broker.|none| |`druid.broker.segment.watchedDataSources`|List of strings|Broker watches the segment announcements from processes serving segments to build cache of which process is serving which segments, this configuration allows to only consider segments being served from a whitelist of dataSources. By default, Broker would consider all datasources. This can be used to configure brokers in partitions so that they are only queryable for specific dataSources.|none| diff --git a/docs/design/broker.md b/docs/design/broker.md index bbd6b94f2b00..800000925e52 100644 --- a/docs/design/broker.md +++ b/docs/design/broker.md @@ -24,7 +24,7 @@ sidebar_label: "Broker" --> -The Broker service routes queries in a distributed cluster setup. It interprets the metadata published to ZooKeeper about segment distribution across services and routes queries accordingly. Additionally, the Broker service consolidates result sets from individual services. +The Broker service routes queries in a distributed cluster setup. It discovers data-serving services via ZooKeeper, polls each one over HTTP for the segments it is serving, and routes queries accordingly. Additionally, the Broker service consolidates result sets from individual services. ## Configuration @@ -46,7 +46,7 @@ org.apache.druid.cli.Main server broker Most Druid queries contain an interval object that indicates a span of time for which data is requested. Similarly, Druid partitions [segments](../design/segments.md) to contain data for some interval of time and distributes the segments across a cluster. Consider a simple datasource with seven segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple services, and hence, the query will likely hit multiple services. -To determine which services to forward queries to, the Broker service first builds a view of the world from information in ZooKeeper. ZooKeeper maintains information about [Historical](../design/historical.md) and streaming ingestion [Peon](../design/peons.md) services and the segments they are serving. For every datasource in ZooKeeper, the Broker service builds a timeline of segments and the services that serve them. When queries are received for a specific datasource and interval, the Broker service performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the services that contain data for the query. The Broker service then forwards down the query to the selected services. +To determine which services to forward queries to, the Broker service first builds a view of the world from the cluster. It uses ZooKeeper to discover [Historical](../design/historical.md) and streaming ingestion [Peon](../design/peons.md) services, then polls each one over HTTP to learn the segments it is serving. For every datasource, the Broker service builds a timeline of segments and the services that serve them. When queries are received for a specific datasource and interval, the Broker service performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the services that contain data for the query. The Broker service then forwards the query to the selected services. ## Caching diff --git a/docs/design/coordinator.md b/docs/design/coordinator.md index bc4c5ebc1cba..5e3b1d37bb43 100644 --- a/docs/design/coordinator.md +++ b/docs/design/coordinator.md @@ -30,16 +30,16 @@ Coordinator service communicates to Historical services to load or drop segments The Coordinator runs its duties periodically and the time between each run is a configurable parameter. On each run, the Coordinator assesses the current state of the cluster before deciding on the appropriate actions to take. -Similar to the Broker and Historical services, the Coordinator maintains a connection to a ZooKeeper cluster for -current cluster information. The Coordinator also maintains a connection to a database containing information about -"used" segments (that is, the segments that *should* be loaded in the cluster) and the loading rules. +The Coordinator uses ZooKeeper for leader election and to discover other Druid services, and polls each Historical +service over HTTP for the current set of segments it is serving. The Coordinator also maintains a connection to a +database containing information about "used" segments (that is, the segments that *should* be loaded in the cluster) +and the loading rules. Before any unassigned segments are serviced by Historical services, the Historical services for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always -assigned to the services with least capacity to maintain a level of balance between services. The Coordinator does not -directly communicate with a Historical service when assigning it a new segment; instead the Coordinator creates some -temporary information about the new segment under load queue path of the Historical service. Once this request is seen, -the Historical service loads the segment and begins servicing it. +assigned to the services with least capacity to maintain a level of balance between services. The Coordinator sends +segment load and drop requests to each Historical service over HTTP. Once this request is received, the Historical +service loads the segment and begins servicing it. ## Configuration diff --git a/docs/design/historical.md b/docs/design/historical.md index d4a0782ba2a9..9bf5935f95ea 100644 --- a/docs/design/historical.md +++ b/docs/design/historical.md @@ -47,13 +47,13 @@ org.apache.druid.cli.Main server historical Each Historical service copies or pulls segment files from deep storage to local disk in an area called the segment cache. To configure the size and location of the segment cache on each Historical service, set the `druid.segmentCache.locations`. For more information, see [Segment cache size](../operations/basic-cluster-tuning.md#segment-cache-size). -The [Coordinator](../design/coordinator.md) controls the assignment of segments to Historicals and the balance of segments between Historicals. Historical services do not communicate directly with each other, nor do they communicate directly with the Coordinator. Instead, the Coordinator creates ephemeral entries in ZooKeeper in a [load queue path](../configuration/index.md#path-configuration). Each Historical service maintains a connection to ZooKeeper, watching those paths for segment information. +The [Coordinator](../design/coordinator.md) controls the assignment of segments to Historicals and the balance of segments between Historicals. Historical services do not communicate directly with each other. The Coordinator sends segment load and drop requests to each Historical over HTTP, and each Historical exposes an HTTP endpoint for the Coordinator to poll for the current state of its segment assignments. -When a Historical service detects a new entry in the ZooKeeper load queue, it checks its own segment cache. If no information about the segment exists there, the Historical service first retrieves metadata from ZooKeeper about the segment, including where the segment is located in deep storage and how it needs to decompress and process it. +When a Historical service receives a load request, it checks its own segment cache. If no information about the segment exists there, the Historical uses the segment metadata included in the request — including where the segment is located in deep storage and how to decompress and process it — to pull the segment from deep storage. For more information about segment metadata and Druid segments in general, see [Segments](../design/segments.md). -After a Historical service pulls down and processes a segment from deep storage, Druid advertises the segment as being available for queries from the Broker. This announcement by the Historical is made via ZooKeeper, in a [served segments path](../configuration/index.md#path-configuration). +After a Historical service pulls down and processes a segment from deep storage, Druid advertises the segment as being available for queries from the Broker. The Historical exposes its current set of served segments via an HTTP endpoint (`/druid-internal/v1/segments`), which the Broker polls to learn what data each Historical is serving. For more information about how the Broker determines what data is available for queries, see [Broker](broker.md). diff --git a/docs/design/zookeeper.md b/docs/design/zookeeper.md index df58b02a21d0..010db8d01071 100644 --- a/docs/design/zookeeper.md +++ b/docs/design/zookeeper.md @@ -29,43 +29,45 @@ Apache Druid uses [Apache ZooKeeper](http://zookeeper.apache.org/) (ZK) for mana Apache Druid supports all stable versions of ZooKeeper. For information about ZooKeeper's stable version, see [ZooKeeper releases](https://zookeeper.apache.org/releases.html). -## ZooKeeper Operations +## ZooKeeper operations -The operations that happen over ZK are +The operations that happen over ZK are: -1. [Coordinator](../design/coordinator.md) leader election -2. Segment "publishing" protocol from [Historical](../design/historical.md) -3. [Overlord](../design/overlord.md) leader election -4. [Overlord](../design/overlord.md) and [Middle Manager](../design/middlemanager.md) task management +1. [Coordinator](../design/coordinator.md) leader election +2. [Overlord](../design/overlord.md) leader election +3. Service (node) announcement and discovery — services announce their presence so other services can find them +4. [Overlord](../design/overlord.md) and [Middle Manager](../design/middlemanager.md) task management -## Coordinator Leader Election +Segment loading, dropping, and discovery no longer use ZooKeeper — they are served over HTTP. -We use the Curator [LeaderLatch](https://curator.apache.org/docs/recipes-leader-latch) recipe to perform leader election at path +## Coordinator leader election + +Druid uses the Curator [LeaderLatch](https://curator.apache.org/docs/recipes-leader-latch) recipe to perform leader election at path ``` ${druid.zk.paths.coordinatorPath}/_COORDINATOR ``` -## Segment "publishing" protocol from Historical and Realtime - -The `announcementsPath` and `liveSegmentsPath` are used for this. +## Overlord leader election -All [Historical](../design/historical.md) processes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at +Druid uses the same [LeaderLatch](https://curator.apache.org/docs/recipes-leader-latch) recipe for Overlord leader election at path ``` -${druid.zk.paths.announcementsPath}/${druid.host} +${druid.zk.paths.overlordPath}/_OVERLORD ``` -Which signifies that they exist. They will also subsequently create a permanent znode at +## Service announcement and discovery + +All Druid services publish their presence by creating an ephemeral znode at ``` -${druid.zk.paths.liveSegmentsPath}/${druid.host} +${druid.zk.paths.announcementsPath}/${druid.host} ``` -And as they load up segments, they will attach ephemeral znodes that look like +In addition, each service announces a richer `DruidNode` record (host, port, role, services) under the internal-discovery path, so that other services can enumerate cluster members by role: ``` -${druid.zk.paths.liveSegmentsPath}/${druid.host}/_segment_identifier_ +${druid.zk.paths.base}/internal-discovery/${nodeRole}/${druid.host} ``` -Processes like the [Coordinator](../design/coordinator.md) and [Broker](../design/broker.md) can then watch these paths to see which processes are currently serving which segments. +Brokers and Coordinators use this internal-discovery path to find Historicals, Peons, and Indexers. They then poll each discovered service's HTTP `/druid-internal/v1/segments` endpoint to get its current set of served segments. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 37d805103e29..78eded234e27 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -65,7 +65,6 @@ import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.tasklogs.TaskLogPusher; import org.apache.druid.timeline.DataSegment; @@ -92,7 +91,6 @@ public class TaskToolbox private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentMover dataSegmentMover; private final DataSegmentAnnouncer segmentAnnouncer; - private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; /** * Using Provider, not {@link QueryRunnerFactoryConglomerate} directly, to not require {@link @@ -153,7 +151,6 @@ public TaskToolbox( DataSegmentMover dataSegmentMover, DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, Provider queryRunnerFactoryConglomerateProvider, Provider processingConfigProvider, @@ -199,7 +196,6 @@ public TaskToolbox( this.dataSegmentMover = dataSegmentMover; this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; - this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; this.queryRunnerFactoryConglomerateProvider = queryRunnerFactoryConglomerateProvider; this.processingConfigProvider = processingConfigProvider; @@ -291,11 +287,6 @@ public DataSegmentAnnouncer getSegmentAnnouncer() return segmentAnnouncer; } - public DataSegmentServerAnnouncer getDataSegmentServerAnnouncer() - { - return serverAnnouncer; - } - public SegmentHandoffNotifierFactory getSegmentHandoffNotifierFactory() { return handoffNotifierFactory; @@ -565,7 +556,6 @@ public static class Builder private DataSegmentMover dataSegmentMover; private DataSegmentArchiver dataSegmentArchiver; private DataSegmentAnnouncer segmentAnnouncer; - private DataSegmentServerAnnouncer serverAnnouncer; private SegmentHandoffNotifierFactory handoffNotifierFactory; private Provider queryRunnerFactoryConglomerateProvider; private Provider processingConfigProvider; @@ -616,7 +606,6 @@ public Builder(TaskToolbox other) this.dataSegmentMover = other.dataSegmentMover; this.dataSegmentArchiver = other.dataSegmentArchiver; this.segmentAnnouncer = other.segmentAnnouncer; - this.serverAnnouncer = other.serverAnnouncer; this.handoffNotifierFactory = other.handoffNotifierFactory; this.queryRunnerFactoryConglomerateProvider = other.queryRunnerFactoryConglomerateProvider; this.processingConfigProvider = other.processingConfigProvider; @@ -716,12 +705,6 @@ public Builder segmentAnnouncer(final DataSegmentAnnouncer segmentAnnouncer) return this; } - public Builder serverAnnouncer(final DataSegmentServerAnnouncer serverAnnouncer) - { - this.serverAnnouncer = serverAnnouncer; - return this; - } - public Builder handoffNotifierFactory(final SegmentHandoffNotifierFactory handoffNotifierFactory) { this.handoffNotifierFactory = handoffNotifierFactory; @@ -927,7 +910,6 @@ public TaskToolbox build() dataSegmentMover, dataSegmentArchiver, segmentAnnouncer, - serverAnnouncer, handoffNotifierFactory, queryRunnerFactoryConglomerateProvider, processingConfigProvider, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 0f38c6152056..2cd5b8dd6096 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -66,7 +66,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.tasklogs.TaskLogPusher; import org.apache.druid.utils.RuntimeInfo; @@ -90,7 +89,6 @@ public class TaskToolboxFactory private final DataSegmentMover dataSegmentMover; private final DataSegmentArchiver dataSegmentArchiver; private final DataSegmentAnnouncer segmentAnnouncer; - private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentHandoffNotifierFactory handoffNotifierFactory; private final Provider queryRunnerFactoryConglomerateProvider; private final Provider processingConfigProvider; @@ -139,7 +137,6 @@ public TaskToolboxFactory( DataSegmentMover dataSegmentMover, DataSegmentArchiver dataSegmentArchiver, DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, SegmentHandoffNotifierFactory handoffNotifierFactory, Provider queryRunnerFactoryConglomerateProvider, Provider processingConfigProvider, @@ -185,7 +182,6 @@ public TaskToolboxFactory( this.dataSegmentMover = dataSegmentMover; this.dataSegmentArchiver = dataSegmentArchiver; this.segmentAnnouncer = segmentAnnouncer; - this.serverAnnouncer = serverAnnouncer; this.handoffNotifierFactory = handoffNotifierFactory; this.queryRunnerFactoryConglomerateProvider = queryRunnerFactoryConglomerateProvider; this.processingConfigProvider = processingConfigProvider; @@ -245,7 +241,6 @@ public TaskToolbox build(TaskConfig config, Task task) .dataSegmentMover(dataSegmentMover) .dataSegmentArchiver(dataSegmentArchiver) .segmentAnnouncer(segmentAnnouncer) - .serverAnnouncer(serverAnnouncer) .handoffNotifierFactory(handoffNotifierFactory) .queryRunnerFactoryConglomerateProvider(queryRunnerFactoryConglomerateProvider) .processingConfigProvider(processingConfigProvider) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c4fa66e1b9e9..47270f74a1c1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -454,7 +454,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception task.newTaskRecordSupplier(toolbox)) { this.recordSupplier = recordSupplier; if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { - toolbox.getDataSegmentServerAnnouncer().announce(); toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); } appenderator = task.newAppenderator(toolbox, segmentGenerationMetrics, rowIngestionMeters, parseExceptionHandler); @@ -949,7 +948,6 @@ public void onFailure(Throwable t) toolbox.removeMonitor(metricsMonitor); if (toolbox.getAppenderatorsManager().shouldTaskMakeNodeAnnouncements()) { toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); - toolbox.getDataSegmentServerAnnouncer().unannounce(); } rejectionPeriodUpdaterExec.shutdown(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 7269b3f176a7..639a3410ab2f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -58,7 +58,6 @@ import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.utils.JvmUtils; import org.apache.druid.utils.RuntimeInfo; @@ -130,7 +129,6 @@ public void setUp() throws IOException mockDataSegmentMover, mockDataSegmentArchiver, mockSegmentAnnouncer, - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), mockHandoffNotifierFactory, () -> mockQueryRunnerFactoryConglomerate, DruidProcessingConfig::new, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 2465ef286f73..51e28cfeff2a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -113,7 +113,6 @@ public void setup() throws IOException new NoopDataSegmentAnnouncer(), null, null, - null, DruidProcessingConfig::new, null, NoopJoinableFactory.INSTANCE, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 3d086c0cc977..e1e6936e6040 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -132,7 +132,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -576,7 +575,6 @@ public void announceSegment(DataSegment segment) } }, - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, () -> queryRunnerFactoryConglomerate, // query runner factory conglomerate corporation unionized collective DruidProcessingConfig::new, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java index 3ed1b08ead2f..b566252f56d1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TestTaskToolboxFactory.java @@ -65,7 +65,6 @@ import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.tasklogs.TaskLogPusher; import org.apache.druid.utils.RuntimeInfo; @@ -94,7 +93,6 @@ public TestTaskToolboxFactory( bob.dataSegmentMover, bob.dataSegmentArchiver, bob.segmentAnnouncer, - bob.serverAnnouncer, bob.handoffNotifierFactory, bob.queryRunnerFactoryConglomerateProvider, bob.processingConfigProvider, @@ -142,7 +140,6 @@ public static class Builder private DataSegmentMover dataSegmentMover; private DataSegmentArchiver dataSegmentArchiver; private DataSegmentAnnouncer segmentAnnouncer; - private DataSegmentServerAnnouncer serverAnnouncer; private SegmentHandoffNotifierFactory handoffNotifierFactory; private Provider queryRunnerFactoryConglomerateProvider; private Provider processingConfigProvider; @@ -244,12 +241,6 @@ public Builder setSegmentAnnouncer(DataSegmentAnnouncer segmentAnnouncer) return this; } - public Builder setServerAnnouncer(DataSegmentServerAnnouncer serverAnnouncer) - { - this.serverAnnouncer = serverAnnouncer; - return this; - } - public Builder setHandoffNotifierFactory(SegmentHandoffNotifierFactory handoffNotifierFactory) { this.handoffNotifierFactory = handoffNotifierFactory; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index e1968a457c4b..797cbffd9372 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -55,7 +55,6 @@ import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.security.AuthTestUtils; @@ -353,7 +352,6 @@ private TaskToolbox createTaskToolbox() .dataNodeService(new DataNodeService(DruidServer.DEFAULT_TIER, 100L, null, ServerType.HISTORICAL, 1)) .lookupNodeService(new LookupNodeService(DruidServer.DEFAULT_TIER)) .appenderatorsManager(new TestAppenderatorsManager()) - .serverAnnouncer(new DataSegmentServerAnnouncer.Noop()) .druidNodeAnnouncer(new NoopDruidNodeAnnouncer()) .jsonMapper(TestHelper.JSON_MAPPER) .emitter(emitter) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 0970c06cbf14..06b1a1c7cab1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -122,7 +122,6 @@ import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.StreamAppenderator; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -676,7 +675,6 @@ public void close() null, // DataSegmentMover null, // DataSegmentArchiver new TestDataSegmentAnnouncer(), - EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), handoffNotifierFactory, this::makeQueryRunnerConglomerate, DruidProcessingConfig::new, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index d10f33fd146a..8455e4b6632c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -141,7 +141,6 @@ private WorkerTaskManager createWorkerTaskManager() null, null, null, - null, notifierFactory, null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 74f86664c751..19b580dffa55 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -186,7 +186,6 @@ private WorkerTaskMonitor createTaskMonitor() null, null, null, - null, notifierFactory, null, null, diff --git a/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java b/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java deleted file mode 100644 index f14666030ec8..000000000000 --- a/server/src/main/java/org/apache/druid/client/BatchServerInventoryView.java +++ /dev/null @@ -1,424 +0,0 @@ -/* - * 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.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; -import org.apache.curator.framework.CuratorFramework; -import org.apache.druid.curator.inventory.CuratorInventoryManager; -import org.apache.druid.curator.inventory.CuratorInventoryManagerStrategy; -import org.apache.druid.curator.inventory.InventoryManagerConfig; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * This class is deprecated. Use {@link HttpServerInventoryView} instead. - */ -@Deprecated -@ManageLifecycle -public class BatchServerInventoryView implements ServerInventoryView, FilteredServerInventoryView -{ - private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class); - - private final CuratorInventoryManager> inventoryManager; - private final AtomicBoolean started = new AtomicBoolean(false); - - private final ConcurrentMap serverCallbacks = new ConcurrentHashMap<>(); - private final ConcurrentMap segmentCallbacks = new ConcurrentHashMap<>(); - - private final ConcurrentMap> zNodes = new ConcurrentHashMap<>(); - private final ConcurrentMap>> segmentPredicates = - new ConcurrentHashMap<>(); - private final Predicate> defaultFilter; - - public BatchServerInventoryView( - final ZkPathsConfig zkPaths, - final CuratorFramework curator, - final ObjectMapper jsonMapper, - final Predicate> defaultFilter, - final String pathChildrenCacheExecPrefix - ) - { - this.inventoryManager = new CuratorInventoryManager<>( - curator, - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return zkPaths.getAnnouncementsPath(); - } - - @Override - public String getInventoryPath() - { - return zkPaths.getLiveSegmentsPath(); - } - }, - Execs.singleThreaded(pathChildrenCacheExecPrefix + "-%s"), - new CuratorInventoryManagerStrategy<>() - { - @Override - public DruidServer deserializeContainer(byte[] bytes) - { - try { - return jsonMapper.readValue(bytes, DruidServer.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public Set deserializeInventory(byte[] bytes) - { - try { - return jsonMapper.readValue(bytes, new TypeReference<>() {}); - } - catch (IOException e) { - log.error(e, "Could not parse json: %s", StringUtils.fromUtf8(bytes)); - throw new RuntimeException(e); - } - } - - @Override - public void newContainer(DruidServer container) - { - log.info("New Server[%s]", container); - runServerCallbacks(callback -> callback.serverAdded(container)); - } - - @Override - public void deadContainer(DruidServer deadContainer) - { - log.info("Server Disappeared[%s]", deadContainer); - runServerCallbacks(callback -> callback.serverRemoved(deadContainer)); - } - - @Override - public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer) - { - return newContainer.addDataSegments(oldContainer); - } - - @Override - public DruidServer addInventory( - final DruidServer container, - String inventoryKey, - final Set inventory - ) - { - return addInnerInventory(container, inventoryKey, inventory); - } - - @Override - public DruidServer updateInventory(DruidServer container, String inventoryKey, Set inventory) - { - return updateInnerInventory(container, inventoryKey, inventory); - } - - @Override - public DruidServer removeInventory(final DruidServer container, String inventoryKey) - { - return removeInnerInventory(container, inventoryKey); - } - - @Override - public void inventoryInitialized() - { - log.info("Inventory Initialized"); - runSegmentCallbacks(SegmentCallback::segmentViewInitialized); - } - } - ); - - this.defaultFilter = Preconditions.checkNotNull(defaultFilter); - } - - @LifecycleStart - public void start() throws Exception - { - synchronized (started) { - if (!started.get()) { - inventoryManager.start(); - started.set(true); - } - } - } - - @LifecycleStop - public void stop() throws IOException - { - synchronized (started) { - if (started.getAndSet(false)) { - inventoryManager.stop(); - } - } - } - - @Override - public boolean isStarted() - { - return started.get(); - } - - @Override - public DruidServer getInventoryValue(String containerKey) - { - return inventoryManager.getInventoryValue(containerKey); - } - - @Override - public Collection getInventory() - { - return inventoryManager.getInventory(); - } - - @Override - public void registerServerCallback(Executor exec, ServerCallback callback) - { - serverCallbacks.put(callback, exec); - } - - @Override - public void registerSegmentCallback(Executor exec, SegmentCallback callback) - { - segmentCallbacks.put(callback, exec); - } - - protected void runSegmentCallbacks( - final Function fn - ) - { - for (final Map.Entry entry : segmentCallbacks.entrySet()) { - entry.getValue().execute( - () -> { - if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { - segmentCallbackRemoved(entry.getKey()); - segmentCallbacks.remove(entry.getKey()); - } - } - ); - } - } - - private void runServerCallbacks(final Function fn) - { - for (final Map.Entry entry : serverCallbacks.entrySet()) { - entry.getValue().execute( - () -> { - if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { - serverCallbacks.remove(entry.getKey()); - } - } - ); - } - } - - protected void addSingleInventory(final DruidServer container, final DataSegment inventory) - { - log.debug("Server[%s] added segment[%s]", container.getName(), inventory.getId()); - - if (container.getSegment(inventory.getId()) != null) { - log.warn( - "Not adding or running callbacks for existing segment[%s] on server[%s]", - inventory.getId(), - container.getName() - ); - - return; - } - - container.addDataSegment(inventory); - - runSegmentCallbacks( - input -> input.segmentAdded(container.getMetadata(), inventory) - ); - } - - void removeSingleInventory(DruidServer container, SegmentId segmentId) - { - log.debug("Server[%s] removed segment[%s]", container.getName(), segmentId); - if (!doRemoveSingleInventory(container, segmentId)) { - log.warn( - "Not running cleanup or callbacks for non-existing segment[%s] on server[%s]", - segmentId, - container.getName() - ); - } - } - - private boolean doRemoveSingleInventory(DruidServer container, SegmentId segmentId) - { - DataSegment segment = container.removeDataSegment(segmentId); - if (segment != null) { - runSegmentCallbacks( - input -> input.segmentRemoved(container.getMetadata(), segment) - ); - return true; - } else { - return false; - } - } - - @Override - public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) - { - try { - DruidServer server = getInventoryValue(serverKey); - return server != null && server.getSegment(segment.getId()) != null; - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - } - - protected DruidServer addInnerInventory( - final DruidServer container, - String inventoryKey, - final Set inventory - ) - { - Set filteredInventory = filterInventory(container, inventory); - zNodes.put(inventoryKey, filteredInventory); - for (DataSegment segment : filteredInventory) { - addSingleInventory(container, segment); - } - return container; - } - - private Set filterInventory(final DruidServer container, Set inventory) - { - Predicate> predicate = Predicates.or( - defaultFilter, - Predicates.or(segmentPredicates.values()) - ); - - // make a copy of the set and not just a filtered view, in order to not keep all the segment data in memory - Set filteredInventory = Sets.newHashSet( - Iterables.transform( - Iterables.filter( - Iterables.transform( - inventory, - new Function<>() - { - @Override - public Pair apply(DataSegment input) - { - return Pair.of(container.getMetadata(), input); - } - } - ), - predicate - ), - new Function, DataSegment>() - { - @Override - public DataSegment apply(Pair input) - { - return DataSegmentInterner.intern(input.rhs); - } - } - ) - ); - return filteredInventory; - } - - protected DruidServer updateInnerInventory(DruidServer container, String inventoryKey, Set inventory) - { - Set filteredInventory = filterInventory(container, inventory); - - Set existing = zNodes.get(inventoryKey); - if (existing == null) { - throw new ISE("Trying to update an inventoryKey[%s] that didn't exist?!", inventoryKey); - } - - for (DataSegment segment : Sets.difference(filteredInventory, existing)) { - addSingleInventory(container, segment); - } - for (DataSegment segment : Sets.difference(existing, filteredInventory)) { - removeSingleInventory(container, segment.getId()); - } - zNodes.put(inventoryKey, filteredInventory); - - return container; - } - - protected DruidServer removeInnerInventory(final DruidServer container, String inventoryKey) - { - log.debug("Server[%s] removed container[%s]", container.getName(), inventoryKey); - Set segments = zNodes.remove(inventoryKey); - - if (segments == null) { - log.warn("Told to remove container[%s], which didn't exist", inventoryKey); - return container; - } - - for (DataSegment segment : segments) { - removeSingleInventory(container, segment.getId()); - } - return container; - } - - @Override - public void registerSegmentCallback( - final Executor exec, - final SegmentCallback callback, - final Predicate> filter - ) - { - SegmentCallback filteringCallback = new FilteringSegmentCallback(callback, filter); - segmentPredicates.put(filteringCallback, filter); - registerSegmentCallback( - exec, - filteringCallback - ); - } - - protected void segmentCallbackRemoved(SegmentCallback callback) - { - segmentPredicates.remove(callback); - } -} diff --git a/server/src/main/java/org/apache/druid/client/BatchServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/BatchServerInventoryViewProvider.java deleted file mode 100644 index 9c555d35b2fc..000000000000 --- a/server/src/main/java/org/apache/druid/client/BatchServerInventoryViewProvider.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicates; -import org.apache.curator.framework.CuratorFramework; -import org.apache.druid.server.initialization.ZkPathsConfig; - -import javax.validation.constraints.NotNull; - -/** - */ -public class BatchServerInventoryViewProvider implements ServerInventoryViewProvider -{ - @JacksonInject - @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; - - @JacksonInject - @NotNull - private ObjectMapper jsonMapper = null; - - @Override - public BatchServerInventoryView get() - { - return new BatchServerInventoryView( - zkPaths, - curator, - jsonMapper, - Predicates.alwaysTrue(), - "BatchServerInventoryView" - ); - } -} diff --git a/server/src/main/java/org/apache/druid/client/FilteredBatchServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/FilteredBatchServerInventoryViewProvider.java deleted file mode 100644 index bb796936af3f..000000000000 --- a/server/src/main/java/org/apache/druid/client/FilteredBatchServerInventoryViewProvider.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicates; -import org.apache.curator.framework.CuratorFramework; -import org.apache.druid.server.initialization.ZkPathsConfig; - -import javax.validation.constraints.NotNull; - -public class FilteredBatchServerInventoryViewProvider implements FilteredServerInventoryViewProvider -{ - @JacksonInject - @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; - - @JacksonInject - @NotNull - private ObjectMapper jsonMapper = null; - - @Override - public BatchServerInventoryView get() - { - return new BatchServerInventoryView( - zkPaths, - curator, - jsonMapper, - Predicates.alwaysFalse(), - "FilteredBatchServerInventoryView" - ); - } -} diff --git a/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java index 5e438af1111c..3d6bfd103909 100644 --- a/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/FilteredServerInventoryViewProvider.java @@ -27,7 +27,6 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = ServerViewModule.TYPE, defaultImpl = FilteredHttpServerInventoryViewProvider.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_BATCH, value = FilteredBatchServerInventoryViewProvider.class), @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_HTTP, value = FilteredHttpServerInventoryViewProvider.class) }) public interface FilteredServerInventoryViewProvider extends Provider diff --git a/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java b/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java index bf3a22dfb572..98476fd9a733 100644 --- a/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java +++ b/server/src/main/java/org/apache/druid/client/ServerInventoryViewProvider.java @@ -28,7 +28,6 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = ServerViewModule.TYPE, defaultImpl = HttpServerInventoryViewProvider.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_BATCH, value = BatchServerInventoryViewProvider.class), @JsonSubTypes.Type(name = ServerViewModule.SERVERVIEW_TYPE_HTTP, value = HttpServerInventoryViewProvider.class), }) public interface ServerInventoryViewProvider extends Provider diff --git a/server/src/main/java/org/apache/druid/curator/discovery/NoopServiceAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/NoopServiceAnnouncer.java deleted file mode 100644 index 15cdd6128fdb..000000000000 --- a/server/src/main/java/org/apache/druid/curator/discovery/NoopServiceAnnouncer.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.curator.discovery; - -import org.apache.druid.server.DruidNode; - -/** - * Does nothing. - */ -@Deprecated -public class NoopServiceAnnouncer implements ServiceAnnouncer -{ - @Override - public void announce(DruidNode node) - { - } - - @Override - public void unannounce(DruidNode node) - { - } -} diff --git a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java index b6a4d472283b..e48822afab6c 100644 --- a/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java +++ b/server/src/main/java/org/apache/druid/guice/AnnouncerModule.java @@ -20,12 +20,10 @@ package org.apache.druid.guice; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Module; import com.google.inject.Provides; import org.apache.curator.framework.CuratorFramework; import org.apache.druid.curator.CuratorConfig; -import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.curator.announcement.NodeAnnouncer; import org.apache.druid.curator.announcement.PathChildrenAnnouncer; import org.apache.druid.curator.announcement.ServiceAnnouncer; @@ -33,39 +31,17 @@ import org.apache.druid.guice.annotations.SingleThreadedAnnouncer; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; -import org.apache.druid.server.coordination.CuratorDataSegmentServerAnnouncer; import org.apache.druid.server.coordination.DataSegmentAnnouncer; -import org.apache.druid.server.coordination.DataSegmentAnnouncerProvider; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; - -import java.util.Properties; /** */ public class AnnouncerModule implements Module { - private boolean isZkEnabled = true; - - @Inject - public void configure(Properties properties) - { - isZkEnabled = ZkEnablementConfig.isEnabled(properties); - } - @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); - JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); - binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); + binder.bind(DataSegmentAnnouncer.class).to(BatchDataSegmentAnnouncer.class); binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleAnnouncements.class); - - if (isZkEnabled) { - binder.bind(DataSegmentServerAnnouncer.class).to(CuratorDataSegmentServerAnnouncer.class).in(LazySingleton.class); - } else { - binder.bind(DataSegmentServerAnnouncer.class).to(DataSegmentServerAnnouncer.Noop.class).in(LazySingleton.class); - } } @Provides diff --git a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java index ee5ac7d939bd..71ee4612ede0 100644 --- a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java +++ b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java @@ -20,6 +20,7 @@ package org.apache.druid.guice; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.FilteredServerInventoryViewProvider; @@ -28,7 +29,9 @@ import org.apache.druid.client.ServerInventoryView; import org.apache.druid.client.ServerInventoryViewProvider; import org.apache.druid.client.ServerView; +import org.apache.druid.error.DruidException; +import java.util.Properties; import java.util.StringJoiner; /** @@ -39,7 +42,24 @@ public class ServerViewModule implements Module public static final String TYPE = "type"; public static final String SERVERVIEW_TYPE_PROPERTY = "druid.serverview.type"; public static final String SERVERVIEW_TYPE_HTTP = "http"; - public static final String SERVERVIEW_TYPE_BATCH = "batch"; + + @Inject + public void configure(Properties properties) + { + String configuredType = properties.getProperty(SERVERVIEW_TYPE_PROPERTY); + if (configuredType != null && !SERVERVIEW_TYPE_HTTP.equals(configuredType)) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "Invalid value[%s] for property[%s]. Only [%s] is supported; the ZooKeeper-based" + + " 'batch' server view has been removed. Remove this property or set it to 'http'." + + " See the Druid upgrade notes for details.", + configuredType, + SERVERVIEW_TYPE_PROPERTY, + SERVERVIEW_TYPE_HTTP + ); + } + } @Override public void configure(Binder binder) diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java index ad721f4ef7ca..61616b9dab61 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -19,291 +19,71 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; -import com.google.inject.Inject; -import com.google.inject.Provider; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.common.utils.UUIDUtils; -import org.apache.druid.curator.ZkEnablementConfig; -import org.apache.druid.curator.announcement.ServiceAnnouncer; -import org.apache.druid.guice.annotations.SingleThreadedAnnouncer; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.druid.server.initialization.ZkPathsConfig; import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.atomic.AtomicLong; -/** - */ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer { private static final Logger log = new Logger(BatchDataSegmentAnnouncer.class); - private final BatchDataSegmentAnnouncerConfig config; - - @Nullable //Null if zk is disabled or isSkipSegmentAnnouncementOnZk = true - private final ServiceAnnouncer announcer; - - private final ObjectMapper jsonMapper; - private final String liveSegmentLocation; - private final DruidServerMetadata server; - - private final Object lock = new Object(); - private final AtomicLong counter = new AtomicLong(0); - - private final Set availableZNodes = new ConcurrentSkipListSet<>(); - private final ConcurrentMap segmentLookup = new ConcurrentHashMap<>(); - private final Function segmentTransformer; + private final Set announcedSegments = ConcurrentHashMap.newKeySet(); private final ChangeRequestHistory changes = new ChangeRequestHistory<>(); private final ConcurrentMap taskSinkSchema = new ConcurrentHashMap<>(); - @Nullable - private final SegmentZNode dummyZnode; - - private final boolean isSkipSegmentAnnouncementOnZk; - - @Inject - public BatchDataSegmentAnnouncer( - DruidServerMetadata server, - final BatchDataSegmentAnnouncerConfig config, - ZkPathsConfig zkPaths, - @SingleThreadedAnnouncer Provider announcerProvider, - ObjectMapper jsonMapper, - ZkEnablementConfig zkEnablementConfig - ) - { - this.config = config; - this.jsonMapper = jsonMapper; - this.server = server; - - this.liveSegmentLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), server.getName()); - segmentTransformer = input -> { - DataSegment rv = input; - if (config.isSkipDimensionsAndMetrics()) { - rv = rv.withDimensions(null).withMetrics(null); - } - if (config.isSkipLoadSpec()) { - rv = rv.withLoadSpec(null); - } - return rv; - }; - - isSkipSegmentAnnouncementOnZk = !zkEnablementConfig.isEnabled() || config.isSkipSegmentAnnouncementOnZk(); - if (isSkipSegmentAnnouncementOnZk) { - dummyZnode = new SegmentZNode("PLACE_HOLDER_ONLY"); - this.announcer = null; - } else { - dummyZnode = null; - this.announcer = announcerProvider.get(); - } - } - - @VisibleForTesting - public BatchDataSegmentAnnouncer( - DruidServerMetadata server, - final BatchDataSegmentAnnouncerConfig config, - ZkPathsConfig zkPaths, - ServiceAnnouncer announcer, - ObjectMapper jsonMapper - ) - { - this(server, config, zkPaths, () -> announcer, jsonMapper, ZkEnablementConfig.ENABLED); - } - @LifecycleStop public void stop() { changes.stop(); } - @Override - public void announceSegment(DataSegment segment) throws IOException + public void announceSegment(DataSegment segment) { - if (segmentLookup.containsKey(segment)) { + if (!announcedSegments.add(segment)) { log.info("Skipping announcement of segment [%s]. Announcement exists already.", segment.getId()); return; } - - synchronized (lock) { - if (segmentLookup.containsKey(segment)) { - log.info("Skipping announcement of segment [%s]. Announcement exists already.", segment.getId()); - return; - } - - DataSegment toAnnounce = segmentTransformer.apply(segment); - - changes.addChangeRequest(new SegmentChangeRequestLoad(toAnnounce)); - - if (isSkipSegmentAnnouncementOnZk) { - segmentLookup.put(segment, dummyZnode); - return; - } - - int newBytesLen = jsonMapper.writeValueAsBytes(toAnnounce).length; - if (newBytesLen > config.getMaxBytesPerNode()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); - } - - boolean done = false; - if (!availableZNodes.isEmpty()) { - // update existing batch - Iterator iter = availableZNodes.iterator(); - while (iter.hasNext() && !done) { - SegmentZNode availableZNode = iter.next(); - if (availableZNode.getBytes().length + newBytesLen < config.getMaxBytesPerNode()) { - availableZNode.addSegment(toAnnounce); - log.info( - "Announcing segment[%s] at existing path[%s]", - toAnnounce.getId(), - availableZNode.getPath() - ); - announcer.update(availableZNode.getPath(), availableZNode.getBytes()); - segmentLookup.put(toAnnounce, availableZNode); - - if (availableZNode.getCount() >= config.getSegmentsPerNode()) { - availableZNodes.remove(availableZNode); - } - done = true; - } else { - // We could have kept the znode around for later use, however we remove it since segment announcements should - // have similar size unless there are significant schema changes. Removing the znode reduces the number of - // znodes that would be scanned at each announcement. - availableZNodes.remove(availableZNode); - } - } - } - - if (!done) { - assert (availableZNodes.isEmpty()); - // create new batch - - SegmentZNode availableZNode = new SegmentZNode(makeServedSegmentPath()); - availableZNode.addSegment(toAnnounce); - - log.info("Announcing %s[%s] at new path[%s]", - toAnnounce.isTombstone() ? DataSegment.TOMBSTONE_LOADSPEC_TYPE : "segment", - toAnnounce.getId(), - availableZNode.getPath() - ); - announcer.announce(availableZNode.getPath(), availableZNode.getBytes()); - segmentLookup.put(toAnnounce, availableZNode); - availableZNodes.add(availableZNode); - } - } + changes.addChangeRequest(new SegmentChangeRequestLoad(segment)); } @Override public void unannounceSegment(DataSegment segment) { - synchronized (lock) { - final SegmentZNode segmentZNode = segmentLookup.remove(segment); - - if (segmentZNode == null) { - log.warn("No path to unannounce segment[%s]", segment.getId()); - return; - } - - changes.addChangeRequest(new SegmentChangeRequestDrop(segment)); - - if (isSkipSegmentAnnouncementOnZk) { - return; - } - - segmentZNode.removeSegment(segment); - - log.info("Unannouncing segment[%s] at path[%s]", segment.getId(), segmentZNode.getPath()); - if (segmentZNode.getCount() == 0) { - availableZNodes.remove(segmentZNode); - announcer.unannounce(segmentZNode.getPath()); - } else { - announcer.update(segmentZNode.getPath(), segmentZNode.getBytes()); - availableZNodes.add(segmentZNode); - } + if (!announcedSegments.remove(segment)) { + log.warn("No announcement to remove for segment[%s]", segment.getId()); + return; } + changes.addChangeRequest(new SegmentChangeRequestDrop(segment)); } @Override - public void announceSegments(Iterable segments) throws IOException + public void announceSegments(Iterable segments) { - SegmentZNode segmentZNode = new SegmentZNode(makeServedSegmentPath()); - Set batch = new HashSet<>(); List changesBatch = new ArrayList<>(); - - int byteSize = 0; - int count = 0; - - synchronized (lock) { - for (DataSegment ds : segments) { - - if (segmentLookup.containsKey(ds)) { - log.info("Skipping announcement of segment [%s]. Announcement exists already.", ds.getId()); - return; - } - - DataSegment segment = segmentTransformer.apply(ds); - + for (DataSegment segment : segments) { + if (announcedSegments.add(segment)) { changesBatch.add(new SegmentChangeRequestLoad(segment)); - - if (isSkipSegmentAnnouncementOnZk) { - segmentLookup.put(segment, dummyZnode); - continue; - } - - int newBytesLen = jsonMapper.writeValueAsBytes(segment).length; - - if (newBytesLen > config.getMaxBytesPerNode()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); - } - - if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxBytesPerNode()) { - segmentZNode.addSegments(batch); - announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); - - segmentZNode = new SegmentZNode(makeServedSegmentPath()); - batch = new HashSet<>(); - count = 0; - byteSize = 0; - } - - log.info("Announcing segment[%s] at path[%s]", segment.getId(), segmentZNode.getPath()); - segmentLookup.put(segment, segmentZNode); - batch.add(segment); - count++; - byteSize += newBytesLen; + } else { + log.info("Skipping announcement of segment [%s]. Announcement exists already.", segment.getId()); } } - - changes.addChangeRequests(changesBatch); - - if (!isSkipSegmentAnnouncementOnZk) { - segmentZNode.addSegments(batch); - announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); + if (!changesBatch.isEmpty()) { + changes.addChangeRequests(changesBatch); } } @@ -319,7 +99,7 @@ public void unannounceSegments(Iterable segments) public void announceSegmentSchemas( String taskId, SegmentSchemas segmentSchemas, - SegmentSchemas segmentSchemasChange + @Nullable SegmentSchemas segmentSchemasChange ) { log.info("Announcing sink schema for task [%s], absolute schema [%s], delta schema [%s].", @@ -346,164 +126,21 @@ public void removeSegmentSchemasForTask(String taskId) public ListenableFuture> getSegmentChangesSince(ChangeRequestHistory.Counter counter) { if (counter.getCounter() < 0) { - synchronized (lock) { - Iterable segments = Iterables.transform( - segmentLookup.keySet(), - SegmentChangeRequestLoad::new - ); - - Iterable sinkSchema = Iterables.transform( - taskSinkSchema.values(), - SegmentSchemasChangeRequest::new - ); - Iterable changeRequestIterables = Iterables.concat(segments, sinkSchema); - SettableFuture> future = SettableFuture.create(); - future.set(ChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(changeRequestIterables))); - return future; - } + Iterable segments = Iterables.transform( + announcedSegments, + SegmentChangeRequestLoad::new + ); + + Iterable sinkSchema = Iterables.transform( + taskSinkSchema.values(), + SegmentSchemasChangeRequest::new + ); + Iterable changeRequestIterables = Iterables.concat(segments, sinkSchema); + SettableFuture> future = SettableFuture.create(); + future.set(ChangeRequestsSnapshot.success(changes.getLastCounter(), Lists.newArrayList(changeRequestIterables))); + return future; } else { return changes.getRequestsSince(counter); } } - - private String makeServedSegmentPath() - { - // server.getName() is already in the zk path - return makeServedSegmentPath( - UUIDUtils.generateUuid( - server.getHost(), - server.getType().toString(), - server.getTier(), - DateTimes.nowUtc().toString() - ) - ); - } - - private String makeServedSegmentPath(String zNode) - { - return ZKPaths.makePath(liveSegmentLocation, StringUtils.format("%s%s", zNode, counter.getAndIncrement())); - } - - private class SegmentZNode implements Comparable - { - private final String path; - - private byte[] bytes = new byte[]{}; - private int count = 0; - - public SegmentZNode(String path) - { - this.path = path; - } - - public String getPath() - { - return path; - } - - public int getCount() - { - return count; - } - - public byte[] getBytes() - { - return bytes; - } - - public Set getSegments() - { - if (bytes.length == 0) { - return new HashSet<>(); - } - try { - return jsonMapper.readValue( - bytes, - new TypeReference<>() {} - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void addSegment(DataSegment segment) - { - Set zkSegments = getSegments(); - zkSegments.add(segment); - - try { - bytes = jsonMapper.writeValueAsBytes(zkSegments); - } - catch (Exception e) { - zkSegments.remove(segment); - throw new RuntimeException(e); - } - - count++; - } - - public void addSegments(Set segments) - { - Set zkSegments = getSegments(); - zkSegments.addAll(segments); - - try { - bytes = jsonMapper.writeValueAsBytes(zkSegments); - } - catch (Exception e) { - zkSegments.removeAll(segments); - throw new RuntimeException(e); - } - - count += segments.size(); - } - - public void removeSegment(DataSegment segment) - { - Set zkSegments = getSegments(); - zkSegments.remove(segment); - - try { - bytes = jsonMapper.writeValueAsBytes(zkSegments); - } - catch (Exception e) { - zkSegments.add(segment); - throw new RuntimeException(e); - } - - count--; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SegmentZNode that = (SegmentZNode) o; - - if (!path.equals(that.path)) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - return path.hashCode(); - } - - @Override - public int compareTo(SegmentZNode segmentZNode) - { - return path.compareTo(segmentZNode.getPath()); - } - } } diff --git a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java b/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java deleted file mode 100644 index b521594d12d4..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import com.fasterxml.jackson.annotation.JacksonInject; - -import javax.annotation.Nullable; - -/** - */ -public class BatchDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider -{ - @JacksonInject - @Nullable - private final BatchDataSegmentAnnouncer batchAnnouncer = null; - - @Override - public DataSegmentAnnouncer get() - { - return batchAnnouncer; - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java deleted file mode 100644 index ec029ba6c9a2..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordination/CuratorDataSegmentServerAnnouncer.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.inject.Inject; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.curator.announcement.ServiceAnnouncer; -import org.apache.druid.guice.annotations.SingleThreadedAnnouncer; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.initialization.ZkPathsConfig; - -/** - * {@link DataSegmentServerAnnouncer} is deprecated. - */ -@Deprecated -public class CuratorDataSegmentServerAnnouncer implements DataSegmentServerAnnouncer -{ - private static final Logger log = new Logger(CuratorDataSegmentServerAnnouncer.class); - - private final DruidServerMetadata server; - private final ZkPathsConfig config; - private final ServiceAnnouncer announcer; - private final ObjectMapper jsonMapper; - - private final Object lock = new Object(); - - private volatile boolean announced = false; - - @Inject - public CuratorDataSegmentServerAnnouncer( - DruidServerMetadata server, - ZkPathsConfig config, - @SingleThreadedAnnouncer ServiceAnnouncer announcer, - ObjectMapper jsonMapper - ) - { - this.server = server; - this.config = config; - this.announcer = announcer; - this.jsonMapper = jsonMapper; - } - - @Override - public void announce() - { - synchronized (lock) { - if (announced) { - return; - } - - try { - final String path = makeAnnouncementPath(); - log.info("Announcing self[%s] at [%s]", server, path); - announcer.announce(path, jsonMapper.writeValueAsBytes(server), false); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - - announced = true; - } - } - - @Override - public void unannounce() - { - synchronized (lock) { - if (!announced) { - return; - } - - final String path = makeAnnouncementPath(); - log.debug("Unannouncing self[%s] at [%s]", server, path); - announcer.unannounce(path); - - announced = false; - } - } - - private String makeAnnouncementPath() - { - return ZKPaths.makePath(config.getAnnouncementsPath(), server.getName()); - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncerProvider.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncerProvider.java deleted file mode 100644 index cb2c1141d973..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentAnnouncerProvider.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = BatchDataSegmentAnnouncerProvider.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "batch", value = BatchDataSegmentAnnouncerProvider.class) -}) -public interface DataSegmentAnnouncerProvider extends Provider -{ -} diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentServerAnnouncer.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentServerAnnouncer.java deleted file mode 100644 index 3b01073bbb80..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentServerAnnouncer.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -/** - * We are gradually migrating usages of this to {@link org.apache.druid.discovery.DruidNodeAnnouncer}. - * - * However, it's still required in some cases. As of this writing (2020-12-03) it's required for any process that - * is serving queryable segments via Curator-based segment discovery. (When using Curator for segment discovery, Brokers - * look for these announcements as part of discovering what segments are available.) - */ -@Deprecated -public interface DataSegmentServerAnnouncer -{ - void announce(); - void unannounce(); - - class Noop implements DataSegmentServerAnnouncer - { - - @Override - public void announce() - { - - } - - @Override - public void unannounce() - { - - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentCacheBootstrapper.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentCacheBootstrapper.java index d0dd28e0efc4..e9e16c9b0518 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentCacheBootstrapper.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentCacheBootstrapper.java @@ -27,7 +27,6 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.common.guava.FutureUtils; import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; @@ -60,8 +59,6 @@ /** * Responsible for bootstrapping segments already cached on disk and bootstrap segments fetched from the coordinator. - * Also responsible for announcing the node as a data server if applicable, once the bootstrapping operations - * are complete. */ @ManageLifecycle public class SegmentCacheBootstrapper @@ -69,9 +66,7 @@ public class SegmentCacheBootstrapper private final SegmentLoadDropHandler loadDropHandler; private final SegmentLoaderConfig config; private final DataSegmentAnnouncer segmentAnnouncer; - private final DataSegmentServerAnnouncer serverAnnouncer; private final SegmentManager segmentManager; - private final ServerTypeConfig serverTypeConfig; private final CoordinatorClient coordinatorClient; private final ServiceEmitter emitter; private final LoadSpecHolder loadSpecHolder; @@ -88,9 +83,7 @@ public SegmentCacheBootstrapper( SegmentLoadDropHandler loadDropHandler, SegmentLoaderConfig config, DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - ServerTypeConfig serverTypeConfig, CoordinatorClient coordinatorClient, ServiceEmitter emitter, LoadSpecHolder loadSpecHolder @@ -99,9 +92,7 @@ public SegmentCacheBootstrapper( this.loadDropHandler = loadDropHandler; this.config = config; this.segmentAnnouncer = segmentAnnouncer; - this.serverAnnouncer = serverAnnouncer; this.segmentManager = segmentManager; - this.serverTypeConfig = serverTypeConfig; this.coordinatorClient = coordinatorClient; this.emitter = emitter; this.loadSpecHolder = loadSpecHolder; @@ -120,10 +111,6 @@ public void start() throws IOException if (segmentManager.canHandleSegments()) { loadSegmentsOnStartup(); } - - if (shouldAnnounce()) { - serverAnnouncer.announce(); - } } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); @@ -144,9 +131,6 @@ public void stop() log.info("Stopping..."); try { - if (shouldAnnounce()) { - serverAnnouncer.unannounce(); - } segmentManager.shutdown(); } catch (Exception e) { @@ -314,21 +298,6 @@ private List getBootstrapSegments() return bootstrapSegments; } - /** - * Returns whether or not we should announce ourselves as a data server using {@link DataSegmentServerAnnouncer}. - * - * Returns true if _either_: - * - *
  • Our {@link #serverTypeConfig} indicates we are a segment server. This is necessary for Brokers to be able - * to detect that we exist.
  • - *
  • The segment manager is able to handle segments. This is necessary for Coordinators to be able to - * assign segments to us.
  • - */ - private boolean shouldAnnounce() - { - return serverTypeConfig.getServerType().isSegmentServer() || segmentManager.canHandleSegments(); - } - private static class BackgroundSegmentAnnouncer implements AutoCloseable { private static final EmittingLogger log = new EmittingLogger(BackgroundSegmentAnnouncer.class); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java deleted file mode 100644 index 72f6affe82f7..000000000000 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.druid.server.initialization.ZkPathsConfig; - -import java.io.IOException; - -/** - * Creates paths for announcing served segments on Zookeeper. - * - * @deprecated as Druid has already migrated to HTTP-based segment loading and - * will soon migrate to HTTP-based inventory view using {@code SegmentListerResource}. - * - * @see org.apache.druid.server.http.SegmentListerResource - */ -@Deprecated -public class ZkCoordinator -{ - private static final EmittingLogger log = new EmittingLogger(ZkCoordinator.class); - - private final Object lock = new Object(); - - private final ZkPathsConfig zkPaths; - private final DruidServerMetadata me; - private final CuratorFramework curator; - private final BatchDataSegmentAnnouncerConfig announcerConfig; - - private volatile boolean started = false; - - @Inject - public ZkCoordinator( - ZkPathsConfig zkPaths, - DruidServerMetadata me, - CuratorFramework curator, - BatchDataSegmentAnnouncerConfig announcerConfig - ) - { - this.zkPaths = zkPaths; - this.me = me; - this.curator = curator; - this.announcerConfig = announcerConfig; - } - - @LifecycleStart - public void start() throws IOException - { - synchronized (lock) { - if (started) { - return; - } - - log.info("Starting zkCoordinator for server[%s]", me.getName()); - - if (announcerConfig.isSkipSegmentAnnouncementOnZk()) { - log.info("Skipping zkPath creation as segment announcement on ZK is disabled."); - started = true; - return; - } - - final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName()); - log.info("Creating zkPath[%s] for announcing live segments.", liveSegmentsLocation); - - try { - curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); - } - catch (Exception e) { - Throwables.propagateIfPossible(e, IOException.class); - throw new RuntimeException(e); - } - - started = true; - } - } - - @LifecycleStop - public void stop() - { - log.info("Stopping ZkCoordinator for [%s]", me); - synchronized (lock) { - if (!started) { - return; - } - - started = false; - } - } -} diff --git a/server/src/main/java/org/apache/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java deleted file mode 100644 index 66ac575d41de..000000000000 --- a/server/src/main/java/org/apache/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.initialization; - -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.validation.constraints.Max; -import javax.validation.constraints.Min; - -/** - */ -public class BatchDataSegmentAnnouncerConfig -{ - @JsonProperty - @Min(1) - private int segmentsPerNode = 50; - - @JsonProperty - @Max(1024 * 1024) - @Min(1024) - private long maxBytesPerNode = 512 * 1024; - - // Skip LoadSpec from segment announcements - @JsonProperty - private boolean skipLoadSpec = false; - - // Skip dimension list from segment announcements - @JsonProperty - private boolean skipDimensionsAndMetrics = false; - - @JsonProperty - private boolean skipSegmentAnnouncementOnZk = false; - - public int getSegmentsPerNode() - { - return segmentsPerNode; - } - - public long getMaxBytesPerNode() - { - return maxBytesPerNode; - } - - public boolean isSkipLoadSpec() - { - return skipLoadSpec; - } - - public boolean isSkipDimensionsAndMetrics() - { - return skipDimensionsAndMetrics; - } - - public boolean isSkipSegmentAnnouncementOnZk() - { - return skipSegmentAnnouncementOnZk; - } -} diff --git a/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java index a47764c65029..0ef35d051dbd 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java @@ -27,42 +27,20 @@ public class ZkPathsConfig @JsonProperty private String base = "druid"; @JsonProperty - private String propertiesPath; - @JsonProperty private String announcementsPath; @JsonProperty - private String liveSegmentsPath; - @JsonProperty private String coordinatorPath; - @JsonProperty - private String connectorPath; public String getBase() { return base; } - public String getPropertiesPath() - { - return (null == propertiesPath) ? defaultPath("properties") : propertiesPath; - } - public String getAnnouncementsPath() { return (null == announcementsPath) ? defaultPath("announcements") : announcementsPath; } - /** - * Path to announce served segments on. - * - * @deprecated Use HTTP-based segment discovery instead. - */ - @Deprecated - public String getLiveSegmentsPath() - { - return (null == liveSegmentsPath) ? defaultPath("segments") : liveSegmentsPath; - } - public String getCoordinatorPath() { return (null == coordinatorPath) ? defaultPath("coordinator") : coordinatorPath; @@ -73,11 +51,6 @@ public String getOverlordPath() return defaultPath("overlord"); } - public String getConnectorPath() - { - return (null == connectorPath) ? defaultPath("connector") : connectorPath; - } - public String getInternalDiscoveryPath() { return defaultPath("internal-discovery"); @@ -101,26 +74,17 @@ public boolean equals(Object other) return false; } ZkPathsConfig otherConfig = (ZkPathsConfig) other; - if (this.getBase().equals(otherConfig.getBase()) && + return this.getBase().equals(otherConfig.getBase()) && this.getAnnouncementsPath().equals(otherConfig.getAnnouncementsPath()) && - this.getConnectorPath().equals(otherConfig.getConnectorPath()) && - this.getLiveSegmentsPath().equals(otherConfig.getLiveSegmentsPath()) && - this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()) && - this.getPropertiesPath().equals(otherConfig.getPropertiesPath())) { - return true; - } - return false; + this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()); } @Override public int hashCode() { int result = base != null ? base.hashCode() : 0; - result = 31 * result + (propertiesPath != null ? propertiesPath.hashCode() : 0); result = 31 * result + (announcementsPath != null ? announcementsPath.hashCode() : 0); - result = 31 * result + (liveSegmentsPath != null ? liveSegmentsPath.hashCode() : 0); result = 31 * result + (coordinatorPath != null ? coordinatorPath.hashCode() : 0); - result = 31 * result + (connectorPath != null ? connectorPath.hashCode() : 0); return result; } } diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java deleted file mode 100644 index 6eff742c11c4..000000000000 --- a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java +++ /dev/null @@ -1,871 +0,0 @@ -/* - * 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.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.fasterxml.jackson.dataformat.smile.SmileGenerator; -import com.google.common.base.Predicates; -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.Lists; -import com.google.common.collect.Sets; -import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; -import org.apache.druid.client.selector.LowestPriorityTierSelectorStrategy; -import org.apache.druid.client.selector.RandomServerSelectorStrategy; -import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.client.selector.TierSelectorStrategy; -import org.apache.druid.curator.CuratorTestBase; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.query.CloneQueryMode; -import org.apache.druid.query.QueryRunnerFactoryConglomerate; -import org.apache.druid.query.QueryWatcher; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordination.TestCoordinatorClient; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineLookup; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.apache.druid.timeline.partition.PartitionHolder; -import org.apache.druid.timeline.partition.SingleElementPartitionChunk; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.stream.Collectors; - -public class BrokerServerViewTest extends CuratorTestBase -{ - private final ObjectMapper jsonMapper; - private final ZkPathsConfig zkPathsConfig; - - private CountDownLatch segmentViewInitLatch; - private CountDownLatch serverAddedLatch; - private CountDownLatch segmentAddedLatch; - private CountDownLatch segmentRemovedLatch; - - private BatchServerInventoryView baseView; - private BrokerServerView brokerServerView; - private BrokerViewOfCoordinatorConfig brokerViewOfCoordinatorConfig; - - public BrokerServerViewTest() - { - jsonMapper = TestHelper.makeJsonMapper(); - zkPathsConfig = new ZkPathsConfig(); - brokerViewOfCoordinatorConfig = new BrokerViewOfCoordinatorConfig(new TestCoordinatorClient()); - } - - @Before - public void setUp() throws Exception - { - setupServerAndCurator(); - brokerViewOfCoordinatorConfig.start(); - curator.start(); - curator.blockUntilConnected(); - } - - @Test - public void testSingleServerAddedRemovedSegment() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(1); - segmentRemovedLatch = new CountDownLatch(1); - - setupViews(); - - final DruidServer druidServer = setupHistoricalServer("default_tier", "localhost:1234", 0); - final DataSegment segment = dataSegmentWithIntervalAndVersion("2014-10-20T00:00:00Z/P1D", "v1"); - final int partition = segment.getShardSpec().getPartitionNum(); - final Interval intervals = Intervals.of("2014-10-20T00:00:00Z/P1D"); - announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource("test_broker_server_view") - ).get(); - List> serverLookupRes = timeline.lookup(intervals); - Assert.assertEquals(1, serverLookupRes.size()); - - TimelineObjectHolder actualTimelineObjectHolder = serverLookupRes.get(0); - Assert.assertEquals(intervals, actualTimelineObjectHolder.getInterval()); - Assert.assertEquals("v1", actualTimelineObjectHolder.getVersion()); - - PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); - Assert.assertTrue(actualPartitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); - - ServerSelector selector = (actualPartitionHolder.iterator().next()).getObject(); - Assert.assertFalse(selector.isEmpty()); - Assert.assertEquals(segment, selector.getSegment()); - Assert.assertEquals(druidServer, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - Assert.assertNotNull(timeline.findChunk(intervals, "v1", partition)); - - unannounceSegmentForServer(druidServer, segment, zkPathsConfig); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - - Assert.assertEquals( - 0, - timeline.lookup(intervals).size() - ); - Assert.assertNull(timeline.findChunk(intervals, "v1", partition)); - } - - @Test - public void testMultipleServerAddedRemovedSegment() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(5); - - // temporarily set latch count to 1 - segmentRemovedLatch = new CountDownLatch(1); - - setupViews(); - - final List druidServers = Lists.transform( - ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), - hostname -> setupHistoricalServer("default_tier", hostname, 0) - ); - - final List segments = Lists.transform( - ImmutableList.of( - Pair.of("2011-04-01/2011-04-03", "v1"), - Pair.of("2011-04-03/2011-04-06", "v1"), - Pair.of("2011-04-01/2011-04-09", "v2"), - Pair.of("2011-04-06/2011-04-09", "v3"), - Pair.of("2011-04-01/2011-04-02", "v3") - ), input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) - ); - - for (int i = 0; i < 5; ++i) { - announceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig, jsonMapper); - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource("test_broker_server_view") - ).get(); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - timeline.lookup( - Intervals.of( - "2011-04-01/2011-04-09" - ) - ) - ); - - // unannounce the segment created by dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-09", "v2") - unannounceSegmentForServer(druidServers.get(2), segments.get(2), zkPathsConfig); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - - // renew segmentRemovedLatch since we still have 4 segments to unannounce - segmentRemovedLatch = new CountDownLatch(4); - - timeline = brokerServerView.getTimeline( - new TableDataSource("test_broker_server_view") - ).get(); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-03", "v1", druidServers.get(0), segments.get(0)), - createExpected("2011-04-03/2011-04-06", "v1", druidServers.get(1), segments.get(1)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - timeline.lookup( - Intervals.of( - "2011-04-01/2011-04-09" - ) - ) - ); - - // unannounce all the segments - for (int i = 0; i < 5; ++i) { - // skip the one that was previously unannounced - if (i != 2) { - unannounceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig); - } - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - - Assert.assertEquals( - 0, - ((List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))).size() - ); - } - - @Test - public void testMultipleServerAndBroker() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - serverAddedLatch = new CountDownLatch(6); - segmentAddedLatch = new CountDownLatch(6); - - // temporarily set latch count to 1 - segmentRemovedLatch = new CountDownLatch(1); - - setupViews(); - - final DruidServer druidBroker = new DruidServer( - "localhost:5", - "localhost:5", - null, - 10000000L, - null, - ServerType.BROKER, - "default_tier", - 0 - ); - - // Materialize this list so all servers are set up - final List druidServers = - ImmutableList.copyOf( - Lists.transform( - ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), - hostname -> setupHistoricalServer("default_tier", hostname, 0) - ) - ); - - setupZNodeForServer(druidBroker, zkPathsConfig, jsonMapper); - - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(serverAddedLatch)); - - // check server metadatas - Assert.assertEquals( - druidServers.stream().map(DruidServer::getMetadata).collect(Collectors.toSet()), - ImmutableSet.copyOf(brokerServerView.getDruidServerMetadatas()) - ); - - final List segments = Lists.transform( - ImmutableList.of( - Pair.of("2011-04-01/2011-04-03", "v1"), - Pair.of("2011-04-03/2011-04-06", "v1"), - Pair.of("2011-04-01/2011-04-09", "v2"), - Pair.of("2011-04-06/2011-04-09", "v3"), - Pair.of("2011-04-01/2011-04-02", "v3") - ), - input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) - ); - - DataSegment brokerSegment = dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-11", "v4"); - announceSegmentForServer(druidBroker, brokerSegment, zkPathsConfig, jsonMapper); - for (int i = 0; i < 5; ++i) { - announceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig, jsonMapper); - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource("test_broker_server_view") - ).get(); - - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - timeline.lookup( - Intervals.of( - "2011-04-01/2011-04-09" - ) - ) - ); - - // unannounce the broker segment should do nothing to announcements - unannounceSegmentForServer(druidBroker, brokerSegment, zkPathsConfig); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - - // renew segmentRemovedLatch since we still have 5 segments to unannounce - segmentRemovedLatch = new CountDownLatch(5); - - timeline = brokerServerView.getTimeline( - new TableDataSource("test_broker_server_view") - ).get(); - - // expect same set of segments as before - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - timeline.lookup( - Intervals.of( - "2011-04-01/2011-04-09" - ) - ) - ); - - // unannounce all the segments - for (int i = 0; i < 5; ++i) { - unannounceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig); - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - } - - @Test - public void testMultipleTiers() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(4); - segmentRemovedLatch = new CountDownLatch(0); - - // Setup a Broker that watches only Tier 2 - final String tier1 = "tier1"; - final String tier2 = "tier2"; - setupViews(Sets.newHashSet(tier2), null, true); - - // Historical Tier 1 has segments 1 and 2, Tier 2 has segments 2 and 3 - final DruidServer server11 = setupHistoricalServer(tier1, "localhost:1", 1); - final DruidServer server21 = setupHistoricalServer(tier2, "localhost:2", 1); - - final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); - announceSegmentForServer(server11, segment1, zkPathsConfig, jsonMapper); - - final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); - announceSegmentForServer(server11, segment2, zkPathsConfig, jsonMapper); - announceSegmentForServer(server21, segment2, zkPathsConfig, jsonMapper); - - final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); - announceSegmentForServer(server21, segment3, zkPathsConfig, jsonMapper); - - // Wait for the segments to be added - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - // Get the timeline for the datasource - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource(segment1.getDataSource()) - ).get(); - - // Verify that the timeline has no entry for the interval of segment 1 - Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); - - // Verify that there is one entry for the interval of segment 2 - List> timelineHolders = - timeline.lookup(segment2.getInterval()); - Assert.assertEquals(1, timelineHolders.size()); - - TimelineObjectHolder timelineHolder = timelineHolders.get(0); - Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); - Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); - - PartitionHolder partitionHolder = timelineHolder.getObject(); - Assert.assertTrue(partitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(partitionHolder)); - - ServerSelector selector = (partitionHolder.iterator().next()).getObject(); - Assert.assertFalse(selector.isEmpty()); - Assert.assertEquals(segment2, selector.getSegment()); - - // Verify that the ServerSelector always picks Tier 1 - for (int i = 0; i < 5; ++i) { - Assert.assertEquals(server21, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - } - Assert.assertEquals(Collections.singletonList(server21.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); - } - - @Test - public void testRealtimeTasksNotWatched() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(4); - segmentRemovedLatch = new CountDownLatch(0); - - // Setup a Broker that watches only Historicals - setupViews(null, null, false); - - // Historical has segments 2 and 3, Realtime has segments 1 and 2 - final DruidServer realtimeServer = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime:1", 1); - final DruidServer historicalServer = setupHistoricalServer("tier1", "historical:2", 1); - - final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); - announceSegmentForServer(realtimeServer, segment1, zkPathsConfig, jsonMapper); - - final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); - announceSegmentForServer(realtimeServer, segment2, zkPathsConfig, jsonMapper); - announceSegmentForServer(historicalServer, segment2, zkPathsConfig, jsonMapper); - - final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); - announceSegmentForServer(historicalServer, segment3, zkPathsConfig, jsonMapper); - - // Wait for the segments to be added - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - // Get the timeline for the datasource - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource(segment1.getDataSource()) - ).get(); - - // Verify that the timeline has no entry for the interval of segment 1 - Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); - - // Verify that there is one entry for the interval of segment 2 - List> timelineHolders = - timeline.lookup(segment2.getInterval()); - Assert.assertEquals(1, timelineHolders.size()); - - TimelineObjectHolder timelineHolder = timelineHolders.get(0); - Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); - Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); - - PartitionHolder partitionHolder = timelineHolder.getObject(); - Assert.assertTrue(partitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(partitionHolder)); - - ServerSelector selector = (partitionHolder.iterator().next()).getObject(); - Assert.assertFalse(selector.isEmpty()); - Assert.assertEquals(segment2, selector.getSegment()); - - // Verify that the ServerSelector always picks the Historical server - for (int i = 0; i < 5; ++i) { - Assert.assertEquals(historicalServer, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - } - Assert.assertEquals(Collections.singletonList(historicalServer.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); - } - - @Test - public void testIgnoredTiers() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(4); - segmentRemovedLatch = new CountDownLatch(0); - - // Setup a Broker that does not watch Tier 1 - final String tier1 = "tier1"; - final String tier2 = "tier2"; - setupViews(null, Sets.newHashSet(tier1), false); - - // Historical Tier 1 has segments 1 and 2, Tier 2 has segments 2 and 3 - final DruidServer server11 = setupHistoricalServer(tier1, "localhost:1", 1); - final DruidServer server21 = setupHistoricalServer(tier2, "localhost:2", 1); - - final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); - announceSegmentForServer(server11, segment1, zkPathsConfig, jsonMapper); - - final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); - announceSegmentForServer(server11, segment2, zkPathsConfig, jsonMapper); - announceSegmentForServer(server21, segment2, zkPathsConfig, jsonMapper); - - final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); - announceSegmentForServer(server21, segment3, zkPathsConfig, jsonMapper); - - // Wait for the segments to be added - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - // Get the timeline for the datasource - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource(segment1.getDataSource()) - ).get(); - - // Verify that the timeline has no entry for the interval of segment 1 - Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); - - // Verify that there is one entry for the interval of segment 2 - List> timelineHolders = - timeline.lookup(segment2.getInterval()); - Assert.assertEquals(1, timelineHolders.size()); - - TimelineObjectHolder timelineHolder = timelineHolders.get(0); - Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); - Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); - - PartitionHolder partitionHolder = timelineHolder.getObject(); - Assert.assertTrue(partitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(partitionHolder)); - - ServerSelector selector = (partitionHolder.iterator().next()).getObject(); - Assert.assertFalse(selector.isEmpty()); - Assert.assertEquals(segment2, selector.getSegment()); - - // Verify that the ServerSelector always picks Tier 1 - for (int i = 0; i < 5; ++i) { - Assert.assertEquals(server21, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - } - Assert.assertEquals(Collections.singletonList(server21.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); - } - - @Test(expected = ISE.class) - public void testInvalidWatchedTiersConfig() throws Exception - { - // Verify that specifying both ignoredTiers and watchedTiers fails startup - final String tier1 = "tier1"; - final String tier2 = "tier2"; - setupViews(Sets.newHashSet(tier2), Sets.newHashSet(tier1), true); - } - - @Test(expected = ISE.class) - public void testEmptyWatchedTiersConfig() throws Exception - { - setupViews(Collections.emptySet(), null, true); - } - - @Test(expected = ISE.class) - public void testEmptyIgnoredTiersConfig() throws Exception - { - setupViews(null, Collections.emptySet(), true); - } - - @Test - public void testDifferentTierStrategiesForHistoricalAndRealtimeServers() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(7); - segmentRemovedLatch = new CountDownLatch(0); - - // Setup a Broker with LowestPriority strategy for historicals and HighestPriority for realtime - setupViews( - new LowestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), - new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) - ); - - // Setup multiple historicals with different priorities and realtime servers - final DruidServer historicalLowPriority = setupHistoricalServer("tier1", "historical-low:1", 0); - final DruidServer historicalHighPriority = setupHistoricalServer("tier1", "historical-high:2", 10); - final DruidServer realtimeLowPriority = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime-low:3", 0); - final DruidServer realtimeHighPriority = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime-high:4", 10); - - // Segment 1: only on historicals with different priorities - final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); - announceSegmentForServer(historicalLowPriority, segment1, zkPathsConfig, jsonMapper); - announceSegmentForServer(historicalHighPriority, segment1, zkPathsConfig, jsonMapper); - - // Segment 2: only on realtime with different priorities - final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); - announceSegmentForServer(realtimeLowPriority, segment2, zkPathsConfig, jsonMapper); - announceSegmentForServer(realtimeHighPriority, segment2, zkPathsConfig, jsonMapper); - - // Segment 3: on both historical and realtime, but pick should prefer historical - final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); - announceSegmentForServer(historicalHighPriority, segment3, zkPathsConfig, jsonMapper); - announceSegmentForServer(realtimeLowPriority, segment3, zkPathsConfig, jsonMapper); - announceSegmentForServer(realtimeHighPriority, segment3, zkPathsConfig, jsonMapper); - - // Wait for the segments to be added - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - // Get the timeline for the datasource - TimelineLookup timeline = brokerServerView.getTimeline( - new TableDataSource(segment1.getDataSource()) - ).get(); - - // Test segment 1: should pick the lowest priority historical (priority 0) - List> holders1 = timeline.lookup(segment1.getInterval()); - Assert.assertEquals(1, holders1.size()); - ServerSelector selector1 = holders1.get(0).getObject().iterator().next().getObject(); - Assert.assertEquals(segment1, selector1.getSegment()); - - // Historical LowestPriorityTierSelectorStrategy should pick the historical servers low and high in order - Assert.assertEquals(historicalLowPriority, selector1.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - Assert.assertEquals(List.of(historicalLowPriority.getMetadata()), selector1.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); - Assert.assertEquals(List.of(historicalLowPriority.getMetadata(), historicalHighPriority.getMetadata()), selector1.getAllServers(CloneQueryMode.EXCLUDECLONES)); - - // Test segment 2: should pick the highest priority realtime (priority 10) - List> holders2 = timeline.lookup(segment2.getInterval()); - Assert.assertEquals(1, holders2.size()); - ServerSelector selector2 = holders2.get(0).getObject().iterator().next().getObject(); - Assert.assertEquals(segment2, selector2.getSegment()); - - // Realtime HighestPriorityTierSelectorStrategy for realtime should pick the realtime servers high and low in order - Assert.assertEquals(realtimeHighPriority, selector2.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - Assert.assertEquals(List.of(realtimeHighPriority.getMetadata()), selector2.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); - Assert.assertEquals(List.of(realtimeHighPriority.getMetadata(), realtimeLowPriority.getMetadata()), selector2.getAllServers(CloneQueryMode.EXCLUDECLONES)); - - // Test segment 3: when both historical and realtime exist, historical is preferred - // and should pick based on historical strategy (lowest priority = 10) - List> holders3 = timeline.lookup(segment3.getInterval()); - Assert.assertEquals(1, holders3.size()); - ServerSelector selector3 = holders3.get(0).getObject().iterator().next().getObject(); - Assert.assertEquals(segment3, selector3.getSegment()); - - // Should prefer historical over realtime servers and in order - Assert.assertEquals(historicalHighPriority, selector3.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - Assert.assertEquals(List.of(historicalHighPriority.getMetadata()), selector3.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); - Assert.assertEquals( - List.of(historicalHighPriority.getMetadata(), realtimeHighPriority.getMetadata(), realtimeLowPriority.getMetadata()), - selector3.getAllServers(CloneQueryMode.EXCLUDECLONES) - ); - } - - /** - * Creates a DruidServer of type HISTORICAL and sets up a ZNode for it. - */ - private DruidServer setupHistoricalServer(String tier, String name, int priority) - { - return setupDruidServer(ServerType.HISTORICAL, tier, name, priority); - } - - /** - * Creates a DruidServer of the specified type and sets up a ZNode for it. - */ - private DruidServer setupDruidServer(ServerType serverType, String tier, String name, int priority) - { - final DruidServer druidServer = new DruidServer( - name, - name, - null, - 1000000, - null, - serverType, - tier, - priority - ); - setupZNodeForServer(druidServer, zkPathsConfig, jsonMapper); - return druidServer; - } - - private Pair>> createExpected( - String intervalStr, - String version, - DruidServer druidServer, - DataSegment segment - ) - { - return Pair.of(Intervals.of(intervalStr), Pair.of(version, Pair.of(druidServer, segment))); - } - - private void assertValues( - List>>> expected, List actual - ) - { - Assert.assertEquals(expected.size(), actual.size()); - - for (int i = 0; i < expected.size(); ++i) { - Pair>> expectedPair = expected.get(i); - TimelineObjectHolder actualTimelineObjectHolder = actual.get(i); - - Assert.assertEquals(expectedPair.lhs, actualTimelineObjectHolder.getInterval()); - Assert.assertEquals(expectedPair.rhs.lhs, actualTimelineObjectHolder.getVersion()); - - PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); - Assert.assertTrue(actualPartitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); - - ServerSelector selector = ((SingleElementPartitionChunk) actualPartitionHolder.iterator() - .next()).getObject(); - Assert.assertFalse(selector.isEmpty()); - Assert.assertEquals(expectedPair.rhs.rhs.lhs, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); - Assert.assertEquals(expectedPair.rhs.rhs.rhs, selector.getSegment()); - } - } - - private void setupViews() throws Exception - { - setupViews(null, null, true); - } - - private void setupViews(TierSelectorStrategy historicalStrategy, TierSelectorStrategy realtimeStrategy) throws Exception - { - setupViews(historicalStrategy, realtimeStrategy, new BrokerSegmentWatcherConfig()); - } - - private void setupViews(Set watchedTiers, Set ignoredTiers, boolean watchRealtimeTasks) throws Exception - { - setupViews( - new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), - new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), - new BrokerSegmentWatcherConfig() - { - @Override - public Set getWatchedTiers() - { - return watchedTiers; - } - - @Override - public boolean isWatchRealtimeTasks() - { - return watchRealtimeTasks; - } - - @Override - public Set getIgnoredTiers() - { - return ignoredTiers; - } - } - ); - } - - private void setupViews(TierSelectorStrategy historicalStrategy, TierSelectorStrategy realtimeStrategy, BrokerSegmentWatcherConfig brokerSegmentWatcherConfig) - throws Exception - { - baseView = new BatchServerInventoryView( - zkPathsConfig, - curator, - jsonMapper, - Predicates.alwaysTrue(), - "test" - ) - { - @Override - public void registerServerCallback(Executor exec, ServerCallback callback) - { - super.registerServerCallback( - exec, - new ServerCallback() { - @Override - public CallbackAction serverAdded(DruidServer server) - { - final CallbackAction res = callback.serverAdded(server); - serverAddedLatch.countDown(); - return res; - } - - @Override - public CallbackAction serverRemoved(DruidServer server) - { - return callback.serverRemoved(server); - } - } - ); - } - - @Override - public void registerSegmentCallback(Executor exec, final SegmentCallback callback) - { - super.registerSegmentCallback( - exec, - new SegmentCallback() - { - @Override - public CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) - { - CallbackAction res = callback.segmentAdded(server, segment); - segmentAddedLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) - { - CallbackAction res = callback.segmentRemoved(server, segment); - segmentRemovedLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentViewInitialized() - { - CallbackAction res = callback.segmentViewInitialized(); - segmentViewInitLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) - { - return CallbackAction.CONTINUE; - } - } - ); - } - }; - - DirectDruidClientFactory druidClientFactory = new DirectDruidClientFactory( - new NoopServiceEmitter(), - EasyMock.createMock(QueryRunnerFactoryConglomerate.class), - EasyMock.createMock(QueryWatcher.class), - getSmileMapper(), - EasyMock.createMock(HttpClient.class) - ); - - brokerServerView = new BrokerServerView( - druidClientFactory, - baseView, - historicalStrategy, - realtimeStrategy, - new NoopServiceEmitter(), - brokerSegmentWatcherConfig, - brokerViewOfCoordinatorConfig - ); - - baseView.start(); - brokerServerView.start(); - } - - private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) - { - return DataSegment.builder() - .dataSource("test_broker_server_view") - .interval(Intervals.of(intervalStr)) - .loadSpec( - ImmutableMap.of( - "type", - "local", - "path", - "somewhere" - ) - ) - .version(version) - .dimensions(ImmutableList.of()) - .metrics(ImmutableList.of()) - .shardSpec(NoneShardSpec.instance()) - .binaryVersion(9) - .size(0) - .build(); - } - - public ObjectMapper getSmileMapper() - { - final SmileFactory smileFactory = new SmileFactory(); - smileFactory.configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false); - smileFactory.delegateToTextual(true); - final ObjectMapper retVal = new DefaultObjectMapper(smileFactory, "broker"); - retVal.getFactory().setCodec(retVal); - return retVal; - } - - @After - public void tearDown() throws Exception - { - baseView.stop(); - tearDownServerAndCurator(); - } -} diff --git a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java deleted file mode 100644 index 503d4333fb4e..000000000000 --- a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java +++ /dev/null @@ -1,502 +0,0 @@ -/* - * 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.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Predicates; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.curator.CuratorTestBase; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineLookup; -import org.apache.druid.timeline.TimelineObjectHolder; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.apache.druid.timeline.partition.PartitionHolder; -import org.easymock.EasyMock; -import org.joda.time.Interval; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; - -@RunWith(Parameterized.class) -public class CoordinatorServerViewTest extends CuratorTestBase -{ - private ObjectMapper jsonMapper; - private ZkPathsConfig zkPathsConfig; - private String inventoryPath; - - private CountDownLatch segmentViewInitLatch; - private CountDownLatch segmentAddedLatch; - private CountDownLatch segmentRemovedLatch; - - private CountDownLatch callbackSegmentViewInitLatch; - private CountDownLatch callbackSegmentAddedLatch; - private CountDownLatch callbackSegmentRemovedLatch; - private CountDownLatch callbackServerSegmentRemovedLatch; - - private BatchServerInventoryView baseView; - private CoordinatorServerView coordinatorServerView; - private ExecutorService callbackExec; - - private boolean setDruidClientFactory; - - @Parameterized.Parameters - public static Object[] data() - { - return new Object[]{true, false}; - } - - public CoordinatorServerViewTest(boolean setDruidClientFactory) - { - this.setDruidClientFactory = setDruidClientFactory; - } - - @Before - public void setUp() throws Exception - { - jsonMapper = TestHelper.makeJsonMapper(); - zkPathsConfig = new ZkPathsConfig(); - inventoryPath = zkPathsConfig.getLiveSegmentsPath(); - callbackExec = Execs.singleThreaded("CoordinatorServerViewTest-%s"); - - setupServerAndCurator(); - curator.start(); - curator.blockUntilConnected(); - } - - @Test - public void testSingleServerAddedRemovedSegment() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(1); - segmentRemovedLatch = new CountDownLatch(1); - callbackSegmentViewInitLatch = new CountDownLatch(1); - callbackSegmentAddedLatch = new CountDownLatch(1); - callbackServerSegmentRemovedLatch = new CountDownLatch(1); - callbackSegmentRemovedLatch = new CountDownLatch(1); - - setupViews(setDruidClientFactory); - - final DruidServer druidServer = new DruidServer( - "localhost:1234", - "localhost:1234", - null, - 10000000L, - null, - ServerType.HISTORICAL, - "default_tier", - 0 - ); - - setupZNodeForServer(druidServer, zkPathsConfig, jsonMapper); - - final DataSegment segment = dataSegmentWithIntervalAndVersion("2014-10-20T00:00:00Z/P1D", "v1"); - final int partition = segment.getShardSpec().getPartitionNum(); - final Interval intervals = Intervals.of("2014-10-20T00:00:00Z/P1D"); - announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentAddedLatch)); - - if (setDruidClientFactory) { - Assert.assertNotNull(coordinatorServerView.getQueryRunner(druidServer.getName())); - } else { - Assert.assertNull(coordinatorServerView.getQueryRunner(druidServer.getName())); - } - - TimelineLookup timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); - List serverLookupRes = (List) timeline.lookup( - intervals - ); - Assert.assertEquals(1, serverLookupRes.size()); - - TimelineObjectHolder actualTimelineObjectHolder = serverLookupRes.get(0); - Assert.assertEquals(intervals, actualTimelineObjectHolder.getInterval()); - Assert.assertEquals("v1", actualTimelineObjectHolder.getVersion()); - - PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); - Assert.assertTrue(actualPartitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); - - SegmentLoadInfo segmentLoadInfo = actualPartitionHolder.iterator().next().getObject(); - Assert.assertFalse(segmentLoadInfo.isEmpty()); - Assert.assertEquals( - druidServer.getMetadata(), - Iterables.getOnlyElement(segmentLoadInfo.toImmutableSegmentLoadInfo().getServers()) - ); - Assert.assertNotNull(timeline.findChunk(intervals, "v1", partition)); - - unannounceSegmentForServer(druidServer, segment); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackServerSegmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentRemovedLatch)); - - Assert.assertEquals( - 0, - ((List) timeline.lookup(Intervals.of("2014-10-20T00:00:00Z/P1D"))).size() - ); - Assert.assertNull(timeline.findChunk(intervals, "v1", partition)); - } - - @Test - public void testMultipleServerAddedRemovedSegment() throws Exception - { - segmentViewInitLatch = new CountDownLatch(1); - segmentAddedLatch = new CountDownLatch(5); - - // temporarily set latch count to 1 - segmentRemovedLatch = new CountDownLatch(1); - - callbackSegmentViewInitLatch = new CountDownLatch(1); - callbackSegmentAddedLatch = new CountDownLatch(5); - callbackServerSegmentRemovedLatch = new CountDownLatch(1); - callbackSegmentRemovedLatch = new CountDownLatch(1); - - setupViews(setDruidClientFactory); - - final List druidServers = Lists.transform( - ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), - new Function<>() - { - @Override - public DruidServer apply(String input) - { - return new DruidServer( - input, - input, - null, - 10000000L, - null, - ServerType.HISTORICAL, - "default_tier", - 0 - ); - } - } - ); - - for (DruidServer druidServer : druidServers) { - setupZNodeForServer(druidServer, zkPathsConfig, jsonMapper); - } - - final List segments = Lists.transform( - ImmutableList.of( - Pair.of("2011-04-01/2011-04-03", "v1"), - Pair.of("2011-04-03/2011-04-06", "v1"), - Pair.of("2011-04-01/2011-04-09", "v2"), - Pair.of("2011-04-06/2011-04-09", "v3"), - Pair.of("2011-04-01/2011-04-02", "v3") - ), new Function<>() - { - @Override - public DataSegment apply(Pair input) - { - return dataSegmentWithIntervalAndVersion(input.lhs, input.rhs); - } - } - ); - - for (int i = 0; i < 5; ++i) { - announceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig, jsonMapper); - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentViewInitLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentAddedLatch)); - - for (int i = 0; i < 5; ++i) { - if (setDruidClientFactory) { - Assert.assertNotNull(coordinatorServerView.getQueryRunner(druidServers.get(i).getName())); - } else { - Assert.assertNull(coordinatorServerView.getQueryRunner(druidServers.get(i).getName())); - } - } - - TimelineLookup timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - (List) timeline.lookup( - Intervals.of( - "2011-04-01/2011-04-09" - ) - ) - ); - - // unannounce the segment created by dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-09", "v2") - unannounceSegmentForServer(druidServers.get(2), segments.get(2)); - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackServerSegmentRemovedLatch)); - - // renew segmentRemovedLatch since we still have 4 segments to unannounce - segmentRemovedLatch = new CountDownLatch(4); - callbackServerSegmentRemovedLatch = new CountDownLatch(4); - callbackSegmentRemovedLatch = new CountDownLatch(4); - - timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), - createExpected("2011-04-02/2011-04-03", "v1", druidServers.get(0), segments.get(0)), - createExpected("2011-04-03/2011-04-06", "v1", druidServers.get(1), segments.get(1)), - createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) - ), - (List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) - ); - - // unannounce all the segments - for (int i = 0; i < 5; ++i) { - // skip the one that was previously unannounced - if (i != 2) { - unannounceSegmentForServer(druidServers.get(i), segments.get(i)); - } - } - Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackSegmentRemovedLatch)); - Assert.assertTrue(timing.forWaiting().awaitLatch(callbackServerSegmentRemovedLatch)); - - Assert.assertEquals( - 0, - ((List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))).size() - ); - } - - private void unannounceSegmentForServer(DruidServer druidServer, DataSegment segment) throws Exception - { - curator - .delete() - .guaranteed() - .forPath(ZKPaths.makePath(inventoryPath, druidServer.getHost(), segment.getId().toString())); - } - - private Pair>> createExpected( - String intervalStr, - String version, - DruidServer druidServer, - DataSegment segment - ) - { - return Pair.of(Intervals.of(intervalStr), Pair.of(version, Pair.of(druidServer, segment))); - } - - private void assertValues( - List>>> expected, List actual - ) - { - Assert.assertEquals(expected.size(), actual.size()); - - for (int i = 0; i < expected.size(); ++i) { - Pair>> expectedPair = expected.get(i); - TimelineObjectHolder actualTimelineObjectHolder = actual.get(i); - - Assert.assertEquals(expectedPair.lhs, actualTimelineObjectHolder.getInterval()); - Assert.assertEquals(expectedPair.rhs.lhs, actualTimelineObjectHolder.getVersion()); - - PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); - Assert.assertTrue(actualPartitionHolder.isComplete()); - Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); - - SegmentLoadInfo segmentLoadInfo = actualPartitionHolder.iterator().next().getObject(); - Assert.assertFalse(segmentLoadInfo.isEmpty()); - Assert.assertEquals(expectedPair.rhs.rhs.lhs.getMetadata(), - Iterables.getOnlyElement(segmentLoadInfo.toImmutableSegmentLoadInfo().getServers())); - Assert.assertEquals(expectedPair.rhs.rhs.lhs.getMetadata(), segmentLoadInfo.pickOne()); - } - } - - private void setupViews(boolean setDruidClientFactory) throws Exception - { - baseView = new BatchServerInventoryView( - zkPathsConfig, - curator, - jsonMapper, - Predicates.alwaysTrue(), - "test" - ) - { - @Override - public void registerSegmentCallback(Executor exec, final SegmentCallback callback) - { - super.registerSegmentCallback( - exec, - new SegmentCallback() - { - @Override - public CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) - { - CallbackAction res = callback.segmentAdded(server, segment); - segmentAddedLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) - { - CallbackAction res = callback.segmentRemoved(server, segment); - segmentRemovedLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentViewInitialized() - { - CallbackAction res = callback.segmentViewInitialized(); - segmentViewInitLatch.countDown(); - return res; - } - - @Override - public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) - { - return CallbackAction.CONTINUE; - } - } - ); - } - }; - - DirectDruidClientFactory druidClientFactory = null; - - if (setDruidClientFactory) { - druidClientFactory = EasyMock.createMock(DirectDruidClientFactory.class); - DirectDruidClient directDruidClient = EasyMock.mock(DirectDruidClient.class); - EasyMock.expect(druidClientFactory.makeDirectClient(EasyMock.anyObject(DruidServer.class))) - .andReturn(directDruidClient) - .anyTimes(); - - EasyMock.replay(druidClientFactory); - } - - coordinatorServerView = new CoordinatorServerView( - baseView, - new CoordinatorSegmentWatcherConfig(), - new NoopServiceEmitter(), - druidClientFactory - ); - - baseView.start(); - initServerViewTimelineCallback(coordinatorServerView); - coordinatorServerView.start(); - } - - private void initServerViewTimelineCallback(final CoordinatorServerView serverView) - { - serverView.registerTimelineCallback( - callbackExec, - new TimelineServerView.TimelineCallback() - { - @Override - public ServerView.CallbackAction timelineInitialized() - { - callbackSegmentViewInitLatch.countDown(); - return ServerView.CallbackAction.CONTINUE; - } - - @Override - public ServerView.CallbackAction segmentAdded(final DruidServerMetadata server, final DataSegment segment) - { - callbackSegmentAddedLatch.countDown(); - return ServerView.CallbackAction.CONTINUE; - } - - @Override - public ServerView.CallbackAction segmentRemoved(final DataSegment segment) - { - callbackSegmentRemovedLatch.countDown(); - return ServerView.CallbackAction.CONTINUE; - } - - @Override - public ServerView.CallbackAction serverSegmentRemoved( - final DruidServerMetadata server, - final DataSegment segment - ) - { - callbackServerSegmentRemovedLatch.countDown(); - return ServerView.CallbackAction.CONTINUE; - } - - @Override - public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) - { - return ServerView.CallbackAction.CONTINUE; - } - } - ); - } - - private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) - { - return DataSegment.builder() - .dataSource("test_overlord_server_view") - .interval(Intervals.of(intervalStr)) - .loadSpec( - ImmutableMap.of( - "type", - "local", - "path", - "somewhere" - ) - ) - .version(version) - .dimensions(ImmutableList.of()) - .metrics(ImmutableList.of()) - .shardSpec(NoneShardSpec.instance()) - .binaryVersion(9) - .size(0) - .build(); - } - - @After - public void tearDown() throws Exception - { - baseView.stop(); - tearDownServerAndCurator(); - } -} diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java deleted file mode 100644 index fb38406ecffc..000000000000 --- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java +++ /dev/null @@ -1,517 +0,0 @@ -/* - * 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.client; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.base.Stopwatch; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.ExponentialBackoffRetry; -import org.apache.curator.test.TestingCluster; -import org.apache.curator.test.Timing; -import org.apache.druid.client.BatchServerInventoryView; -import org.apache.druid.client.DruidServer; -import org.apache.druid.client.ServerView; -import org.apache.druid.curator.PotentiallyGzippedCompressionProvider; -import org.apache.druid.curator.announcement.NodeAnnouncer; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; -import org.apache.druid.server.coordination.CuratorDataSegmentServerAnnouncer; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.timeline.DataSegment; -import org.easymock.EasyMock; -import org.easymock.IAnswer; -import org.easymock.LogicalOperator; -import org.joda.time.DateTime; -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 javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * - */ -public class BatchServerInventoryViewTest -{ - private static final String TEST_BASE_PATH = "/test"; - public static final DateTime SEGMENT_INTERVAL_START = DateTimes.of("2013-01-01"); - public static final int INITIAL_SEGMENTS = 100; - private static final Timing TIMING = new Timing(); - - private TestingCluster testingCluster; - private CuratorFramework cf; - private ObjectMapper jsonMapper; - private NodeAnnouncer nodeAnnouncer; - private BatchDataSegmentAnnouncer segmentAnnouncer; - private DataSegmentServerAnnouncer serverAnnouncer; - private Set testSegments; - private BatchServerInventoryView batchServerInventoryView; - private BatchServerInventoryView filteredBatchServerInventoryView; - private final AtomicInteger inventoryUpdateCounter = new AtomicInteger(); - - @Rule - public ExpectedException exception = ExpectedException.none(); - - @Before - public void setUp() throws Exception - { - testingCluster = new TestingCluster(1); - testingCluster.start(); - - cf = CuratorFrameworkFactory.builder() - .connectString(testingCluster.getConnectString()) - .retryPolicy(new ExponentialBackoffRetry(1, 10)) - .compressionProvider(new PotentiallyGzippedCompressionProvider(true)) - .build(); - cf.start(); - cf.blockUntilConnected(); - cf.create().creatingParentsIfNeeded().forPath(TEST_BASE_PATH); - - jsonMapper = TestHelper.makeJsonMapper(); - - nodeAnnouncer = new NodeAnnouncer(cf, Execs.directExecutor()); - nodeAnnouncer.start(); - - DruidServerMetadata serverMetadata = new DruidServerMetadata( - "id", - "host", - null, - Long.MAX_VALUE, - null, - ServerType.HISTORICAL, - "tier", - 0 - ); - - ZkPathsConfig zkPathsConfig = new ZkPathsConfig() - { - @Override - public String getBase() - { - return TEST_BASE_PATH; - } - }; - - serverAnnouncer = new CuratorDataSegmentServerAnnouncer( - serverMetadata, - zkPathsConfig, - nodeAnnouncer, - jsonMapper - ); - serverAnnouncer.announce(); - - segmentAnnouncer = new BatchDataSegmentAnnouncer( - serverMetadata, - new BatchDataSegmentAnnouncerConfig() - { - @Override - public int getSegmentsPerNode() - { - return 50; - } - }, - zkPathsConfig, - nodeAnnouncer, - jsonMapper - ); - - testSegments = Sets.newConcurrentHashSet(); - for (int i = 0; i < INITIAL_SEGMENTS; i++) { - testSegments.add(makeSegment(i)); - } - - batchServerInventoryView = new BatchServerInventoryView( - new ZkPathsConfig() - { - @Override - public String getBase() - { - return TEST_BASE_PATH; - } - }, - cf, - jsonMapper, - Predicates.alwaysTrue(), - "test" - ); - - batchServerInventoryView.start(); - inventoryUpdateCounter.set(0); - filteredBatchServerInventoryView = new BatchServerInventoryView( - new ZkPathsConfig() - { - @Override - public String getBase() - { - return TEST_BASE_PATH; - } - }, - cf, - jsonMapper, - new Predicate<>() - { - @Override - public boolean apply(@Nullable Pair input) - { - return input.rhs.getInterval().getStart().isBefore(SEGMENT_INTERVAL_START.plusDays(INITIAL_SEGMENTS)); - } - }, - "test" - ) - { - @Override - protected DruidServer addInnerInventory(DruidServer container, String inventoryKey, Set inventory) - { - DruidServer server = super.addInnerInventory(container, inventoryKey, inventory); - inventoryUpdateCounter.incrementAndGet(); - return server; - } - }; - filteredBatchServerInventoryView.start(); - } - - @After - public void tearDown() throws Exception - { - batchServerInventoryView.stop(); - filteredBatchServerInventoryView.stop(); - serverAnnouncer.unannounce(); - nodeAnnouncer.stop(); - cf.close(); - testingCluster.stop(); - } - - @Test - public void testRun() throws Exception - { - segmentAnnouncer.announceSegments(testSegments); - - waitForSync(batchServerInventoryView, testSegments); - - DruidServer server = Iterables.get(batchServerInventoryView.getInventory(), 0); - Set segments = Sets.newHashSet(server.iterateAllSegments()); - - Assert.assertEquals(testSegments, segments); - - DataSegment segment1 = makeSegment(101); - DataSegment segment2 = makeSegment(102); - - segmentAnnouncer.announceSegment(segment1); - segmentAnnouncer.announceSegment(segment2); - testSegments.add(segment1); - testSegments.add(segment2); - - waitForSync(batchServerInventoryView, testSegments); - - Assert.assertEquals(testSegments, Sets.newHashSet(server.iterateAllSegments())); - - segmentAnnouncer.unannounceSegment(segment1); - segmentAnnouncer.unannounceSegment(segment2); - testSegments.remove(segment1); - testSegments.remove(segment2); - - waitForSync(batchServerInventoryView, testSegments); - - Assert.assertEquals(testSegments, Sets.newHashSet(server.iterateAllSegments())); - } - - @Test - public void testRunWithFilter() throws Exception - { - segmentAnnouncer.announceSegments(testSegments); - - waitForSync(filteredBatchServerInventoryView, testSegments); - - DruidServer server = Iterables.get(filteredBatchServerInventoryView.getInventory(), 0); - Set segments = Sets.newHashSet(server.iterateAllSegments()); - - Assert.assertEquals(testSegments, segments); - int prevUpdateCount = inventoryUpdateCounter.get(); - // segment outside the range of default filter - DataSegment segment1 = makeSegment(101); - segmentAnnouncer.announceSegment(segment1); - testSegments.add(segment1); - - waitForUpdateEvents(prevUpdateCount + 1); - Assert.assertNull( - Iterables.getOnlyElement(filteredBatchServerInventoryView.getInventory()) - .getSegment(segment1.getId()) - ); - } - - @Test - public void testRunWithFilterCallback() throws Exception - { - final CountDownLatch removeCallbackLatch = new CountDownLatch(1); - - segmentAnnouncer.announceSegments(testSegments); - - waitForSync(filteredBatchServerInventoryView, testSegments); - - DruidServer server = Iterables.get(filteredBatchServerInventoryView.getInventory(), 0); - Set segments = Sets.newHashSet(server.iterateAllSegments()); - - Assert.assertEquals(testSegments, segments); - - ServerView.SegmentCallback callback = EasyMock.createStrictMock(ServerView.SegmentCallback.class); - Comparator dataSegmentComparator = - Comparator.comparing(DataSegment::getInterval, Comparators.intervalsByStartThenEnd()); - - EasyMock - .expect( - callback.segmentAdded( - EasyMock.anyObject(), - EasyMock.cmp(makeSegment(INITIAL_SEGMENTS + 2), dataSegmentComparator, LogicalOperator.EQUAL) - ) - ) - .andReturn(ServerView.CallbackAction.CONTINUE) - .times(1); - - EasyMock - .expect( - callback.segmentRemoved( - EasyMock.anyObject(), - EasyMock.cmp(makeSegment(INITIAL_SEGMENTS + 2), dataSegmentComparator, LogicalOperator.EQUAL) - ) - ) - .andAnswer( - new IAnswer<>() - { - @Override - public ServerView.CallbackAction answer() - { - removeCallbackLatch.countDown(); - return ServerView.CallbackAction.CONTINUE; - } - } - ) - .times(1); - - - EasyMock.replay(callback); - - filteredBatchServerInventoryView.registerSegmentCallback( - Execs.directExecutor(), - callback, - new Predicate<>() - { - @Override - public boolean apply(@Nullable Pair input) - { - return input.rhs.getInterval().getStart().equals(SEGMENT_INTERVAL_START.plusDays(INITIAL_SEGMENTS + 2)); - } - } - ); - - DataSegment segment2 = makeSegment(INITIAL_SEGMENTS + 2); - segmentAnnouncer.announceSegment(segment2); - testSegments.add(segment2); - - DataSegment oldSegment = makeSegment(-1); - segmentAnnouncer.announceSegment(oldSegment); - testSegments.add(oldSegment); - - segmentAnnouncer.unannounceSegment(oldSegment); - testSegments.remove(oldSegment); - - waitForSync(filteredBatchServerInventoryView, testSegments); - - segmentAnnouncer.unannounceSegment(segment2); - testSegments.remove(segment2); - - waitForSync(filteredBatchServerInventoryView, testSegments); - TIMING.forWaiting().awaitLatch(removeCallbackLatch); - - EasyMock.verify(callback); - } - - private DataSegment makeSegment(int offset) - { - return DataSegment.builder() - .dataSource("foo") - .interval( - new Interval( - SEGMENT_INTERVAL_START.plusDays(offset), - SEGMENT_INTERVAL_START.plusDays(offset + 1) - ) - ) - .version(DateTimes.nowUtc().toString()) - .size(0) - .build(); - } - - private static void waitForSync(BatchServerInventoryView batchServerInventoryView, Set testSegments) - throws Exception - { - final Timing forWaitingTiming = TIMING.forWaiting(); - Stopwatch stopwatch = Stopwatch.createStarted(); - while (Iterables.isEmpty(batchServerInventoryView.getInventory()) - || Iterables.size(Iterables.get(batchServerInventoryView.getInventory(), 0).iterateAllSegments()) != - testSegments.size()) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > forWaitingTiming.milliseconds()) { - throw new ISE("BatchServerInventoryView is not updating"); - } - } - } - - private void waitForUpdateEvents(int count) - throws Exception - { - final Timing forWaitingTiming = TIMING.forWaiting(); - Stopwatch stopwatch = Stopwatch.createStarted(); - while (inventoryUpdateCounter.get() != count) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > forWaitingTiming.milliseconds()) { - throw new ISE( - "BatchServerInventoryView is not updating counter expected[%d] value[%d]", - count, - inventoryUpdateCounter.get() - ); - } - } - } - - @Test - public void testSameTimeZnode() throws Exception - { - final int numThreads = INITIAL_SEGMENTS / 10; - final ListeningExecutorService executor = MoreExecutors.listeningDecorator(Execs.multiThreaded( - numThreads, - "BatchServerInventoryViewTest-%d" - )); - - segmentAnnouncer.announceSegments(testSegments); - - waitForSync(batchServerInventoryView, testSegments); - - DruidServer server = Iterables.get(batchServerInventoryView.getInventory(), 0); - final Set segments = Sets.newHashSet(server.iterateAllSegments()); - - Assert.assertEquals(testSegments, segments); - - final CountDownLatch latch = new CountDownLatch(numThreads); - - final List> futures = new ArrayList<>(); - for (int i = 0; i < numThreads; ++i) { - final int ii = i; - futures.add( - executor.submit( - new Callable<>() - { - @Override - public BatchDataSegmentAnnouncer call() - { - BatchDataSegmentAnnouncer segmentAnnouncer = new BatchDataSegmentAnnouncer( - new DruidServerMetadata( - "id", - "host", - null, - Long.MAX_VALUE, - null, - ServerType.HISTORICAL, - "tier", - 0 - ), - new BatchDataSegmentAnnouncerConfig() - { - @Override - public int getSegmentsPerNode() - { - return 50; - } - }, - new ZkPathsConfig() - { - @Override - public String getBase() - { - return TEST_BASE_PATH; - } - }, - nodeAnnouncer, - jsonMapper - ); - List segments = new ArrayList<>(); - try { - for (int j = 0; j < INITIAL_SEGMENTS / numThreads; ++j) { - segments.add(makeSegment(INITIAL_SEGMENTS + ii + numThreads * j)); - } - latch.countDown(); - latch.await(); - segmentAnnouncer.announceSegments(segments); - testSegments.addAll(segments); - } - catch (Exception e) { - throw new RuntimeException(e); - } - return segmentAnnouncer; - } - } - ) - ); - } - final List announcers = Futures.allAsList(futures).get(); - Assert.assertEquals(INITIAL_SEGMENTS * 2, testSegments.size()); - waitForSync(batchServerInventoryView, testSegments); - - Assert.assertEquals(testSegments, Sets.newHashSet(server.iterateAllSegments())); - - for (int i = 0; i < INITIAL_SEGMENTS; ++i) { - final DataSegment segment = makeSegment(100 + i); - segmentAnnouncer.unannounceSegment(segment); - testSegments.remove(segment); - } - - waitForSync(batchServerInventoryView, testSegments); - - Assert.assertEquals(testSegments, Sets.newHashSet(server.iterateAllSegments())); - } -} diff --git a/server/src/test/java/org/apache/druid/curator/CuratorTestBase.java b/server/src/test/java/org/apache/druid/curator/CuratorTestBase.java index b7bf33316fc4..c58999565d71 100644 --- a/server/src/test/java/org/apache/druid/curator/CuratorTestBase.java +++ b/server/src/test/java/org/apache/druid/curator/CuratorTestBase.java @@ -19,19 +19,11 @@ package org.apache.druid.curator; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableSet; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryOneTime; import org.apache.curator.test.TestingServer; import org.apache.curator.test.Timing; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.client.DruidServer; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.timeline.DataSegment; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; import java.io.IOException; @@ -57,83 +49,6 @@ public void setupServerAndCurator() throws Exception .build(); } - protected void setupZNodeForServer(DruidServer server, ZkPathsConfig zkPathsConfig, ObjectMapper jsonMapper) - { - final String announcementsPath = zkPathsConfig.getAnnouncementsPath(); - final String inventoryPath = zkPathsConfig.getLiveSegmentsPath(); - - String zkPath = ZKPaths.makePath(announcementsPath, server.getHost()); - try { - curator.create() - .creatingParentsIfNeeded() - .forPath(zkPath, jsonMapper.writeValueAsBytes(server.getMetadata())); - curator.create() - .creatingParentsIfNeeded() - .forPath(ZKPaths.makePath(inventoryPath, server.getHost())); - } - catch (KeeperException.NodeExistsException e) { - /* - * For some reason, Travis build sometimes fails here because of - * org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = NodeExists, though it should never - * happen because zookeeper should be in a clean state for each run of tests. - * Address issue: https://github.com/apache/druid/issues/1512 - */ - try { - curator.setData() - .forPath(zkPath, jsonMapper.writeValueAsBytes(server.getMetadata())); - curator.setData() - .forPath(ZKPaths.makePath(inventoryPath, server.getHost())); - } - catch (Exception e1) { - throw new RuntimeException(e1); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - protected void announceSegmentForServer( - DruidServer druidServer, - DataSegment segment, - ZkPathsConfig zkPathsConfig, - ObjectMapper jsonMapper - ) - { - final String segmentAnnouncementPath = - ZKPaths.makePath(zkPathsConfig.getLiveSegmentsPath(), druidServer.getHost(), segment.getId().toString()); - - try { - curator.create() - .compressed() - .withMode(CreateMode.EPHEMERAL) - .forPath(segmentAnnouncementPath, jsonMapper.writeValueAsBytes(ImmutableSet.of(segment))); - } - catch (KeeperException.NodeExistsException e) { - try { - curator.setData() - .forPath(segmentAnnouncementPath, jsonMapper.writeValueAsBytes(ImmutableSet.of(segment))); - } - catch (Exception e1) { - throw new RuntimeException(e1); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - protected void unannounceSegmentForServer(DruidServer druidServer, DataSegment segment, ZkPathsConfig zkPathsConfig) - throws Exception - { - String path = ZKPaths.makePath( - zkPathsConfig.getLiveSegmentsPath(), - druidServer.getHost(), - segment.getId().toString() - ); - curator.delete().guaranteed().forPath(path); - } - public void tearDownServerAndCurator() { try { diff --git a/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java b/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java index 146e5adf326a..54ee16c891d2 100644 --- a/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java +++ b/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java @@ -54,11 +54,8 @@ public void testOverrideBaseOnlyConfig() propertyValues.clear(); propertyValues.put(StringUtils.format("%s.base", CONFIG_PREFIX), base); - propertyValues.put(StringUtils.format("%s.propertiesPath", CONFIG_PREFIX), ZKPaths.makePath(base, "properties")); propertyValues.put(StringUtils.format("%s.announcementsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "announcements")); - propertyValues.put(StringUtils.format("%s.liveSegmentsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "segments")); propertyValues.put(StringUtils.format("%s.coordinatorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "coordinator")); - propertyValues.put(StringUtils.format("%s.connectorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "connector")); ZkPathsConfig zkPathsConfigObj = zkPathsConfig.get(); validateEntries(zkPathsConfigObj); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperCacheTest.java index d0a7ca819bdd..bf8edc619563 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperCacheTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter; @@ -63,7 +62,6 @@ public class SegmentCacheBootstrapperCacheTest private File infoDir; private File cacheDir; private TestDataSegmentAnnouncer segmentAnnouncer; - private TestDataServerAnnouncer serverAnnouncer; private SegmentManager segmentManager; private SegmentLoaderConfig loaderConfig; private SegmentLocalCacheManager cacheManager; @@ -106,7 +104,6 @@ public List getLocations() objectMapper ); segmentManager = new SegmentManager(cacheManager); - serverAnnouncer = new TestDataServerAnnouncer(); segmentAnnouncer = new TestDataSegmentAnnouncer(); coordinatorClient = new TestCoordinatorClient(); emitter = new StubServiceEmitter(); @@ -138,19 +135,15 @@ public void testLoadStartStopWithEmptyLocations() throws IOException loadDropHandler, loaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, emitter, new DefaultLoadSpecHolder() ); bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); bootstrapper.stop(); - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -166,19 +159,15 @@ public void testLoadStartStop() throws IOException loadDropHandler, loaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, emitter, new DefaultLoadSpecHolder() ); bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); bootstrapper.stop(); - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } @Test @@ -206,16 +195,13 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException loadDropHandler, loaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, emitter, new DefaultLoadSpecHolder() ); bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); // Verify the expected announcements Assert.assertTrue(segmentAnnouncer.getObservedSegments().containsAll(expectedSegments)); @@ -231,6 +217,5 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); bootstrapper.stop(); - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperTest.java index d64f39fb8ee4..594554022c75 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentCacheBootstrapperTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.StubServiceEmitter; @@ -53,7 +52,6 @@ public class SegmentCacheBootstrapperTest private static final int COUNT = 50; private TestDataSegmentAnnouncer segmentAnnouncer; - private TestDataServerAnnouncer serverAnnouncer; private SegmentLoaderConfig segmentLoaderConfig; private TestCoordinatorClient coordinatorClient; private StubServiceEmitter serviceEmitter; @@ -67,7 +65,6 @@ public void setUp() throws IOException final File segmentCacheDir = temporaryFolder.newFolder(); segmentAnnouncer = new TestDataSegmentAnnouncer(); - serverAnnouncer = new TestDataServerAnnouncer(); segmentLoaderConfig = new SegmentLoaderConfig() { @Override @@ -126,9 +123,7 @@ public void testStartStop() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new DefaultLoadSpecHolder() @@ -137,7 +132,6 @@ public void testStartStop() throws Exception Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -154,7 +148,6 @@ public void testStartStop() throws Exception bootstrapper.stop(); - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); Assert.assertEquals(1, cacheManager.getObservedShutdownBootstrapCount().get()); } @@ -185,9 +178,7 @@ public void testLoadCachedSegments() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new DefaultLoadSpecHolder() @@ -197,7 +188,6 @@ public void testLoadCachedSegments() throws Exception bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -214,7 +204,6 @@ public void testLoadCachedSegments() throws Exception bootstrapper.stop(); - Assert.assertEquals(0, serverAnnouncer.getObservedCount()); Assert.assertEquals(1, cacheManager.getObservedShutdownBootstrapCount().get()); } @@ -241,9 +230,7 @@ public void testLoadBootstrapSegments() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new DefaultLoadSpecHolder() @@ -253,7 +240,6 @@ public void testLoadBootstrapSegments() throws Exception bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); for (int i = 0; i < COUNT; ++i) { @@ -295,9 +281,7 @@ public void testLoadNoBootstrapSegments() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new TestLoadSpecHolder(LookupLoadingSpec.ALL, BroadcastDatasourceLoadingSpec.NONE) @@ -307,7 +291,6 @@ public void testLoadNoBootstrapSegments() throws Exception bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); final ImmutableList expectedBootstrapSegments = ImmutableList.of(); @@ -344,9 +327,7 @@ public void testLoadOnlyRequiredBootstrapSegments() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new TestLoadSpecHolder(LookupLoadingSpec.NONE, BroadcastDatasourceLoadingSpec.loadOnly(Set.of("test1"))) @@ -356,7 +337,6 @@ public void testLoadOnlyRequiredBootstrapSegments() throws Exception bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertFalse(segmentManager.getDataSourceCounts().isEmpty()); Assert.assertEquals(ImmutableSet.of("test1"), segmentManager.getDataSourceNames()); @@ -385,9 +365,7 @@ public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception handler, segmentLoaderConfig, segmentAnnouncer, - serverAnnouncer, segmentManager, - new ServerTypeConfig(ServerType.HISTORICAL), coordinatorClient, serviceEmitter, new DefaultLoadSpecHolder() @@ -397,7 +375,6 @@ public void testLoadBootstrapSegmentsWhenExceptionThrown() throws Exception bootstrapper.start(); - Assert.assertEquals(1, serverAnnouncer.getObservedCount()); Assert.assertTrue(segmentManager.getDataSourceCounts().isEmpty()); Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index f66f4c5530d5..607e5a3fa89a 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -149,7 +149,7 @@ public void testSegmentLoading1() // Make sure the scheduled runnable that "deletes" segment files has been executed. // Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in - // ZkCoordinator, the scheduled runnable will not actually delete segment files. + // SegmentLoadDropHandler, the scheduled runnable will not actually delete segment files. for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -197,7 +197,7 @@ public void testSegmentLoading2() // Make sure the scheduled runnable that "deletes" segment files has been executed. // Because another addSegment() call is executed, which removes the segment from segmentsToDelete field in - // ZkCoordinator, the scheduled runnable will not actually delete segment files. + // SegmentLoadDropHandler, the scheduled runnable will not actually delete segment files. for (Runnable runnable : scheduledRunnable) { runnable.run(); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java b/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java deleted file mode 100644 index d88b753f5ff6..000000000000 --- a/server/src/test/java/org/apache/druid/server/coordination/TestDataServerAnnouncer.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import java.util.concurrent.atomic.AtomicInteger; - -/** - * A test data server announcer that tracks the count of all announcements and unannouncements. - * The counter is incremented and decremented on each announce and unannounce respectively. - */ -public class TestDataServerAnnouncer implements DataSegmentServerAnnouncer -{ - private final AtomicInteger observedCount; - - TestDataServerAnnouncer() - { - this.observedCount = new AtomicInteger(0); - } - - @Override - public void announce() - { - observedCount.incrementAndGet(); - } - - @Override - public void unannounce() - { - observedCount.decrementAndGet(); - } - - public int getObservedCount() - { - return observedCount.get(); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java deleted file mode 100644 index cbfab8773f6d..000000000000 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.coordination; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.utils.EnsurePath; -import org.apache.curator.utils.ZKPaths; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.druid.server.initialization.ZkPathsConfig; -import org.easymock.EasyMock; -import org.junit.Test; - -public class ZkCoordinatorTest -{ - private final DruidServerMetadata me = new DruidServerMetadata( - "dummyServer", - "dummyHost", - null, - 0, - null, - ServerType.HISTORICAL, - "normal", - 0 - ); - private final ZkPathsConfig zkPaths = new ZkPathsConfig() - { - @Override - public String getBase() - { - return "/druid"; - } - }; - - @Test(timeout = 60_000L) - public void testSegmentPathIsCreatedIfZkAnnouncementIsEnabled() throws Exception - { - testSegmentPathCreated(true); - } - - @Test(timeout = 60_000L) - public void testSegmentPathIsNotCreatedIfZkAnnouncementIsDisabled() throws Exception - { - testSegmentPathCreated(false); - } - - private void testSegmentPathCreated(boolean announceSegmentsOnZk) throws Exception - { - final String liveSegmentsPath = ZKPaths.makePath( - zkPaths.getLiveSegmentsPath(), - me.getName() - ); - - final EnsurePath mockEnsurePath = EasyMock.mock(EnsurePath.class); - final CuratorFramework mockCurator = EasyMock.mock(CuratorFramework.class); - - if (announceSegmentsOnZk) { - EasyMock.expect(mockCurator.newNamespaceAwareEnsurePath(liveSegmentsPath)) - .andReturn(mockEnsurePath).once(); - - EasyMock.expect(mockCurator.getZookeeperClient()) - .andReturn(null).once(); - - mockEnsurePath.ensure(EasyMock.anyObject()); - EasyMock.expectLastCall().once(); - } - - EasyMock.replay(mockCurator, mockEnsurePath); - final ZkCoordinator zkCoordinator = new ZkCoordinator( - zkPaths, - me, - mockCurator, - new BatchDataSegmentAnnouncerConfig() { - @Override - public boolean isSkipSegmentAnnouncementOnZk() - { - return !announceSegmentsOnZk; - } - } - ); - - zkCoordinator.start(); - EasyMock.verify(); - zkCoordinator.stop(); - } -} diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 6b679ccd0ff1..21a51407162f 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -19,628 +19,106 @@ package org.apache.druid.server.coordination.coordination; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.ExponentialBackoffRetry; -import org.apache.curator.test.TestingCluster; -import org.apache.druid.curator.PotentiallyGzippedCompressionProvider; -import org.apache.druid.curator.announcement.NodeAnnouncer; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.DruidServerMetadata; -import org.apache.druid.server.coordination.SegmentSchemasChangeRequest; -import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.druid.server.initialization.ZkPathsConfig; +import org.apache.druid.server.coordination.SegmentChangeRequestDrop; +import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -/** - * - */ public class BatchDataSegmentAnnouncerTest { - private static final String TEST_BASE_PATH = "/test"; - private static final String TEST_SEGMENTS_PATH = "/test/segments/id"; - private static final Joiner JOINER = Joiner.on("/"); - private static final int NUM_THREADS = 4; - - private TestingCluster testingCluster; - private CuratorFramework cf; - private ObjectMapper jsonMapper; - private TestAnnouncer announcer; - private SegmentReader segmentReader; - private BatchDataSegmentAnnouncer segmentAnnouncer; - private Set testSegments; - - private final AtomicInteger maxBytesPerNode = new AtomicInteger(512 * 1024); - private Boolean skipDimensionsAndMetrics; - private Boolean skipLoadSpec; - - private ExecutorService exec; + private BatchDataSegmentAnnouncer announcer; @Before - public void setUp() throws Exception - { - testingCluster = new TestingCluster(1); - testingCluster.start(); - - cf = CuratorFrameworkFactory.builder() - .connectString(testingCluster.getConnectString()) - .retryPolicy(new ExponentialBackoffRetry(1, 10)) - .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) - .build(); - cf.start(); - cf.blockUntilConnected(); - cf.create().creatingParentsIfNeeded().forPath(TEST_BASE_PATH); - - jsonMapper = TestHelper.makeJsonMapper(); - - announcer = new TestAnnouncer( - cf, - Execs.directExecutor() - ); - announcer.start(); - - segmentReader = new SegmentReader(cf, jsonMapper); - skipDimensionsAndMetrics = false; - skipLoadSpec = false; - segmentAnnouncer = new BatchDataSegmentAnnouncer( - new DruidServerMetadata( - "id", - "host", - null, - Long.MAX_VALUE, - null, - ServerType.HISTORICAL, - "tier", - 0 - ), - new BatchDataSegmentAnnouncerConfig() - { - @Override - public int getSegmentsPerNode() - { - return 50; - } - - @Override - public long getMaxBytesPerNode() - { - return maxBytesPerNode.get(); - } - - @Override - public boolean isSkipDimensionsAndMetrics() - { - return skipDimensionsAndMetrics; - } - - @Override - public boolean isSkipLoadSpec() - { - return skipLoadSpec; - } - }, - new ZkPathsConfig() - { - @Override - public String getBase() - { - return TEST_BASE_PATH; - } - }, - announcer, - jsonMapper - ); - - testSegments = new HashSet<>(); - for (int i = 0; i < 100; i++) { - testSegments.add(makeSegment(i)); - } - - exec = Execs.multiThreaded(NUM_THREADS, "BatchDataSegmentAnnouncerTest-%d"); - } - - @After - public void tearDown() throws Exception + public void setUp() { - announcer.stop(); - cf.close(); - testingCluster.stop(); - exec.shutdownNow(); - } - - @Test - public void testSingleAnnounce() throws Exception - { - Iterator segIter = testSegments.iterator(); - DataSegment firstSegment = segIter.next(); - DataSegment secondSegment = segIter.next(); - - segmentAnnouncer.announceSegment(firstSegment); - - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - - for (String zNode : zNodes) { - Set segments = segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)); - Assert.assertEquals(segments.iterator().next(), firstSegment); - } - - segmentAnnouncer.announceSegment(secondSegment); - - for (String zNode : zNodes) { - Set segments = segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)); - Assert.assertEquals(Sets.newHashSet(firstSegment, secondSegment), segments); - } - - ChangeRequestsSnapshot snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(2, snapshot.getRequests().size()); - Assert.assertEquals(2, snapshot.getCounter().getCounter()); - - segmentAnnouncer.unannounceSegment(firstSegment); - - for (String zNode : zNodes) { - Set segments = segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)); - Assert.assertEquals(segments.iterator().next(), secondSegment); - } - - segmentAnnouncer.unannounceSegment(secondSegment); - - Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty()); - - snapshot = segmentAnnouncer.getSegmentChangesSince( - snapshot.getCounter() - ).get(); - Assert.assertEquals(2, snapshot.getRequests().size()); - Assert.assertEquals(4, snapshot.getCounter().getCounter()); - - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(0, snapshot.getRequests().size()); - Assert.assertEquals(4, snapshot.getCounter().getCounter()); + announcer = new BatchDataSegmentAnnouncer(); } @Test - public void testSingleTombstoneAnnounce() throws Exception + public void testAnnounceAndUnannounceProduceChangeHistory() throws Exception { - DataSegment firstSegment = makeSegment(0, true); - - segmentAnnouncer.announceSegment(firstSegment); - - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - - for (String zNode : zNodes) { - Set segments = segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode)); - Assert.assertEquals(segments.iterator().next(), firstSegment); - } + DataSegment segmentA = makeSegment(0); + DataSegment segmentB = makeSegment(1); - ChangeRequestsSnapshot snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(1, snapshot.getRequests().size()); - Assert.assertEquals(1, snapshot.getCounter().getCounter()); - - segmentAnnouncer.unannounceSegment(firstSegment); - - Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty()); + announcer.announceSegment(segmentA); + announcer.announceSegment(segmentB); + announcer.unannounceSegment(segmentA); - snapshot = segmentAnnouncer.getSegmentChangesSince( - snapshot.getCounter() - ).get(); + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(new ChangeRequestHistory.Counter(-1, -1)).get(); + // When counter is negative, getSegmentChangesSince returns the current snapshot of announced segments, which + // at this point is just segmentB. Assert.assertEquals(1, snapshot.getRequests().size()); - Assert.assertEquals(2, snapshot.getCounter().getCounter()); - - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(0, snapshot.getRequests().size()); - Assert.assertEquals(2, snapshot.getCounter().getCounter()); - } - - @Test - public void testSkipDimensions() throws Exception - { - skipDimensionsAndMetrics = true; - Iterator segIter = testSegments.iterator(); - DataSegment firstSegment = segIter.next(); - - segmentAnnouncer.announceSegment(firstSegment); - - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - - for (String zNode : zNodes) { - DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join( - TEST_SEGMENTS_PATH, - zNode - ))); - Assert.assertEquals(announcedSegment, firstSegment); - Assert.assertTrue(announcedSegment.getDimensions().isEmpty()); - Assert.assertTrue(announcedSegment.getMetrics().isEmpty()); - } - - segmentAnnouncer.unannounceSegment(firstSegment); - - Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty()); + Assert.assertTrue(snapshot.getRequests().get(0) instanceof SegmentChangeRequestLoad); } @Test - public void testSkipLoadSpec() throws Exception + public void testAnnounceSegmentsBatchesChangeRequests() throws Exception { - skipLoadSpec = true; - Iterator segIter = testSegments.iterator(); - DataSegment firstSegment = segIter.next(); + List segments = ImmutableList.of(makeSegment(0), makeSegment(1), makeSegment(2)); - segmentAnnouncer.announceSegment(firstSegment); + announcer.announceSegments(segments); - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - - for (String zNode : zNodes) { - DataSegment announcedSegment = Iterables.getOnlyElement(segmentReader.read(JOINER.join( - TEST_SEGMENTS_PATH, - zNode - ))); - Assert.assertEquals(announcedSegment, firstSegment); - Assert.assertNull(announcedSegment.getLoadSpec()); + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(ChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(segments.size(), snapshot.getRequests().size()); + for (DataSegmentChangeRequest request : snapshot.getRequests()) { + Assert.assertTrue(request instanceof SegmentChangeRequestLoad); } - - segmentAnnouncer.unannounceSegment(firstSegment); - - Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty()); } @Test - public void testSingleAnnounceManyTimes() throws Exception + public void testUnannounceAfterAnnounceProducesDropRequest() throws Exception { - int prevMax = maxBytesPerNode.get(); - maxBytesPerNode.set(2048); - // each segment is about 348 bytes long and that makes 2048 / 348 = 5 segments included per node - // so 100 segments makes 100 / 5 = 20 nodes - try { - for (DataSegment segment : testSegments) { - segmentAnnouncer.announceSegment(segment); - } - } - finally { - maxBytesPerNode.set(prevMax); - } + DataSegment segment = makeSegment(0); - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - Assert.assertEquals(20, zNodes.size()); + announcer.announceSegment(segment); + announcer.unannounceSegment(segment); - Set segments = Sets.newHashSet(testSegments); - for (String zNode : zNodes) { - for (DataSegment segment : segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode))) { - Assert.assertTrue("Invalid segment " + segment, segments.remove(segment)); - } - } - Assert.assertTrue("Failed to find segments " + segments, segments.isEmpty()); - } - - @Test - public void testBatchAnnounce() throws Exception - { - testBatchAnnounce(true); - } - - @Test - public void testMultipleBatchAnnounce() throws Exception - { - for (int i = 0; i < 10; i++) { - testBatchAnnounce(false); - } + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(ChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(2, snapshot.getRequests().size()); + Assert.assertTrue(snapshot.getRequests().get(0) instanceof SegmentChangeRequestLoad); + Assert.assertTrue(snapshot.getRequests().get(1) instanceof SegmentChangeRequestDrop); } @Test - public void testSchemaAnnounce() throws Exception + public void testDuplicateAnnouncementIsIgnored() throws Exception { - String dataSource = "foo"; - String segmentId = "id"; - String taskId = "t1"; - SegmentSchemas.SegmentSchema absoluteSchema1 = - new SegmentSchemas.SegmentSchema( - dataSource, - segmentId, - false, - 20, - ImmutableList.of("dim1", "dim2"), - Collections.emptyList(), - ImmutableMap.of("dim1", ColumnType.STRING, "dim2", ColumnType.STRING) - ); - - - SegmentSchemas.SegmentSchema absoluteSchema2 = - new SegmentSchemas.SegmentSchema( - dataSource, - segmentId, - false, - 40, - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of(), - ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim2", ColumnType.STRING, "dim3", ColumnType.STRING) - ); - - SegmentSchemas.SegmentSchema deltaSchema = - new SegmentSchemas.SegmentSchema( - dataSource, - segmentId, - true, - 40, - ImmutableList.of("dim3"), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim3", ColumnType.STRING) - ); - - segmentAnnouncer.announceSegmentSchemas( - taskId, - new SegmentSchemas(Collections.singletonList(absoluteSchema1)), - new SegmentSchemas(Collections.singletonList(absoluteSchema1)) - ); - - ChangeRequestsSnapshot snapshot; + DataSegment segment = makeSegment(0); - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(1, snapshot.getRequests().size()); - Assert.assertEquals(1, snapshot.getCounter().getCounter()); - - Assert.assertEquals( - absoluteSchema1, - ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) - .getSegmentSchemas() - .getSegmentSchemaList() - .get(0) - ); - segmentAnnouncer.announceSegmentSchemas( - taskId, - new SegmentSchemas(Collections.singletonList(absoluteSchema2)), - new SegmentSchemas(Collections.singletonList(deltaSchema)) - ); - - snapshot = segmentAnnouncer.getSegmentChangesSince(snapshot.getCounter()).get(); + announcer.announceSegment(segment); + announcer.announceSegment(segment); - Assert.assertEquals( - deltaSchema, - ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) - .getSegmentSchemas() - .getSegmentSchemaList() - .get(0) - ); - Assert.assertEquals(1, snapshot.getRequests().size()); - Assert.assertEquals(2, snapshot.getCounter().getCounter()); - - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals( - absoluteSchema2, - ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) - .getSegmentSchemas() - .getSegmentSchemaList() - .get(0) - ); + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(ChangeRequestHistory.Counter.ZERO).get(); Assert.assertEquals(1, snapshot.getRequests().size()); - Assert.assertEquals(2, snapshot.getCounter().getCounter()); - } - - private void testBatchAnnounce(boolean testHistory) throws Exception - { - segmentAnnouncer.announceSegments(testSegments); - - List zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH); - - Assert.assertEquals(2, zNodes.size()); - - Set allSegments = new HashSet<>(); - for (String zNode : zNodes) { - allSegments.addAll(segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode))); - } - Assert.assertEquals(allSegments, testSegments); - - ChangeRequestsSnapshot snapshot = null; - - if (testHistory) { - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); - Assert.assertEquals(testSegments.size(), snapshot.getCounter().getCounter()); - } - - segmentAnnouncer.unannounceSegments(testSegments); - - Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty()); - - if (testHistory) { - snapshot = segmentAnnouncer.getSegmentChangesSince( - snapshot.getCounter() - ).get(); - Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); - Assert.assertEquals(2 * testSegments.size(), snapshot.getCounter().getCounter()); - - snapshot = segmentAnnouncer.getSegmentChangesSince( - new ChangeRequestHistory.Counter(-1, -1) - ).get(); - Assert.assertEquals(0, snapshot.getRequests().size()); - Assert.assertEquals(2 * testSegments.size(), snapshot.getCounter().getCounter()); - } - } - - @Test(timeout = 5000L) - public void testAnnounceSegmentsWithSameSegmentConcurrently() throws ExecutionException, InterruptedException - { - final List futures = new ArrayList<>(NUM_THREADS); - - for (int i = 0; i < NUM_THREADS; i++) { - futures.add( - exec.submit(() -> { - try { - segmentAnnouncer.announceSegments(testSegments); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) - ); - } - - for (Future future : futures) { - future.get(); - } - - // Announcing 100 segments requires 2 nodes because of maxBytesPerNode configuration. - Assert.assertEquals(2, announcer.numPathAnnounced.size()); - for (ConcurrentHashMap eachMap : announcer.numPathAnnounced.values()) { - for (Entry entry : eachMap.entrySet()) { - Assert.assertEquals(1, entry.getValue().get()); - } - } - } - - @Test(timeout = 5000L) - public void testAnnounceSegmentWithSameSegmentConcurrently() throws ExecutionException, InterruptedException - { - final List futures = new ArrayList<>(NUM_THREADS); - - final DataSegment segment1 = makeSegment(0); - final DataSegment segment2 = makeSegment(1); - final DataSegment segment3 = makeSegment(2); - final DataSegment segment4 = makeSegment(3); - - for (int i = 0; i < NUM_THREADS; i++) { - futures.add( - exec.submit(() -> { - try { - segmentAnnouncer.announceSegment(segment1); - segmentAnnouncer.announceSegment(segment2); - segmentAnnouncer.announceSegment(segment3); - segmentAnnouncer.announceSegment(segment4); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) - ); - } - - for (Future future : futures) { - future.get(); - } - - Assert.assertEquals(1, announcer.numPathAnnounced.size()); - for (ConcurrentHashMap eachMap : announcer.numPathAnnounced.values()) { - for (Entry entry : eachMap.entrySet()) { - Assert.assertEquals(1, entry.getValue().get()); - } - } } - private static DataSegment makeSegment(int offset, boolean isTombstone) + private DataSegment makeSegment(int offset) { - Interval interval = new Interval( - DateTimes.of("2013-01-01").plusDays(offset), - DateTimes.of("2013-01-02").plusDays(offset) + return new DataSegment( + "test", + new Interval(DateTimes.utc(0), DateTimes.utc(offset + 1)), + "v1", + null, + ImmutableList.of("dim"), + ImmutableList.of("met"), + NoneShardSpec.instance(), + 0, + 1 ); - SegmentId segmentId = SegmentId.of("foo", interval, DateTimes.nowUtc().toString(), null); - DataSegment.Builder builder = DataSegment.builder(segmentId) - .loadSpec(ImmutableMap.of("type", "local")) - .dimensions(ImmutableList.of("dim1", "dim2")) - .metrics(ImmutableList.of("met1", "met2")) - .projections(ImmutableList.of("proj1", "proj2")) - .size(0); - if (isTombstone) { - builder.loadSpec(Collections.singletonMap("type", DataSegment.TOMBSTONE_LOADSPEC_TYPE)); - } - - return builder.build(); - } - - private static DataSegment makeSegment(int offset) - { - return makeSegment(offset, false); - } - - private static class SegmentReader - { - private final CuratorFramework cf; - private final ObjectMapper jsonMapper; - - public SegmentReader(CuratorFramework cf, ObjectMapper jsonMapper) - { - this.cf = cf; - this.jsonMapper = jsonMapper; - } - - public Set read(String path) - { - try { - if (cf.checkExists().forPath(path) != null) { - return jsonMapper.readValue( - cf.getData().forPath(path), new TypeReference<>() {} - ); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - - return new HashSet<>(); - } - } - - private static class TestAnnouncer extends NodeAnnouncer - { - private final ConcurrentHashMap> numPathAnnounced = new ConcurrentHashMap<>(); - - private TestAnnouncer(CuratorFramework curator, ExecutorService exec) - { - super(curator, exec); - } - - @Override - public void announce(String path, byte[] bytes, boolean removeParentIfCreated) - { - numPathAnnounced.computeIfAbsent(path, k -> new ConcurrentHashMap<>()) - .computeIfAbsent(bytes, k -> new AtomicInteger(0)) - .incrementAndGet(); - super.announce(path, bytes, removeParentIfCreated); - } } } diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index c622abc4fcc4..180ca60b6b32 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -22,7 +22,6 @@ import com.github.rvesse.airline.annotations.Command; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.name.Names; @@ -44,7 +43,6 @@ import org.apache.druid.client.selector.ServerSelectorStrategy; import org.apache.druid.client.selector.StrictTierSelectorStrategyConfig; import org.apache.druid.client.selector.TierSelectorStrategy; -import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.BrokerProcessingModule; import org.apache.druid.guice.BrokerServiceModule; @@ -81,7 +79,6 @@ import org.apache.druid.server.SubqueryGuardrailHelperProvider; import org.apache.druid.server.coordination.SegmentCacheBootstrapper; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.BrokerResource; import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; @@ -108,19 +105,11 @@ public class CliBroker extends ServerRunnable { private static final Logger log = new Logger(CliBroker.class); - private boolean isZkEnabled = true; - public CliBroker() { super(log); } - @Inject - public void configure(Properties properties) - { - isZkEnabled = ZkEnablementConfig.isEnabled(properties); - } - @Override protected Set getNodeRoles(Properties properties) { @@ -190,14 +179,10 @@ protected List getModules() binder.bind(SegmentManager.class).in(LazySingleton.class); binder.bind(BrokerViewOfCoordinatorConfig.class).in(ManageLifecycle.class); binder.bind(BrokerViewOfBrokerConfig.class).in(ManageLifecycle.class); - binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.BROKER)); Jerseys.addResource(binder, HistoricalResource.class); Jerseys.addResource(binder, SegmentListerResource.class); - if (isZkEnabled) { - LifecycleModule.register(binder, ZkCoordinator.class); - } LifecycleModule.register(binder, SegmentCacheBootstrapper.class); bindAnnouncer( diff --git a/services/src/main/java/org/apache/druid/cli/CliHistorical.java b/services/src/main/java/org/apache/druid/cli/CliHistorical.java index 3c48dbc35656..ae1145f860fd 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHistorical.java +++ b/services/src/main/java/org/apache/druid/cli/CliHistorical.java @@ -24,12 +24,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.name.Names; import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; @@ -39,7 +37,6 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; -import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.SegmentWranglerModule; @@ -58,7 +55,6 @@ import org.apache.druid.server.ServerManager; import org.apache.druid.server.coordination.SegmentCacheBootstrapper; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.http.SelfDiscoveryResource; @@ -80,19 +76,11 @@ public class CliHistorical extends ServerRunnable { private static final Logger log = new Logger(CliHistorical.class); - private boolean isZkEnabled = true; - public CliHistorical() { super(log); } - @Inject - public void configure(Properties properties) - { - isZkEnabled = ZkEnablementConfig.isEnabled(properties); - } - @Override protected Set getNodeRoles(Properties properties) { @@ -116,11 +104,9 @@ protected List getModules() binder.bindConstant().annotatedWith(PruneLastCompactionState.class).to(true); binder.bind(ResponseContextConfig.class).toInstance(ResponseContextConfig.newConfig(true)); - // register Server before binding ZkCoordinator to ensure HTTP endpoints are available immediately LifecycleModule.register(binder, Server.class); binder.bind(ServerManager.class).in(LazySingleton.class); binder.bind(SegmentManager.class).in(LazySingleton.class); - binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); bindQuerySegmentWalker(binder); binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.HISTORICAL)); @@ -131,9 +117,6 @@ protected List getModules() Jerseys.addResource(binder, HistoricalResource.class); LifecycleModule.register(binder, QueryResource.class); - if (isZkEnabled) { - LifecycleModule.register(binder, ZkCoordinator.class); - } LifecycleModule.register(binder, SegmentCacheBootstrapper.class); JsonConfigProvider.bind(binder, "druid.historical.cache", CacheConfig.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 9a06be1bfb28..acbeae18f6ad 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -45,7 +45,6 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; -import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; @@ -78,7 +77,6 @@ import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentCacheBootstrapper; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.http.SelfDiscoveryResource; @@ -182,12 +180,8 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); binder.bind(SegmentManager.class).in(LazySingleton.class); - binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); Jerseys.addResource(binder, HistoricalResource.class); - if (isZkEnabled) { - LifecycleModule.register(binder, ZkCoordinator.class); - } LifecycleModule.register(binder, SegmentCacheBootstrapper.class); bindAnnouncer( diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 8d1daee9c1e4..d234a6f37fec 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -43,7 +43,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.druid.client.cache.CacheConfig; -import org.apache.druid.curator.ZkEnablementConfig; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; @@ -124,7 +123,6 @@ import org.apache.druid.server.coordination.BroadcastDatasourceLoadingSpec; import org.apache.druid.server.coordination.SegmentCacheBootstrapper; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.initialization.jetty.ChatHandlerServerModule; @@ -172,8 +170,6 @@ public class CliPeon extends GuiceRunnable @Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK") public String serverType = "indexer-executor"; - private boolean isZkEnabled = true; - /** *

    This option is deprecated, see {@link #loadBroadcastDatasourcesMode} option.

    * @@ -211,7 +207,6 @@ public CliPeon() public void configure(Properties properties) { this.properties = properties; - isZkEnabled = ZkEnablementConfig.isEnabled(properties); } @Override @@ -561,12 +556,8 @@ public class BroadcastSegmentLoadingModule implements Module public void configure(Binder binder) { binder.bind(SegmentManager.class).in(LazySingleton.class); - binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); Jerseys.addResource(binder, HistoricalResource.class); - if (isZkEnabled) { - LifecycleModule.register(binder, ZkCoordinator.class); - } LifecycleModule.register(binder, SegmentCacheBootstrapper.class); } diff --git a/services/src/test/java/org/apache/druid/cli/CliBrokerTest.java b/services/src/test/java/org/apache/druid/cli/CliBrokerTest.java index ad7c14c3a579..0c614d667e27 100644 --- a/services/src/test/java/org/apache/druid/cli/CliBrokerTest.java +++ b/services/src/test/java/org/apache/druid/cli/CliBrokerTest.java @@ -443,7 +443,6 @@ private Injector makeBrokerInjector(final Properties props) ); final CliBroker broker = new CliBroker(); - broker.configure(props); broker.configure(props, baseInjector); return broker.makeInjector(Set.of(NodeRole.BROKER)); } From a9b924aa2218f6d88bbcd10d04c4308bdc502a4d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 27 Apr 2026 01:20:38 -0700 Subject: [PATCH 2/4] got a bit carried away --- .../druid/client/BrokerServerViewTest.java | 865 ++++++++++++++++++ .../client/CoordinatorServerViewTest.java | 472 ++++++++++ .../BatchDataSegmentAnnouncerTest.java | 198 +++- 3 files changed, 1522 insertions(+), 13 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java create mode 100644 server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java new file mode 100644 index 000000000000..e39aabf984c7 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java @@ -0,0 +1,865 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.fasterxml.jackson.dataformat.smile.SmileGenerator; +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.Lists; +import com.google.common.collect.Sets; +import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; +import org.apache.druid.client.selector.LowestPriorityTierSelectorStrategy; +import org.apache.druid.client.selector.RandomServerSelectorStrategy; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.query.CloneQueryMode; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryWatcher; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.TestCoordinatorClient; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.timeline.partition.SingleElementPartitionChunk; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class BrokerServerViewTest +{ + private static final long AWAIT_SECONDS = 10; + + private CountDownLatch segmentViewInitLatch; + private CountDownLatch serverAddedLatch; + private CountDownLatch segmentAddedLatch; + private CountDownLatch segmentRemovedLatch; + + private TestServerInventoryView baseView; + private BrokerServerView brokerServerView; + private final BrokerViewOfCoordinatorConfig brokerViewOfCoordinatorConfig; + + public BrokerServerViewTest() + { + brokerViewOfCoordinatorConfig = new BrokerViewOfCoordinatorConfig(new TestCoordinatorClient()); + } + + @Before + public void setUp() + { + brokerViewOfCoordinatorConfig.start(); + } + + @Test + public void testSingleServerAddedRemovedSegment() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(1); + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); + + final DruidServer druidServer = setupHistoricalServer("default_tier", "localhost:1234", 0); + final DataSegment segment = dataSegmentWithIntervalAndVersion("2014-10-20T00:00:00Z/P1D", "v1"); + final int partition = segment.getShardSpec().getPartitionNum(); + final Interval intervals = Intervals.of("2014-10-20T00:00:00Z/P1D"); + baseView.addSegment(druidServer, segment); + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource("test_broker_server_view") + ).get(); + List> serverLookupRes = timeline.lookup(intervals); + Assert.assertEquals(1, serverLookupRes.size()); + + TimelineObjectHolder actualTimelineObjectHolder = serverLookupRes.get(0); + Assert.assertEquals(intervals, actualTimelineObjectHolder.getInterval()); + Assert.assertEquals("v1", actualTimelineObjectHolder.getVersion()); + + PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); + Assert.assertTrue(actualPartitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); + + ServerSelector selector = (actualPartitionHolder.iterator().next()).getObject(); + Assert.assertFalse(selector.isEmpty()); + Assert.assertEquals(segment, selector.getSegment()); + Assert.assertEquals(druidServer, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + Assert.assertNotNull(timeline.findChunk(intervals, "v1", partition)); + + baseView.removeSegment(druidServer, segment); + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + + Assert.assertEquals( + 0, + timeline.lookup(intervals).size() + ); + Assert.assertNull(timeline.findChunk(intervals, "v1", partition)); + } + + @Test + public void testMultipleServerAddedRemovedSegment() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(5); + + // temporarily set latch count to 1 + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); + + final List druidServers = Lists.transform( + ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), + hostname -> setupHistoricalServer("default_tier", hostname, 0) + ); + + final List segments = Lists.transform( + ImmutableList.of( + Pair.of("2011-04-01/2011-04-03", "v1"), + Pair.of("2011-04-03/2011-04-06", "v1"), + Pair.of("2011-04-01/2011-04-09", "v2"), + Pair.of("2011-04-06/2011-04-09", "v3"), + Pair.of("2011-04-01/2011-04-02", "v3") + ), input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + ); + + for (int i = 0; i < 5; ++i) { + baseView.addSegment(druidServers.get(i), segments.get(i)); + } + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource("test_broker_server_view") + ).get(); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce the segment created by dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-09", "v2") + baseView.removeSegment(druidServers.get(2), segments.get(2)); + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + + // renew segmentRemovedLatch since we still have 4 segments to unannounce + segmentRemovedLatch = new CountDownLatch(4); + + timeline = brokerServerView.getTimeline( + new TableDataSource("test_broker_server_view") + ).get(); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-03", "v1", druidServers.get(0), segments.get(0)), + createExpected("2011-04-03/2011-04-06", "v1", druidServers.get(1), segments.get(1)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce all the segments + for (int i = 0; i < 5; ++i) { + // skip the one that was previously unannounced + if (i != 2) { + baseView.removeSegment(druidServers.get(i), segments.get(i)); + } + } + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + + Assert.assertEquals( + 0, + ((List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))).size() + ); + } + + @Test + public void testMultipleServerAndBroker() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + serverAddedLatch = new CountDownLatch(6); + segmentAddedLatch = new CountDownLatch(6); + + // temporarily set latch count to 1 + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); + + final DruidServer druidBroker = new DruidServer( + "localhost:5", + "localhost:5", + null, + 10000000L, + null, + ServerType.BROKER, + "default_tier", + 0 + ); + + // Materialize this list so all servers are set up + final List druidServers = + ImmutableList.copyOf( + Lists.transform( + ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), + hostname -> setupHistoricalServer("default_tier", hostname, 0) + ) + ); + + baseView.addServer(druidBroker); + + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(serverAddedLatch)); + + // check server metadatas + Assert.assertEquals( + druidServers.stream().map(DruidServer::getMetadata).collect(Collectors.toSet()), + ImmutableSet.copyOf(brokerServerView.getDruidServerMetadatas()) + ); + + final List segments = Lists.transform( + ImmutableList.of( + Pair.of("2011-04-01/2011-04-03", "v1"), + Pair.of("2011-04-03/2011-04-06", "v1"), + Pair.of("2011-04-01/2011-04-09", "v2"), + Pair.of("2011-04-06/2011-04-09", "v3"), + Pair.of("2011-04-01/2011-04-02", "v3") + ), + input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + ); + + DataSegment brokerSegment = dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-11", "v4"); + baseView.addSegment(druidBroker, brokerSegment); + for (int i = 0; i < 5; ++i) { + baseView.addSegment(druidServers.get(i), segments.get(i)); + } + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource("test_broker_server_view") + ).get(); + + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce the broker segment should do nothing to announcements + baseView.removeSegment(druidBroker, brokerSegment); + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + + // renew segmentRemovedLatch since we still have 5 segments to unannounce + segmentRemovedLatch = new CountDownLatch(5); + + timeline = brokerServerView.getTimeline( + new TableDataSource("test_broker_server_view") + ).get(); + + // expect same set of segments as before + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce all the segments + for (int i = 0; i < 5; ++i) { + baseView.removeSegment(druidServers.get(i), segments.get(i)); + } + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + } + + @Test + public void testMultipleTiers() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(4); + segmentRemovedLatch = new CountDownLatch(0); + + // Setup a Broker that watches only Tier 2 + final String tier1 = "tier1"; + final String tier2 = "tier2"; + setupViews(Sets.newHashSet(tier2), null, true); + + // Historical Tier 1 has segments 1 and 2, Tier 2 has segments 2 and 3 + final DruidServer server11 = setupHistoricalServer(tier1, "localhost:1", 1); + final DruidServer server21 = setupHistoricalServer(tier2, "localhost:2", 1); + + final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); + baseView.addSegment(server11, segment1); + + final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); + baseView.addSegment(server11, segment2); + baseView.addSegment(server21, segment2); + + final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); + baseView.addSegment(server21, segment3); + + // Wait for the segments to be added + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + // Get the timeline for the datasource + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource(segment1.getDataSource()) + ).get(); + + // Verify that the timeline has no entry for the interval of segment 1 + Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); + + // Verify that there is one entry for the interval of segment 2 + List> timelineHolders = + timeline.lookup(segment2.getInterval()); + Assert.assertEquals(1, timelineHolders.size()); + + TimelineObjectHolder timelineHolder = timelineHolders.get(0); + Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); + Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); + + PartitionHolder partitionHolder = timelineHolder.getObject(); + Assert.assertTrue(partitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(partitionHolder)); + + ServerSelector selector = (partitionHolder.iterator().next()).getObject(); + Assert.assertFalse(selector.isEmpty()); + Assert.assertEquals(segment2, selector.getSegment()); + + // Verify that the ServerSelector always picks Tier 1 + for (int i = 0; i < 5; ++i) { + Assert.assertEquals(server21, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + } + Assert.assertEquals(Collections.singletonList(server21.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); + } + + @Test + public void testRealtimeTasksNotWatched() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(4); + segmentRemovedLatch = new CountDownLatch(0); + + // Setup a Broker that watches only Historicals + setupViews(null, null, false); + + // Historical has segments 2 and 3, Realtime has segments 1 and 2 + final DruidServer realtimeServer = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime:1", 1); + final DruidServer historicalServer = setupHistoricalServer("tier1", "historical:2", 1); + + final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); + baseView.addSegment(realtimeServer, segment1); + + final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); + baseView.addSegment(realtimeServer, segment2); + baseView.addSegment(historicalServer, segment2); + + final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); + baseView.addSegment(historicalServer, segment3); + + // Wait for the segments to be added + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + // Get the timeline for the datasource + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource(segment1.getDataSource()) + ).get(); + + // Verify that the timeline has no entry for the interval of segment 1 + Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); + + // Verify that there is one entry for the interval of segment 2 + List> timelineHolders = + timeline.lookup(segment2.getInterval()); + Assert.assertEquals(1, timelineHolders.size()); + + TimelineObjectHolder timelineHolder = timelineHolders.get(0); + Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); + Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); + + PartitionHolder partitionHolder = timelineHolder.getObject(); + Assert.assertTrue(partitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(partitionHolder)); + + ServerSelector selector = (partitionHolder.iterator().next()).getObject(); + Assert.assertFalse(selector.isEmpty()); + Assert.assertEquals(segment2, selector.getSegment()); + + // Verify that the ServerSelector always picks the Historical server + for (int i = 0; i < 5; ++i) { + Assert.assertEquals(historicalServer, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + } + Assert.assertEquals(Collections.singletonList(historicalServer.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); + } + + @Test + public void testIgnoredTiers() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(4); + segmentRemovedLatch = new CountDownLatch(0); + + // Setup a Broker that does not watch Tier 1 + final String tier1 = "tier1"; + final String tier2 = "tier2"; + setupViews(null, Sets.newHashSet(tier1), false); + + // Historical Tier 1 has segments 1 and 2, Tier 2 has segments 2 and 3 + final DruidServer server11 = setupHistoricalServer(tier1, "localhost:1", 1); + final DruidServer server21 = setupHistoricalServer(tier2, "localhost:2", 1); + + final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); + baseView.addSegment(server11, segment1); + + final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); + baseView.addSegment(server11, segment2); + baseView.addSegment(server21, segment2); + + final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); + baseView.addSegment(server21, segment3); + + // Wait for the segments to be added + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + // Get the timeline for the datasource + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource(segment1.getDataSource()) + ).get(); + + // Verify that the timeline has no entry for the interval of segment 1 + Assert.assertTrue(timeline.lookup(segment1.getInterval()).isEmpty()); + + // Verify that there is one entry for the interval of segment 2 + List> timelineHolders = + timeline.lookup(segment2.getInterval()); + Assert.assertEquals(1, timelineHolders.size()); + + TimelineObjectHolder timelineHolder = timelineHolders.get(0); + Assert.assertEquals(segment2.getInterval(), timelineHolder.getInterval()); + Assert.assertEquals(segment2.getVersion(), timelineHolder.getVersion()); + + PartitionHolder partitionHolder = timelineHolder.getObject(); + Assert.assertTrue(partitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(partitionHolder)); + + ServerSelector selector = (partitionHolder.iterator().next()).getObject(); + Assert.assertFalse(selector.isEmpty()); + Assert.assertEquals(segment2, selector.getSegment()); + + // Verify that the ServerSelector always picks Tier 1 + for (int i = 0; i < 5; ++i) { + Assert.assertEquals(server21, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + } + Assert.assertEquals(Collections.singletonList(server21.getMetadata()), selector.getCandidates(2, CloneQueryMode.EXCLUDECLONES)); + } + + @Test(expected = ISE.class) + public void testInvalidWatchedTiersConfig() throws Exception + { + // Verify that specifying both ignoredTiers and watchedTiers fails startup + final String tier1 = "tier1"; + final String tier2 = "tier2"; + setupViews(Sets.newHashSet(tier2), Sets.newHashSet(tier1), true); + } + + @Test(expected = ISE.class) + public void testEmptyWatchedTiersConfig() throws Exception + { + setupViews(Collections.emptySet(), null, true); + } + + @Test(expected = ISE.class) + public void testEmptyIgnoredTiersConfig() throws Exception + { + setupViews(null, Collections.emptySet(), true); + } + + @Test + public void testDifferentTierStrategiesForHistoricalAndRealtimeServers() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(7); + segmentRemovedLatch = new CountDownLatch(0); + + // Setup a Broker with LowestPriority strategy for historicals and HighestPriority for realtime + setupViews( + new LowestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), + new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()) + ); + + // Setup multiple historicals with different priorities and realtime servers + final DruidServer historicalLowPriority = setupHistoricalServer("tier1", "historical-low:1", 0); + final DruidServer historicalHighPriority = setupHistoricalServer("tier1", "historical-high:2", 10); + final DruidServer realtimeLowPriority = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime-low:3", 0); + final DruidServer realtimeHighPriority = setupDruidServer(ServerType.INDEXER_EXECUTOR, null, "realtime-high:4", 10); + + // Segment 1: only on historicals with different priorities + final DataSegment segment1 = dataSegmentWithIntervalAndVersion("2020-01-01/P1D", "v1"); + baseView.addSegment(historicalLowPriority, segment1); + baseView.addSegment(historicalHighPriority, segment1); + + // Segment 2: only on realtime with different priorities + final DataSegment segment2 = dataSegmentWithIntervalAndVersion("2020-01-02/P1D", "v1"); + baseView.addSegment(realtimeLowPriority, segment2); + baseView.addSegment(realtimeHighPriority, segment2); + + // Segment 3: on both historical and realtime, but pick should prefer historical + final DataSegment segment3 = dataSegmentWithIntervalAndVersion("2020-01-03/P1D", "v1"); + baseView.addSegment(historicalHighPriority, segment3); + baseView.addSegment(realtimeLowPriority, segment3); + baseView.addSegment(realtimeHighPriority, segment3); + + // Wait for the segments to be added + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + // Get the timeline for the datasource + TimelineLookup timeline = brokerServerView.getTimeline( + new TableDataSource(segment1.getDataSource()) + ).get(); + + // Test segment 1: should pick the lowest priority historical (priority 0) + List> holders1 = timeline.lookup(segment1.getInterval()); + Assert.assertEquals(1, holders1.size()); + ServerSelector selector1 = holders1.get(0).getObject().iterator().next().getObject(); + Assert.assertEquals(segment1, selector1.getSegment()); + + // Historical LowestPriorityTierSelectorStrategy should pick the historical servers low and high in order + Assert.assertEquals(historicalLowPriority, selector1.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + Assert.assertEquals(List.of(historicalLowPriority.getMetadata()), selector1.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); + Assert.assertEquals(List.of(historicalLowPriority.getMetadata(), historicalHighPriority.getMetadata()), selector1.getAllServers(CloneQueryMode.EXCLUDECLONES)); + + // Test segment 2: should pick the highest priority realtime (priority 10) + List> holders2 = timeline.lookup(segment2.getInterval()); + Assert.assertEquals(1, holders2.size()); + ServerSelector selector2 = holders2.get(0).getObject().iterator().next().getObject(); + Assert.assertEquals(segment2, selector2.getSegment()); + + // Realtime HighestPriorityTierSelectorStrategy for realtime should pick the realtime servers high and low in order + Assert.assertEquals(realtimeHighPriority, selector2.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + Assert.assertEquals(List.of(realtimeHighPriority.getMetadata()), selector2.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); + Assert.assertEquals(List.of(realtimeHighPriority.getMetadata(), realtimeLowPriority.getMetadata()), selector2.getAllServers(CloneQueryMode.EXCLUDECLONES)); + + // Test segment 3: when both historical and realtime exist, historical is preferred + // and should pick based on historical strategy (lowest priority = 10) + List> holders3 = timeline.lookup(segment3.getInterval()); + Assert.assertEquals(1, holders3.size()); + ServerSelector selector3 = holders3.get(0).getObject().iterator().next().getObject(); + Assert.assertEquals(segment3, selector3.getSegment()); + + // Should prefer historical over realtime servers and in order + Assert.assertEquals(historicalHighPriority, selector3.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + Assert.assertEquals(List.of(historicalHighPriority.getMetadata()), selector3.getCandidates(1, CloneQueryMode.EXCLUDECLONES)); + Assert.assertEquals( + List.of(historicalHighPriority.getMetadata(), realtimeHighPriority.getMetadata(), realtimeLowPriority.getMetadata()), + selector3.getAllServers(CloneQueryMode.EXCLUDECLONES) + ); + } + + /** + * Creates a DruidServer of type HISTORICAL and adds it to the test inventory view. + */ + private DruidServer setupHistoricalServer(String tier, String name, int priority) + { + return setupDruidServer(ServerType.HISTORICAL, tier, name, priority); + } + + /** + * Creates a DruidServer of the specified type and adds it to the test inventory view. + */ + private DruidServer setupDruidServer(ServerType serverType, String tier, String name, int priority) + { + final DruidServer druidServer = new DruidServer( + name, + name, + null, + 1000000, + null, + serverType, + tier, + priority + ); + baseView.addServer(druidServer); + return druidServer; + } + + private boolean awaitLatch(CountDownLatch latch) throws InterruptedException + { + return latch.await(AWAIT_SECONDS, TimeUnit.SECONDS); + } + + private Pair>> createExpected( + String intervalStr, + String version, + DruidServer druidServer, + DataSegment segment + ) + { + return Pair.of(Intervals.of(intervalStr), Pair.of(version, Pair.of(druidServer, segment))); + } + + private void assertValues( + List>>> expected, List actual + ) + { + Assert.assertEquals(expected.size(), actual.size()); + + for (int i = 0; i < expected.size(); ++i) { + Pair>> expectedPair = expected.get(i); + TimelineObjectHolder actualTimelineObjectHolder = actual.get(i); + + Assert.assertEquals(expectedPair.lhs, actualTimelineObjectHolder.getInterval()); + Assert.assertEquals(expectedPair.rhs.lhs, actualTimelineObjectHolder.getVersion()); + + PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); + Assert.assertTrue(actualPartitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); + + ServerSelector selector = ((SingleElementPartitionChunk) actualPartitionHolder.iterator() + .next()).getObject(); + Assert.assertFalse(selector.isEmpty()); + Assert.assertEquals(expectedPair.rhs.rhs.lhs, selector.pick(null, CloneQueryMode.EXCLUDECLONES).getServer()); + Assert.assertEquals(expectedPair.rhs.rhs.rhs, selector.getSegment()); + } + } + + private void setupViews() throws InterruptedException + { + setupViews(null, null, true); + } + + private void setupViews(TierSelectorStrategy historicalStrategy, TierSelectorStrategy realtimeStrategy) throws InterruptedException + { + setupViews(historicalStrategy, realtimeStrategy, new BrokerSegmentWatcherConfig()); + } + + private void setupViews(Set watchedTiers, Set ignoredTiers, boolean watchRealtimeTasks) throws InterruptedException + { + setupViews( + new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), + new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy()), + new BrokerSegmentWatcherConfig() + { + @Override + public Set getWatchedTiers() + { + return watchedTiers; + } + + @Override + public boolean isWatchRealtimeTasks() + { + return watchRealtimeTasks; + } + + @Override + public Set getIgnoredTiers() + { + return ignoredTiers; + } + } + ); + } + + private void setupViews(TierSelectorStrategy historicalStrategy, TierSelectorStrategy realtimeStrategy, BrokerSegmentWatcherConfig brokerSegmentWatcherConfig) + throws InterruptedException + { + baseView = new TestServerInventoryView() + { + @Override + public void registerServerCallback(Executor exec, ServerCallback callback) + { + super.registerServerCallback( + exec, + new ServerCallback() { + @Override + public CallbackAction serverAdded(DruidServer server) + { + final CallbackAction res = callback.serverAdded(server); + if (serverAddedLatch != null) { + serverAddedLatch.countDown(); + } + return res; + } + + @Override + public CallbackAction serverRemoved(DruidServer server) + { + return callback.serverRemoved(server); + } + } + ); + } + + @Override + public void registerSegmentCallback(Executor exec, final SegmentCallback callback) + { + super.registerSegmentCallback( + exec, + new SegmentCallback() + { + @Override + public CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) + { + CallbackAction res = callback.segmentAdded(server, segment); + segmentAddedLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) + { + CallbackAction res = callback.segmentRemoved(server, segment); + segmentRemovedLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentViewInitialized() + { + CallbackAction res = callback.segmentViewInitialized(); + segmentViewInitLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } + } + ); + } + }; + + DirectDruidClientFactory druidClientFactory = new DirectDruidClientFactory( + new NoopServiceEmitter(), + EasyMock.createMock(QueryRunnerFactoryConglomerate.class), + EasyMock.createMock(QueryWatcher.class), + getSmileMapper(), + EasyMock.createMock(HttpClient.class) + ); + + brokerServerView = new BrokerServerView( + druidClientFactory, + baseView, + historicalStrategy, + realtimeStrategy, + new NoopServiceEmitter(), + brokerSegmentWatcherConfig, + brokerViewOfCoordinatorConfig + ); + + baseView.start(); + baseView.markInventoryInitialized(); + brokerServerView.start(); + } + + private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) + { + return DataSegment.builder() + .dataSource("test_broker_server_view") + .interval(Intervals.of(intervalStr)) + .loadSpec( + ImmutableMap.of( + "type", + "local", + "path", + "somewhere" + ) + ) + .version(version) + .dimensions(ImmutableList.of()) + .metrics(ImmutableList.of()) + .shardSpec(NoneShardSpec.instance()) + .binaryVersion(9) + .size(0) + .build(); + } + + public ObjectMapper getSmileMapper() + { + final SmileFactory smileFactory = new SmileFactory(); + smileFactory.configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false); + smileFactory.delegateToTextual(true); + final ObjectMapper retVal = new DefaultObjectMapper(smileFactory, "broker"); + retVal.getFactory().setCodec(retVal); + return retVal; + } + + @After + public void tearDown() + { + if (baseView != null) { + baseView.stop(); + } + } +} diff --git a/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java new file mode 100644 index 000000000000..7290eacfac5e --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/CoordinatorServerViewTest.java @@ -0,0 +1,472 @@ +/* + * 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.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.TimelineLookup; +import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.PartitionHolder; +import org.easymock.EasyMock; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +@RunWith(Parameterized.class) +public class CoordinatorServerViewTest +{ + private static final long AWAIT_SECONDS = 10; + + private CountDownLatch segmentViewInitLatch; + private CountDownLatch segmentAddedLatch; + private CountDownLatch segmentRemovedLatch; + + private CountDownLatch callbackSegmentViewInitLatch; + private CountDownLatch callbackSegmentAddedLatch; + private CountDownLatch callbackSegmentRemovedLatch; + private CountDownLatch callbackServerSegmentRemovedLatch; + + private TestServerInventoryView baseView; + private CoordinatorServerView coordinatorServerView; + private ExecutorService callbackExec; + + private boolean setDruidClientFactory; + + @Parameterized.Parameters + public static Object[] data() + { + return new Object[]{true, false}; + } + + public CoordinatorServerViewTest(boolean setDruidClientFactory) + { + this.setDruidClientFactory = setDruidClientFactory; + } + + @Before + public void setUp() + { + callbackExec = Execs.singleThreaded("CoordinatorServerViewTest-%s"); + } + + @Test + public void testSingleServerAddedRemovedSegment() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(1); + segmentRemovedLatch = new CountDownLatch(1); + callbackSegmentViewInitLatch = new CountDownLatch(1); + callbackSegmentAddedLatch = new CountDownLatch(1); + callbackServerSegmentRemovedLatch = new CountDownLatch(1); + callbackSegmentRemovedLatch = new CountDownLatch(1); + + setupViews(setDruidClientFactory); + + final DruidServer druidServer = new DruidServer( + "localhost:1234", + "localhost:1234", + null, + 10000000L, + null, + ServerType.HISTORICAL, + "default_tier", + 0 + ); + + baseView.addServer(druidServer); + baseView.markInventoryInitialized(); + + final DataSegment segment = dataSegmentWithIntervalAndVersion("2014-10-20T00:00:00Z/P1D", "v1"); + final int partition = segment.getShardSpec().getPartitionNum(); + final Interval intervals = Intervals.of("2014-10-20T00:00:00Z/P1D"); + baseView.addSegment(druidServer, segment); + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + + Assert.assertTrue(awaitLatch(callbackSegmentViewInitLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentAddedLatch)); + + if (setDruidClientFactory) { + Assert.assertNotNull(coordinatorServerView.getQueryRunner(druidServer.getName())); + } else { + Assert.assertNull(coordinatorServerView.getQueryRunner(druidServer.getName())); + } + + TimelineLookup timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + List serverLookupRes = (List) timeline.lookup( + intervals + ); + Assert.assertEquals(1, serverLookupRes.size()); + + TimelineObjectHolder actualTimelineObjectHolder = serverLookupRes.get(0); + Assert.assertEquals(intervals, actualTimelineObjectHolder.getInterval()); + Assert.assertEquals("v1", actualTimelineObjectHolder.getVersion()); + + PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); + Assert.assertTrue(actualPartitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); + + SegmentLoadInfo segmentLoadInfo = actualPartitionHolder.iterator().next().getObject(); + Assert.assertFalse(segmentLoadInfo.isEmpty()); + Assert.assertEquals( + druidServer.getMetadata(), + Iterables.getOnlyElement(segmentLoadInfo.toImmutableSegmentLoadInfo().getServers()) + ); + Assert.assertNotNull(timeline.findChunk(intervals, "v1", partition)); + + baseView.removeSegment(druidServer, segment); + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackServerSegmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentRemovedLatch)); + + Assert.assertEquals( + 0, + ((List) timeline.lookup(Intervals.of("2014-10-20T00:00:00Z/P1D"))).size() + ); + Assert.assertNull(timeline.findChunk(intervals, "v1", partition)); + } + + @Test + public void testMultipleServerAddedRemovedSegment() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(5); + + // temporarily set latch count to 1 + segmentRemovedLatch = new CountDownLatch(1); + + callbackSegmentViewInitLatch = new CountDownLatch(1); + callbackSegmentAddedLatch = new CountDownLatch(5); + callbackServerSegmentRemovedLatch = new CountDownLatch(1); + callbackSegmentRemovedLatch = new CountDownLatch(1); + + setupViews(setDruidClientFactory); + + final List druidServers = Lists.transform( + ImmutableList.of("localhost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), + input -> new DruidServer( + input, + input, + null, + 10000000L, + null, + ServerType.HISTORICAL, + "default_tier", + 0 + ) + ); + + for (DruidServer druidServer : druidServers) { + baseView.addServer(druidServer); + } + baseView.markInventoryInitialized(); + + final List segments = Lists.transform( + ImmutableList.of( + Pair.of("2011-04-01/2011-04-03", "v1"), + Pair.of("2011-04-03/2011-04-06", "v1"), + Pair.of("2011-04-01/2011-04-09", "v2"), + Pair.of("2011-04-06/2011-04-09", "v3"), + Pair.of("2011-04-01/2011-04-02", "v3") + ), + input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + ); + + for (int i = 0; i < 5; ++i) { + baseView.addSegment(druidServers.get(i), segments.get(i)); + } + Assert.assertTrue(awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(awaitLatch(segmentAddedLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentViewInitLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentAddedLatch)); + + for (int i = 0; i < 5; ++i) { + if (setDruidClientFactory) { + Assert.assertNotNull(coordinatorServerView.getQueryRunner(druidServers.get(i).getName())); + } else { + Assert.assertNull(coordinatorServerView.getQueryRunner(druidServers.get(i).getName())); + } + } + + TimelineLookup timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + (List) timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce the segment created by dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-09", "v2") + baseView.removeSegment(druidServers.get(2), segments.get(2)); + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackServerSegmentRemovedLatch)); + + // renew segmentRemovedLatch since we still have 4 segments to unannounce + segmentRemovedLatch = new CountDownLatch(4); + callbackServerSegmentRemovedLatch = new CountDownLatch(4); + callbackSegmentRemovedLatch = new CountDownLatch(4); + + timeline = coordinatorServerView.getTimeline(new TableDataSource("test_overlord_server_view")); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-03", "v1", druidServers.get(0), segments.get(0)), + createExpected("2011-04-03/2011-04-06", "v1", druidServers.get(1), segments.get(1)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + (List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) + ); + + // unannounce all the segments + for (int i = 0; i < 5; ++i) { + // skip the one that was previously unannounced + if (i != 2) { + baseView.removeSegment(druidServers.get(i), segments.get(i)); + } + } + Assert.assertTrue(awaitLatch(segmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackSegmentRemovedLatch)); + Assert.assertTrue(awaitLatch(callbackServerSegmentRemovedLatch)); + + Assert.assertEquals( + 0, + ((List) timeline.lookup(Intervals.of("2011-04-01/2011-04-09"))).size() + ); + } + + private boolean awaitLatch(CountDownLatch latch) throws InterruptedException + { + return latch.await(AWAIT_SECONDS, TimeUnit.SECONDS); + } + + private Pair>> createExpected( + String intervalStr, + String version, + DruidServer druidServer, + DataSegment segment + ) + { + return Pair.of(Intervals.of(intervalStr), Pair.of(version, Pair.of(druidServer, segment))); + } + + private void assertValues( + List>>> expected, List actual + ) + { + Assert.assertEquals(expected.size(), actual.size()); + + for (int i = 0; i < expected.size(); ++i) { + Pair>> expectedPair = expected.get(i); + TimelineObjectHolder actualTimelineObjectHolder = actual.get(i); + + Assert.assertEquals(expectedPair.lhs, actualTimelineObjectHolder.getInterval()); + Assert.assertEquals(expectedPair.rhs.lhs, actualTimelineObjectHolder.getVersion()); + + PartitionHolder actualPartitionHolder = actualTimelineObjectHolder.getObject(); + Assert.assertTrue(actualPartitionHolder.isComplete()); + Assert.assertEquals(1, Iterables.size(actualPartitionHolder)); + + SegmentLoadInfo segmentLoadInfo = actualPartitionHolder.iterator().next().getObject(); + Assert.assertFalse(segmentLoadInfo.isEmpty()); + Assert.assertEquals(expectedPair.rhs.rhs.lhs.getMetadata(), + Iterables.getOnlyElement(segmentLoadInfo.toImmutableSegmentLoadInfo().getServers())); + Assert.assertEquals(expectedPair.rhs.rhs.lhs.getMetadata(), segmentLoadInfo.pickOne()); + } + } + + private void setupViews(boolean setDruidClientFactory) throws InterruptedException + { + baseView = new TestServerInventoryView() + { + @Override + public void registerSegmentCallback(Executor exec, final SegmentCallback callback) + { + super.registerSegmentCallback( + exec, + new SegmentCallback() + { + @Override + public CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) + { + CallbackAction res = callback.segmentAdded(server, segment); + segmentAddedLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) + { + CallbackAction res = callback.segmentRemoved(server, segment); + segmentRemovedLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentViewInitialized() + { + CallbackAction res = callback.segmentViewInitialized(); + segmentViewInitLatch.countDown(); + return res; + } + + @Override + public CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return CallbackAction.CONTINUE; + } + } + ); + } + }; + + DirectDruidClientFactory druidClientFactory = null; + + if (setDruidClientFactory) { + druidClientFactory = EasyMock.createMock(DirectDruidClientFactory.class); + DirectDruidClient directDruidClient = EasyMock.mock(DirectDruidClient.class); + EasyMock.expect(druidClientFactory.makeDirectClient(EasyMock.anyObject(DruidServer.class))) + .andReturn(directDruidClient) + .anyTimes(); + + EasyMock.replay(druidClientFactory); + } + + coordinatorServerView = new CoordinatorServerView( + baseView, + new CoordinatorSegmentWatcherConfig(), + new NoopServiceEmitter(), + druidClientFactory + ); + + baseView.start(); + baseView.markInventoryInitialized(); + initServerViewTimelineCallback(coordinatorServerView); + coordinatorServerView.start(); + } + + private void initServerViewTimelineCallback(final CoordinatorServerView serverView) + { + serverView.registerTimelineCallback( + callbackExec, + new TimelineServerView.TimelineCallback() + { + @Override + public ServerView.CallbackAction timelineInitialized() + { + callbackSegmentViewInitLatch.countDown(); + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction segmentAdded(final DruidServerMetadata server, final DataSegment segment) + { + callbackSegmentAddedLatch.countDown(); + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction segmentRemoved(final DataSegment segment) + { + callbackSegmentRemovedLatch.countDown(); + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction serverSegmentRemoved( + final DruidServerMetadata server, + final DataSegment segment + ) + { + callbackServerSegmentRemovedLatch.countDown(); + return ServerView.CallbackAction.CONTINUE; + } + + @Override + public ServerView.CallbackAction segmentSchemasAnnounced(SegmentSchemas segmentSchemas) + { + return ServerView.CallbackAction.CONTINUE; + } + } + ); + } + + private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String version) + { + return DataSegment.builder() + .dataSource("test_overlord_server_view") + .interval(Intervals.of(intervalStr)) + .loadSpec( + ImmutableMap.of( + "type", + "local", + "path", + "somewhere" + ) + ) + .version(version) + .dimensions(ImmutableList.of()) + .metrics(ImmutableList.of()) + .shardSpec(NoneShardSpec.instance()) + .binaryVersion(9) + .size(0) + .build(); + } + + @After + public void tearDown() + { + if (baseView != null) { + baseView.stop(); + } + if (callbackExec != null) { + callbackExec.shutdownNow(); + } + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 21a51407162f..8485c075ac61 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -20,30 +20,60 @@ package org.apache.druid.server.coordination.coordination; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.realtime.appenderator.SegmentSchemas; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.SegmentChangeRequestDrop; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; +import org.apache.druid.server.coordination.SegmentSchemasChangeRequest; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; public class BatchDataSegmentAnnouncerTest { + private static final int NUM_THREADS = 4; + private BatchDataSegmentAnnouncer announcer; + private Set testSegments; + private ExecutorService exec; @Before public void setUp() { announcer = new BatchDataSegmentAnnouncer(); + testSegments = new HashSet<>(); + for (int i = 0; i < 100; i++) { + testSegments.add(makeSegment(i)); + } + exec = Execs.multiThreaded(NUM_THREADS, "BatchDataSegmentAnnouncerTest-%d"); + } + + @After + public void tearDown() + { + announcer.stop(); + exec.shutdownNow(); } @Test @@ -56,10 +86,10 @@ public void testAnnounceAndUnannounceProduceChangeHistory() throws Exception announcer.announceSegment(segmentB); announcer.unannounceSegment(segmentA); + // When the counter is negative, getSegmentChangesSince returns the current snapshot of + // announced segments, which at this point is just segmentB. ChangeRequestsSnapshot snapshot = announcer.getSegmentChangesSince(new ChangeRequestHistory.Counter(-1, -1)).get(); - // When counter is negative, getSegmentChangesSince returns the current snapshot of announced segments, which - // at this point is just segmentB. Assert.assertEquals(1, snapshot.getRequests().size()); Assert.assertTrue(snapshot.getRequests().get(0) instanceof SegmentChangeRequestLoad); } @@ -107,18 +137,160 @@ public void testDuplicateAnnouncementIsIgnored() throws Exception Assert.assertEquals(1, snapshot.getRequests().size()); } - private DataSegment makeSegment(int offset) + @Test + public void testSchemaAnnounce() throws Exception + { + String dataSource = "foo"; + String segmentId = "id"; + String taskId = "t1"; + SegmentSchemas.SegmentSchema absoluteSchema1 = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + false, + 20, + ImmutableList.of("dim1", "dim2"), + Collections.emptyList(), + ImmutableMap.of("dim1", ColumnType.STRING, "dim2", ColumnType.STRING) + ); + + + SegmentSchemas.SegmentSchema absoluteSchema2 = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + false, + 40, + ImmutableList.of("dim1", "dim2", "dim3"), + ImmutableList.of(), + ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim2", ColumnType.STRING, "dim3", ColumnType.STRING) + ); + + SegmentSchemas.SegmentSchema deltaSchema = + new SegmentSchemas.SegmentSchema( + dataSource, + segmentId, + true, + 40, + ImmutableList.of("dim3"), + ImmutableList.of("dim1"), + ImmutableMap.of("dim1", ColumnType.UNKNOWN_COMPLEX, "dim3", ColumnType.STRING) + ); + + announcer.announceSegmentSchemas( + taskId, + new SegmentSchemas(Collections.singletonList(absoluteSchema1)), + new SegmentSchemas(Collections.singletonList(absoluteSchema1)) + ); + + ChangeRequestsSnapshot snapshot; + + snapshot = announcer.getSegmentChangesSince( + new ChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(1, snapshot.getCounter().getCounter()); + + Assert.assertEquals( + absoluteSchema1, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + announcer.announceSegmentSchemas( + taskId, + new SegmentSchemas(Collections.singletonList(absoluteSchema2)), + new SegmentSchemas(Collections.singletonList(deltaSchema)) + ); + + snapshot = announcer.getSegmentChangesSince(snapshot.getCounter()).get(); + + Assert.assertEquals( + deltaSchema, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + + snapshot = announcer.getSegmentChangesSince( + new ChangeRequestHistory.Counter(-1, -1) + ).get(); + Assert.assertEquals( + absoluteSchema2, + ((SegmentSchemasChangeRequest) snapshot.getRequests().get(0)) + .getSegmentSchemas() + .getSegmentSchemaList() + .get(0) + ); + Assert.assertEquals(1, snapshot.getRequests().size()); + Assert.assertEquals(2, snapshot.getCounter().getCounter()); + } + + @Test(timeout = 5000L) + public void testAnnounceSegmentsWithSameSegmentConcurrently() throws ExecutionException, InterruptedException + { + final List> futures = new ArrayList<>(NUM_THREADS); + + for (int i = 0; i < NUM_THREADS; i++) { + futures.add(exec.submit(() -> announcer.announceSegments(testSegments))); + } + + for (Future future : futures) { + future.get(); + } + + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(ChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(testSegments.size(), snapshot.getRequests().size()); + } + + @Test(timeout = 5000L) + public void testAnnounceSegmentWithSameSegmentConcurrently() throws ExecutionException, InterruptedException + { + final List> futures = new ArrayList<>(NUM_THREADS); + + final DataSegment segment1 = makeSegment(0); + final DataSegment segment2 = makeSegment(1); + final DataSegment segment3 = makeSegment(2); + final DataSegment segment4 = makeSegment(3); + + for (int i = 0; i < NUM_THREADS; i++) { + futures.add( + exec.submit(() -> { + announcer.announceSegment(segment1); + announcer.announceSegment(segment2); + announcer.announceSegment(segment3); + announcer.announceSegment(segment4); + }) + ); + } + + for (Future future : futures) { + future.get(); + } + + ChangeRequestsSnapshot snapshot = + announcer.getSegmentChangesSince(ChangeRequestHistory.Counter.ZERO).get(); + Assert.assertEquals(4, snapshot.getRequests().size()); + } + + private static DataSegment makeSegment(int offset) { - return new DataSegment( - "test", - new Interval(DateTimes.utc(0), DateTimes.utc(offset + 1)), - "v1", - null, - ImmutableList.of("dim"), - ImmutableList.of("met"), - NoneShardSpec.instance(), - 0, - 1 + Interval interval = new Interval( + DateTimes.of("2013-01-01").plusDays(offset), + DateTimes.of("2013-01-02").plusDays(offset) ); + SegmentId segmentId = SegmentId.of("foo", interval, DateTimes.nowUtc().toString(), null); + return DataSegment.builder(segmentId) + .loadSpec(ImmutableMap.of("type", "local")) + .dimensions(ImmutableList.of("dim1", "dim2")) + .metrics(ImmutableList.of("met1", "met2")) + .shardSpec(NoneShardSpec.instance()) + .size(0) + .build(); } } From ab33b8b67a664d1933a60cacba53995395dfe728 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 27 Apr 2026 02:56:26 -0700 Subject: [PATCH 3/4] remove more stuff, add missing file --- docs/api-reference/tasks-api.md | 1 - docs/configuration/logging.md | 35 -- .../druid/guice/StartupInjectorBuilder.java | 24 + .../guice/StartupInjectorBuilderTest.java | 27 ++ .../CuratorDruidNodeDiscoveryProvider.java | 1 - .../inventory/CuratorInventoryManager.java | 457 ------------------ .../CuratorInventoryManagerStrategy.java | 41 -- .../inventory/InventoryManagerConfig.java | 53 -- .../apache/druid/guice/ServerViewModule.java | 21 - .../druid/client/TestServerInventoryView.java | 157 ++++++ .../CuratorInventoryManagerTest.java | 274 ----------- .../java/org/apache/druid/cli/MainTest.java | 66 +-- 12 files changed, 225 insertions(+), 932 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java delete mode 100644 server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManagerStrategy.java delete mode 100644 server/src/main/java/org/apache/druid/curator/inventory/InventoryManagerConfig.java create mode 100644 server/src/test/java/org/apache/druid/client/TestServerInventoryView.java delete mode 100644 server/src/test/java/org/apache/druid/curator/inventory/CuratorInventoryManagerTest.java diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index f53037f84e1d..6eaa00d7f463 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -1079,7 +1079,6 @@ Host: http://ROUTER_IP:ROUTER_PORT 2023-07-03T22:11:18,009 INFO [main] org.eclipse.jetty.server.session - node0 Stopped scavenging 2023-07-03T22:11:18,012 INFO [main] org.eclipse.jetty.server.handler.ContextHandler - Stopped o.e.j.s.ServletContextHandler@742aa00a{/,null,STOPPED} 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [NORMAL] - 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.ZkCoordinator - Stopping ZkCoordinator for [DruidServerMetadata{name='localhost:8100', hostAndPort='localhost:8100', hostAndTlsPort='null', maxSize=0, tier='_default_tier', type=indexer-executor, priority=0}] 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.SegmentLoadDropHandler - Stopping... 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.SegmentLoadDropHandler - Stopped. 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.indexing.overlord.SingleTaskBackgroundRunner - Starting graceful shutdown of task[index_kafka_social_media_0e905aa31037879_nommnaeg]. diff --git a/docs/configuration/logging.md b/docs/configuration/logging.md index dcfa9a9267bc..76749c8f6dad 100644 --- a/docs/configuration/logging.md +++ b/docs/configuration/logging.md @@ -133,38 +133,3 @@ Java runtime itself. This file is not rotated, but it is generally small due to the low volume of messages. If necessary, you can truncate it using the Linux command `truncate --size 0 log/historical.stdout.log`. -## Set the logs to asynchronously write - -If your logs are really chatty, you can set them to write asynchronously. -The following example shows a `log4j2.xml` that configures some of the more chatty classes to write asynchronously: - -``` - - - - - - - - - - - - - - - - - - - - - - - - - - - - -``` diff --git a/processing/src/main/java/org/apache/druid/guice/StartupInjectorBuilder.java b/processing/src/main/java/org/apache/druid/guice/StartupInjectorBuilder.java index b98f9e251eb1..954610c4e287 100644 --- a/processing/src/main/java/org/apache/druid/guice/StartupInjectorBuilder.java +++ b/processing/src/main/java/org/apache/druid/guice/StartupInjectorBuilder.java @@ -53,6 +53,8 @@ public class StartupInjectorBuilder extends BaseInjectorBuilder - * The logic of the InventoryManager just maintains a local cache of the containers and inventory it sees on ZK. It - * provides methods for getting at the container objects, which house the actual individual pieces of inventory. - *

    - * A Strategy is provided to the constructor of an Inventory manager, this strategy provides all of the - * object-specific logic to serialize, deserialize, compose and alter the container and inventory objects. - */ -@Deprecated -public class CuratorInventoryManager -{ - private static final Logger log = new Logger(CuratorInventoryManager.class); - - private final Object lock = new Object(); - - private final CuratorFramework curatorFramework; - private final InventoryManagerConfig config; - private final CuratorInventoryManagerStrategy strategy; - - private final ConcurrentMap containers; - private final Set uninitializedInventory; - private final PathChildrenCacheFactory cacheFactory; - private final ExecutorService pathChildrenCacheExecutor; - - private volatile @Nullable PathChildrenCache childrenCache; - - public CuratorInventoryManager( - CuratorFramework curatorFramework, - InventoryManagerConfig config, - ExecutorService exec, - CuratorInventoryManagerStrategy strategy - ) - { - this.curatorFramework = curatorFramework; - this.config = config; - this.strategy = strategy; - - this.containers = new ConcurrentHashMap<>(); - this.uninitializedInventory = Sets.newConcurrentHashSet(); - - this.pathChildrenCacheExecutor = exec; - this.cacheFactory = new PathChildrenCacheFactory.Builder() - //NOTE: cacheData is temporarily set to false and we get data directly from ZK on each event. - //this is a workaround to solve curator's out-of-order events problem - //https://issues.apache.org/jira/browse/CURATOR-191 - // This is also done in CuratorDruidNodeDiscoveryProvider. - .withCacheData(false) - .withCompressed(true) - .withExecutorService(pathChildrenCacheExecutor) - .withShutdownExecutorOnClose(false) - .build(); - } - - public void start() throws Exception - { - PathChildrenCache childrenCache; - synchronized (lock) { - childrenCache = this.childrenCache; - if (childrenCache != null) { - return; - } - - this.childrenCache = childrenCache = cacheFactory.make(curatorFramework, config.getContainerPath()); - } - - childrenCache.getListenable().addListener(new ContainerCacheListener()); - - try { - childrenCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); - } - catch (Exception e) { - synchronized (lock) { - try { - stop(); - } - catch (IOException e1) { - log.error(e1, "Exception when stopping InventoryManager that couldn't start."); - } - } - throw e; - } - } - - public void stop() throws IOException - { - synchronized (lock) { - PathChildrenCache childrenCache = this.childrenCache; - if (childrenCache == null) { - return; - } - - // This close() call actually calls shutdownNow() on the executor registered with the Cache object... - childrenCache.close(); - this.childrenCache = null; - } - - Closer closer = Closer.create(); - for (ContainerHolder containerHolder : containers.values()) { - closer.register(containerHolder.getCache()); - } - try { - closer.close(); - } - finally { - pathChildrenCacheExecutor.shutdown(); - } - } - - public InventoryManagerConfig getConfig() - { - return config; - } - - @Nullable - public ContainerClass getInventoryValue(String containerKey) - { - final ContainerHolder containerHolder = containers.get(containerKey); - return containerHolder == null ? null : containerHolder.getContainer(); - } - - public Collection getInventory() - { - return containers.values() - .stream() - .map(ContainerHolder::getContainer) - .collect(Collectors.toList()); - } - - /** - * Doing this instead of a simple call to {@link ChildData#getData()} because data cache is turned off, see a comment - * in {@link #CuratorInventoryManager}. - */ - @Nullable - private byte[] getZkDataForNode(ChildData child) - { - try { - return curatorFramework.getData().decompressed().forPath(child.getPath()); - } - catch (Exception ex) { - log.warn(ex, "Exception while getting data for node %s", child.getPath()); - return null; - } - } - - private class ContainerHolder - { - private final AtomicReference container; - private final PathChildrenCache cache; - private boolean initialized = false; - - ContainerHolder( - ContainerClass container, - PathChildrenCache cache - ) - { - this.container = new AtomicReference<>(container); - this.cache = cache; - } - - private ContainerClass getContainer() - { - return container.get(); - } - - private void setContainer(ContainerClass newContainer) - { - container.set(newContainer); - } - - private PathChildrenCache getCache() - { - return cache; - } - } - - private class ContainerCacheListener implements PathChildrenCacheListener - { - private volatile boolean containersInitialized = false; - private volatile boolean doneInitializing = false; - - @Override - public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception - { - switch (event.getType()) { - case CHILD_ADDED: - synchronized (lock) { - final ChildData child = event.getData(); - - byte[] data = getZkDataForNode(child); - if (data == null) { - log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", - event.getType(), - child.getPath(), - child.getStat().getVersion()); - return; - } - - final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); - - final ContainerClass container = strategy.deserializeContainer(data); - - // This would normally be a race condition, but the only thing that should be mutating the containers - // map is this listener, which should never run concurrently. If the same container is going to disappear - // and come back, we expect a removed event in between. - if (containers.containsKey(containerKey)) { - log.error("New node[%s] but there was already one. That's not good, ignoring new one.", child.getPath()); - } else { - final String inventoryPath = StringUtils.format("%s/%s", config.getInventoryPath(), containerKey); - PathChildrenCache inventoryCache = cacheFactory.make(curatorFramework, inventoryPath); - inventoryCache.getListenable().addListener(new InventoryCacheListener(containerKey, inventoryPath)); - - containers.put(containerKey, new ContainerHolder(container, inventoryCache)); - - log.debug("Starting inventory cache for %s, inventoryPath %s", containerKey, inventoryPath); - strategy.newContainer(container); - inventoryCache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); - } - } - break; - case CHILD_REMOVED: - synchronized (lock) { - final ChildData child = event.getData(); - final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); - - final ContainerHolder removed = containers.remove(containerKey); - if (removed == null) { - log.error("Container[%s] removed that wasn't a container!?", child.getPath()); - break; - } - - // This close() call actually calls shutdownNow() on the executor registered with the Cache object, it - // better have its own executor or ignore shutdownNow() calls... - log.debug("Closing inventory cache for %s. Also removing listeners.", containerKey); - removed.getCache().close(); - strategy.deadContainer(removed.getContainer()); - - // also remove node from uninitilized, in case a nodes gets removed while we are starting up - synchronized (removed) { - markInventoryInitialized(removed); - } - } - break; - case CHILD_UPDATED: - synchronized (lock) { - final ChildData child = event.getData(); - - byte[] data = getZkDataForNode(child); - if (data == null) { - log.warn( - "Ignoring event: Type - %s , Path - %s , Version - %s", - event.getType(), - child.getPath(), - child.getStat().getVersion() - ); - return; - } - - final String containerKey = ZKPaths.getNodeFromPath(child.getPath()); - - final ContainerClass container = strategy.deserializeContainer(data); - - log.debug("Container[%s] updated.", child.getPath()); - ContainerHolder holder = containers.get(containerKey); - if (holder == null) { - log.error("Container update[%s], but the old container didn't exist!? Ignoring.", child.getPath()); - } else { - synchronized (holder) { - holder.setContainer(strategy.updateContainer(holder.getContainer(), container)); - } - } - } - break; - case INITIALIZED: - synchronized (lock) { - // must await initialized of all container holders - for (ContainerHolder holder : containers.values()) { - synchronized (holder) { - if (!holder.initialized) { - uninitializedInventory.add(holder); - } - } - } - containersInitialized = true; - maybeDoneInitializing(); - } - break; - case CONNECTION_SUSPENDED: - case CONNECTION_RECONNECTED: - case CONNECTION_LOST: - // do nothing - } - } - - // must be run in synchronized(lock) { synchronized(holder) { ... } } block - private void markInventoryInitialized(final ContainerHolder holder) - { - holder.initialized = true; - uninitializedInventory.remove(holder); - maybeDoneInitializing(); - } - - private void maybeDoneInitializing() - { - if (doneInitializing) { - return; - } - - // only fire if we are done initializing the parent PathChildrenCache - if (containersInitialized && uninitializedInventory.isEmpty()) { - doneInitializing = true; - strategy.inventoryInitialized(); - } - } - - private class InventoryCacheListener implements PathChildrenCacheListener - { - private final String containerKey; - - public InventoryCacheListener(String containerKey, String inventoryPath) - { - this.containerKey = containerKey; - log.debug("Created new InventoryCacheListener for %s", inventoryPath); - } - - @Override - public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) - { - final ContainerHolder holder = containers.get(containerKey); - if (holder == null) { - return; - } - - switch (event.getType()) { - case CHILD_ADDED: { - final ChildData child = event.getData(); - - byte[] data = getZkDataForNode(child); - if (data == null) { - log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", - event.getType(), - child.getPath(), - child.getStat().getVersion()); - return; - } - - final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.debug("CHILD_ADDED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); - - final InventoryClass addedInventory = strategy.deserializeInventory(data); - - synchronized (holder) { - holder.setContainer(strategy.addInventory(holder.getContainer(), inventoryKey, addedInventory)); - } - break; - } - - case CHILD_UPDATED: { - final ChildData child = event.getData(); - - byte[] data = getZkDataForNode(child); - if (data == null) { - log.warn("Ignoring event: Type - %s , Path - %s , Version - %s", - event.getType(), - child.getPath(), - child.getStat().getVersion()); - return; - } - - final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.debug("CHILD_UPDATED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); - - final InventoryClass updatedInventory = strategy.deserializeInventory(data); - - synchronized (holder) { - holder.setContainer(strategy.updateInventory(holder.getContainer(), inventoryKey, updatedInventory)); - } - - break; - } - - case CHILD_REMOVED: { - final ChildData child = event.getData(); - final String inventoryKey = ZKPaths.getNodeFromPath(child.getPath()); - log.debug("CHILD_REMOVED[%s] with version[%s]", child.getPath(), event.getData().getStat().getVersion()); - - synchronized (holder) { - holder.setContainer(strategy.removeInventory(holder.getContainer(), inventoryKey)); - } - - break; - } - case INITIALIZED: { - // make sure to acquire locks in (lock -> holder) order - synchronized (lock) { - synchronized (holder) { - markInventoryInitialized(holder); - } - } - - break; - } - case CONNECTION_SUSPENDED: - case CONNECTION_RECONNECTED: - case CONNECTION_LOST: - // do nothing - } - } - } - } -} diff --git a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManagerStrategy.java b/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManagerStrategy.java deleted file mode 100644 index 2a1d79c4a51d..000000000000 --- a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManagerStrategy.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.curator.inventory; - -/** - * Concurrency guarantees: all calls to {@link #newContainer}, {@link #deadContainer}, {@link #updateContainer} and - * {@link #inventoryInitialized} (all done within {@link CuratorInventoryManager#pathChildrenCacheExecutor}) are - * linearizable. Calls to {@link #addInventory}, {@link #updateInventory} and {@link #removeInventory} are linearizable - * for the each container in independence. - */ -public interface CuratorInventoryManagerStrategy -{ - ContainerClass deserializeContainer(byte[] bytes); - - InventoryClass deserializeInventory(byte[] bytes); - - void newContainer(ContainerClass newContainer); - void deadContainer(ContainerClass deadContainer); - ContainerClass updateContainer(ContainerClass oldContainer, ContainerClass newContainer); - ContainerClass addInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); - ContainerClass updateInventory(ContainerClass container, String inventoryKey, InventoryClass inventory); - ContainerClass removeInventory(ContainerClass container, String inventoryKey); - void inventoryInitialized(); -} diff --git a/server/src/main/java/org/apache/druid/curator/inventory/InventoryManagerConfig.java b/server/src/main/java/org/apache/druid/curator/inventory/InventoryManagerConfig.java deleted file mode 100644 index ddd36dd88aee..000000000000 --- a/server/src/main/java/org/apache/druid/curator/inventory/InventoryManagerConfig.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.curator.inventory; - -/** - */ -public interface InventoryManagerConfig -{ - /** - * The ContainerPath is the path where the InventoryManager should look for new containers of inventory. - * - * Because ZK does not allow for children under ephemeral nodes, the common interaction for registering Inventory - * that might be ephemeral is to - * - * 1) Create a permanent node underneath the InventoryPath - * 2) Create an ephemeral node under the ContainerPath with the same name as the permanent node under InventoryPath - * 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1) - * - * @return the containerPath - */ - String getContainerPath(); - - /** - * The InventoryPath is the path where the InventoryManager should look for new inventory. - * - * Because ZK does not allow for children under ephemeral nodes, the common interaction for registering Inventory - * that might be ephemeral is to - * - * 1) Create a permanent node underneath the InventoryPath - * 2) Create an ephemeral node under the ContainerPath with the same name as the permanent node under InventoryPath - * 3) For each piece of "inventory", create an ephemeral node as a child of the node created in step (1) - * - * @return the inventoryPath - */ - String getInventoryPath(); -} diff --git a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java index 71ee4612ede0..873daa8573f5 100644 --- a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java +++ b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java @@ -20,7 +20,6 @@ package org.apache.druid.guice; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Module; import org.apache.druid.client.FilteredServerInventoryView; import org.apache.druid.client.FilteredServerInventoryViewProvider; @@ -29,9 +28,7 @@ import org.apache.druid.client.ServerInventoryView; import org.apache.druid.client.ServerInventoryViewProvider; import org.apache.druid.client.ServerView; -import org.apache.druid.error.DruidException; -import java.util.Properties; import java.util.StringJoiner; /** @@ -43,24 +40,6 @@ public class ServerViewModule implements Module public static final String SERVERVIEW_TYPE_PROPERTY = "druid.serverview.type"; public static final String SERVERVIEW_TYPE_HTTP = "http"; - @Inject - public void configure(Properties properties) - { - String configuredType = properties.getProperty(SERVERVIEW_TYPE_PROPERTY); - if (configuredType != null && !SERVERVIEW_TYPE_HTTP.equals(configuredType)) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build( - "Invalid value[%s] for property[%s]. Only [%s] is supported; the ZooKeeper-based" - + " 'batch' server view has been removed. Remove this property or set it to 'http'." - + " See the Druid upgrade notes for details.", - configuredType, - SERVERVIEW_TYPE_PROPERTY, - SERVERVIEW_TYPE_HTTP - ); - } - } - @Override public void configure(Binder binder) { diff --git a/server/src/test/java/org/apache/druid/client/TestServerInventoryView.java b/server/src/test/java/org/apache/druid/client/TestServerInventoryView.java new file mode 100644 index 000000000000..4536c1f6af94 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/TestServerInventoryView.java @@ -0,0 +1,157 @@ +/* + * 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.base.Function; +import com.google.common.base.Predicate; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.timeline.DataSegment; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +/** + * In-memory test fake for {@link ServerInventoryView} / {@link FilteredServerInventoryView}. + * + * Tests drive it directly via {@link #addServer}, {@link #removeServer}, {@link #addSegment}, + * {@link #removeSegment}, and {@link #markInventoryInitialized} instead of going through + * ZooKeeper. Each driver method synchronously dispatches the corresponding callback. + */ +public class TestServerInventoryView implements ServerInventoryView, FilteredServerInventoryView +{ + private final ConcurrentHashMap inventory = new ConcurrentHashMap<>(); + private final ConcurrentHashMap serverCallbacks = new ConcurrentHashMap<>(); + private final ConcurrentHashMap segmentCallbacks = new ConcurrentHashMap<>(); + + private volatile boolean started = false; + + public void start() + { + started = true; + } + + public void stop() + { + started = false; + } + + public void addServer(DruidServer server) + { + if (inventory.putIfAbsent(server.getName(), server) != null) { + return; + } + runServerCallbacks(callback -> callback.serverAdded(server)); + } + + public void removeServer(DruidServer server) + { + inventory.remove(server.getName()); + runServerCallbacks(callback -> callback.serverRemoved(server)); + } + + public void addSegment(DruidServer server, DataSegment segment) + { + server.addDataSegment(segment); + runSegmentCallbacks(callback -> callback.segmentAdded(server.getMetadata(), segment)); + } + + public void removeSegment(DruidServer server, DataSegment segment) + { + server.removeDataSegment(segment.getId()); + runSegmentCallbacks(callback -> callback.segmentRemoved(server.getMetadata(), segment)); + } + + public void markInventoryInitialized() + { + runSegmentCallbacks(SegmentCallback::segmentViewInitialized); + } + + @Override + public DruidServer getInventoryValue(String serverKey) + { + return inventory.get(serverKey); + } + + @Override + public Collection getInventory() + { + return inventory.values(); + } + + @Override + public boolean isStarted() + { + return started; + } + + @Override + public boolean isSegmentLoadedByServer(String serverKey, DataSegment segment) + { + DruidServer server = getInventoryValue(serverKey); + return server != null && server.getSegment(segment.getId()) != null; + } + + @Override + public void registerServerCallback(Executor exec, ServerCallback callback) + { + serverCallbacks.put(callback, exec); + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + segmentCallbacks.put(callback, exec); + } + + @Override + public void registerSegmentCallback( + Executor exec, + SegmentCallback callback, + Predicate> filter + ) + { + registerSegmentCallback(exec, new FilteringSegmentCallback(callback, filter)); + } + + protected void runServerCallbacks(final Function fn) + { + for (final Map.Entry entry : serverCallbacks.entrySet()) { + entry.getValue().execute(() -> { + if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { + serverCallbacks.remove(entry.getKey()); + } + }); + } + } + + protected void runSegmentCallbacks(final Function fn) + { + for (final Map.Entry entry : segmentCallbacks.entrySet()) { + entry.getValue().execute(() -> { + if (CallbackAction.UNREGISTER == fn.apply(entry.getKey())) { + segmentCallbacks.remove(entry.getKey()); + } + }); + } + } +} diff --git a/server/src/test/java/org/apache/druid/curator/inventory/CuratorInventoryManagerTest.java b/server/src/test/java/org/apache/druid/curator/inventory/CuratorInventoryManagerTest.java deleted file mode 100644 index 4190b10744d8..000000000000 --- a/server/src/test/java/org/apache/druid/curator/inventory/CuratorInventoryManagerTest.java +++ /dev/null @@ -1,274 +0,0 @@ -/* - * 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.curator.inventory; - -import com.google.common.collect.Iterables; -import com.google.common.primitives.Ints; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.api.CuratorEvent; -import org.apache.curator.framework.api.CuratorEventType; -import org.apache.curator.framework.api.CuratorListener; -import org.apache.druid.curator.CuratorTestBase; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.Watcher; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Map; -import java.util.TreeMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; - -/** - */ -public class CuratorInventoryManagerTest extends CuratorTestBase -{ - private ExecutorService exec; - - @Before - public void setUp() throws Exception - { - setupServerAndCurator(); - exec = Execs.singleThreaded("curator-inventory-manager-test-%s"); - } - - @After - public void tearDown() - { - tearDownServerAndCurator(); - } - - @Test - public void testSanity() throws Exception - { - final MapStrategy strategy = new MapStrategy(); - CuratorInventoryManager, Integer> manager = new CuratorInventoryManager<>( - curator, - new StringInventoryManagerConfig("/container", "/inventory"), - exec, - strategy - ); - - curator.start(); - curator.blockUntilConnected(); - - manager.start(); - - Assert.assertTrue(Iterables.isEmpty(manager.getInventory())); - - CountDownLatch containerLatch = new CountDownLatch(1); - strategy.setNewContainerLatch(containerLatch); - curator.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath("/container/billy", new byte[]{}); - - Assert.assertTrue(timing.awaitLatch(containerLatch)); - strategy.setNewContainerLatch(null); - - final Iterable> inventory = manager.getInventory(); - Assert.assertTrue(Iterables.getOnlyElement(inventory).isEmpty()); - - CountDownLatch inventoryLatch = new CountDownLatch(2); - strategy.setNewInventoryLatch(inventoryLatch); - curator.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/1", Ints.toByteArray(100)); - curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/bob", Ints.toByteArray(2287)); - - Assert.assertTrue(timing.awaitLatch(inventoryLatch)); - strategy.setNewInventoryLatch(null); - - verifyInventory(manager); - - CountDownLatch deleteLatch = new CountDownLatch(1); - strategy.setDeadInventoryLatch(deleteLatch); - curator.delete().forPath("/inventory/billy/1"); - - Assert.assertTrue(timing.awaitLatch(deleteLatch)); - strategy.setDeadInventoryLatch(null); - - Assert.assertEquals(1, manager.getInventoryValue("billy").size()); - Assert.assertEquals(2287, manager.getInventoryValue("billy").get("bob").intValue()); - - inventoryLatch = new CountDownLatch(1); - strategy.setNewInventoryLatch(inventoryLatch); - curator.create().withMode(CreateMode.EPHEMERAL).forPath("/inventory/billy/1", Ints.toByteArray(100)); - - Assert.assertTrue(timing.awaitLatch(inventoryLatch)); - strategy.setNewInventoryLatch(null); - - verifyInventory(manager); - - final CountDownLatch latch = new CountDownLatch(1); - curator.getCuratorListenable().addListener( - new CuratorListener() { - @Override - public void eventReceived(CuratorFramework client, CuratorEvent event) - { - if (event.getType() == CuratorEventType.WATCHED - && event.getWatchedEvent().getState() == Watcher.Event.KeeperState.Disconnected) { - latch.countDown(); - } - } - } - ); - - server.stop(); - Assert.assertTrue(timing.awaitLatch(latch)); - - verifyInventory(manager); - - Thread.sleep(50); // Wait a bit - - verifyInventory(manager); - } - - private void verifyInventory(CuratorInventoryManager, Integer> manager) - { - final Map vals = manager.getInventoryValue("billy"); - Assert.assertEquals(2, vals.size()); - Assert.assertEquals(100, vals.get("1").intValue()); - Assert.assertEquals(2287, vals.get("bob").intValue()); - } - - private static class StringInventoryManagerConfig implements InventoryManagerConfig - { - private final String containerPath; - private final String inventoryPath; - - private StringInventoryManagerConfig( - String containerPath, - String inventoryPath - ) - { - this.containerPath = containerPath; - this.inventoryPath = inventoryPath; - } - - @Override - public String getContainerPath() - { - return containerPath; - } - - @Override - public String getInventoryPath() - { - return inventoryPath; - } - } - - private static class MapStrategy implements CuratorInventoryManagerStrategy, Integer> - { - private volatile CountDownLatch newContainerLatch = null; - private volatile CountDownLatch deadContainerLatch = null; - private volatile CountDownLatch newInventoryLatch = null; - private volatile CountDownLatch deadInventoryLatch = null; - private volatile boolean initialized = false; - - @Override - public Map deserializeContainer(byte[] bytes) - { - return new TreeMap<>(); - } - - @Override - public Integer deserializeInventory(byte[] bytes) - { - return Ints.fromByteArray(bytes); - } - - @Override - public void newContainer(Map newContainer) - { - if (newContainerLatch != null) { - newContainerLatch.countDown(); - } - } - - @Override - public void deadContainer(Map deadContainer) - { - if (deadContainerLatch != null) { - deadContainerLatch.countDown(); - } - } - - @Override - public Map updateContainer(Map oldContainer, Map newContainer) - { - newContainer.putAll(oldContainer); - return newContainer; - } - - @Override - public Map addInventory(Map container, String inventoryKey, Integer inventory) - { - container.put(inventoryKey, inventory); - if (newInventoryLatch != null) { - newInventoryLatch.countDown(); - } - return container; - } - - @Override - public Map updateInventory( - Map container, String inventoryKey, Integer inventory - ) - { - return addInventory(container, inventoryKey, inventory); - } - - @Override - public Map removeInventory(Map container, String inventoryKey) - { - container.remove(inventoryKey); - if (deadInventoryLatch != null) { - deadInventoryLatch.countDown(); - } - return container; - } - - private void setNewContainerLatch(CountDownLatch newContainerLatch) - { - this.newContainerLatch = newContainerLatch; - } - - private void setDeadContainerLatch(CountDownLatch deadContainerLatch) - { - this.deadContainerLatch = deadContainerLatch; - } - - private void setNewInventoryLatch(CountDownLatch newInventoryLatch) - { - this.newInventoryLatch = newInventoryLatch; - } - - private void setDeadInventoryLatch(CountDownLatch deadInventoryLatch) - { - this.deadInventoryLatch = deadInventoryLatch; - } - - @Override - public void inventoryInitialized() - { - initialized = true; - } - } -} diff --git a/services/src/test/java/org/apache/druid/cli/MainTest.java b/services/src/test/java/org/apache/druid/cli/MainTest.java index 246f037bb7fc..33fb605d1f60 100644 --- a/services/src/test/java/org/apache/druid/cli/MainTest.java +++ b/services/src/test/java/org/apache/druid/cli/MainTest.java @@ -19,68 +19,36 @@ package org.apache.druid.cli; -import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import org.apache.druid.guice.GuiceInjectors; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.util.Properties; +import java.util.stream.Stream; -@RunWith(Parameterized.class) public class MainTest { - @Parameterized.Parameters(name = "{0}") - public static Iterable constructorFeeder() + private static Stream runnables() { - return ImmutableList.of( - new Object[]{new CliOverlord()}, - new Object[]{new CliBroker()}, - new Object[]{new CliHistorical()}, - new Object[]{new CliCoordinator()}, - new Object[]{new CliMiddleManager()}, - new Object[]{new CliRouter()}, - new Object[]{new CliIndexer()} + return Stream.of( + new CliOverlord(), + new CliBroker(), + new CliHistorical(), + new CliCoordinator(), + new CliMiddleManager(), + new CliRouter(), + new CliIndexer() ); } - private final ServerRunnable runnable; - - public MainTest(ServerRunnable runnable) - { - this.runnable = runnable; - } - - @Test - public void testSimpleInjection() + @ParameterizedTest(name = "{0}") + @MethodSource("runnables") + public void testSimpleInjection(ServerRunnable runnable) { final Injector injector = GuiceInjectors.makeStartupInjector(); injector.injectMembers(runnable); - Assert.assertNotNull(runnable.makeInjector(runnable.getNodeRoles(new Properties()))); - } - - @Test(expected = RuntimeException.class) - public void testSimpleInjection_centralizedDatasourceSchemaEnabled() - { - // Do not run the test for CliRouter or CliHistorical - Assume.assumeFalse(runnable instanceof CliRouter || runnable instanceof CliHistorical); - - try { - System.setProperty("druid.centralizedDatasourceSchema.enabled", "true"); - System.setProperty("druid.serverview.type", "batch"); - System.setProperty("druid.server.http.numThreads", "2"); - - final Injector injector = GuiceInjectors.makeStartupInjector(); - injector.injectMembers(runnable); - Assert.assertNotNull(runnable.makeInjector(runnable.getNodeRoles(new Properties()))); - } - finally { - System.clearProperty("druid.centralizedDatasourceSchema.enabled"); - System.clearProperty("druid.serverview.type"); - System.clearProperty("druid.server.http.numThreads"); - } + Assertions.assertNotNull(runnable.makeInjector(runnable.getNodeRoles(new Properties()))); } } From 4f3ea3312bd63e8a0bf949517d9162ce0658fb58 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 27 Apr 2026 09:30:18 -0700 Subject: [PATCH 4/4] fix build --- .../druid/guice/StartupInjectorBuilderTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/guice/StartupInjectorBuilderTest.java b/processing/src/test/java/org/apache/druid/guice/StartupInjectorBuilderTest.java index 12efe8359291..ece5e78cbf72 100644 --- a/processing/src/test/java/org/apache/druid/guice/StartupInjectorBuilderTest.java +++ b/processing/src/test/java/org/apache/druid/guice/StartupInjectorBuilderTest.java @@ -195,13 +195,13 @@ public void testValidator_rejectsNonHttpServerViewType() props.setProperty(StartupInjectorBuilder.SERVERVIEW_TYPE_CONFIG_STRING, "batch"); final StartupInjectorBuilder builder = new StartupInjectorBuilder().withExtensions().withProperties(props); - MatcherAssert.assertThat( - Assert.assertThrows(ISE.class, builder::build), - ExceptionMatcher.of(ISE.class).expectMessageIs( - "Invalid value[batch] for property[druid.serverview.type]. Only [http] is supported;" - + " the ZooKeeper-based 'batch' server view has been removed. Remove this property or" - + " set it to 'http'. See the Druid upgrade notes for details." - ) + + Throwable t = Assertions.assertThrows(ISE.class, builder::build); + Assertions.assertEquals( + "Invalid value[batch] for property[druid.serverview.type]. Only [http] is supported;" + + " the ZooKeeper-based 'batch' server view has been removed. Remove this property or" + + " set it to 'http'. See the Druid upgrade notes for details.", + t.getMessage() ); }