diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 48dccd11a84..b974253888c 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -140,4 +140,4 @@ This is a specification of the properties that tell the job how to update metada |password|password for db|yes| |segmentTable|table to use in DB|yes| -These properties should parrot what you have configured for your [Master](Master.html). +These properties should parrot what you have configured for your [Coordinator](Coordinator.html). diff --git a/docs/content/Broker.md b/docs/content/Broker.md index be5e2142dbf..225be9f93b9 100644 --- a/docs/content/Broker.md +++ b/docs/content/Broker.md @@ -11,13 +11,13 @@ Forwarding Queries Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](Segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 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 nodes, and hence, the query will likely hit multiple nodes. -To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Compute](Compute.html) and [Realtime](Realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker node then forwards down the query to the selected nodes. +To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](Historical.html) and [Realtime](Realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker node then forwards down the query to the selected nodes. Caching ------- Broker nodes employ a distributed cache with a LRU cache invalidation strategy. The broker cache stores per segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the -compute nodes. Once the compute nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable. +historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable. Running ------- diff --git a/docs/content/Compute.md b/docs/content/Compute.md deleted file mode 100644 index 2994fed97c2..00000000000 --- a/docs/content/Compute.md +++ /dev/null @@ -1,40 +0,0 @@ ---- -layout: doc_page ---- -Compute -======= - -Compute nodes are the work horses of a cluster. They load up historical segments and expose them for querying. - -Loading and Serving Segments ----------------------------- - -Each compute node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Compute nodes do not communicate directly with each other or with the master nodes but instead rely on Zookeeper for coordination. - -The [Master](Master.html) node is responsible for assigning new segments to compute nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a compute node. For more information on how the master assigns segments to compute nodes, please see [Master](Master.html). - -When a compute node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the compute node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [Segments](Segments.html). Once a compute node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying. - -Loading and Serving Segments From Cache ---------------------------------------- - -Recall that when a compute node notices a new segment entry in its load queue path, the compute node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the compute node will directly read the segment binary files from disk and load the segment. - -The segment cache is also leveraged when a compute node is first started. On startup, a compute node will search through its cache directory and immediately load and serve all segments that are found. This feature allows compute nodes to be queried as soon they come online. - -Querying Segments ------------------ - -Please see [Querying](Querying.html) for more information on querying compute nodes. - -For every query that a compute node services, it will log the query and report metrics on the time taken to run the query. - -Running -------- - -Compute nodes can be run using the `com.metamx.druid.http.ComputeMain` class. - -Configuration -------------- - -See [Configuration](Configuration.html). diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 2faf27b3d6a..c7b94eebdbf 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -72,7 +72,6 @@ druid.storage.s3.bucket= druid.storage.s3.baseKey= druid.bard.cache.sizeInBytes=40000000 -druid.master.merger.service=blah_blah ``` Configuration groupings @@ -80,7 +79,7 @@ Configuration groupings ### S3 Access -These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [Compute](Compute.html), [Realtime](Realtime.html) and [Indexing service](Indexing service.html) nodes. +These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [Historical](Historical.html), [Realtime](Realtime.html) and [Indexing service](Indexing service.html) nodes. |Property|Description|Default| |--------|-----------|-------| @@ -91,25 +90,25 @@ These properties are for connecting with S3 and using it to pull down segments. ### JDBC connection -These properties specify the jdbc connection and other configuration around the "segments table" database. The only processes that connect to the DB with these properties are the [Master](Master.html) and [Indexing service](Indexing-service.html). This is tested on MySQL. +These properties specify the jdbc connection and other configuration around the "segments table" database. The only processes that connect to the DB with these properties are the [Coordinator](Coordinator.html) and [Indexing service](Indexing-service.html). This is tested on MySQL. |Property|Description|Default| |--------|-----------|-------| |`druid.database.connectURI`|The jdbc connection uri|none| |`druid.database.user`|The username to connect with|none| |`druid.database.password`|The password to connect with|none| -|`druid.database.poll.duration`|The duration between polls the Master does for updates to the set of active segments. Generally defines the amount of lag time it can take for the master to notice new segments|PT1M| +|`druid.database.poll.duration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments|PT1M| |`druid.database.segmentTable`|The table to use to look for segments.|none| |`druid.database.ruleTable`|The table to use to look for segment load/drop rules.|none| |`druid.database.configTable`|The table to use to look for configs.|none| -### Master properties +### Coordinator properties |Property|Description|Default| |--------|-----------|-------| -|`druid.master.period`|The run period for the master. The master’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs|PT60S| -|`druid.master.removedSegmentLifetime`|When a node disappears, the master can provide a grace period for how long it waits before deciding that the node really isn’t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the master.|1| -|`druid.master.startDelay`|The operation of the Master works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Master to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data|PT600S| +|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs|PT60S| +|`druid.coordinator.removedSegmentLifetime`|When a node disappears, the coordinator can provide a grace period for how long it waits before deciding that the node really isn’t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the coordinator.|1| +|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data|PT600S| ### Zk properties @@ -121,28 +120,28 @@ These are properties that define various service/HTTP server aspects |Property|Description|Default| |--------|-----------|-------| -|`druid.client.http.connections`|Size of connection pool for the Broker to connect to compute nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|none| -|`druid.paths.indexCache`|Segments assigned to a compute node are first stored on the local file system and then served by the compute node. This path defines where that local cache resides. Directory will be created if needed|none| -|`druid.paths.segmentInfoCache`|Compute nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the master to reassign. This path defines where this metadata is kept. Directory will be created if needed|none| +|`druid.client.http.connections`|Size of connection pool for the Broker to connect to historical nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|none| +|`druid.paths.indexCache`|Segments assigned to a historical node are first stored on the local file system and then served by the historical node. This path defines where that local cache resides. Directory will be created if needed|none| +|`druid.paths.segmentInfoCache`|historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed|none| |`druid.http.numThreads`|The number of HTTP worker threads.|10| |`druid.http.maxIdleTimeMillis`|The amount of time a connection can remain idle before it is terminated|300000 (5 min)| -|`druid.request.logging.dir`|Compute, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none| +|`druid.request.logging.dir`|Historical, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none| |`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| |`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| -|`druid.processing.formatString`|Realtime and Compute nodes use this format string to name their processing threads.|none| +|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|none| |`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|none| -|`druid.computation.buffer.size`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Compute and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| +|`druid.computation.buffer.size`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| |`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| |`druid.bard.cache.sizeInBytes`|The Broker (called Bard internally) instance has the ability to store results of queries in an in-memory cache. This specifies the number of bytes to use for that cache|none| -### Compute Properties +### Historical Node Properties -These are properties that the compute nodes use +These are properties that the historical nodes use |Property|Description|Default| |--------|-----------|-------| -|`druid.server.maxSize`|The maximum number of bytes worth of segment that the node wants assigned to it. This is not a limit that the compute nodes actually enforce, they just publish it to the master and trust the master to do the right thing|none| -|`druid.server.type`|Specifies the type of the node. This is published via ZK and depending on the value the node will be treated specially by the Master/Broker. Allowed values are "realtime" or "historical". This is a configuration parameter because the plan is to allow for a more configurable cluster composition. At the current time, all realtime nodes should just be "realtime" and all compute nodes should just be "compute"|none| +|`druid.server.maxSize`|The maximum number of bytes worth of segment that the node wants assigned to it. This is not a limit that the historical nodes actually enforce, they just publish it to the coordinator and trust the coordinator to do the right thing|none| +|`druid.server.type`|Specifies the type of the node. This is published via ZK and depending on the value the node will be treated specially by the Coordinator/Broker. Allowed values are "realtime" or "historical". This is a configuration parameter because the plan is to allow for a more configurable cluster composition. At the current time, all realtime nodes should just be "realtime" and all historical nodes should just be "historical"|none| ### Emitter Properties diff --git a/docs/content/Coordinator.md b/docs/content/Coordinator.md new file mode 100644 index 00000000000..694003b3b3a --- /dev/null +++ b/docs/content/Coordinator.md @@ -0,0 +1,136 @@ +--- +layout: doc_page +--- +Coordinator +====== + +The Druid coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid coordinator node communicates to historical nodes to load or drop segments based on configurations. The Druid coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load. + +The Druid coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the broker and historical nodes, the Druid coordinator maintains a connection to a Zookeeper cluster for current cluster information. The coordinator also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled. + +Before any unassigned segments are serviced by historical nodes, the available historical nodes 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 nodes with least capacity to maintain a level of balance between nodes. The coordinator does not directly communicate with a historical node when assigning it a new segment; instead the coordinator creates some temporary information about the new segment under load queue path of the historical node. Once this request is seen, the historical node will load the segment and begin servicing it. + +Rules +----- + +Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different historical node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The coordinator loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule + +For more information on rules, see [Rule Configuration](Rule-Configuration.html). + +Cleaning Up Segments +-------------------- + +Each run, the Druid coordinator compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped. + +Segment Availability +-------------------- + +If a historical node restarts or becomes unavailable for any reason, the Druid coordinator will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other historical nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime represents a period of time in which the coordinator will not reassign a dropped segment. Hence, if a historical node becomes unavailable and available again within a short period of time, the historical node will start up and serve segments from its cache without any those segments being reassigned across the cluster. + +Balancing Segment Load +---------------------- + +To ensure an even distribution of segments across historical nodes in the cluster, the coordinator node will find the total size of all segments being served by every historical node each time the coordinator runs. For every historical node tier in the cluster, the coordinator node will determine the historical node with the highest utilization and the historical node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized node to the lowest utilized node. There is a configurable limit on the number of segments that can be moved from one node to another each time the coordinator runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased. + +HTTP Endpoints +-------------- + +The coordinator node exposes several HTTP endpoints for interactions. + +### GET + +* `/info/coordinator` + + Returns the current true coordinator of the cluster as a JSON object. + +* `/info/cluster` + + Returns JSON data about every node and segment in the cluster. Information about each node and each segment on each node will be returned. + +* `/info/servers` + + Returns information about servers in the cluster. Set the `?full` query parameter to get full metadata about all servers and their segments in the cluster. + +* `/info/servers/{serverName}` + + Returns full metadata about a specific server. + +* `/info/servers/{serverName}/segments` + + Returns a list of all segments for a server. Set the `?full` query parameter to get all segment metadata included + +* `/info/servers/{serverName}/segments/{segmentId}` + + Returns full metadata for a specific segment. + +* `/info/segments` + + Returns all segments in the cluster as a list. Set the `?full` flag to get all metadata about segments in the cluster + +* `/info/segments/{segmentId}` + + Returns full metadata for a specific segment + +* `/info/datasources` + + Returns a list of datasources in the cluster. Set the `?full` flag to get all metadata for every datasource in the cluster + +* `/info/datasources/{dataSourceName}` + + Returns full metadata for a datasource + +* `/info/datasources/{dataSourceName}/segments` + + Returns a list of all segments for a datasource. Set the `?full` flag to get full segment metadata for a datasource + +* `/info/datasources/{dataSourceName}/segments/{segmentId}` + + Returns full segment metadata for a specific segment + +* `/info/rules` + + Returns all rules for all data sources in the cluster including the default datasource. + +* `/info/rules/{dataSourceName}` + + Returns all rules for a specified datasource + +### POST + +* `/info/rules/{dataSourceName}` + + POST with a list of rules in JSON form to update rules. + +The Coordinator Console +------------------ + +The Druid coordinator exposes a web GUI for displaying cluster information and rule configuration. After the coordinator starts, the console can be accessed at http://HOST:PORT/static/. There exists a full cluster view, as well as views for individual historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table. + +The coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited. + +FAQ +--- + +1. **Do clients ever contact the coordinator node?** + + The coordinator is not involved in a query. + + historical nodes never directly contact the coordinator node. The Druid coordinator tells the historical nodes to load/drop data via Zookeeper, but the historical nodes are completely unaware of the coordinator. + + Brokers also never contact the coordinator. Brokers base their understanding of the data topology on metadata exposed by the historical nodes via ZK and are completely unaware of the coordinator. + +2. **Does it matter if the coordinator node starts up before or after other processes?** + + No. If the Druid coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the coordinator node can be started up at any time, and after a configurable delay, will start running coordinator tasks. + + This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology. + +Running +------- + +Coordinator nodes can be run using the `io.druid.cli.Main` class with program parameters "server coordinator". + +Configuration +------------- + +See [Configuration](Configuration.html). diff --git a/docs/content/Design.md b/docs/content/Design.md index cdd3d5e6322..668994f4067 100644 --- a/docs/content/Design.md +++ b/docs/content/Design.md @@ -27,13 +27,13 @@ Druid is architected as a grouping of systems each with a distinct role and toge The node types that currently exist are: -* **Compute** nodes are the workhorses that handle storage and querying on "historical" data (non-realtime) -* **Realtime** nodes ingest data in real-time, they are in charge of listening to a stream of incoming data and making it available immediately inside the Druid system. As data they have ingested ages, they hand it off to the compute nodes. -* **Master** nodes act as coordinators. They look over the grouping of computes and make sure that data is available, replicated and in a generally "optimal" configuration. +* **Historical** nodes are the workhorses that handle storage and querying on "historical" data (non-realtime) +* **Realtime** nodes ingest data in real-time, they are in charge of listening to a stream of incoming data and making it available immediately inside the Druid system. As data they have ingested ages, they hand it off to the historical nodes. +* **Coordinator** nodes look over the grouping of historical nodes and make sure that data is available, replicated and in a generally "optimal" configuration. * **Broker** nodes understand the topology of data across all of the other nodes in the cluster and re-write and route queries accordingly * **Indexer** nodes form a cluster of workers to load batch and real-time data into the system as well as allow for alterations to the data stored in the system (also known as the Indexing Service) -This separation allows each node to only care about what it is best at. By separating Compute and Realtime, we separate the memory concerns of listening on a real-time stream of data and processing it for entry into the system. By separating the Master and Broker, we separate the needs for querying from the needs for maintaining "good" data distribution across the cluster. +This separation allows each node to only care about what it is best at. By separating Historical and Realtime, we separate the memory concerns of listening on a real-time stream of data and processing it for entry into the system. By separating the Coordinator and Broker, we separate the needs for querying from the needs for maintaining "good" data distribution across the cluster. All nodes can be run in some highly available fashion. Either as symmetric peers in a share-nothing cluster or as hot-swap failover nodes. @@ -55,27 +55,27 @@ Getting data into the Druid system requires an indexing process. This gives the - Bitmap compression - RLE (on the roadmap, but not yet implemented) -The output of the indexing process is stored in a "deep storage" LOB store/file system ([Deep Storage](Deep Storage.html) for information about potential options). Data is then loaded by compute nodes by first downloading the data to their local disk and then memory mapping it before serving queries. +The output of the indexing process is stored in a "deep storage" LOB store/file system ([Deep Storage](Deep Storage.html) for information about potential options). Data is then loaded by historical nodes by first downloading the data to their local disk and then memory mapping it before serving queries. -If a compute node dies, it will no longer serve its segments, but given that the segments are still available on the "deep storage" any other node can simply download the segment and start serving it. This means that it is possible to actually remove all compute nodes from the cluster and then re-provision them without any data loss. It also means that if the "deep storage" is not available, the nodes can continue to serve the segments they have already pulled down (i.e. the cluster goes stale, not down). +If a historical node dies, it will no longer serve its segments, but given that the segments are still available on the "deep storage" any other node can simply download the segment and start serving it. This means that it is possible to actually remove all historical nodes from the cluster and then re-provision them without any data loss. It also means that if the "deep storage" is not available, the nodes can continue to serve the segments they have already pulled down (i.e. the cluster goes stale, not down). -In order for a segment to exist inside of the cluster, an entry has to be added to a table in a MySQL instance. This entry is a self-describing bit of metadata about the segment, it includes things like the schema of the segment, the size, and the location on deep storage. These entries are what the Master uses to know what data **should** be available on the cluster. +In order for a segment to exist inside of the cluster, an entry has to be added to a table in a MySQL instance. This entry is a self-describing bit of metadata about the segment, it includes things like the schema of the segment, the size, and the location on deep storage. These entries are what the Coordinator uses to know what data **should** be available on the cluster. ### Fault Tolerance -- **Compute** As discussed above, if a compute node dies, another compute node can take its place and there is no fear of data loss -- **Master** Can be run in a hot fail-over configuration. If no masters are running, then changes to the data topology will stop happening (no new data and no data balancing decisions), but the system will continue to run. +- **Historical** As discussed above, if a historical node dies, another historical node can take its place and there is no fear of data loss +- **Coordinator** Can be run in a hot fail-over configuration. If no coordinators are running, then changes to the data topology will stop happening (no new data and no data balancing decisions), but the system will continue to run. - **Broker** Can be run in parallel or in hot fail-over. -- **Realtime** Depending on the semantics of the delivery stream, multiple of these can be run in parallel processing the exact same stream. They periodically checkpoint to disk and eventually push out to the Computes. Steps are taken to be able to recover from process death, but loss of access to the local disk can result in data loss if this is the only method of adding data to the system. +- **Realtime** Depending on the semantics of the delivery stream, multiple of these can be run in parallel processing the exact same stream. They periodically checkpoint to disk and eventually push out to the Historical nodes. Steps are taken to be able to recover from process death, but loss of access to the local disk can result in data loss if this is the only method of adding data to the system. - **"deep storage" file system** If this is not available, new data will not be able to enter the cluster, but the cluster will continue operating as is. -- **MySQL** If this is not available, the master will be unable to find out about new segments in the system, but it will continue with its current view of the segments that should exist in the cluster. +- **MySQL** If this is not available, the coordinator will be unable to find out about new segments in the system, but it will continue with its current view of the segments that should exist in the cluster. - **ZooKeeper** If this is not available, data topology changes will not be able to be made, but the Brokers will maintain their most recent view of the data topology and continue serving requests accordingly. ### Query processing -A query first enters the Broker, where the broker will match the query with the data segments that are known to exist. It will then pick a set of machines that are serving those segments and rewrite the query for each server to specify the segment(s) targetted. The Compute/Realtime nodes will take in the query, process them and return results. The Broker then takes the results and merges them together to get the final answer, which it returns. In this way, the broker can prune all of the data that doesn’t match a query before ever even looking at a single row of data. +A query first enters the Broker, where the broker will match the query with the data segments that are known to exist. It will then pick a set of machines that are serving those segments and rewrite the query for each server to specify the segment(s) targetted. The Historical/Realtime nodes will take in the query, process them and return results. The Broker then takes the results and merges them together to get the final answer, which it returns. In this way, the broker can prune all of the data that doesn’t match a query before ever even looking at a single row of data. -For filters at a more granular level than what the Broker can prune based on, the indexing structures inside each segment allows the compute nodes to figure out which (if any) rows match the filter set before looking at any row of data. It can do all of the boolean algebra of the filter on the bitmap indices and never actually look directly at a row of data. +For filters at a more granular level than what the Broker can prune based on, the indexing structures inside each segment allows the historical nodes to figure out which (if any) rows match the filter set before looking at any row of data. It can do all of the boolean algebra of the filter on the bitmap indices and never actually look directly at a row of data. Once it knows the rows that match the current query, it can access the columns it cares about for those rows directly without having to load data that it is just going to throw away. diff --git a/docs/content/Druid-Personal-Demo-Cluster.md b/docs/content/Druid-Personal-Demo-Cluster.md index 01112a4725c..928ddc8fad7 100644 --- a/docs/content/Druid-Personal-Demo-Cluster.md +++ b/docs/content/Druid-Personal-Demo-Cluster.md @@ -14,9 +14,9 @@ This guide walks you through the steps to create the cluster and then how to cre ## What’s in this Druid Demo Cluster? -1. A single "Master" node. This node co-locates the [Master](Master.html) process, the [Broker](Broker.html) process, Zookeeper, and the MySQL instance. You can read more about Druid architecture [Design](Design.html). +1. A single "Coordinator" node. This node co-locates the [Coordinator](Coordinator.html) process, the [Broker](Broker.html) process, Zookeeper, and the MySQL instance. You can read more about Druid architecture [Design](Design.html). -1. Three compute nodes; these compute nodes, have been pre-configured to work with the Master node and should automatically load up the Wikipedia edit stream data (no specific setup is required). +1. Three historical nodes; these historical nodes, have been pre-configured to work with the Coordinator node and should automatically load up the Wikipedia edit stream data (no specific setup is required). ## Setup Instructions 1. Log in to your AWS account: Start by logging into the [Console page](https://console.aws.amazon.com) of your AWS account; if you don’t have one, follow this link to sign up for one [http://aws.amazon.com/](http://aws.amazon.com/). @@ -54,19 +54,19 @@ This guide walks you through the steps to create the cluster and then how to cre ![CloudFormations](images/demo/setup-09-events.png) -1. Get the IP address of your Druid Master Node: +1. Get the IP address of your Druid Coordinator Node: 1. Go to the following URL: [https://console.aws.amazon.com/ec2](https://console.aws.amazon.com/ec2) 1. Click **Instances** in the left pane – you should see something similar to the following figure. - 1. Select the **DruidMaster** instance - 1. Your IP address is right under the heading: **EC2 Instance: DruidMaster**. Select and copy that entire line, which ends with `amazonaws.com`. + 1. Select the **DruidCoordinator** instance + 1. Your IP address is right under the heading: **EC2 Instance: DruidCoordinator**. Select and copy that entire line, which ends with `amazonaws.com`. ![EC2 Instances](images/demo/setup-10-ip.png) ## Querying Data -1. Use the following URL to bring up the Druid Demo Cluster query interface (replace **IPAddressDruidMaster** with the actual druid master IP Address): +1. Use the following URL to bring up the Druid Demo Cluster query interface (replace **IPAddressDruidCoordinator** with the actual druid coordinator IP Address): -**`http://IPAddressDruidMaster:8082/druid/v3/demoServlet`** +**`http://IPAddressDruidCoordinator:8082/druid/v3/demoServlet`** As you can see from the image below, there are default values in the Dimensions and Granularity fields. Clicking **Execute** will produce a basic query result. ![Demo Query Interface](images/demo/query-1.png) diff --git a/docs/content/Druid-vs-redshift.md b/docs/content/Druid-vs-redshift.md index 52ec6250a66..4fe06586467 100644 --- a/docs/content/Druid-vs-redshift.md +++ b/docs/content/Druid-vs-redshift.md @@ -19,7 +19,7 @@ It’s write semantics aren’t as fluid and does not support joins. ParAccel is ###Data distribution model -Druid’s data distribution, is segment based which exists on highly available "deep" storage, like S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of compute nodes does not result in data loss because new compute nodes can always be brought up by reading data from "deep" storage. +Druid’s data distribution, is segment based which exists on highly available "deep" storage, like S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of historical nodes does not result in data loss because new historical nodes can always be brought up by reading data from "deep" storage. To contrast, ParAccel’s data distribution model is hash-based. Expanding the cluster requires re-hashing the data across the nodes, making it difficult to perform without taking downtime. Amazon’s Redshift works around this issue with a multi-step process: diff --git a/docs/content/Historical.md b/docs/content/Historical.md new file mode 100644 index 00000000000..bf801415b14 --- /dev/null +++ b/docs/content/Historical.md @@ -0,0 +1,40 @@ +--- +layout: doc_page +--- +Historical +======= + +Historical nodes load up historical segments and expose them for querying. + +Loading and Serving Segments +---------------------------- + +Each historical node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Historical nodes do not communicate directly with each other or with the coordinator nodes but instead rely on Zookeeper for coordination. + +The [Coordinator](Coordinator.html) node is responsible for assigning new segments to historical nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a historical node. For more information on how the coordinator assigns segments to historical nodes, please see [Coordinator](Coordinator.html). + +When a historical node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the historical node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [Segments](Segments.html). Once a historical node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying. + +Loading and Serving Segments From Cache +--------------------------------------- + +Recall that when a historical node notices a new segment entry in its load queue path, the historical node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the historical node will directly read the segment binary files from disk and load the segment. + +The segment cache is also leveraged when a historical node is first started. On startup, a historical node will search through its cache directory and immediately load and serve all segments that are found. This feature allows historical nodes to be queried as soon they come online. + +Querying Segments +----------------- + +Please see [Querying](Querying.html) for more information on querying historical nodes. + +For every query that a historical node services, it will log the query and report metrics on the time taken to run the query. + +Running +------- + +Historical nodes can be run using the `io.druid.cli.Main` class with program arguments "server historical". + +Configuration +------------- + +See [Configuration](Configuration.html). diff --git a/docs/content/Indexing-Service.md b/docs/content/Indexing-Service.md index e4bb5252310..271c4bb09c6 100644 --- a/docs/content/Indexing-Service.md +++ b/docs/content/Indexing-Service.md @@ -57,7 +57,7 @@ http://:/static/console.html The coordinator retrieves worker setup metadata from the Druid [MySQL](MySQL.html) config table. This metadata contains information about the version of workers to create, the maximum and minimum number of workers in the cluster at one time, and additional information required to automatically create workers. -Tasks are assigned to workers by creating entries under specific /tasks paths associated with a worker, similar to how the Druid master node assigns segments to compute nodes. See [Worker Configuration](Indexing-Service#configuration-1). Once a worker picks up a task, it deletes the task entry and announces a task status under a /status path associated with the worker. Tasks are submitted to a worker until the worker hits capacity. If all workers in a cluster are at capacity, the indexer coordinator node automatically creates new worker resources. +Tasks are assigned to workers by creating entries under specific /tasks paths associated with a worker, similar to how the Druid coordinator node assigns segments to historical nodes. See [Worker Configuration](Indexing-Service#configuration-1). Once a worker picks up a task, it deletes the task entry and announces a task status under a /status path associated with the worker. Tasks are submitted to a worker until the worker hits capacity. If all workers in a cluster are at capacity, the indexer coordinator node automatically creates new worker resources. #### Autoscaling @@ -180,7 +180,7 @@ Worker nodes require [basic service configuration](https://github.com/metamx/dru -Ddruid.indexer.taskDir=/mnt/persistent/task/ -Ddruid.indexer.hadoopWorkingPath=/tmp/druid-indexing --Ddruid.worker.masterService=druid:sample_cluster:indexer +-Ddruid.worker.coordinatorService=druid:sample_cluster:indexer -Ddruid.indexer.fork.hostpattern=:%d -Ddruid.indexer.fork.startport=8080 @@ -205,7 +205,7 @@ Many of the configurations for workers are similar to those for basic service co |`druid.worker.version`|Version identifier for the worker.|0| |`druid.worker.capacity`|Maximum number of tasks the worker can accept.|1| |`druid.indexer.threads`|Number of processing threads per worker.|1| -|`druid.worker.masterService`|Name of the indexer coordinator used for service discovery.|none| +|`druid.worker.coordinatorService`|Name of the indexer coordinator used for service discovery.|none| |`druid.indexer.fork.hostpattern`|The format of the host name.|none| |`druid.indexer.fork.startport`|Port in which child JVM starts from.|none| |`druid.indexer.fork.opts`|JVM options for child JVMs.|none| diff --git a/docs/content/Loading-Your-Data.md b/docs/content/Loading-Your-Data.md index 30dc1df7f84..ee35dce10ae 100644 --- a/docs/content/Loading-Your-Data.md +++ b/docs/content/Loading-Your-Data.md @@ -11,8 +11,8 @@ Each type of node needs its own config file and directory, so create them as sub ```bash mkdir config mkdir config/realtime -mkdir config/master -mkdir config/compute +mkdir config/coordinator +mkdir config/historical mkdir config/broker ``` @@ -211,7 +211,7 @@ GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; CREATE database druid; ``` -The [Master](Master.html) node will create the tables it needs based on its configuration. +The [Coordinator](Coordinator.html) node will create the tables it needs based on its configuration. ### Make sure you have ZooKeeper Running ### @@ -232,11 +232,11 @@ cp conf/zoo_sample.cfg conf/zoo.cfg cd .. ``` -### Launch a Master Node ### +### Launch a Coordinator Node ### -If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [Realtime](Realtime.html) node, we use 8081 for the [Master](Master.html). +If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [Realtime](Realtime.html) node, we use 8081 for the [Coordinator](Coordinator.html). -1. Setup a configuration file called config/master/runtime.properties similar to: +1. Setup a configuration file called config/coordinator/runtime.properties similar to: ```properties druid.host=0.0.0.0:8081 @@ -251,7 +251,7 @@ If you've already setup a realtime node, be aware that although you can run mult # emitting, opaque marker druid.service=example - druid.master.startDelay=PT60s + druid.coordinator.startDelay=PT60s druid.request.logging.dir=/tmp/example/log druid.realtime.specFile=realtime.spec com.metamx.emitter.logging=true @@ -281,17 +281,17 @@ If you've already setup a realtime node, be aware that although you can run mult druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache ``` -2. Launch the [Master](Master.html) node +2. Launch the [Coordinator](Coordinator.html) node ```bash java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ - -classpath lib/*:config/master \ - com.metamx.druid.http.MasterMain + -classpath lib/*:config/coordinator \ + com.metamx.druid.http.CoordinatorMain ``` -### Launch a Compute/Historical Node ### +### Launch a Historical Node ### -1. Create a configuration file in config/compute/runtime.properties similar to: +1. Create a configuration file in config/historical/runtime.properties similar to: ```properties druid.host=0.0.0.0:8082 @@ -338,12 +338,12 @@ If you've already setup a realtime node, be aware that although you can run mult druid.storage.local=true ``` -2. Launch the compute node: +2. Launch the historical node: ```bash java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ - -classpath lib/*:config/compute \ - com.metamx.druid.http.ComputeMain + -classpath lib/*:config/historical \ + io.druid.cli.Main server historical ``` ### Create a File of Records ### @@ -360,7 +360,7 @@ We can use the same records we have been, in a file called records.json: ### Run the Hadoop Job ### -Now its time to run the Hadoop [Batch-ingestion](Batch-ingestion.html) job, HadoopDruidIndexer, which will fill a historical [Compute](Compute.html) node with data. First we'll need to configure the job. +Now its time to run the Hadoop [Batch-ingestion](Batch-ingestion.html) job, HadoopDruidIndexer, which will fill a historical [Historical](Historical.html) node with data. First we'll need to configure the job. 1. Create a config called batchConfig.json similar to: diff --git a/docs/content/Master.md b/docs/content/Master.md deleted file mode 100644 index d6ba2e78998..00000000000 --- a/docs/content/Master.md +++ /dev/null @@ -1,136 +0,0 @@ ---- -layout: doc_page ---- -Master -====== - -The Druid master node is primarily responsible for segment management and distribution. More specifically, the Druid master node communicates to compute nodes to load or drop segments based on configurations. The Druid master is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load. - -The Druid master runs periodically and the time between each run is a configurable parameter. Each time the Druid master runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the broker and compute nodes, the Druid master maintains a connection to a Zookeeper cluster for current cluster information. The master also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled. - -Before any unassigned segments are serviced by compute nodes, the available compute nodes 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 nodes with least capacity to maintain a level of balance between nodes. The master does not directly communicate with a compute node when assigning it a new segment; instead the master creates some temporary information about the new segment under load queue path of the compute node. Once this request is seen, the compute node will load the segment and begin servicing it. - -Rules ------ - -Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different compute node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The master loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The master will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule - -For more information on rules, see [Rule Configuration](Rule-Configuration.html). - -Cleaning Up Segments --------------------- - -Each run, the Druid master compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped. - -Segment Availability --------------------- - -If a compute node restarts or becomes unavailable for any reason, the Druid master will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other compute nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime represents a period of time in which the master will not reassign a dropped segment. Hence, if a compute node becomes unavailable and available again within a short period of time, the compute node will start up and serve segments from its cache without any those segments being reassigned across the cluster. - -Balancing Segment Load ----------------------- - -To ensure an even distribution of segments across compute nodes in the cluster, the master node will find the total size of all segments being served by every compute node each time the master runs. For every compute node tier in the cluster, the master node will determine the compute node with the highest utilization and the compute node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized node to the lowest utilized node. There is a configurable limit on the number of segments that can be moved from one node to another each time the master runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased. - -HTTP Endpoints --------------- - -The master node exposes several HTTP endpoints for interactions. - -### GET - -* `/info/master` - - Returns the current true master of the cluster as a JSON object. - -* `/info/cluster` - - Returns JSON data about every node and segment in the cluster. Information about each node and each segment on each node will be returned. - -* `/info/servers` - - Returns information about servers in the cluster. Set the `?full` query parameter to get full metadata about all servers and their segments in the cluster. - -* `/info/servers/{serverName}` - - Returns full metadata about a specific server. - -* `/info/servers/{serverName}/segments` - - Returns a list of all segments for a server. Set the `?full` query parameter to get all segment metadata included - -* `/info/servers/{serverName}/segments/{segmentId}` - - Returns full metadata for a specific segment. - -* `/info/segments` - - Returns all segments in the cluster as a list. Set the `?full` flag to get all metadata about segments in the cluster - -* `/info/segments/{segmentId}` - - Returns full metadata for a specific segment - -* `/info/datasources` - - Returns a list of datasources in the cluster. Set the `?full` flag to get all metadata for every datasource in the cluster - -* `/info/datasources/{dataSourceName}` - - Returns full metadata for a datasource - -* `/info/datasources/{dataSourceName}/segments` - - Returns a list of all segments for a datasource. Set the `?full` flag to get full segment metadata for a datasource - -* `/info/datasources/{dataSourceName}/segments/{segmentId}` - - Returns full segment metadata for a specific segment - -* `/info/rules` - - Returns all rules for all data sources in the cluster including the default datasource. - -* `/info/rules/{dataSourceName}` - - Returns all rules for a specified datasource - -### POST - -* `/info/rules/{dataSourceName}` - - POST with a list of rules in JSON form to update rules. - -The Master Console ------------------- - -The Druid master exposes a web GUI for displaying cluster information and rule configuration. After the master starts, the console can be accessed at http://HOST:PORT/static/. There exists a full cluster view, as well as views for individual compute nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table. - -The master console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited. - -FAQ ---- - -1. **Do clients ever contact the master node?** - - The master is not involved in a query. - - Compute nodes never directly contact the master node. The Druid master tells the compute nodes to load/drop data via Zookeeper, but the compute nodes are completely unaware of the master. - - Brokers also never contact the master. Brokers base their understanding of the data topology on metadata exposed by the compute nodes via ZK and are completely unaware of the master. - -2. **Does it matter if the master node starts up before or after other processes?** - - No. If the Druid master is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the master node can be started up at any time, and after a configurable delay, will start running master tasks. - - This also means that if you have a working cluster and all of your masters die, the cluster will continue to function, it just won’t experience any changes to its data topology. - -Running -------- - -Master nodes can be run using the `com.metamx.druid.http.MasterMain` class. - -Configuration -------------- - -See [Configuration](Configuration.html). diff --git a/docs/content/MySQL.md b/docs/content/MySQL.md index b28bd6951ff..e81703792cf 100644 --- a/docs/content/MySQL.md +++ b/docs/content/MySQL.md @@ -8,7 +8,7 @@ Segments Table This is dictated by the `druid.database.segmentTable` property (Note that these properties are going to change in the next stable version after 0.4.12). -This table stores metadata about the segments that are available in the system. The table is polled by the [Master](Master.html) to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes. +This table stores metadata about the segments that are available in the system. The table is polled by the [Coordinator](Coordinator.html) to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes. The `used` column is a boolean "tombstone". A 1 means that the segment should be "used" by the cluster (i.e. it should be loaded and available for requests). A 0 means that the segment should not be actively loaded into the cluster. We do this as a means of removing segments from the cluster without actually removing their metadata (which allows for simpler rolling back if that is ever an issue). @@ -36,7 +36,7 @@ Note that the format of this blob can and will change from time-to-time. Rule Table ---------- -The rule table is used to store the various rules about where segments should land. These rules are used by the [Master](Master.html) when making segment (re-)allocation decisions about the cluster. +The rule table is used to store the various rules about where segments should land. These rules are used by the [Coordinator](Coordinator.html) when making segment (re-)allocation decisions about the cluster. Config Table ------------ diff --git a/docs/content/Querying-your-data.md b/docs/content/Querying-your-data.md index 379be367594..da0fc007e5c 100644 --- a/docs/content/Querying-your-data.md +++ b/docs/content/Querying-your-data.md @@ -3,7 +3,7 @@ layout: doc_page --- # Setup # -Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading-Your-Data.html) we setup a [Realtime](Realtime.html), [Compute](Compute.html) and [Master](Master.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. +Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading-Your-Data.html) we setup a [Realtime](Realtime.html), [Historical](Historical.html) and [Coordinator](Coordinator.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. ## Booting a Broker Node ## @@ -65,16 +65,16 @@ Before we start querying druid, we're going to finish setting up a complete clus com.metamx.druid.http.BrokerMain ``` -## Booting a Master Node ## +## Booting a Coordinator Node ## -1. Setup a config file at config/master/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818870](https://gist.github.com/rjurney/5818870) +1. Setup a config file at config/coordinator/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818870](https://gist.github.com/rjurney/5818870) -2. Run the master node: +2. Run the coordinator node: ```bash java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ - -classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/master \ - com.metamx.druid.http.MasterMain + -classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/coordinator \ + io.druid.cli.Main server coordinator ``` ## Booting a Realtime Node ## @@ -92,15 +92,15 @@ Before we start querying druid, we're going to finish setting up a complete clus com.metamx.druid.realtime.RealtimeMain ``` -## Booting a Compute Node ## +## Booting a historical node ## -1. Setup a config file at config/compute/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818885](https://gist.github.com/rjurney/5818885) -2. Run the compute node: +1. Setup a config file at config/historical/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818885](https://gist.github.com/rjurney/5818885) +2. Run the historical node: ```bash java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ - -classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/compute \ - com.metamx.druid.http.ComputeMain + -classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/historical \ + io.druid.cli.Main server historical ``` # Querying Your Data # @@ -109,7 +109,7 @@ Now that we have a complete cluster setup on localhost, we need to load data. To ## Querying Different Nodes ## -As a shared-nothing system, there are three ways to query druid, against the [Realtime](Realtime.html), [Compute](Compute.html) or [Broker](Broker.html) node. Querying a Realtime node returns only realtime data, querying a compute node returns only historical segments. Querying the broker will query both realtime and compute segments and compose an overall result for the query. This is the normal mode of operation for queries in druid. +As a shared-nothing system, there are three ways to query druid, against the [Realtime](Realtime.html), [Historical](Historical.html) or [Broker](Broker.html) node. Querying a Realtime node returns only realtime data, querying a historical node returns only historical segments. Querying the broker will query both realtime and historical segments and compose an overall result for the query. This is the normal mode of operation for queries in druid. ### Construct a Query ### @@ -148,7 +148,7 @@ See our result: } ] ``` -### Querying the Compute Node ### +### Querying the historical node ### Run the query against port 8082: ```bash diff --git a/docs/content/Querying.md b/docs/content/Querying.md index a09d969f76d..13f7b5a91ce 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -4,7 +4,7 @@ layout: doc_page Querying ======== -Queries are made using an HTTP REST style request to a [Broker](Broker.html), [Compute](Compute.html), or [Realtime](Realtime.html) node. The query is expressed in JSON and each of these node types expose the same REST query interface. +Queries are made using an HTTP REST style request to a [Broker](Broker.html), [Historical](Historical.html), or [Realtime](Realtime.html) node. The query is expressed in JSON and each of these node types expose the same REST query interface. We start by describing an example query with additional comments that mention possible variations. Query operators are also summarized in a table below. diff --git a/docs/content/Realtime.md b/docs/content/Realtime.md index 459f7d20175..e2f85bbaeec 100644 --- a/docs/content/Realtime.md +++ b/docs/content/Realtime.md @@ -4,7 +4,7 @@ layout: doc_page Realtime ======== -Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data they’ve collected over some span of time and hand these segments off to [Compute](Compute.html) nodes. +Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data they’ve collected over some span of time and hand these segments off to [Historical](Historical.html) nodes. Running ------- diff --git a/docs/content/Rule-Configuration.md b/docs/content/Rule-Configuration.md index 80e4b649422..cba6bdd3924 100644 --- a/docs/content/Rule-Configuration.md +++ b/docs/content/Rule-Configuration.md @@ -1,7 +1,7 @@ --- layout: doc_page --- -Note: It is recommended that the master console is used to configure rules. However, the master node does have HTTP endpoints to programmatically configure rules. +Note: It is recommended that the coordinator console is used to configure rules. However, the coordinator node does have HTTP endpoints to programmatically configure rules. Load Rules ---------- @@ -22,7 +22,7 @@ Interval load rules are of the form: * `type` - this should always be "loadByInterval" * `interval` - A JSON Object representing ISO-8601 Intervals -* `tier` - the configured compute node tier +* `tier` - the configured historical node tier ### Period Load Rule @@ -38,7 +38,7 @@ Period load rules are of the form: * `type` - this should always be "loadByPeriod" * `period` - A JSON Object representing ISO-8601 Periods -* `tier` - the configured compute node tier +* `tier` - the configured historical node tier The interval of a segment will be compared against the specified period. The rule matches if the period overlaps the interval. diff --git a/docs/content/Segments.md b/docs/content/Segments.md index 8f1af704df6..644fea72579 100644 --- a/docs/content/Segments.md +++ b/docs/content/Segments.md @@ -4,7 +4,7 @@ layout: doc_page Segments ======== -Segments are the fundamental structure to store data in Druid. [Compute](Compute.html) and [Realtime](Realtime.html) nodes load and serve segments for querying. To construct segments, Druid will always shard data by a time partition. Data may be further sharded based on dimension cardinality and row count. +Segments are the fundamental structure to store data in Druid. [Historical](Historical.html) and [Realtime](Realtime.html) nodes load and serve segments for querying. To construct segments, Druid will always shard data by a time partition. Data may be further sharded based on dimension cardinality and row count. The latest Druid segment version is `v9`. diff --git a/docs/content/Stand-Alone-With-Riak-CS.md b/docs/content/Stand-Alone-With-Riak-CS.md index 1efb0ea3398..3b9cc1a6d09 100644 --- a/docs/content/Stand-Alone-With-Riak-CS.md +++ b/docs/content/Stand-Alone-With-Riak-CS.md @@ -22,9 +22,9 @@ We started with a minimal CentOS installation but you can use any other compatib 1. A Kafka Broker 1. A single-node Zookeeper ensemble 1. A single-node Riak-CS cluster -1. A Druid [Master](Master.html) +1. A Druid [Coordinator](Coordinator.html) 1. A Druid [Broker](Broker.html) -1. A Druid [Compute](Compute.html) +1. A Druid [Historical](Historical.html) 1. A Druid [Realtime](Realtime.html) This just walks through getting the relevant software installed and running. You will then need to configure the [Realtime](Realtime.html) node to take in your data. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index cb5c463ba76..7f284ce9eaf 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -94,14 +94,14 @@ mkdir config If you are interested in learning more about Druid configuration files, check out this [link](https://github.com/metamx/druid/wiki/Configuration). Many aspects of Druid are customizable. For the purposes of this tutorial, we are going to use default values for most things. -### Start a Master Node ### +### Start a Coordinator Node ### -Master nodes are in charge of load assignment and distribution. Master nodes monitor the status of the cluster and command compute nodes to assign and drop segments. +Coordinator nodes are in charge of load assignment and distribution. Coordinator nodes monitor the status of the cluster and command historical nodes to assign and drop segments. -To create the master config file: +To create the coordinator config file: ``` -mkdir config/master +mkdir config/coordinator ``` Under the directory we just created, create the file `runtime.properties` with the following contents: @@ -109,7 +109,7 @@ Under the directory we just created, create the file `runtime.properties` with t ``` druid.host=127.0.0.1:8082 druid.port=8082 -druid.service=master +druid.service=coordinator # logging com.metamx.emitter.logging=true @@ -132,24 +132,24 @@ druid.database.connectURI=jdbc:mysql://localhost:3306/druid druid.database.ruleTable=rules druid.database.configTable=config -# master runtime configs -druid.master.startDelay=PT60S +# coordinator runtime configs +druid.coordinator.startDelay=PT60S ``` -To start the master node: +To start the coordinator node: ```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/master com.metamx.druid.http.MasterMain +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/coordinator io.druid.cli.Main server coordinator ``` -### Start a Compute Node ### +### Start a historical node ### -Compute nodes are the workhorses of a cluster and are in charge of loading historical segments and making them available for queries. Our Wikipedia segment will be downloaded by a compute node. +historical nodes are the workhorses of a cluster and are in charge of loading historical segments and making them available for queries. Our Wikipedia segment will be downloaded by a historical node. -To create the compute config file: +To create the historical config file: ``` -mkdir config/compute +mkdir config/historical ``` Under the directory we just created, create the file `runtime.properties` with the following contents: @@ -157,7 +157,7 @@ Under the directory we just created, create the file `runtime.properties` with t ``` druid.host=127.0.0.1:8081 druid.port=8081 -druid.service=compute +druid.service=historical # logging com.metamx.emitter.logging=true @@ -185,15 +185,15 @@ druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache druid.server.maxSize=100000000 ``` -To start the compute node: +To start the historical node: ```bash -java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/compute com.metamx.druid.http.ComputeMain +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/historical io.druid.cli.Main server historical ``` ### Start a Broker Node ### -Broker nodes are responsible for figuring out which compute and/or realtime nodes correspond to which queries. They also merge partial results from these nodes in a scatter/gather fashion. +Broker nodes are responsible for figuring out which historical and/or realtime nodes correspond to which queries. They also merge partial results from these nodes in a scatter/gather fashion. To create the broker config file: @@ -229,7 +229,7 @@ java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/ ## Loading the Data ## -The MySQL dependency we introduced earlier on contains a 'segments' table that contains entries for segments that should be loaded into our cluster. The Druid master compares this table with segments that already exist in the cluster to determine what should be loaded and dropped. To load our wikipedia segment, we need to create an entry in our MySQL segment table. +The MySQL dependency we introduced earlier on contains a 'segments' table that contains entries for segments that should be loaded into our cluster. The Druid coordinator compares this table with segments that already exist in the cluster to determine what should be loaded and dropped. To load our wikipedia segment, we need to create an entry in our MySQL segment table. Usually, when new segments are created, these MySQL entries are created directly so you never have to do this by hand. For this tutorial, we can do this manually by going back into MySQL and issuing: @@ -238,14 +238,14 @@ use druid; INSERT INTO segments (id, dataSource, created_date, start, end, partitioned, version, used, payload) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}'); ``` -If you look in your master node logs, you should, after a maximum of a minute or so, see logs of the following form: +If you look in your coordinator node logs, you should, after a maximum of a minute or so, see logs of the following form: ``` -2013-08-08 22:48:41,967 INFO [main-EventThread] com.metamx.druid.master.LoadQueuePeon - Server[/druid/loadQueue/127.0.0.1:8081] done processing [/druid/loadQueue/127.0.0.1:8081/wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z] +2013-08-08 22:48:41,967 INFO [main-EventThread] com.metamx.druid.coordinator.LoadQueuePeon - Server[/druid/loadQueue/127.0.0.1:8081] done processing [/druid/loadQueue/127.0.0.1:8081/wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z] 2013-08-08 22:48:41,969 INFO [ServerInventoryView-0] com.metamx.druid.client.SingleServerInventoryView - Server[127.0.0.1:8081] added segment[wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z] ``` -When the segment completes downloading and ready for queries, you should see the following message on your compute node logs: +When the segment completes downloading and ready for queries, you should see the following message on your historical node logs: ``` 2013-08-08 22:48:41,959 INFO [ZkCoordinator-0] com.metamx.druid.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z] at path[/druid/segments/127.0.0.1:8081/2013-08-08T22:48:41.959Z] diff --git a/docs/content/ZooKeeper.md b/docs/content/ZooKeeper.md index 771a60f41e0..e38f272070f 100644 --- a/docs/content/ZooKeeper.md +++ b/docs/content/ZooKeeper.md @@ -3,9 +3,9 @@ layout: doc_page --- Druid uses ZooKeeper (ZK) for management of current cluster state. The operations that happen over ZK are -1. [Master](Master.html) leader election -2. Segment "publishing" protocol from [Compute](Compute.html) and [Realtime](Realtime.html) -3. Segment load/drop protocol between [Master](Master.html) and [Compute](Compute.html) +1. [Coordinator](Coordinator.html) leader election +2. Segment "publishing" protocol from [Historical](Historical.html) and [Realtime](Realtime.html) +3. Segment load/drop protocol between [Coordinator](Coordinator.html) and [Historical](Historical.html) ### Property Configuration @@ -24,7 +24,7 @@ druid.zk.paths.propertiesPath=${druid.zk.paths.base}/properties druid.zk.paths.announcementsPath=${druid.zk.paths.base}/announcements druid.zk.paths.servedSegmentsPath=${druid.zk.paths.base}/servedSegments druid.zk.paths.loadQueuePath=${druid.zk.paths.base}/loadQueue -druid.zk.paths.masterPath=${druid.zk.paths.base}/master +druid.zk.paths.coordinatorPath=${druid.zk.paths.base}/coordinator druid.zk.paths.indexer.announcementsPath=${druid.zk.paths.base}/indexer/announcements druid.zk.paths.indexer.tasksPath=${druid.zk.paths.base}/indexer/tasks druid.zk.paths.indexer.statusPath=${druid.zk.paths.base}/indexer/status @@ -37,19 +37,19 @@ NOTE: We also use Curator’s service discovery module to expose some services v druid.zk.paths.discoveryPath ``` -### Master Leader Election +### Coordinator Leader Election We use the Curator LeadershipLatch recipe to do leader election at path ``` -${druid.zk.paths.masterPath}/_MASTER +${druid.zk.paths.coordinatorPath}/_COORDINATOR ``` -### Segment "publishing" protocol from Compute and Realtime +### Segment "publishing" protocol from Historical and Realtime The `announcementsPath` and `servedSegmentsPath` are used for this. -All [Compute](Compute.html) and [Realtime](Realtime.html) nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at +All [Historical](Historical.html) and [Realtime](Realtime.html) nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at ``` ${druid.zk.paths.announcementsPath}/${druid.host} @@ -67,16 +67,16 @@ And as they load up segments, they will attach ephemeral znodes that look like ${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_ ``` -Nodes like the [Master](Master.html) and [Broker](Broker.html) can then watch these paths to see which nodes are currently serving which segments. +Nodes like the [Coordinator](Coordinator.html) and [Broker](Broker.html) can then watch these paths to see which nodes are currently serving which segments. -### Segment load/drop protocol between Master and Compute +### Segment load/drop protocol between Coordinator and Historical The `loadQueuePath` is used for this. -When the [Master](Master.html) decides that a [Compute](Compute.html) node should load or drop a segment, it writes an ephemeral znode to +When the [Coordinator](Coordinator.html) decides that a [Historical](Historical.html) node should load or drop a segment, it writes an ephemeral znode to ``` -${druid.zk.paths.loadQueuePath}/_host_of_compute_node/_segment_identifier +${druid.zk.paths.loadQueuePath}/_host_of_historical_node/_segment_identifier ``` -This node will contain a payload that indicates to the Compute node what it should do with the given segment. When the Compute node is done with the work, it will delete the znode in order to signify to the Master that it is complete. +This node will contain a payload that indicates to the historical node what it should do with the given segment. When the historical node is done with the work, it will delete the znode in order to signify to the Coordinator that it is complete. diff --git a/docs/content/toc.textile b/docs/content/toc.textile index e4ed908a559..e9c8cbe2470 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -43,9 +43,9 @@ h2. Architecture * "Design":./Design.html * "Segments":./Segments.html * Node Types -** "Compute":./Compute.html +** "Historical":./Historical.html ** "Broker":./Broker.html -** "Master":./Master.html +** "Coordinator":./Coordinator.html *** "Rule Configuration":./Rule-Configuration.html ** "Realtime":./Realtime.html *** "Firehose":./Firehose.html diff --git a/examples/bin/ec2/env.sh b/examples/bin/ec2/env.sh deleted file mode 100644 index a0a05719df1..00000000000 --- a/examples/bin/ec2/env.sh +++ /dev/null @@ -1,27 +0,0 @@ -# Setup Oracle Java -sudo apt-get update -sudo add-apt-repository -y ppa:webupd8team/java -sudo apt-get update - -# Setup yes answer to license question -echo debconf shared/accepted-oracle-license-v1-1 select true | sudo debconf-set-selections -echo debconf shared/accepted-oracle-license-v1-1 seen true | sudo debconf-set-selections -sudo apt-get -y -q install oracle-java7-installer - -# Automated Kafka setup -curl http://static.druid.io/artifacts/kafka-0.7.2-incubating-bin.tar.gz -o /tmp/kafka-0.7.2-incubating-bin.tar.gz -tar -xvzf /tmp/kafka-0.7.2-incubating-bin.tar.gz -cd kafka-0.7.2-incubating-bin -cat config/zookeeper.properties -nohup bin/zookeeper-server-start.sh config/zookeeper.properties 2>&1 > /dev/null & -# in a new console -nohup bin/kafka-server-start.sh config/server.properties 2>&1 > /dev/null & - -# Install dependencies - mysql must be built from source, as the 12.04 apt-get hangs -export DEBIAN_FRONTEND=noninteractive -sudo debconf-set-selections <<< 'mysql-server-5.5 mysql-server/root_password password diurd' -sudo debconf-set-selections <<< 'mysql-server-5.5 mysql-server/root_password_again password diurd' -sudo apt-get -q -y -V --force-yes --reinstall install mysql-server-5.5 - -echo "ALL DONE with druid environment setup! Hit CTRL-C to proceed." -exit 0 diff --git a/examples/bin/ec2/run.sh b/examples/bin/ec2/run.sh deleted file mode 100644 index df7438ccf5b..00000000000 --- a/examples/bin/ec2/run.sh +++ /dev/null @@ -1,22 +0,0 @@ -# Is localhost expected with multi-node? -mysql -u root -pdiurd -e "GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; CREATE database druid;" 2>&1 > /dev/null - -tar -xvzf druid-services-*-bin.tar.gz 2>&1 > /dev/null -cd druid-services-* 2>&1 > /dev/null - -mkdir logs 2>&1 > /dev/null - -# Now start a realtime node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime io.druid.cli.Main server realtime 2>&1 > logs/realtime.log & - -# And a master node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master io.druid.cli.Main server coordinator 2>&1 > logs/master.log & - -# And a compute node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute io.druid.cli.Main server historical 2>&1 > logs/compute.log & - -# And a broker node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/broker io.druid.cli.Main server broker 2>&1 > logs/broker.log & - -echo "Hit CTRL-C to continue..." -exit 0 diff --git a/examples/cassandra/README.md b/examples/cassandra/README.md index 4fc71784057..7a8f5f99195 100644 --- a/examples/cassandra/README.md +++ b/examples/cassandra/README.md @@ -2,7 +2,7 @@ Druid can use Cassandra as a deep storage mechanism. Segments and their metadata are stored in Cassandra in two tables: `index_storage` and `descriptor_storage`. Underneath the hood, the Cassandra integration leverages Astyanax. The index storage table is a [Chunked Object](https://github.com/Netflix/astyanax/wiki/Chunked-Object-Store) repository. It contains -compressed segments for distribution to compute nodes. Since segments can be large, the Chunked Object storage allows the integration to multi-thread +compressed segments for distribution to historical nodes. Since segments can be large, the Chunked Object storage allows the integration to multi-thread the write to Cassandra, and spreads the data across all the nodes in a cluster. The descriptor storage table is a normal C* table that stores the segment metadatak. diff --git a/examples/config/coordinator/runtime.properties b/examples/config/coordinator/runtime.properties index 1dca923d921..59e328216b8 100644 --- a/examples/config/coordinator/runtime.properties +++ b/examples/config/coordinator/runtime.properties @@ -1,5 +1,5 @@ druid.host=localhost -druid.service=master +druid.service=coordinator druid.port=8082 druid.zk.service.host=localhost diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index e0fb60d6a25..23b341ab275 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -1,5 +1,5 @@ druid.host=localhost -druid.service=compute +druid.service=historical druid.port=8081 druid.zk.service.host=localhost diff --git a/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java b/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java index 5303638f5aa..72a983d6884 100644 --- a/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java +++ b/indexing-service/src/main/java/io/druid/guice/IndexingServiceModuleHelper.java @@ -20,8 +20,8 @@ package io.druid.guice; import com.google.inject.Binder; -import io.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; /** */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java index 7ecad840a54..8bb23918b01 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClient.java @@ -21,7 +21,7 @@ package io.druid.indexing.common.actions; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.TaskStorage; +import io.druid.indexing.overlord.TaskStorage; import java.io.IOException; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClientFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClientFactory.java index 0dfc94b2e08..db89dc5c895 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClientFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/LocalTaskActionClientFactory.java @@ -21,7 +21,7 @@ package io.druid.indexing.common.actions; import com.google.inject.Inject; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.TaskStorage; +import io.druid.indexing.overlord.TaskStorage; /** */ diff --git a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java index 7907af6402a..b7e78e0c2be 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/actions/TaskActionToolbox.java @@ -25,9 +25,9 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.IndexerDBCoordinator; -import io.druid.indexing.coordinator.TaskLockbox; -import io.druid.indexing.coordinator.TaskQueue; +import io.druid.indexing.overlord.IndexerDBCoordinator; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskQueue; import io.druid.timeline.DataSegment; import java.util.List; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index d9d2a58d6a9..bb30c351732 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -271,7 +271,7 @@ public class RealtimeIndexTask extends AbstractTask // NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means // NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip and - // NOTE: descriptor.json to mismatch, or it can cause compute nodes to load different instances of the + // NOTE: descriptor.json to mismatch, or it can cause historical nodes to load different instances of the // NOTE: "same" segment. realtimePlumberSchool.setDataSegmentPusher(toolbox.getSegmentPusher()); realtimePlumberSchool.setConglomerate(toolbox.getQueryRunnerFactoryConglomerate()); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java index cd14e3e84b7..03ad653abc3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java @@ -22,8 +22,8 @@ package io.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.io.InputSupplier; import com.google.inject.Inject; -import io.druid.indexing.coordinator.TaskMaster; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskRunner; import java.io.IOException; import java.io.InputStream; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/DbTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/DbTaskStorage.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java index 5589533fe49..5100f6ca613 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/DbTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/DbTaskStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index e18a806f338..1e69516db95 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; @@ -45,7 +45,7 @@ import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.tasklogs.TaskLogPusher; import io.druid.indexing.common.tasklogs.TaskLogStreamer; -import io.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.server.DruidNode; import org.apache.commons.io.FileUtils; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java similarity index 94% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunnerFactory.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java index 513910e6734..4a090f4fa47 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/ForkingTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import io.druid.guice.annotations.Self; import io.druid.indexing.common.tasklogs.TaskLogPusher; -import io.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.server.DruidNode; import java.util.Properties; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/HeapMemoryTaskStorage.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java index 1178028ce48..7ebaa69dab9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/HeapMemoryTaskStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Optional; import com.google.common.base.Preconditions; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/IndexerDBCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/IndexerDBCoordinator.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java index 51042138473..b3056c274ac 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/IndexerDBCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/IndexerDBCoordinator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 23767a87cb3..fb212a164d9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; @@ -46,8 +46,8 @@ import io.druid.curator.cache.PathChildrenCacheFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.tasklogs.TaskLogStreamer; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.server.initialization.ZkPathsConfig; @@ -86,7 +86,7 @@ import java.util.concurrent.TimeUnit; *

* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will * fail. The RemoteTaskRunner depends on another component to create additional worker resources. - * For example, {@link io.druid.indexing.coordinator.scaling.ResourceManagementScheduler} can take care of these duties. + * For example, {@link io.druid.indexing.overlord.scaling.ResourceManagementScheduler} can take care of these duties. *

* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker. *

diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java similarity index 93% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java index 947d0d22f1d..aa7e1720e89 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerFactory.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; @@ -25,8 +25,8 @@ import com.google.inject.Inject; import com.metamx.http.client.HttpClient; import io.druid.curator.cache.SimplePathChildrenCacheFactory; import io.druid.guice.annotations.Global; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java index 692ba50ae6b..1c1dc7a17a9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.util.concurrent.SettableFuture; import io.druid.indexing.common.TaskStatus; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkQueue.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkQueue.java index ebdffd05526..33705c220b5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkQueue.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import org.joda.time.DateTime; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskExistsException.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskExistsException.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskExistsException.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskExistsException.java index a670f8640f9..3b752e086bb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskExistsException.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskExistsException.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; public class TaskExistsException extends RuntimeException { diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskLockbox.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskLockbox.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java index 33a8a2a013f..3dc024530b3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskLockbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskLockbox.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Function; import com.google.common.base.Objects; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java index ab8d410574f..e079a94551b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskMaster.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -32,9 +32,9 @@ import io.druid.guice.annotations.Self; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.exec.TaskConsumer; -import io.druid.indexing.coordinator.scaling.ResourceManagementScheduler; -import io.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import io.druid.indexing.overlord.exec.TaskConsumer; +import io.druid.indexing.overlord.scaling.ResourceManagementScheduler; +import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerFactory; import io.druid.server.DruidNode; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskQueue.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskQueue.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java index d58d636e78a..e033ff60ce9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskQueue.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskQueue.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Optional; import com.google.common.base.Preconditions; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java index e7174d9d680..6509c975cdf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.util.concurrent.ListenableFuture; import io.druid.indexing.common.TaskStatus; @@ -27,7 +27,7 @@ import java.util.Collection; import java.util.List; /** - * Interface for handing off tasks. Used by a {@link io.druid.indexing.coordinator.exec.TaskConsumer} to + * Interface for handing off tasks. Used by a {@link io.druid.indexing.overlord.exec.TaskConsumer} to * run tasks that have been locked. */ public interface TaskRunner diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerFactory.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerFactory.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerFactory.java index 73bd9c90ffe..f29102855d8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerFactory.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; public interface TaskRunnerFactory { diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerWorkItem.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java index 4c474bce5ba..4d4cac6ef70 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ComparisonChain; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorage.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorage.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java index c2f81d588a1..b74dc0d9c1a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorage.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Optional; import io.druid.indexing.common.TaskLock; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorageQueryAdapter.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java index a4829ae48e8..db03ab67ff7 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskStorageQueryAdapter.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Function; import com.google.common.base.Optional; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/ThreadPoolTaskRunner.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index d0ab2611e45..78e4b9e30ec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Function; import com.google.common.base.Throwables; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java similarity index 99% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/ZkWorker.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java index 8580bf16674..edb0c3df685 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ZkWorker.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/EC2AutoScalingStrategyConfig.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/config/EC2AutoScalingStrategyConfig.java index 1c553dfc1ed..ddd52bf4ded 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/EC2AutoScalingStrategyConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.config; +package io.druid.indexing.overlord.config; import org.skife.config.Config; import org.skife.config.Default; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java index d48b6b212f6..6f941874076 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.config; +package io.druid.indexing.overlord.config; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java index aaaecb9c346..3318975c7ca 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/IndexerDbConnectorConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.config; +package io.druid.indexing.overlord.config; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.db.DbConnectorConfig; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/OverlordConfig.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/config/OverlordConfig.java index 0671aabe135..6fe2e07b7fa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/OverlordConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.config; +package io.druid.indexing.overlord.config; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; @@ -29,7 +29,7 @@ import java.util.Set; /** */ -public abstract class IndexerCoordinatorConfig extends ZkPathsConfig +public abstract class OverlordConfig extends ZkPathsConfig { private volatile Set whitelistDatasources = null; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/RemoteTaskRunnerConfig.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/config/RemoteTaskRunnerConfig.java index 59f5159bac0..79c9c2b9186 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/RemoteTaskRunnerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.config; +package io.druid.indexing.overlord.config; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/exec/TaskConsumer.java b/indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/exec/TaskConsumer.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java index 7c40fee27b1..d75cad14f08 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/exec/TaskConsumer.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/exec/TaskConsumer.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.exec; +package io.druid.indexing.overlord.exec; import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; @@ -31,8 +31,8 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.TaskQueue; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunner; public class TaskConsumer implements Runnable { diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java similarity index 82% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java index b8a6f679df8..96c66ba8dde 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.http; +package io.druid.indexing.overlord.http; import com.google.inject.Inject; import io.druid.common.config.JacksonConfigManager; import io.druid.indexing.common.tasklogs.TaskLogStreamer; -import io.druid.indexing.coordinator.TaskMaster; -import io.druid.indexing.coordinator.TaskStorageQueryAdapter; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskStorageQueryAdapter; import javax.ws.rs.Path; @@ -31,10 +31,10 @@ import javax.ws.rs.Path; */ @Deprecated @Path("/mmx/merger/v1") -public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource +public class OldOverlordResource extends OverlordResource { @Inject - public OldIndexerCoordinatorResource( + public OldOverlordResource( TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, TaskLogStreamer taskLogStreamer, diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/OverlordRedirectInfo.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java similarity index 94% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/http/OverlordRedirectInfo.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java index e9b93aa73e5..7238610e6e8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/OverlordRedirectInfo.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordRedirectInfo.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.http; +package io.druid.indexing.overlord.http; import com.google.common.base.Throwables; import com.google.inject.Inject; -import io.druid.indexing.coordinator.TaskMaster; +import io.druid.indexing.overlord.TaskMaster; import io.druid.server.http.RedirectInfo; import java.net.URL; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/IndexerCoordinatorResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java similarity index 95% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/http/IndexerCoordinatorResource.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 9a2638e8d67..2ffc124a5bb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/http/IndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.http; +package io.druid.indexing.overlord.http; import com.google.common.base.Function; import com.google.common.base.Optional; @@ -32,13 +32,13 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionHolder; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.tasklogs.TaskLogStreamer; -import io.druid.indexing.coordinator.TaskMaster; -import io.druid.indexing.coordinator.TaskQueue; -import io.druid.indexing.coordinator.TaskRunner; -import io.druid.indexing.coordinator.TaskRunnerWorkItem; -import io.druid.indexing.coordinator.TaskStorageQueryAdapter; -import io.druid.indexing.coordinator.scaling.ResourceManagementScheduler; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TaskRunnerWorkItem; +import io.druid.indexing.overlord.TaskStorageQueryAdapter; +import io.druid.indexing.overlord.scaling.ResourceManagementScheduler; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.timeline.DataSegment; import javax.ws.rs.Consumes; @@ -59,9 +59,9 @@ import java.util.concurrent.atomic.AtomicReference; /** */ @Path("/druid/indexer/v1") -public class IndexerCoordinatorResource +public class OverlordResource { - private static final Logger log = new Logger(IndexerCoordinatorResource.class); + private static final Logger log = new Logger(OverlordResource.class); private static Function> simplifyTaskFn = new Function>() @@ -92,7 +92,7 @@ public class IndexerCoordinatorResource private AtomicReference workerSetupDataRef = null; @Inject - public IndexerCoordinatorResource( + public OverlordResource( TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, TaskLogStreamer taskLogStreamer, diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingData.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java index 7852e6a73f6..7a0ab258310 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingStrategy.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingStrategy.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingStrategy.java index 19c98c9247f..392919df039 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/AutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/AutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import java.util.List; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java index 8933fb16041..2426d08890a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.amazonaws.services.ec2.AmazonEC2; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; @@ -35,9 +35,9 @@ import com.google.common.collect.Lists; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.guice.annotations.Json; -import io.druid.indexing.coordinator.setup.EC2NodeData; -import io.druid.indexing.coordinator.setup.GalaxyUserData; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.setup.EC2NodeData; +import io.druid.indexing.overlord.setup.GalaxyUserData; +import io.druid.indexing.overlord.setup.WorkerSetupData; import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopAutoScalingStrategy.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopAutoScalingStrategy.java index ef6b5291f61..893f69ca9f4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopAutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.metamx.emitter.EmittingLogger; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopResourceManagementScheduler.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopResourceManagementScheduler.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopResourceManagementScheduler.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopResourceManagementScheduler.java index 9d09dfab0bd..0176ec450cb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/NoopResourceManagementScheduler.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/NoopResourceManagementScheduler.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.metamx.common.logger.Logger; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementScheduler.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementScheduler.java index c8c63cee742..dfba06f983a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementScheduler.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.granularity.PeriodGranularity; -import io.druid.indexing.coordinator.RemoteTaskRunner; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.RemoteTaskRunner; +import io.druid.indexing.overlord.TaskRunner; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerConfig.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerConfig.java index b4cae15a122..534b548138d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.DateTime; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactory.java similarity index 90% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactory.java index 10924daa02e..cde36e61188 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactory.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; -import io.druid.indexing.coordinator.RemoteTaskRunner; +import io.druid.indexing.overlord.RemoteTaskRunner; /** */ diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactoryImpl.java similarity index 94% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactoryImpl.java index a898394298e..1df06cab129 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementSchedulerFactoryImpl.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.google.inject.Inject; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import io.druid.indexing.coordinator.RemoteTaskRunner; +import io.druid.indexing.overlord.RemoteTaskRunner; /** */ diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementStrategy.java similarity index 89% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementStrategy.java index 9838c40d383..e58eccced3e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ResourceManagementStrategy.java @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; -import io.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; -import io.druid.indexing.coordinator.ZkWorker; +import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; +import io.druid.indexing.overlord.ZkWorker; import java.util.Collection; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ScalingStats.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ScalingStats.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ScalingStats.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ScalingStats.java index 7af8de7b6ab..937aa1d48d4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/ScalingStats.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/ScalingStats.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementConfig.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementConfig.java index 6e659b01148..429b8317e16 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index f96ce562b11..25afe9feedc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.google.common.base.Function; import com.google.common.base.Predicate; @@ -28,10 +28,10 @@ import com.google.common.collect.Sets; import com.google.inject.Inject; import com.metamx.common.guava.FunctionalIterable; import com.metamx.emitter.EmittingLogger; -import io.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; -import io.druid.indexing.coordinator.TaskRunnerWorkItem; -import io.druid.indexing.coordinator.ZkWorker; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; +import io.druid.indexing.overlord.TaskRunnerWorkItem; +import io.druid.indexing.overlord.ZkWorker; +import io.druid.indexing.overlord.setup.WorkerSetupData; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/EC2NodeData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2NodeData.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/setup/EC2NodeData.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2NodeData.java index 66c8026ebe5..6be42500ab8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/EC2NodeData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2NodeData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.setup; +package io.druid.indexing.overlord.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/GalaxyUserData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java similarity index 97% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/setup/GalaxyUserData.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java index e7a57f6c386..ee721e9ac8c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/GalaxyUserData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.setup; +package io.druid.indexing.overlord.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/WorkerSetupData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/coordinator/setup/WorkerSetupData.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java index 332edadcf39..7764b9f5fea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/setup/WorkerSetupData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.setup; +package io.druid.indexing.overlord.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index 812eee3d4cb..6669556580b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -30,7 +30,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.CreateMode; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index c1382c994e6..51ad98adaf2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -27,7 +27,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java index 5c3d15e9d2d..bc8879b6960 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -31,7 +31,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.concurrent.Execs; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.TaskRunner; import java.io.File; import java.io.IOException; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleFactory.java b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleFactory.java index a434c352821..8ad0455ec5a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/executor/ExecutorLifecycleFactory.java @@ -20,7 +20,7 @@ package io.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.TaskRunner; import java.io.File; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java index 9ab4396b2cc..f38acb23982 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.io.InputSupplier; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import io.druid.indexing.coordinator.ForkingTaskRunner; +import io.druid.indexing.overlord.ForkingTaskRunner; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/RealtimeishTask.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java similarity index 99% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/RealtimeishTask.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java index 7e371da4cd2..b6f93f04704 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/RealtimeishTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RealtimeishTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java similarity index 99% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index e9eebfaba67..6bf97fa85c8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.repackaged.com.google.common.base.Throwables; @@ -39,7 +39,7 @@ import io.druid.indexing.common.TestRealtimeTask; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.jackson.DefaultObjectMapper; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java similarity index 99% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index c011ca07e8f..5575b87eb2e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -58,7 +58,7 @@ import io.druid.indexing.common.task.IndexTask; import io.druid.indexing.common.task.KillTask; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; -import io.druid.indexing.coordinator.exec.TaskConsumer; +import io.druid.indexing.overlord.exec.TaskConsumer; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java similarity index 99% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java index cab98ca3afb..2b1d0e560c1 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskQueueTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TestRemoteTaskRunnerConfig.java similarity index 92% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/TestRemoteTaskRunnerConfig.java index c006535251a..aa714a30b25 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TestRemoteTaskRunnerConfig.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.overlord; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import org.joda.time.Period; /** diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java similarity index 95% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java index c07f04d6fd7..368901fb605 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.amazonaws.services.ec2.AmazonEC2Client; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; @@ -29,9 +29,9 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Lists; import io.druid.common.guava.DSuppliers; -import io.druid.indexing.coordinator.setup.EC2NodeData; -import io.druid.indexing.coordinator.setup.GalaxyUserData; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.setup.EC2NodeData; +import io.druid.indexing.overlord.setup.GalaxyUserData; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; import org.junit.After; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java similarity index 98% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 03fd983437c..8cda83e48ea 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -29,9 +29,9 @@ import io.druid.common.guava.DSuppliers; import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; -import io.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; -import io.druid.indexing.coordinator.ZkWorker; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; +import io.druid.indexing.overlord.ZkWorker; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.jackson.DefaultObjectMapper; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/TestAutoScalingStrategy.java similarity index 96% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java rename to indexing-service/src/test/java/io/druid/indexing/overlord/scaling/TestAutoScalingStrategy.java index 5b6cc4bf195..8ef2e0513f6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/TestAutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator.scaling; +package io.druid.indexing.overlord.scaling; import java.util.List; diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 6a818119cea..f80ca3cd8db 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -34,8 +34,8 @@ import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.TestRealtimeTask; import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.config.TaskConfig; -import io.druid.indexing.coordinator.TestRemoteTaskRunnerConfig; -import io.druid.indexing.coordinator.ThreadPoolTaskRunner; +import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; +import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.loading.DataSegmentPuller; diff --git a/pom.xml b/pom.xml index d0fee2251c6..8d1a3d034cc 100644 --- a/pom.xml +++ b/pom.xml @@ -356,7 +356,7 @@ org.antlr - antlr4-master + antlr4-coordinator 4.0 diff --git a/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java index 43f90a1ac01..37a054a3f38 100644 --- a/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java @@ -32,12 +32,12 @@ import java.util.Set; public class RegexDimExtractionFnTest { private static final String[] paths = { - "/druid/prod/compute", - "/druid/prod/bard", - "/druid/prod/master", - "/druid/demo-east/compute", - "/druid/demo-east/bard", - "/druid/demo-east/master", + "/druid/prod/historical", + "/druid/prod/broker", + "/druid/prod/coordinator", + "/druid/demo/historical", + "/druid/demo/broker", + "/druid/demo/coordinator", "/dash/aloe", "/dash/baloo" }; @@ -80,7 +80,7 @@ public class RegexDimExtractionFnTest Assert.assertEquals(4, extracted.size()); Assert.assertTrue(extracted.contains("druid/prod")); - Assert.assertTrue(extracted.contains("druid/demo-east")); + Assert.assertTrue(extracted.contains("druid/demo")); Assert.assertTrue(extracted.contains("dash/aloe")); Assert.assertTrue(extracted.contains("dash/baloo")); } diff --git a/publications/vldb/.gitignore b/publications/whitepaper/.gitignore similarity index 100% rename from publications/vldb/.gitignore rename to publications/whitepaper/.gitignore diff --git a/publications/vldb/Makefile b/publications/whitepaper/Makefile similarity index 100% rename from publications/vldb/Makefile rename to publications/whitepaper/Makefile diff --git a/publications/vldb/druid.bib b/publications/whitepaper/druid.bib similarity index 100% rename from publications/vldb/druid.bib rename to publications/whitepaper/druid.bib diff --git a/publications/vldb/druid.tex b/publications/whitepaper/druid.tex similarity index 95% rename from publications/vldb/druid.tex rename to publications/whitepaper/druid.tex index 7f3be31ba4c..55083998d62 100644 --- a/publications/vldb/druid.tex +++ b/publications/whitepaper/druid.tex @@ -186,7 +186,7 @@ Figure~\ref{fig:druid_cluster}. Recall that the Druid data model has the notion of historical and real-time segments. The Druid cluster is architected to reflect this conceptual separation of data. Real-time nodes are responsible for ingesting, storing, and responding to queries for the most recent -events. Similarly, historical compute nodes are responsible for +events. Similarly, historical historical nodes are responsible for loading and responding to queries for historical events. Data in Druid is stored on storage nodes. Storage nodes can be either @@ -195,7 +195,7 @@ typically first hit a layer of broker nodes. Broker nodes are responsible for finding and routing queries down to the storage nodes that host the pertinent data. The storage nodes compute their portion of the query response in parallel and return their results to the -brokers. Broker nodes, compute nodes, and realtime nodes are jointly +brokers. Broker nodes, historical nodes, and realtime nodes are jointly classified as queryable nodes. Druid also has a set of coordination nodes to manage load assignment, @@ -207,51 +207,51 @@ Druid relies on Apache Zookeeper \cite{hunt2010zookeeper} for coordination. Most intra-cluster communication is over Zookeeper, although queries are typically forwarded over HTTP. -\subsection{Historical Compute Nodes} -Historical compute nodes are the main workers of a Druid cluster and -are self-contained and self-sufficient. Compute nodes load historical +\subsection{Historical historical nodes} +Historical historical nodes are the main workers of a Druid cluster and +are self-contained and self-sufficient. historical nodes load historical segments from permanent/deep storage and expose them for querying. There is no single point of contention between the nodes and -nodes have no knowledge of one another. Compute nodes are +nodes have no knowledge of one another. historical nodes are operationally simple; they only know how to perform the tasks they are -assigned. To help other services discover compute nodes and the data -they hold, every compute node maintains a constant Zookeeper -connection. Compute nodes announce their online state and the segments +assigned. To help other services discover historical nodes and the data +they hold, every historical node maintains a constant Zookeeper +connection. historical nodes announce their online state and the segments they serve by creating ephemeral nodes under specifically configured -Zookeeper paths. Instructions for a given compute node to load new +Zookeeper paths. Instructions for a given historical node to load new segments or drop existing segments are sent by creating ephemeral znodes under a special “load queue” path associated with the compute -node. Figure~\ref{fig:zookeeper} illustrates a simple compute node and Zookeeper interaction. -Each compute node announces themselves under an "announcements" path when they come online +node. Figure~\ref{fig:zookeeper} illustrates a simple historical node and Zookeeper interaction. +Each historical node announces themselves under an "announcements" path when they come online and each compute has a load queue path associated with it. \begin{figure} \centering \includegraphics[width = 2.8in]{zookeeper} -\caption{Compute nodes create ephemeral znodes under specifically configured Zookeeper paths.} +\caption{historical nodes create ephemeral znodes under specifically configured Zookeeper paths.} \label{fig:zookeeper} \end{figure} -To expose a segment for querying, a compute node must first possess a -local copy of the segment. Before a compute node downloads a segment +To expose a segment for querying, a historical node must first possess a +local copy of the segment. Before a historical node downloads a segment from deep storage, it first checks a local disk directory (cache) to see if the segment already exists in local storage. If no cache -information about the segment is present, the compute node will +information about the segment is present, the historical node will download metadata about the segment from Zookeeper. This metadata includes information about where the segment is located in deep storage and about how to decompress and process the segment. Once a -compute node completes processing a segment, the node announces (in +historical node completes processing a segment, the node announces (in Zookeeper) that it is serving the segment. At this point, the segment is queryable. \subsubsection{Tiers} \label{sec:tiers} -Compute nodes can be grouped in different tiers, where all nodes in a +historical nodes can be grouped in different tiers, where all nodes in a given tier are identically configured. Different performance and fault-tolerance parameters can be set for each tier. The purpose of tiered nodes is to enable higher or lower priority segments to be distributed according to their importance. For example, it is possible -to spin up a “hot” tier of compute nodes that have a high number of +to spin up a “hot” tier of historical nodes that have a high number of cores and a large RAM capacity. The “hot” cluster can be configured to download more frequently accessed segments. A parallel “cold” cluster can also be created with much less powerful backing hardware. The @@ -290,12 +290,12 @@ all indexes on a node to be queried. On a periodic basis, the nodes will schedule a background task that searches for all persisted indexes of a data source. The task merges these indexes together and builds a historical segment. The nodes will upload the segment to deep storage -and provide a signal for the historical compute nodes to begin serving +and provide a signal for the historical historical nodes to begin serving the segment. The ingest, persist, merge, and handoff steps are fluid; there is no data loss as a real-time node converts a real-time segment to a historical one. Figure~\ref{fig:data-durability} illustrates the process. -Similar to compute nodes, real-time nodes announce segments in +Similar to historical nodes, real-time nodes announce segments in Zookeeper. Unlike historical segments, real-time segments may represent a period of time that extends into the future. For example, a real-time node may announce it is serving a segment that contains @@ -384,7 +384,7 @@ query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query -to the compute nodes. Once the compute nodes return their results, +to the historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually @@ -412,16 +412,16 @@ table can be updated by any service that creates historical segments. The MySQL database also contains a rule table that governs how segments are created, destroyed, and replicated in the cluster. -The master does not directly communicate with a compute node when +The master does not directly communicate with a historical node when assigning it work; instead the master creates an ephemeral znode in -Zookeeper containing information about what the compute node should -do. The compute node maintains a similar connection to Zookeeper to +Zookeeper containing information about what the historical node should +do. The historical node maintains a similar connection to Zookeeper to monitor for new work. \subsubsection{Rules} Rules govern how historical segments are loaded and dropped from the cluster. Rules indicate how segments should be assigned to -different compute node tiers and how many replicates of a segment +different historical node tiers and how many replicates of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. Rules are usually set for a period of time. For example, a user may use rules to load the most recent one month's worth of segments into a "hot" cluster, @@ -435,7 +435,7 @@ match each segment with the first rule that applies to it. \subsubsection{Load Balancing} In a typical production environment, queries often hit dozens or even -hundreds of data segments. Since each compute node has limited +hundreds of data segments. Since each historical node has limited resources, historical segments must be distributed among the cluster to ensure that the cluster load is not too imbalanced. Determining optimal load distribution requires some knowledge about query patterns @@ -445,7 +445,7 @@ access smaller segments are faster. These query patterns suggest replicating recent historical segments at a higher rate, spreading out large segments that are close in time to -different compute nodes, and co-locating segments from different data +different historical nodes, and co-locating segments from different data sources. To optimally distribute and balance segments among the cluster, we developed a cost-based optimization procedure that takes into account the segment data source, recency, and size. The exact @@ -628,7 +628,7 @@ Druid replicates historical segments on multiple hosts. The number of replicates in each tier of the historical compute cluster is fully configurable. Setups that require high levels of fault tolerance can be configured to have a high number of replicates. Replicates are -assigned to compute nodes by coordination nodes using the same load +assigned to historical nodes by coordination nodes using the same load distribution algorithm discussed in Section~\ref{sec:caching}. Broker nodes forward queries to the first node they find that contain a segment required for the query. Real-time segments follow a different replication model as real-time @@ -641,7 +641,7 @@ reload any indexes that were persisted to disk and read from the message bus from the point it last committed an offset. \subsection{Failure Detection} -If a compute node completely fails and becomes unavailable, the +If a historical node completely fails and becomes unavailable, the ephemeral Zookeeper znodes it created are deleted. The master node will notice that certain segments are insufficiently replicated or missing altogether. Additional replicates will be created and @@ -742,7 +742,7 @@ count, achieving scan rates of 33M rows/second/core. We believe the 75 node cluster was actually overprovisioned for the test dataset, explaining the modest improvement over the 50 node cluster. Druid's concurrency model is based on shards: one thread will scan one -shard. If the number of segments on a compute node modulo the number +shard. If the number of segments on a historical node modulo the number of cores is small (e.g. 17 segments and 15 cores), then many of the cores will be idle during the last round of the computation. diff --git a/publications/vldb/figures/Druid_cluster.png b/publications/whitepaper/figures/Druid_cluster.png similarity index 100% rename from publications/vldb/figures/Druid_cluster.png rename to publications/whitepaper/figures/Druid_cluster.png diff --git a/publications/vldb/figures/cluster_scan_rate.pdf b/publications/whitepaper/figures/cluster_scan_rate.pdf similarity index 100% rename from publications/vldb/figures/cluster_scan_rate.pdf rename to publications/whitepaper/figures/cluster_scan_rate.pdf diff --git a/publications/vldb/figures/concise_plot.pdf b/publications/whitepaper/figures/concise_plot.pdf similarity index 100% rename from publications/vldb/figures/concise_plot.pdf rename to publications/whitepaper/figures/concise_plot.pdf diff --git a/publications/vldb/figures/core_scan_rate.pdf b/publications/whitepaper/figures/core_scan_rate.pdf similarity index 100% rename from publications/vldb/figures/core_scan_rate.pdf rename to publications/whitepaper/figures/core_scan_rate.pdf diff --git a/publications/vldb/figures/data_reading.png b/publications/whitepaper/figures/data_reading.png similarity index 100% rename from publications/vldb/figures/data_reading.png rename to publications/whitepaper/figures/data_reading.png diff --git a/publications/vldb/figures/druid_message_bus.png b/publications/whitepaper/figures/druid_message_bus.png similarity index 100% rename from publications/vldb/figures/druid_message_bus.png rename to publications/whitepaper/figures/druid_message_bus.png diff --git a/publications/vldb/figures/druid_realtime_flow.png b/publications/whitepaper/figures/druid_realtime_flow.png similarity index 100% rename from publications/vldb/figures/druid_realtime_flow.png rename to publications/whitepaper/figures/druid_realtime_flow.png diff --git a/publications/vldb/figures/druid_segment.png b/publications/whitepaper/figures/druid_segment.png similarity index 100% rename from publications/vldb/figures/druid_segment.png rename to publications/whitepaper/figures/druid_segment.png diff --git a/publications/vldb/figures/zookeeper.png b/publications/whitepaper/figures/zookeeper.png similarity index 100% rename from publications/vldb/figures/zookeeper.png rename to publications/whitepaper/figures/zookeeper.png diff --git a/publications/vldb/src/druid_plot.R b/publications/whitepaper/src/druid_plot.R similarity index 100% rename from publications/vldb/src/druid_plot.R rename to publications/whitepaper/src/druid_plot.R diff --git a/publications/vldb/src/druid_tables.R b/publications/whitepaper/src/druid_tables.R similarity index 100% rename from publications/vldb/src/druid_tables.R rename to publications/whitepaper/src/druid_tables.R diff --git a/publications/vldb/vldb.cls b/publications/whitepaper/vldb.cls similarity index 100% rename from publications/vldb/vldb.cls rename to publications/whitepaper/vldb.cls diff --git a/realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java b/realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java index afa68fba764..91bf87440ac 100644 --- a/realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java +++ b/realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java @@ -31,7 +31,7 @@ import java.io.IOException; /** * A Fire Department has a Firehose and a Plumber. * - * This is a metaphor for a realtime stream (Firehose) and a master of sinks (Plumber). The Firehose provides the + * This is a metaphor for a realtime stream (Firehose) and a coordinator of sinks (Plumber). The Firehose provides the * realtime stream of data. The Plumber directs each drop of water from the firehose into the correct sink and makes * sure that the sinks don't overflow. */ diff --git a/server/src/main/java/io/druid/db/DatabaseRuleManager.java b/server/src/main/java/io/druid/db/DatabaseRuleManager.java index c6968362caa..0ba44c52c85 100644 --- a/server/src/main/java/io/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/io/druid/db/DatabaseRuleManager.java @@ -34,8 +34,8 @@ import com.metamx.common.logger.Logger; import io.druid.concurrent.Execs; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Json; -import io.druid.server.master.rules.PeriodLoadRule; -import io.druid.server.master.rules.Rule; +import io.druid.server.coordinator.rules.PeriodLoadRule; +import io.druid.server.coordinator.rules.Rule; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/io/druid/server/master/BalancerSegmentHolder.java b/server/src/main/java/io/druid/server/coordinator/BalancerSegmentHolder.java similarity index 97% rename from server/src/main/java/io/druid/server/master/BalancerSegmentHolder.java rename to server/src/main/java/io/druid/server/coordinator/BalancerSegmentHolder.java index 7eb01282b48..ab9914612ef 100644 --- a/server/src/main/java/io/druid/server/master/BalancerSegmentHolder.java +++ b/server/src/main/java/io/druid/server/coordinator/BalancerSegmentHolder.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.client.DruidServer; import io.druid.timeline.DataSegment; diff --git a/server/src/main/java/io/druid/server/master/BalancerStrategy.java b/server/src/main/java/io/druid/server/coordinator/BalancerStrategy.java similarity index 90% rename from server/src/main/java/io/druid/server/master/BalancerStrategy.java rename to server/src/main/java/io/druid/server/coordinator/BalancerStrategy.java index 0fb10eb2b6b..77181f690dc 100644 --- a/server/src/main/java/io/druid/server/master/BalancerStrategy.java +++ b/server/src/main/java/io/druid/server/coordinator/BalancerStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.timeline.DataSegment; @@ -31,5 +31,5 @@ public interface BalancerStrategy public BalancerSegmentHolder pickSegmentToMove(final List serverHolders); - public void emitStats(String tier, MasterStats stats, List serverHolderList); + public void emitStats(String tier, CoordinatorStats stats, List serverHolderList); } diff --git a/server/src/main/java/io/druid/server/master/BalancerStrategyFactory.java b/server/src/main/java/io/druid/server/coordinator/BalancerStrategyFactory.java similarity index 96% rename from server/src/main/java/io/druid/server/master/BalancerStrategyFactory.java rename to server/src/main/java/io/druid/server/coordinator/BalancerStrategyFactory.java index abdb9e88020..870e0f7daa1 100644 --- a/server/src/main/java/io/druid/server/master/BalancerStrategyFactory.java +++ b/server/src/main/java/io/druid/server/coordinator/BalancerStrategyFactory.java @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import org.joda.time.DateTime; diff --git a/server/src/main/java/io/druid/server/master/MasterDynamicConfig.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java similarity index 95% rename from server/src/main/java/io/druid/server/master/MasterDynamicConfig.java rename to server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java index b9ded33bf29..e4c553644fd 100644 --- a/server/src/main/java/io/druid/server/master/MasterDynamicConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -16,14 +16,14 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -public class MasterDynamicConfig +public class CoordinatorDynamicConfig { - public static final String CONFIG_KEY = "master.config"; + public static final String CONFIG_KEY = "coordinator.config"; private final long millisToWaitBeforeDeleting; private final long mergeBytesLimit; @@ -34,7 +34,7 @@ public class MasterDynamicConfig private final boolean emitBalancingStats; @JsonCreator - public MasterDynamicConfig( + public CoordinatorDynamicConfig( @JsonProperty("millisToWaitBeforeDeleting") long millisToWaitBeforeDeleting, @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @@ -165,9 +165,9 @@ public class MasterDynamicConfig return this; } - public MasterDynamicConfig build() + public CoordinatorDynamicConfig build() { - return new MasterDynamicConfig( + return new CoordinatorDynamicConfig( millisToWaitBeforeDeleting, mergeBytesLimit, mergeSegmentsLimit, diff --git a/server/src/main/java/io/druid/server/master/MasterStats.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java similarity index 93% rename from server/src/main/java/io/druid/server/master/MasterStats.java rename to server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java index ed23b9f0b79..e694e7609d2 100644 --- a/server/src/main/java/io/druid/server/master/MasterStats.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Maps; import io.druid.collections.CountingMap; @@ -27,12 +27,12 @@ import java.util.concurrent.atomic.AtomicLong; /** */ -public class MasterStats +public class CoordinatorStats { private final Map> perTierStats; private final CountingMap globalStats; - public MasterStats() + public CoordinatorStats() { perTierStats = Maps.newHashMap(); globalStats = new CountingMap(); @@ -63,7 +63,7 @@ public class MasterStats globalStats.add(statName, value); } - public MasterStats accumulate(MasterStats stats) + public CoordinatorStats accumulate(CoordinatorStats stats) { for (Map.Entry> entry : stats.perTierStats.entrySet()) { CountingMap theStat = perTierStats.get(entry.getKey()); diff --git a/server/src/main/java/io/druid/server/master/CostBalancerStrategy.java b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java similarity index 97% rename from server/src/main/java/io/druid/server/master/CostBalancerStrategy.java rename to server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java index 06d5aa0836d..a5cb45e7e7f 100644 --- a/server/src/main/java/io/druid/server/master/CostBalancerStrategy.java +++ b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.metamx.common.Pair; import com.metamx.emitter.EmittingLogger; @@ -190,13 +190,13 @@ public class CostBalancerStrategy implements BalancerStrategy /** * Calculates the cost normalization. This is such that the normalized cost is lower bounded - * by 1 (e.g. when each segment gets its own compute node). + * by 1 (e.g. when each segment gets its own historical node). * * @param serverHolders A list of ServerHolders for a particular tier. * * @return The normalization value (the sum of the diagonal entries in the * pairwise cost matrix). This is the cost of a cluster if each - * segment were to get its own compute node. + * segment were to get its own historical node. */ public double calculateNormalization(final List serverHolders) { @@ -212,7 +212,7 @@ public class CostBalancerStrategy implements BalancerStrategy @Override public void emitStats( String tier, - MasterStats stats, List serverHolderList + CoordinatorStats stats, List serverHolderList ) { final double initialTotalCost = calculateInitialTotalCost(serverHolderList); diff --git a/server/src/main/java/io/druid/server/master/CostBalancerStrategyFactory.java b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategyFactory.java similarity index 96% rename from server/src/main/java/io/druid/server/master/CostBalancerStrategyFactory.java rename to server/src/main/java/io/druid/server/coordinator/CostBalancerStrategyFactory.java index 832ee851277..29342318a83 100644 --- a/server/src/main/java/io/druid/server/master/CostBalancerStrategyFactory.java +++ b/server/src/main/java/io/druid/server/coordinator/CostBalancerStrategyFactory.java @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import org.joda.time.DateTime; diff --git a/server/src/main/java/io/druid/server/master/DatasourceWhitelist.java b/server/src/main/java/io/druid/server/coordinator/DatasourceWhitelist.java similarity index 93% rename from server/src/main/java/io/druid/server/master/DatasourceWhitelist.java rename to server/src/main/java/io/druid/server/coordinator/DatasourceWhitelist.java index 428a0cb3863..7c14f933446 100644 --- a/server/src/main/java/io/druid/server/master/DatasourceWhitelist.java +++ b/server/src/main/java/io/druid/server/coordinator/DatasourceWhitelist.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; @@ -29,7 +29,7 @@ import java.util.Set; */ public class DatasourceWhitelist { - public static final String CONFIG_KEY = "master.whitelist"; + public static final String CONFIG_KEY = "coordinator.whitelist"; private final Set dataSources; diff --git a/server/src/main/java/io/druid/server/master/DruidCluster.java b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java similarity index 98% rename from server/src/main/java/io/druid/server/master/DruidCluster.java rename to server/src/main/java/io/druid/server/coordinator/DruidCluster.java index dd9d0c33bc5..1059159e7e9 100644 --- a/server/src/main/java/io/druid/server/master/DruidCluster.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCluster.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; diff --git a/server/src/main/java/io/druid/server/master/DruidMaster.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java similarity index 81% rename from server/src/main/java/io/druid/server/master/DruidMaster.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 97347124dca..cef7067d559 100644 --- a/server/src/main/java/io/druid/server/master/DruidMaster.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.base.Function; import com.google.common.base.Predicate; @@ -74,19 +74,19 @@ import java.util.concurrent.atomic.AtomicReference; /** */ @ManageLifecycle -public class DruidMaster +public class DruidCoordinator { - public static final String MASTER_OWNER_NODE = "_MASTER"; + public static final String COORDINATOR_OWNER_NODE = "_COORDINATOR"; - private static final EmittingLogger log = new EmittingLogger(DruidMaster.class); + private static final EmittingLogger log = new EmittingLogger(DruidCoordinator.class); private final Object lock = new Object(); private volatile boolean started = false; - private volatile boolean master = false; - private volatile AtomicReference dynamicConfigs; + private volatile boolean leader = false; + private volatile AtomicReference dynamicConfigs; - private final DruidMasterConfig config; + private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; private final DatabaseSegmentManager databaseSegmentManager; @@ -101,8 +101,8 @@ public class DruidMaster private final AtomicReference leaderLatch; @Inject - public DruidMaster( - DruidMasterConfig config, + public DruidCoordinator( + DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, DatabaseSegmentManager databaseSegmentManager, @@ -131,8 +131,8 @@ public class DruidMaster ); } - DruidMaster( - DruidMasterConfig config, + DruidCoordinator( + DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, DatabaseSegmentManager databaseSegmentManager, @@ -158,16 +158,16 @@ public class DruidMaster this.indexingServiceClient = indexingServiceClient; this.taskMaster = taskMaster; - this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); + this.exec = scheduledExecutorFactory.create(1, "Coordinator-Exec--%d"); this.leaderLatch = new AtomicReference<>(null); this.dynamicConfigs = new AtomicReference<>(null); this.loadManagementPeons = loadQueuePeonMap; } - public boolean isClusterMaster() + public boolean isLeader() { - return master; + return leader; } public Map getLoadStatus() @@ -215,7 +215,7 @@ public class DruidMaster return loadStatus; } - public MasterDynamicConfig getDynamicConfigs() + public CoordinatorDynamicConfig getDynamicConfigs() { return dynamicConfigs.get(); } @@ -236,7 +236,7 @@ public class DruidMaster databaseSegmentManager.enableDatasource(ds); } - public String getCurrentMaster() + public String getCurrentLeader() { try { final LeaderLatch latch = leaderLatch.get(); @@ -414,7 +414,7 @@ public class DruidMaster private LeaderLatch createNewLeaderLatch() { final LeaderLatch newLeaderLatch = new LeaderLatch( - curator, ZKPaths.makePath(zkPaths.getMasterPath(), MASTER_OWNER_NODE), config.getHost() + curator, ZKPaths.makePath(zkPaths.getCoordinatorPath(), COORDINATOR_OWNER_NODE), config.getHost() ); newLeaderLatch.addListener( @@ -423,16 +423,16 @@ public class DruidMaster @Override public void isLeader() { - DruidMaster.this.becomeMaster(); + DruidCoordinator.this.becomeLeader(); } @Override public void notLeader() { - DruidMaster.this.stopBeingMaster(); + DruidCoordinator.this.stopBeingLeader(); } }, - Execs.singleThreaded("MasterLeader-%s") + Execs.singleThreaded("CoordinatorLeader-%s") ); return leaderLatch.getAndSet(newLeaderLatch); @@ -446,7 +446,7 @@ public class DruidMaster return; } - stopBeingMaster(); + stopBeingLeader(); try { leaderLatch.get().close(); @@ -461,54 +461,59 @@ public class DruidMaster } } - private void becomeMaster() + private void becomeLeader() { synchronized (lock) { if (!started) { return; } - log.info("I am the master, all must bow!"); + log.info("I am the leader of the coordinators, all must bow!"); try { - master = true; + leader = true; databaseSegmentManager.start(); databaseRuleManager.start(); serverInventoryView.start(); - final List> masterRunnables = Lists.newArrayList(); + final List> coordinatorRunnables = Lists.newArrayList(); dynamicConfigs = configManager.watch( - MasterDynamicConfig.CONFIG_KEY, - MasterDynamicConfig.class, - new MasterDynamicConfig.Builder().build() + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.class, + new CoordinatorDynamicConfig.Builder().build() ); - masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod())); + coordinatorRunnables.add(Pair.of(new CoordinatorHistoricalManagerRunnable(), config.getCoordinatorPeriod())); if (indexingServiceClient != null) { - masterRunnables.add( + coordinatorRunnables.add( Pair.of( - new MasterIndexingServiceRunnable( - makeIndexingServiceHelpers(configManager.watch(DatasourceWhitelist.CONFIG_KEY, DatasourceWhitelist.class)) + new CoordinatorIndexingServiceRunnable( + makeIndexingServiceHelpers( + configManager.watch( + DatasourceWhitelist.CONFIG_KEY, + DatasourceWhitelist.class + ) + ) ), - config.getMasterSegmentMergerPeriod() + config.getCoordinatorSegmentMergerPeriod() ) ); } - for (final Pair masterRunnable : masterRunnables) { + for (final Pair coordinatorRunnable : coordinatorRunnables) { ScheduledExecutors.scheduleWithFixedDelay( exec, - config.getMasterStartDelay(), - masterRunnable.rhs, + config.getCoordinatorStartDelay(), + coordinatorRunnable.rhs, new Callable() { - private final MasterRunnable theRunnable = masterRunnable.lhs; + private final CoordinatorRunnable theRunnable = coordinatorRunnable.lhs; @Override public ScheduledExecutors.Signal call() { - if (master) { + if (leader) { theRunnable.run(); } - if (master) { // (We might no longer be master) + if (leader) { // (We might no longer be coordinator) return ScheduledExecutors.Signal.REPEAT; } else { return ScheduledExecutors.Signal.STOP; @@ -519,7 +524,7 @@ public class DruidMaster } } catch (Exception e) { - log.makeAlert(e, "Unable to become master") + log.makeAlert(e, "Unable to become leader") .emit(); final LeaderLatch oldLatch = createNewLeaderLatch(); Closeables.closeQuietly(oldLatch); @@ -527,7 +532,7 @@ public class DruidMaster leaderLatch.get().start(); } catch (Exception e1) { - // If an exception gets thrown out here, then the master will zombie out 'cause it won't be looking for + // If an exception gets thrown out here, then the coordinator will zombie out 'cause it won't be looking for // the latch anymore. I don't believe it's actually possible for an Exception to throw out here, but // Curator likes to have "throws Exception" on methods so it might happen... log.makeAlert(e1, "I am a zombie") @@ -537,11 +542,11 @@ public class DruidMaster } } - private void stopBeingMaster() + private void stopBeingLeader() { synchronized (lock) { try { - log.info("I am no longer the master..."); + log.info("I am no longer the leader..."); for (String server : loadManagementPeons.keySet()) { LoadQueuePeon peon = loadManagementPeons.remove(server); @@ -551,37 +556,37 @@ public class DruidMaster databaseSegmentManager.stop(); serverInventoryView.stop(); - master = false; + leader = false; } catch (Exception e) { - log.makeAlert(e, "Unable to stopBeingMaster").emit(); + log.makeAlert(e, "Unable to stopBeingLeader").emit(); } } } - private List makeIndexingServiceHelpers(final AtomicReference whitelistRef) + private List makeIndexingServiceHelpers(final AtomicReference whitelistRef) { - List helpers = Lists.newArrayList(); + List helpers = Lists.newArrayList(); - helpers.add(new DruidMasterSegmentInfoLoader(DruidMaster.this)); + helpers.add(new DruidCoordinatorSegmentInfoLoader(DruidCoordinator.this)); if (config.isConvertSegments()) { - helpers.add(new DruidMasterVersionConverter(indexingServiceClient, whitelistRef)); + helpers.add(new DruidCoordinatorVersionConverter(indexingServiceClient, whitelistRef)); } if (config.isMergeSegments()) { - helpers.add(new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef)); + helpers.add(new DruidCoordinatorSegmentMerger(indexingServiceClient, whitelistRef)); helpers.add( - new DruidMasterHelper() + new DruidCoordinatorHelper() { @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - MasterStats stats = params.getMasterStats(); + CoordinatorStats stats = params.getCoordinatorStats(); log.info("Issued merge requests for %s segments", stats.getGlobalStats().get("mergedCount").get()); params.getEmitter().emit( new ServiceMetricEvent.Builder().build( - "master/merge/count", stats.getGlobalStats().get("mergedCount") + "coordinator/merge/count", stats.getGlobalStats().get("mergedCount") ) ); @@ -594,12 +599,12 @@ public class DruidMaster return ImmutableList.copyOf(helpers); } - public static class DruidMasterVersionConverter implements DruidMasterHelper + public static class DruidCoordinatorVersionConverter implements DruidCoordinatorHelper { private final IndexingServiceClient indexingServiceClient; private final AtomicReference whitelistRef; - public DruidMasterVersionConverter( + public DruidCoordinatorVersionConverter( IndexingServiceClient indexingServiceClient, AtomicReference whitelistRef ) @@ -609,7 +614,7 @@ public class DruidMaster } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { DatasourceWhitelist whitelist = whitelistRef.get(); @@ -628,12 +633,12 @@ public class DruidMaster } } - public abstract class MasterRunnable implements Runnable + public abstract class CoordinatorRunnable implements Runnable { private final long startTime = System.currentTimeMillis(); - private final List helpers; + private final List helpers; - protected MasterRunnable(List helpers) + protected CoordinatorRunnable(List helpers) { this.helpers = helpers; } @@ -645,8 +650,8 @@ public class DruidMaster synchronized (lock) { final LeaderLatch latch = leaderLatch.get(); if (latch == null || !latch.hasLeadership()) { - log.info("LEGGO MY EGGO. [%s] is master.", latch == null ? null : latch.getLeader().getId()); - stopBeingMaster(); + log.info("LEGGO MY EGGO. [%s] is leader.", latch == null ? null : latch.getLeader().getId()); + stopBeingLeader(); return; } } @@ -658,14 +663,14 @@ public class DruidMaster for (Boolean aBoolean : allStarted) { if (!aBoolean) { log.error("InventoryManagers not started[%s]", allStarted); - stopBeingMaster(); + stopBeingLeader(); return; } } - // Do master stuff. - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + // Do coordinator stuff. + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withStartTime(startTime) .withDatasources(databaseSegmentManager.getInventory()) .withDynamicConfigs(dynamicConfigs.get()) @@ -673,7 +678,7 @@ public class DruidMaster .build(); - for (DruidMasterHelper helper : helpers) { + for (DruidCoordinatorHelper helper : helpers) { params = helper.run(params); } } @@ -683,17 +688,17 @@ public class DruidMaster } } - private class MasterComputeManagerRunnable extends MasterRunnable + private class CoordinatorHistoricalManagerRunnable extends CoordinatorRunnable { - private MasterComputeManagerRunnable() + private CoordinatorHistoricalManagerRunnable() { super( ImmutableList.of( - new DruidMasterSegmentInfoLoader(DruidMaster.this), - new DruidMasterHelper() + new DruidCoordinatorSegmentInfoLoader(DruidCoordinator.this), + new DruidCoordinatorHelper() { @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { // Display info about all historical servers Iterable servers = FunctionalIterable @@ -759,18 +764,18 @@ public class DruidMaster .build(); } }, - new DruidMasterRuleRunner(DruidMaster.this), - new DruidMasterCleanup(DruidMaster.this), - new DruidMasterBalancer(DruidMaster.this), - new DruidMasterLogger() + new DruidCoordinatorRuleRunner(DruidCoordinator.this), + new DruidCoordinatorCleanup(DruidCoordinator.this), + new DruidCoordinatorBalancer(DruidCoordinator.this), + new DruidCoordinatorLogger() ) ); } } - private class MasterIndexingServiceRunnable extends MasterRunnable + private class CoordinatorIndexingServiceRunnable extends CoordinatorRunnable { - private MasterIndexingServiceRunnable(List helpers) + private CoordinatorIndexingServiceRunnable(List helpers) { super(helpers); } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorBalancer.java similarity index 89% rename from server/src/main/java/io/druid/server/master/DruidMasterBalancer.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorBalancer.java index 5a4ae3e9dbe..0ca4ff32fe8 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorBalancer.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -35,7 +35,7 @@ import java.util.concurrent.ConcurrentHashMap; /** */ -public class DruidMasterBalancer implements DruidMasterHelper +public class DruidCoordinatorBalancer implements DruidCoordinatorHelper { public static final Comparator percentUsedComparator = Comparators.inverse( new Comparator() @@ -47,17 +47,17 @@ public class DruidMasterBalancer implements DruidMasterHelper } } ); - protected static final EmittingLogger log = new EmittingLogger(DruidMasterBalancer.class); + protected static final EmittingLogger log = new EmittingLogger(DruidCoordinatorBalancer.class); - protected final DruidMaster master; + protected final DruidCoordinator coordinator; protected final Map> currentlyMovingSegments = Maps.newHashMap(); - public DruidMasterBalancer( - DruidMaster master + public DruidCoordinatorBalancer( + DruidCoordinator coordinator ) { - this.master = master; + this.coordinator = coordinator; } protected void reduceLifetimes(String tier) @@ -74,12 +74,12 @@ public class DruidMasterBalancer implements DruidMasterHelper } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - final MasterStats stats = new MasterStats(); + final CoordinatorStats stats = new CoordinatorStats(); final DateTime referenceTimestamp = params.getBalancerReferenceTimestamp(); final BalancerStrategy strategy = params.getBalancerStrategyFactory().createBalancerStrategy(referenceTimestamp); - final int maxSegmentsToMove = params.getMasterDynamicConfig().getMaxSegmentsToMove(); + final int maxSegmentsToMove = params.getCoordinatorDynamicConfig().getMaxSegmentsToMove(); for (Map.Entry> entry : params.getDruidCluster().getCluster().entrySet()) { @@ -124,7 +124,7 @@ public class DruidMasterBalancer implements DruidMasterHelper } } stats.addToTieredStat("movedCount", tier, currentlyMovingSegments.get(tier).size()); - if (params.getMasterDynamicConfig().emitBalancingStats()) { + if (params.getCoordinatorDynamicConfig().emitBalancingStats()) { strategy.emitStats(tier, stats, serverHolderList); } @@ -135,14 +135,14 @@ public class DruidMasterBalancer implements DruidMasterHelper } return params.buildFromExisting() - .withMasterStats(stats) + .withCoordinatorStats(stats) .build(); } protected void moveSegment( final BalancerSegmentHolder segment, final DruidServer toServer, - final DruidMasterRuntimeParams params + final DruidCoordinatorRuntimeParams params ) { final String toServerName = toServer.getName(); @@ -171,7 +171,7 @@ public class DruidMasterBalancer implements DruidMasterHelper } } }; - master.moveSegment( + coordinator.moveSegment( fromServerName, toServerName, segmentToMove.getIdentifier(), diff --git a/server/src/main/java/io/druid/server/master/DruidMasterCleanup.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanup.java similarity index 87% rename from server/src/main/java/io/druid/server/master/DruidMasterCleanup.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanup.java index e31b1ae45c7..659391514aa 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterCleanup.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanup.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; @@ -34,23 +34,23 @@ import java.util.Set; /** */ -public class DruidMasterCleanup implements DruidMasterHelper +public class DruidCoordinatorCleanup implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidMasterCleanup.class); + private static final Logger log = new Logger(DruidCoordinatorCleanup.class); - private final DruidMaster master; + private final DruidCoordinator coordinator; - public DruidMasterCleanup( - DruidMaster master + public DruidCoordinatorCleanup( + DruidCoordinator coordinator ) { - this.master = master; + this.coordinator = coordinator; } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - MasterStats stats = new MasterStats(); + CoordinatorStats stats = new CoordinatorStats(); Set availableSegments = params.getAvailableSegments(); DruidCluster cluster = params.getDruidCluster(); @@ -110,7 +110,7 @@ public class DruidMasterCleanup implements DruidMasterHelper for (VersionedIntervalTimeline timeline : timelines.values()) { for (TimelineObjectHolder holder : timeline.findOvershadowed()) { for (DataSegment dataSegment : holder.getObject().payloads()) { - master.removeSegment(dataSegment); + coordinator.removeSegment(dataSegment); stats.addToGlobalStat("overShadowedCount", 1); } } @@ -118,7 +118,7 @@ public class DruidMasterCleanup implements DruidMasterHelper } return params.buildFromExisting() - .withMasterStats(stats) + .withCoordinatorStats(stats) .build(); } } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterConfig.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java similarity index 71% rename from server/src/main/java/io/druid/server/master/DruidMasterConfig.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java index d6a4fbf491d..a1319a7b7f5 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import org.joda.time.Duration; import org.skife.config.Config; @@ -25,36 +25,36 @@ import org.skife.config.Default; /** */ -public abstract class DruidMasterConfig +public abstract class DruidCoordinatorConfig { @Config("druid.host") public abstract String getHost(); - @Config("druid.master.startDelay") + @Config("druid.coordinator.startDelay") @Default("PT60s") - public abstract Duration getMasterStartDelay(); + public abstract Duration getCoordinatorStartDelay(); - @Config("druid.master.period") + @Config("druid.coordinator.period") @Default("PT60s") - public abstract Duration getMasterPeriod(); + public abstract Duration getCoordinatorPeriod(); - @Config("druid.master.period.segmentMerger") + @Config("druid.coordinator.period.segmentMerger") @Default("PT1800s") - public abstract Duration getMasterSegmentMergerPeriod(); + public abstract Duration getCoordinatorSegmentMergerPeriod(); - @Config("druid.master.merge.on") + @Config("druid.coordinator.merge.on") public boolean isMergeSegments() { return false; } - @Config("druid.master.conversion.on") + @Config("druid.coordinator.conversion.on") public boolean isConvertSegments() { return false; } - @Config("druid.master.load.timeout") + @Config("druid.coordinator.load.timeout") public Duration getLoadTimeoutDelay() { return new Duration(15 * 60 * 1000); diff --git a/server/src/main/java/io/druid/server/master/DruidMasterHelper.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorHelper.java similarity index 83% rename from server/src/main/java/io/druid/server/master/DruidMasterHelper.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorHelper.java index 860a8546e7a..e6eb7bb3997 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterHelper.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorHelper.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; /** */ -public interface DruidMasterHelper +public interface DruidCoordinatorHelper { - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params); + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params); } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterLogger.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorLogger.java similarity index 86% rename from server/src/main/java/io/druid/server/master/DruidMasterLogger.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorLogger.java index 49a771177d3..b6c8bb8c240 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterLogger.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorLogger.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; @@ -35,9 +35,9 @@ import java.util.concurrent.atomic.AtomicLong; /** */ -public class DruidMasterLogger implements DruidMasterHelper +public class DruidCoordinatorLogger implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidMasterLogger.class); + private static final Logger log = new Logger(DruidCoordinatorLogger.class); private void emitTieredStats( final ServiceEmitter emitter, @@ -59,10 +59,10 @@ public class DruidMasterLogger implements DruidMasterHelper } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { DruidCluster cluster = params.getDruidCluster(); - MasterStats stats = params.getMasterStats(); + CoordinatorStats stats = params.getCoordinatorStats(); ServiceEmitter emitter = params.getEmitter(); Map assigned = stats.getPerTierStats().get("assignedCount"); @@ -86,29 +86,29 @@ public class DruidMasterLogger implements DruidMasterHelper } emitTieredStats( - emitter, "master/%s/cost/raw", + emitter, "coordinator/%s/cost/raw", stats.getPerTierStats().get("initialCost") ); emitTieredStats( - emitter, "master/%s/cost/normalization", + emitter, "coordinator/%s/cost/normalization", stats.getPerTierStats().get("normalization") ); emitTieredStats( - emitter, "master/%s/moved/count", + emitter, "coordinator/%s/moved/count", stats.getPerTierStats().get("movedCount") ); emitTieredStats( - emitter, "master/%s/deleted/count", + emitter, "coordinator/%s/deleted/count", stats.getPerTierStats().get("deletedCount") ); Map normalized = stats.getPerTierStats().get("normalizedInitialCostTimesOneThousand"); if (normalized != null) { emitTieredStats( - emitter, "master/%s/cost/normalized", + emitter, "coordinator/%s/cost/normalized", Maps.transformEntries( normalized, new Maps.EntryTransformer() @@ -135,7 +135,7 @@ public class DruidMasterLogger implements DruidMasterHelper emitter.emit( new ServiceMetricEvent.Builder().build( - "master/overShadowed/count", stats.getGlobalStats().get("overShadowedCount") + "coordinator/overShadowed/count", stats.getGlobalStats().get("overShadowedCount") ) ); @@ -174,7 +174,7 @@ public class DruidMasterLogger implements DruidMasterHelper } } - // Emit master metrics + // Emit coordinator metrics final Set> peonEntries = params.getLoadManagementPeons().entrySet(); for (Map.Entry entry : peonEntries) { String serverName = entry.getKey(); @@ -182,25 +182,25 @@ public class DruidMasterLogger implements DruidMasterHelper emitter.emit( new ServiceMetricEvent.Builder() .setUser1(serverName).build( - "master/loadQueue/size", queuePeon.getLoadQueueSize() + "coordinator/loadQueue/size", queuePeon.getLoadQueueSize() ) ); emitter.emit( new ServiceMetricEvent.Builder() .setUser1(serverName).build( - "master/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() + "coordinator/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() ) ); emitter.emit( new ServiceMetricEvent.Builder() .setUser1(serverName).build( - "master/loadQueue/count", queuePeon.getSegmentsToLoad().size() + "coordinator/loadQueue/count", queuePeon.getSegmentsToLoad().size() ) ); emitter.emit( new ServiceMetricEvent.Builder() .setUser1(serverName).build( - "master/dropQueue/count", queuePeon.getSegmentsToDrop().size() + "coordinator/dropQueue/count", queuePeon.getSegmentsToDrop().size() ) ); } @@ -220,7 +220,7 @@ public class DruidMasterLogger implements DruidMasterHelper emitter.emit( new ServiceMetricEvent.Builder() .setUser1(dataSource).build( - "master/segment/size", size + "coordinator/segment/size", size ) ); } @@ -230,7 +230,7 @@ public class DruidMasterLogger implements DruidMasterHelper emitter.emit( new ServiceMetricEvent.Builder() .setUser1(dataSource).build( - "master/segment/count", count + "coordinator/segment/count", count ) ); } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterRuleRunner.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuleRunner.java similarity index 73% rename from server/src/main/java/io/druid/server/master/DruidMasterRuleRunner.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuleRunner.java index a10832c36ee..594889201cf 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuleRunner.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.metamx.emitter.EmittingLogger; import io.druid.db.DatabaseRuleManager; -import io.druid.server.master.rules.Rule; +import io.druid.server.coordinator.rules.Rule; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -29,35 +29,35 @@ import java.util.List; /** */ -public class DruidMasterRuleRunner implements DruidMasterHelper +public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper { - private static final EmittingLogger log = new EmittingLogger(DruidMasterRuleRunner.class); + private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class); private final ReplicationThrottler replicatorThrottler; - private final DruidMaster master; + private final DruidCoordinator coordinator; - public DruidMasterRuleRunner(DruidMaster master) + public DruidCoordinatorRuleRunner(DruidCoordinator coordinator) { this( new ReplicationThrottler( - master.getDynamicConfigs().getReplicationThrottleLimit(), - master.getDynamicConfigs().getReplicantLifetime() + coordinator.getDynamicConfigs().getReplicationThrottleLimit(), + coordinator.getDynamicConfigs().getReplicantLifetime() ), - master + coordinator ); } - public DruidMasterRuleRunner(ReplicationThrottler replicatorThrottler, DruidMaster master) + public DruidCoordinatorRuleRunner(ReplicationThrottler replicatorThrottler, DruidCoordinator coordinator) { this.replicatorThrottler = replicatorThrottler; - this.master = master; + this.coordinator = coordinator; } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - MasterStats stats = new MasterStats(); + CoordinatorStats stats = new CoordinatorStats(); DruidCluster cluster = params.getDruidCluster(); if (cluster.isEmpty()) { @@ -70,7 +70,7 @@ public class DruidMasterRuleRunner implements DruidMasterHelper replicatorThrottler.updateTerminationState(tier); } - DruidMasterRuntimeParams paramsWithReplicationManager = params.buildFromExisting() + DruidCoordinatorRuntimeParams paramsWithReplicationManager = params.buildFromExisting() .withReplicationManager(replicatorThrottler) .build(); @@ -82,7 +82,7 @@ public class DruidMasterRuleRunner implements DruidMasterHelper boolean foundMatchingRule = false; for (Rule rule : rules) { if (rule.appliesTo(segment, now)) { - stats.accumulate(rule.run(master, paramsWithReplicationManager, segment)); + stats.accumulate(rule.run(coordinator, paramsWithReplicationManager, segment)); foundMatchingRule = true; break; } @@ -99,7 +99,7 @@ public class DruidMasterRuleRunner implements DruidMasterHelper } return paramsWithReplicationManager.buildFromExisting() - .withMasterStats(stats) + .withCoordinatorStats(stats) .build(); } } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterRuntimeParams.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java similarity index 86% rename from server/src/main/java/io/druid/server/master/DruidMasterRuntimeParams.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index c1e497eff9b..6ce41051400 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -35,7 +35,7 @@ import java.util.Set; /** */ -public class DruidMasterRuntimeParams +public class DruidCoordinatorRuntimeParams { private final long startTime; private final DruidCluster druidCluster; @@ -46,12 +46,12 @@ public class DruidMasterRuntimeParams private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; - private final MasterDynamicConfig masterDynamicConfig; - private final MasterStats stats; + private final CoordinatorDynamicConfig coordinatorDynamicConfig; + private final CoordinatorStats stats; private final DateTime balancerReferenceTimestamp; private final BalancerStrategyFactory strategyFactory; - public DruidMasterRuntimeParams( + public DruidCoordinatorRuntimeParams( long startTime, DruidCluster druidCluster, DatabaseRuleManager databaseRuleManager, @@ -61,8 +61,8 @@ public class DruidMasterRuntimeParams Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, - MasterDynamicConfig masterDynamicConfig, - MasterStats stats, + CoordinatorDynamicConfig coordinatorDynamicConfig, + CoordinatorStats stats, DateTime balancerReferenceTimestamp, BalancerStrategyFactory strategyFactory ) @@ -76,7 +76,7 @@ public class DruidMasterRuntimeParams this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; - this.masterDynamicConfig = masterDynamicConfig; + this.coordinatorDynamicConfig = coordinatorDynamicConfig; this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.strategyFactory = strategyFactory; @@ -127,12 +127,12 @@ public class DruidMasterRuntimeParams return emitter; } - public MasterDynamicConfig getMasterDynamicConfig() + public CoordinatorDynamicConfig getCoordinatorDynamicConfig() { - return masterDynamicConfig; + return coordinatorDynamicConfig; } - public MasterStats getMasterStats() + public CoordinatorStats getCoordinatorStats() { return stats; } @@ -149,7 +149,7 @@ public class DruidMasterRuntimeParams public boolean hasDeletionWaitTimeElapsed() { - return (System.currentTimeMillis() - getStartTime() > masterDynamicConfig.getMillisToWaitBeforeDeleting()); + return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting()); } public static Builder newBuilder() @@ -169,7 +169,7 @@ public class DruidMasterRuntimeParams loadManagementPeons, replicationManager, emitter, - masterDynamicConfig, + coordinatorDynamicConfig, stats, balancerReferenceTimestamp, strategyFactory @@ -187,8 +187,8 @@ public class DruidMasterRuntimeParams private final Map loadManagementPeons; private ReplicationThrottler replicationManager; private ServiceEmitter emitter; - private MasterDynamicConfig masterDynamicConfig; - private MasterStats stats; + private CoordinatorDynamicConfig coordinatorDynamicConfig; + private CoordinatorStats stats; private DateTime balancerReferenceTimestamp; private BalancerStrategyFactory strategyFactory; @@ -203,8 +203,8 @@ public class DruidMasterRuntimeParams this.loadManagementPeons = Maps.newHashMap(); this.replicationManager = null; this.emitter = null; - this.stats = new MasterStats(); - this.masterDynamicConfig = new MasterDynamicConfig.Builder().build(); + this.stats = new CoordinatorStats(); + this.coordinatorDynamicConfig = new CoordinatorDynamicConfig.Builder().build(); this.balancerReferenceTimestamp = null; this.strategyFactory = new CostBalancerStrategyFactory(); } @@ -219,8 +219,8 @@ public class DruidMasterRuntimeParams Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, - MasterDynamicConfig masterDynamicConfig, - MasterStats stats, + CoordinatorDynamicConfig coordinatorDynamicConfig, + CoordinatorStats stats, DateTime balancerReferenceTimestamp, BalancerStrategyFactory strategyFactory ) @@ -234,15 +234,15 @@ public class DruidMasterRuntimeParams this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; - this.masterDynamicConfig = masterDynamicConfig; + this.coordinatorDynamicConfig = coordinatorDynamicConfig; this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.strategyFactory=strategyFactory; } - public DruidMasterRuntimeParams build() + public DruidCoordinatorRuntimeParams build() { - return new DruidMasterRuntimeParams( + return new DruidCoordinatorRuntimeParams( startTime, druidCluster, databaseRuleManager, @@ -252,7 +252,7 @@ public class DruidMasterRuntimeParams loadManagementPeons, replicationManager, emitter, - masterDynamicConfig, + coordinatorDynamicConfig, stats, balancerReferenceTimestamp, strategyFactory @@ -313,15 +313,15 @@ public class DruidMasterRuntimeParams return this; } - public Builder withMasterStats(MasterStats stats) + public Builder withCoordinatorStats(CoordinatorStats stats) { this.stats.accumulate(stats); return this; } - public Builder withDynamicConfigs(MasterDynamicConfig configs) + public Builder withDynamicConfigs(CoordinatorDynamicConfig configs) { - this.masterDynamicConfig = configs; + this.coordinatorDynamicConfig = configs; return this; } diff --git a/server/src/main/java/io/druid/server/master/DruidMasterSegmentInfoLoader.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentInfoLoader.java similarity index 71% rename from server/src/main/java/io/druid/server/master/DruidMasterSegmentInfoLoader.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentInfoLoader.java index b80eaf8b42d..69f8ccdbc43 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterSegmentInfoLoader.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentInfoLoader.java @@ -17,29 +17,29 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.metamx.common.logger.Logger; import io.druid.timeline.DataSegment; import java.util.Set; -public class DruidMasterSegmentInfoLoader implements DruidMasterHelper +public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper { - private final DruidMaster master; + private final DruidCoordinator coordinator; - private static final Logger log = new Logger(DruidMasterSegmentInfoLoader.class); + private static final Logger log = new Logger(DruidCoordinatorSegmentInfoLoader.class); - public DruidMasterSegmentInfoLoader(DruidMaster master) + public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) { - this.master = master; + this.coordinator = coordinator; } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { // Display info about all available segments - final Set availableSegments = master.getAvailableDataSegments(); + final Set availableSegments = coordinator.getAvailableDataSegments(); if (log.isDebugEnabled()) { log.debug("Available DataSegments"); for (DataSegment dataSegment : availableSegments) { diff --git a/server/src/main/java/io/druid/server/master/DruidMasterSegmentMerger.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentMerger.java similarity index 92% rename from server/src/main/java/io/druid/server/master/DruidMasterSegmentMerger.java rename to server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentMerger.java index 7c57d7dbbef..839cda93563 100644 --- a/server/src/main/java/io/druid/server/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorSegmentMerger.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -47,14 +47,14 @@ import java.util.concurrent.atomic.AtomicReference; /** */ -public class DruidMasterSegmentMerger implements DruidMasterHelper +public class DruidCoordinatorSegmentMerger implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidMasterSegmentMerger.class); + private static final Logger log = new Logger(DruidCoordinatorSegmentMerger.class); private final IndexingServiceClient indexingServiceClient; private final AtomicReference whiteListRef; - public DruidMasterSegmentMerger( + public DruidCoordinatorSegmentMerger( IndexingServiceClient indexingServiceClient, AtomicReference whitelistRef ) @@ -64,11 +64,11 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper } @Override - public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { DatasourceWhitelist whitelist = whiteListRef.get(); - MasterStats stats = new MasterStats(); + CoordinatorStats stats = new CoordinatorStats(); Map> dataSources = Maps.newHashMap(); // Find serviced segments by using a timeline @@ -99,9 +99,9 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper for (int i = 0; i < timelineObjects.size(); i++) { if (!segmentsToMerge.add(timelineObjects.get(i)) - || segmentsToMerge.getByteCount() > params.getMasterDynamicConfig().getMergeBytesLimit() - || segmentsToMerge.getSegmentCount() >= params.getMasterDynamicConfig().getMergeSegmentsLimit()) { - i -= segmentsToMerge.backtrack(params.getMasterDynamicConfig().getMergeBytesLimit()); + || segmentsToMerge.getByteCount() > params.getCoordinatorDynamicConfig().getMergeBytesLimit() + || segmentsToMerge.getSegmentCount() >= params.getCoordinatorDynamicConfig().getMergeSegmentsLimit()) { + i -= segmentsToMerge.backtrack(params.getCoordinatorDynamicConfig().getMergeBytesLimit()); if (segmentsToMerge.getSegmentCount() > 1) { stats.addToGlobalStat("mergedCount", mergeSegments(segmentsToMerge, entry.getKey())); @@ -117,14 +117,14 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper } // Finish any timelineObjects to merge that may have not hit threshold - segmentsToMerge.backtrack(params.getMasterDynamicConfig().getMergeBytesLimit()); + segmentsToMerge.backtrack(params.getCoordinatorDynamicConfig().getMergeBytesLimit()); if (segmentsToMerge.getSegmentCount() > 1) { stats.addToGlobalStat("mergedCount", mergeSegments(segmentsToMerge, entry.getKey())); } } return params.buildFromExisting() - .withMasterStats(stats) + .withCoordinatorStats(stats) .build(); } diff --git a/server/src/main/java/io/druid/server/master/LoadPeonCallback.java b/server/src/main/java/io/druid/server/coordinator/LoadPeonCallback.java similarity index 96% rename from server/src/main/java/io/druid/server/master/LoadPeonCallback.java rename to server/src/main/java/io/druid/server/coordinator/LoadPeonCallback.java index 50a513961e9..ff0cea085ca 100644 --- a/server/src/main/java/io/druid/server/master/LoadPeonCallback.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadPeonCallback.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; /** */ diff --git a/server/src/main/java/io/druid/server/master/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java similarity index 98% rename from server/src/main/java/io/druid/server/master/LoadQueuePeon.java rename to server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index ece30058a3f..dcd9a77a8f8 100644 --- a/server/src/main/java/io/druid/server/master/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -72,7 +72,7 @@ public class LoadQueuePeon private final String basePath; private final ObjectMapper jsonMapper; private final ScheduledExecutorService zkWritingExecutor; - private final DruidMasterConfig config; + private final DruidCoordinatorConfig config; private final AtomicLong queuedSize = new AtomicLong(0); private final AtomicInteger failedAssignCount = new AtomicInteger(0); @@ -93,7 +93,7 @@ public class LoadQueuePeon String basePath, ObjectMapper jsonMapper, ScheduledExecutorService zkWritingExecutor, - DruidMasterConfig config + DruidCoordinatorConfig config ) { this.curator = curator; @@ -386,7 +386,7 @@ public class LoadQueuePeon synchronized (lock) { log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading); failedAssignCount.getAndIncrement(); - // Act like it was completed so that the master gives it to someone else + // Act like it was completed so that the coordinator gives it to someone else actionCompleted(); doNext(); } diff --git a/server/src/main/java/io/druid/server/master/LoadQueueTaskMaster.java b/server/src/main/java/io/druid/server/coordinator/LoadQueueTaskMaster.java similarity index 93% rename from server/src/main/java/io/druid/server/master/LoadQueueTaskMaster.java rename to server/src/main/java/io/druid/server/coordinator/LoadQueueTaskMaster.java index 1d1d114d6dc..74841853589 100644 --- a/server/src/main/java/io/druid/server/master/LoadQueueTaskMaster.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueueTaskMaster.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; @@ -33,14 +33,14 @@ public class LoadQueueTaskMaster private final CuratorFramework curator; private final ObjectMapper jsonMapper; private final ScheduledExecutorService peonExec; - private final DruidMasterConfig config; + private final DruidCoordinatorConfig config; @Inject public LoadQueueTaskMaster( CuratorFramework curator, ObjectMapper jsonMapper, ScheduledExecutorService peonExec, - DruidMasterConfig config + DruidCoordinatorConfig config ) { this.curator = curator; diff --git a/server/src/main/java/io/druid/server/master/RandomBalancerStrategy.java b/server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategy.java similarity index 94% rename from server/src/main/java/io/druid/server/master/RandomBalancerStrategy.java rename to server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategy.java index cc8e62afec6..70389162da9 100644 --- a/server/src/main/java/io/druid/server/master/RandomBalancerStrategy.java +++ b/server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.timeline.DataSegment; @@ -64,7 +64,7 @@ public class RandomBalancerStrategy implements BalancerStrategy @Override public void emitStats( - String tier, MasterStats stats, List serverHolderList + String tier, CoordinatorStats stats, List serverHolderList ) { } diff --git a/server/src/main/java/io/druid/server/master/RandomBalancerStrategyFactory.java b/server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategyFactory.java similarity index 96% rename from server/src/main/java/io/druid/server/master/RandomBalancerStrategyFactory.java rename to server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategyFactory.java index e2f4641ed56..62c1994a080 100644 --- a/server/src/main/java/io/druid/server/master/RandomBalancerStrategyFactory.java +++ b/server/src/main/java/io/druid/server/coordinator/RandomBalancerStrategyFactory.java @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import org.joda.time.DateTime; diff --git a/server/src/main/java/io/druid/server/master/ReplicationThrottler.java b/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java similarity index 99% rename from server/src/main/java/io/druid/server/master/ReplicationThrottler.java rename to server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java index 9584bf46f8c..2559bc5ce51 100644 --- a/server/src/main/java/io/druid/server/master/ReplicationThrottler.java +++ b/server/src/main/java/io/druid/server/coordinator/ReplicationThrottler.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/server/src/main/java/io/druid/server/master/ReservoirSegmentSampler.java b/server/src/main/java/io/druid/server/coordinator/ReservoirSegmentSampler.java similarity index 97% rename from server/src/main/java/io/druid/server/master/ReservoirSegmentSampler.java rename to server/src/main/java/io/druid/server/coordinator/ReservoirSegmentSampler.java index b50cfc8ace0..1b30ed3d3dd 100644 --- a/server/src/main/java/io/druid/server/master/ReservoirSegmentSampler.java +++ b/server/src/main/java/io/druid/server/coordinator/ReservoirSegmentSampler.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.timeline.DataSegment; diff --git a/server/src/main/java/io/druid/server/master/SegmentReplicantLookup.java b/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java similarity index 99% rename from server/src/main/java/io/druid/server/master/SegmentReplicantLookup.java rename to server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java index d455a89c5f9..a90eeba4cae 100644 --- a/server/src/main/java/io/druid/server/master/SegmentReplicantLookup.java +++ b/server/src/main/java/io/druid/server/coordinator/SegmentReplicantLookup.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Maps; diff --git a/server/src/main/java/io/druid/server/master/ServerHolder.java b/server/src/main/java/io/druid/server/coordinator/ServerHolder.java similarity index 98% rename from server/src/main/java/io/druid/server/master/ServerHolder.java rename to server/src/main/java/io/druid/server/coordinator/ServerHolder.java index 1a6d40b57b5..dd3d4f698cf 100644 --- a/server/src/main/java/io/druid/server/master/ServerHolder.java +++ b/server/src/main/java/io/druid/server/coordinator/ServerHolder.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.metamx.common.logger.Logger; import io.druid.client.DruidServer; diff --git a/server/src/main/java/io/druid/server/master/rules/DropRule.java b/server/src/main/java/io/druid/server/coordinator/rules/DropRule.java similarity index 71% rename from server/src/main/java/io/druid/server/master/rules/DropRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/DropRule.java index f9040cb46b7..03cdab88766 100644 --- a/server/src/main/java/io/druid/server/master/rules/DropRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/DropRule.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.DruidMasterRuntimeParams; -import io.druid.server.master.MasterStats; +import io.druid.server.coordinator.CoordinatorStats; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; import io.druid.timeline.DataSegment; /** @@ -30,10 +30,10 @@ import io.druid.timeline.DataSegment; public abstract class DropRule implements Rule { @Override - public MasterStats run(DruidMaster master, DruidMasterRuntimeParams params, DataSegment segment) + public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { - MasterStats stats = new MasterStats(); - master.removeSegment(segment); + CoordinatorStats stats = new CoordinatorStats(); + coordinator.removeSegment(segment); stats.addToGlobalStat("deletedCount", 1); return stats; } diff --git a/server/src/main/java/io/druid/server/master/rules/IntervalDropRule.java b/server/src/main/java/io/druid/server/coordinator/rules/IntervalDropRule.java similarity index 97% rename from server/src/main/java/io/druid/server/master/rules/IntervalDropRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/IntervalDropRule.java index f5b570ca626..8ea0bd30574 100644 --- a/server/src/main/java/io/druid/server/master/rules/IntervalDropRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/IntervalDropRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/master/rules/IntervalLoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java similarity index 98% rename from server/src/main/java/io/druid/server/master/rules/IntervalLoadRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java index 436c2e98e0a..4fe01521e66 100644 --- a/server/src/main/java/io/druid/server/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/IntervalLoadRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/master/rules/LoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java similarity index 89% rename from server/src/main/java/io/druid/server/master/rules/LoadRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java index 6caafeb15c3..792d76d01b2 100644 --- a/server/src/main/java/io/druid/server/master/rules/LoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/LoadRule.java @@ -17,18 +17,18 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.emitter.EmittingLogger; -import io.druid.server.master.BalancerStrategy; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.DruidMasterRuntimeParams; -import io.druid.server.master.LoadPeonCallback; -import io.druid.server.master.MasterStats; -import io.druid.server.master.ReplicationThrottler; -import io.druid.server.master.ServerHolder; +import io.druid.server.coordinator.CoordinatorStats; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import io.druid.server.coordinator.LoadPeonCallback; +import io.druid.server.coordinator.BalancerStrategy; +import io.druid.server.coordinator.ReplicationThrottler; +import io.druid.server.coordinator.ServerHolder; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -44,9 +44,9 @@ public abstract class LoadRule implements Rule private static final EmittingLogger log = new EmittingLogger(LoadRule.class); @Override - public MasterStats run(DruidMaster master, DruidMasterRuntimeParams params, DataSegment segment) + public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { - MasterStats stats = new MasterStats(); + CoordinatorStats stats = new CoordinatorStats(); int expectedReplicants = getReplicants(); int totalReplicants = params.getSegmentReplicantLookup().getTotalReplicants(segment.getIdentifier(), getTier()); @@ -79,7 +79,7 @@ public abstract class LoadRule implements Rule return stats; } - private MasterStats assign( + private CoordinatorStats assign( final ReplicationThrottler replicationManager, final int expectedReplicants, int totalReplicants, @@ -88,7 +88,7 @@ public abstract class LoadRule implements Rule final DataSegment segment ) { - final MasterStats stats = new MasterStats(); + final CoordinatorStats stats = new CoordinatorStats(); while (totalReplicants < expectedReplicants) { boolean replicate = totalReplicants > 0; @@ -138,14 +138,14 @@ public abstract class LoadRule implements Rule return stats; } - private MasterStats drop( + private CoordinatorStats drop( int expectedReplicants, int clusterReplicants, final DataSegment segment, - final DruidMasterRuntimeParams params + final DruidCoordinatorRuntimeParams params ) { - MasterStats stats = new MasterStats(); + CoordinatorStats stats = new CoordinatorStats(); final ReplicationThrottler replicationManager = params.getReplicationManager(); if (!params.hasDeletionWaitTimeElapsed()) { diff --git a/server/src/main/java/io/druid/server/master/rules/PeriodDropRule.java b/server/src/main/java/io/druid/server/coordinator/rules/PeriodDropRule.java similarity index 97% rename from server/src/main/java/io/druid/server/master/rules/PeriodDropRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/PeriodDropRule.java index 7a490bccf91..1e7b1208628 100644 --- a/server/src/main/java/io/druid/server/master/rules/PeriodDropRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/PeriodDropRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/master/rules/PeriodLoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/PeriodLoadRule.java similarity index 98% rename from server/src/main/java/io/druid/server/master/rules/PeriodLoadRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/PeriodLoadRule.java index e75734a0737..a2048756894 100644 --- a/server/src/main/java/io/druid/server/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/PeriodLoadRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/master/rules/Rule.java b/server/src/main/java/io/druid/server/coordinator/rules/Rule.java similarity index 83% rename from server/src/main/java/io/druid/server/master/rules/Rule.java rename to server/src/main/java/io/druid/server/coordinator/rules/Rule.java index c576dc3a9fd..7c720e1aa2f 100644 --- a/server/src/main/java/io/druid/server/master/rules/Rule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/Rule.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.DruidMasterRuntimeParams; -import io.druid.server.master.MasterStats; +import io.druid.server.coordinator.CoordinatorStats; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -45,5 +45,5 @@ public interface Rule public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp); - public MasterStats run(DruidMaster master, DruidMasterRuntimeParams params, DataSegment segment); + public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment); } diff --git a/server/src/main/java/io/druid/server/master/rules/RuleMap.java b/server/src/main/java/io/druid/server/coordinator/rules/RuleMap.java similarity index 97% rename from server/src/main/java/io/druid/server/master/rules/RuleMap.java rename to server/src/main/java/io/druid/server/coordinator/rules/RuleMap.java index a8f80ee1b2d..412beb67e7e 100644 --- a/server/src/main/java/io/druid/server/master/rules/RuleMap.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/RuleMap.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.google.common.collect.Lists; diff --git a/server/src/main/java/io/druid/server/master/rules/SizeDropRule.java b/server/src/main/java/io/druid/server/coordinator/rules/SizeDropRule.java similarity index 97% rename from server/src/main/java/io/druid/server/master/rules/SizeDropRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/SizeDropRule.java index f7ad948a54b..219d5559b69 100644 --- a/server/src/main/java/io/druid/server/master/rules/SizeDropRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/SizeDropRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/master/rules/SizeLoadRule.java b/server/src/main/java/io/druid/server/coordinator/rules/SizeLoadRule.java similarity index 98% rename from server/src/main/java/io/druid/server/master/rules/SizeLoadRule.java rename to server/src/main/java/io/druid/server/coordinator/rules/SizeLoadRule.java index df33cb0f50d..6fdc10f822c 100644 --- a/server/src/main/java/io/druid/server/master/rules/SizeLoadRule.java +++ b/server/src/main/java/io/druid/server/coordinator/rules/SizeLoadRule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/http/BackwardsCompatiableInfoResource.java b/server/src/main/java/io/druid/server/http/BackwardsCompatiableInfoResource.java index 2d49b0c6884..11fe97da9be 100644 --- a/server/src/main/java/io/druid/server/http/BackwardsCompatiableInfoResource.java +++ b/server/src/main/java/io/druid/server/http/BackwardsCompatiableInfoResource.java @@ -24,7 +24,7 @@ import io.druid.client.InventoryView; import io.druid.client.indexing.IndexingServiceClient; import io.druid.db.DatabaseRuleManager; import io.druid.db.DatabaseSegmentManager; -import io.druid.server.master.DruidMaster; +import io.druid.server.coordinator.DruidCoordinator; import javax.annotation.Nullable; import javax.ws.rs.Path; @@ -36,13 +36,13 @@ public class BackwardsCompatiableInfoResource extends InfoResource { @Inject public BackwardsCompatiableInfoResource( - DruidMaster master, + DruidCoordinator coordinator, InventoryView serverInventoryView, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, @Nullable IndexingServiceClient indexingServiceClient ) { - super(master, serverInventoryView, databaseSegmentManager, databaseRuleManager, indexingServiceClient); + super(coordinator, serverInventoryView, databaseSegmentManager, databaseRuleManager, indexingServiceClient); } } diff --git a/server/src/main/java/io/druid/server/http/MasterDynamicConfigsResource.java b/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java similarity index 78% rename from server/src/main/java/io/druid/server/http/MasterDynamicConfigsResource.java rename to server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java index b1993cd8a5d..7efa753053b 100644 --- a/server/src/main/java/io/druid/server/http/MasterDynamicConfigsResource.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java @@ -20,7 +20,7 @@ package io.druid.server.http; import io.druid.common.config.JacksonConfigManager; -import io.druid.server.master.MasterDynamicConfig; +import io.druid.server.coordinator.CoordinatorDynamicConfig; import javax.inject.Inject; import javax.ws.rs.Consumes; @@ -32,13 +32,13 @@ import javax.ws.rs.core.Response; /** */ -@Path("/master/config") -public class MasterDynamicConfigsResource +@Path("/coordinator/config") +public class CoordinatorDynamicConfigsResource { private final JacksonConfigManager manager; @Inject - public MasterDynamicConfigsResource( + public CoordinatorDynamicConfigsResource( JacksonConfigManager manager ) { @@ -51,17 +51,17 @@ public class MasterDynamicConfigsResource { return Response.ok( manager.watch( - MasterDynamicConfig.CONFIG_KEY, - MasterDynamicConfig.class + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.class ).get() ).build(); } @POST @Consumes("application/json") - public Response setDynamicConfigs(final MasterDynamicConfig dynamicConfig) + public Response setDynamicConfigs(final CoordinatorDynamicConfig dynamicConfig) { - if (!manager.set(MasterDynamicConfig.CONFIG_KEY, dynamicConfig)) { + if (!manager.set(CoordinatorDynamicConfig.CONFIG_KEY, dynamicConfig)) { return Response.status(Response.Status.BAD_REQUEST).build(); } return Response.ok().build(); diff --git a/server/src/main/java/io/druid/server/http/MasterRedirectInfo.java b/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java similarity index 74% rename from server/src/main/java/io/druid/server/http/MasterRedirectInfo.java rename to server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java index 3857ea04d63..e3c89d6e6d6 100644 --- a/server/src/main/java/io/druid/server/http/MasterRedirectInfo.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorRedirectInfo.java @@ -21,37 +21,37 @@ package io.druid.server.http; import com.google.common.base.Throwables; import com.google.inject.Inject; -import io.druid.server.master.DruidMaster; +import io.druid.server.coordinator.DruidCoordinator; import java.net.URL; /** */ -public class MasterRedirectInfo implements RedirectInfo +public class CoordinatorRedirectInfo implements RedirectInfo { - private final DruidMaster master; + private final DruidCoordinator coordinator; @Inject - public MasterRedirectInfo(DruidMaster master) { - this.master = master; + public CoordinatorRedirectInfo(DruidCoordinator coordinator) { + this.coordinator = coordinator; } @Override public boolean doLocal() { - return master.isClusterMaster(); + return coordinator.isLeader(); } @Override public URL getRedirectURL(String queryString, String requestURI) { try { - final String currentMaster = master.getCurrentMaster(); - if (currentMaster == null) { + final String leader = coordinator.getCurrentLeader(); + if (leader == null) { return null; } - String location = String.format("http://%s%s", currentMaster, requestURI); + String location = String.format("http://%s%s", leader, requestURI); if (queryString != null) { location = String.format("%s?%s", location, queryString); diff --git a/server/src/main/java/io/druid/server/http/MasterResource.java b/server/src/main/java/io/druid/server/http/CoordinatorResource.java similarity index 86% rename from server/src/main/java/io/druid/server/http/MasterResource.java rename to server/src/main/java/io/druid/server/http/CoordinatorResource.java index f44bb93b72d..1c70dd39c8c 100644 --- a/server/src/main/java/io/druid/server/http/MasterResource.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorResource.java @@ -20,9 +20,8 @@ package io.druid.server.http; import com.google.inject.Inject; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.LoadPeonCallback; -import io.druid.server.master.MasterDynamicConfig; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.LoadPeonCallback; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -34,17 +33,17 @@ import java.util.List; /** */ -@Path("/master") -public class MasterResource +@Path("/coordinator") +public class CoordinatorResource { - private final DruidMaster master; + private final DruidCoordinator coordinator; @Inject - public MasterResource( - DruidMaster master + public CoordinatorResource( + DruidCoordinator coordinator ) { - this.master = master; + this.coordinator = coordinator; } @POST @@ -55,7 +54,7 @@ public class MasterResource Response resp = Response.status(Response.Status.OK).build(); for (SegmentToMove segmentToMove : segmentsToMove) { try { - master.moveSegment( + coordinator.moveSegment( segmentToMove.getFromServer(), segmentToMove.getToServer(), segmentToMove.getSegmentName(), @@ -88,7 +87,7 @@ public class MasterResource Response resp = Response.status(Response.Status.OK).build(); for (SegmentToDrop segmentToDrop : segmentsToDrop) { try { - master.dropSegment( + coordinator.dropSegment( segmentToDrop.getFromServer(), segmentToDrop.getSegmentName(), new LoadPeonCallback() { @Override @@ -115,6 +114,6 @@ public class MasterResource @Produces("application/json") public Response getLoadStatus() { - return Response.ok(master.getLoadStatus()).build(); + return Response.ok(coordinator.getLoadStatus()).build(); } } \ No newline at end of file diff --git a/server/src/main/java/io/druid/server/http/InfoResource.java b/server/src/main/java/io/druid/server/http/InfoResource.java index 670942d06cf..c702c4450ca 100644 --- a/server/src/main/java/io/druid/server/http/InfoResource.java +++ b/server/src/main/java/io/druid/server/http/InfoResource.java @@ -33,8 +33,8 @@ import io.druid.client.InventoryView; import io.druid.client.indexing.IndexingServiceClient; import io.druid.db.DatabaseRuleManager; import io.druid.db.DatabaseSegmentManager; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.rules.Rule; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.rules.Rule; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -96,7 +96,7 @@ public class InfoResource } }; - private final DruidMaster master; + private final DruidCoordinator coordinator; private final InventoryView serverInventoryView; private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; @@ -105,7 +105,7 @@ public class InfoResource @Inject public InfoResource( - DruidMaster master, + DruidCoordinator coordinator, InventoryView serverInventoryView, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, @@ -113,7 +113,7 @@ public class InfoResource IndexingServiceClient indexingServiceClient ) { - this.master = master; + this.coordinator = coordinator; this.serverInventoryView = serverInventoryView; this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; @@ -121,12 +121,12 @@ public class InfoResource } @GET - @Path("/master") + @Path("/coordinator") @Produces("application/json") public Response getMaster() { return Response.status(Response.Status.OK) - .entity(master.getCurrentMaster()) + .entity(coordinator.getCurrentLeader()) .build(); } diff --git a/server/src/main/java/io/druid/server/http/RedirectFilter.java b/server/src/main/java/io/druid/server/http/RedirectFilter.java index 671a2cb207e..0968cdd5c92 100644 --- a/server/src/main/java/io/druid/server/http/RedirectFilter.java +++ b/server/src/main/java/io/druid/server/http/RedirectFilter.java @@ -74,7 +74,7 @@ public class RedirectFilter implements Filter log.debug("Forwarding request to [%s]", url); if (url == null) { - // We apparently have no master, so let's do a Service Unavailable + // We apparently have no coordinator, so let's do a Service Unavailable response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE); return; } diff --git a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java index 0635938a6e5..ab296698548 100644 --- a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java @@ -60,10 +60,10 @@ public abstract class ZkPathsConfig return defaultPath("loadQueue"); } - @Config("druid.zk.paths.masterPath") - public String getMasterPath() + @Config("druid.zk.paths.coordinatorPath") + public String getCoordinatorPath() { - return defaultPath("master"); + return defaultPath("coordinator"); } @Config("druid.zk.paths.indexer.announcementsPath") diff --git a/server/src/main/resources/static/cluster.html b/server/src/main/resources/static/cluster.html index 5bc9b857999..3f4512b8713 100644 --- a/server/src/main/resources/static/cluster.html +++ b/server/src/main/resources/static/cluster.html @@ -20,10 +20,10 @@ - Druid Master Console + Druid Coordinator Console - + @@ -44,7 +44,7 @@

Cluster Segments
-
+
Loading segment data... this may take a few minutes
diff --git a/server/src/main/resources/static/config.html b/server/src/main/resources/static/config.html index 7ca2cb4f7a2..bf6ddca9722 100644 --- a/server/src/main/resources/static/config.html +++ b/server/src/main/resources/static/config.html @@ -20,10 +20,10 @@ - Druid Master Console - Rule Configuration + Druid Coordinator Console - Rule Configuration - + @@ -36,7 +36,7 @@
-
Master Configuration
+
Coordinator Configuration
diff --git a/server/src/main/resources/static/enable.html b/server/src/main/resources/static/enable.html index 265ae7780df..ae080b0a995 100644 --- a/server/src/main/resources/static/enable.html +++ b/server/src/main/resources/static/enable.html @@ -20,10 +20,10 @@ - Druid Master Console - Enable/Disable Datasources + Druid Coordinator Console - Enable/Disable Datasources - + diff --git a/server/src/main/resources/static/index.html b/server/src/main/resources/static/index.html index b158c671bef..cec3d620e88 100644 --- a/server/src/main/resources/static/index.html +++ b/server/src/main/resources/static/index.html @@ -18,13 +18,13 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + - Druid Master Console + Druid Coordinator Console - + @@ -36,10 +36,10 @@ View Information about the Cluster
- Configure Compute Node Rules + Configure Assignment Rules
- Configure Master Parameters + Configure Coordinator Parameters
Enable/Disable Datasources diff --git a/server/src/main/resources/static/js/config-0.0.1.js b/server/src/main/resources/static/js/config-0.0.1.js index 32bdfb7b093..5b092a4e76f 100644 --- a/server/src/main/resources/static/js/config-0.0.1.js +++ b/server/src/main/resources/static/js/config-0.0.1.js @@ -20,7 +20,7 @@ function domToConfig(configDiv) { } function getConfigs() { - $.getJSON("/master/config", function(data) { + $.getJSON("/coordinator/config", function(data) { $('#config_list').empty(); $.each(data, function (key, value) { @@ -72,7 +72,7 @@ $(document).ready(function() { $.ajax({ type: 'POST', - url:'/master/config', + url:'/coordinator/config', data: JSON.stringify(configs), contentType:"application/json; charset=utf-8", dataType:"json", diff --git a/server/src/main/resources/static/js/init-0.0.2.js b/server/src/main/resources/static/js/init-0.0.2.js index b18a51751ac..914b9b1521a 100644 --- a/server/src/main/resources/static/js/init-0.0.2.js +++ b/server/src/main/resources/static/js/init-0.0.2.js @@ -100,8 +100,8 @@ $(document).ready(function() { } // Execution stuff - $.get('/info/master', function(data) { - $("#master").html('Current Cluster Master: ' + data.host); + $.get('/info/coordinator', function(data) { + $("#coordinator").html('Current Cluster Coordinator: ' + data.host); }); $('#move_segment').submit(function() { @@ -120,7 +120,7 @@ $(document).ready(function() { /* $.ajax({ - url:"/master/move", + url:"/coordinator/move", type: "POST", data: JSON.stringify(data), contentType:"application/json; charset=utf-8", @@ -154,7 +154,7 @@ $(document).ready(function() { } $.ajax({ - url:"/master/drop", + url:"/coordinator/drop", type: "POST", data: JSON.stringify(data), contentType:"application/json; charset=utf-8", diff --git a/server/src/main/resources/static/kill.html b/server/src/main/resources/static/kill.html index 8741fd25f41..9fe3af9e6be 100644 --- a/server/src/main/resources/static/kill.html +++ b/server/src/main/resources/static/kill.html @@ -20,10 +20,10 @@ - Druid Master Console - Enable/Disable Datasources + Druid Coordinator Console - Enable/Disable Datasources - + diff --git a/server/src/main/resources/static/rules.html b/server/src/main/resources/static/rules.html index 7630c16c7cd..85729f00ca6 100644 --- a/server/src/main/resources/static/rules.html +++ b/server/src/main/resources/static/rules.html @@ -20,10 +20,10 @@ - Druid Master Console - Rule Configuration + Druid Coordinator Console - Rule Configuration - + diff --git a/server/src/main/resources/static/view.html b/server/src/main/resources/static/view.html index 1d4c1f931d3..80432597eb6 100644 --- a/server/src/main/resources/static/view.html +++ b/server/src/main/resources/static/view.html @@ -20,10 +20,10 @@ - Druid Master Console + Druid Coordinator Console - + diff --git a/server/src/test/java/io/druid/server/master/DruidMasterBalancerProfiler.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java similarity index 87% rename from server/src/test/java/io/druid/server/master/DruidMasterBalancerProfiler.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index 568a8d8b863..2d95fec765b 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterBalancerProfiler.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.base.Stopwatch; import com.google.common.collect.ImmutableList; @@ -29,8 +29,8 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.DruidServer; import io.druid.db.DatabaseRuleManager; -import io.druid.server.master.rules.PeriodLoadRule; -import io.druid.server.master.rules.Rule; +import io.druid.server.coordinator.rules.PeriodLoadRule; +import io.druid.server.coordinator.rules.Rule; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -44,10 +44,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -public class DruidMasterBalancerProfiler +public class DruidCoordinatorBalancerProfiler { private static final int MAX_SEGMENTS_TO_MOVE = 5; - private DruidMaster master; + private DruidCoordinator coordinator; private DruidServer druidServer1; private DruidServer druidServer2; Map segments = Maps.newHashMap(); @@ -59,7 +59,7 @@ public class DruidMasterBalancerProfiler @Before public void setUp() throws Exception { - master = EasyMock.createMock(DruidMaster.class); + coordinator = EasyMock.createMock(DruidCoordinator.class); druidServer1 = EasyMock.createMock(DruidServer.class); druidServer2 = EasyMock.createMock(DruidServer.class); emitter = EasyMock.createMock(ServiceEmitter.class); @@ -77,14 +77,14 @@ public class DruidMasterBalancerProfiler EasyMock.expect(manager.getRulesWithDefault(EasyMock.anyObject())).andReturn(rules).anyTimes(); EasyMock.replay(manager); - master.moveSegment( + coordinator.moveSegment( EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(master); + EasyMock.replay(coordinator); List serverList = Lists.newArrayList(); Map peonMap = Maps.newHashMap(); @@ -128,13 +128,13 @@ public class DruidMasterBalancerProfiler serverHolderList.add(new ServerHolder(server, peon)); } - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( serverHolderList ) @@ -146,7 +146,7 @@ public class DruidMasterBalancerProfiler ) .withAvailableSegments(segmentMap.values()) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMaxSegmentsToMove( + new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).withReplicantLifetime(500) .withReplicationThrottleLimit(5) @@ -161,7 +161,7 @@ public class DruidMasterBalancerProfiler new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( serverHolderList ) @@ -171,11 +171,11 @@ public class DruidMasterBalancerProfiler ) .build(); - DruidMasterBalancerTester tester = new DruidMasterBalancerTester(master); - DruidMasterRuleRunner runner = new DruidMasterRuleRunner(master); + DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); + DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(coordinator); watch.start(); - DruidMasterRuntimeParams balanceParams = tester.run(params); - DruidMasterRuntimeParams assignParams = runner.run(params); + DruidCoordinatorRuntimeParams balanceParams = tester.run(params); + DruidCoordinatorRuntimeParams assignParams = runner.run(params); System.out.println(watch.stop()); } @@ -201,22 +201,22 @@ public class DruidMasterBalancerProfiler EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); - master.moveSegment( + coordinator.moveSegment( EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(master); + EasyMock.replay(coordinator); - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( Arrays.asList( new ServerHolder(druidServer1, fromPeon), @@ -236,15 +236,15 @@ public class DruidMasterBalancerProfiler ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMaxSegmentsToMove( + new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).build() ) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterBalancerTester tester = new DruidMasterBalancerTester(master); + DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); watch.start(); - DruidMasterRuntimeParams balanceParams = tester.run(params); + DruidCoordinatorRuntimeParams balanceParams = tester.run(params); System.out.println(watch.stop()); } diff --git a/server/src/test/java/io/druid/server/master/DruidMasterBalancerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java similarity index 88% rename from server/src/test/java/io/druid/server/master/DruidMasterBalancerTest.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 4b25de80e01..6c64722696d 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -40,10 +40,10 @@ import java.util.Map; /** */ -public class DruidMasterBalancerTest +public class DruidCoordinatorBalancerTest { private static final int MAX_SEGMENTS_TO_MOVE = 5; - private DruidMaster master; + private DruidCoordinator coordinator; private DruidServer druidServer1; private DruidServer druidServer2; private DruidServer druidServer3; @@ -57,7 +57,7 @@ public class DruidMasterBalancerTest @Before public void setUp() throws Exception { - master = EasyMock.createMock(DruidMaster.class); + coordinator = EasyMock.createMock(DruidCoordinator.class); druidServer1 = EasyMock.createMock(DruidServer.class); druidServer2 = EasyMock.createMock(DruidServer.class); druidServer3 = EasyMock.createMock(DruidServer.class); @@ -125,7 +125,7 @@ public class DruidMasterBalancerTest @After public void tearDown() throws Exception { - EasyMock.verify(master); + EasyMock.verify(coordinator); EasyMock.verify(druidServer1); EasyMock.verify(druidServer2); EasyMock.verify(druidServer3); @@ -154,26 +154,26 @@ public class DruidMasterBalancerTest EasyMock.replay(druidServer3); EasyMock.replay(druidServer4); - // Mock stuff that the master needs - master.moveSegment( + // Mock stuff that the coordinator needs + coordinator.moveSegment( EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(master); + EasyMock.replay(coordinator); LoadQueuePeonTester fromPeon = new LoadQueuePeonTester(); LoadQueuePeonTester toPeon = new LoadQueuePeonTester(); - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( Arrays.asList( new ServerHolder(druidServer1, fromPeon), @@ -193,16 +193,16 @@ public class DruidMasterBalancerTest ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMaxSegmentsToMove( + new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).build() ) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - params = new DruidMasterBalancerTester(master).run(params); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() < segments.size()); + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() < segments.size()); } @@ -231,25 +231,25 @@ public class DruidMasterBalancerTest EasyMock.replay(druidServer3); EasyMock.replay(druidServer4); - // Mock stuff that the master needs - master.moveSegment( + // Mock stuff that the coordinator needs + coordinator.moveSegment( EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(master); + EasyMock.replay(coordinator); LoadQueuePeonTester fromPeon = new LoadQueuePeonTester(); LoadQueuePeonTester toPeon = new LoadQueuePeonTester(); - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( Arrays.asList( new ServerHolder(druidServer1, fromPeon), @@ -269,14 +269,14 @@ public class DruidMasterBalancerTest ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) + new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) .build() ) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - params = new DruidMasterBalancerTester(master).run(params); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); } @@ -315,28 +315,28 @@ public class DruidMasterBalancerTest EasyMock.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); - // Mock stuff that the master needs - master.moveSegment( + // Mock stuff that the coordinator needs + coordinator.moveSegment( EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(master); + EasyMock.replay(coordinator); LoadQueuePeonTester peon1 = new LoadQueuePeonTester(); LoadQueuePeonTester peon2 = new LoadQueuePeonTester(); LoadQueuePeonTester peon3 = new LoadQueuePeonTester(); LoadQueuePeonTester peon4 = new LoadQueuePeonTester(); - DruidMasterRuntimeParams params = - DruidMasterRuntimeParams.newBuilder() + DruidCoordinatorRuntimeParams params = + DruidCoordinatorRuntimeParams.newBuilder() .withDruidCluster( new DruidCluster( ImmutableMap.>of( "normal", - MinMaxPriorityQueue.orderedBy(DruidMasterBalancerTester.percentUsedComparator) + MinMaxPriorityQueue.orderedBy(DruidCoordinatorBalancerTester.percentUsedComparator) .create( Arrays.asList( new ServerHolder(druidServer1, peon1), @@ -362,15 +362,15 @@ public class DruidMasterBalancerTest ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMaxSegmentsToMove( + new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).build() ) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - params = new DruidMasterBalancerTester(master).run(params); - Assert.assertTrue(params.getMasterStats().getPerTierStats().get("movedCount").get("normal").get() > 0); + params = new DruidCoordinatorBalancerTester(coordinator).run(params); + Assert.assertTrue(params.getCoordinatorStats().getPerTierStats().get("movedCount").get("normal").get() > 0); } } diff --git a/server/src/test/java/io/druid/server/master/DruidMasterBalancerTester.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java similarity index 90% rename from server/src/test/java/io/druid/server/master/DruidMasterBalancerTester.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java index e2afe47c909..77808f2fe86 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterBalancerTester.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTester.java @@ -17,23 +17,23 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.client.DruidServer; import io.druid.timeline.DataSegment; -public class DruidMasterBalancerTester extends DruidMasterBalancer +public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer { - public DruidMasterBalancerTester(DruidMaster master) + public DruidCoordinatorBalancerTester(DruidCoordinator coordinator) { - super(master); + super(coordinator); } @Override protected void moveSegment( final BalancerSegmentHolder segment, final DruidServer toServer, - final DruidMasterRuntimeParams params + final DruidCoordinatorRuntimeParams params ) { final String toServerName = toServer.getName(); diff --git a/server/src/test/java/io/druid/server/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java similarity index 89% rename from server/src/test/java/io/druid/server/master/DruidMasterRuleRunnerTest.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 65948802ef1..9c8fdf0de2a 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -31,9 +31,9 @@ import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.client.DruidServer; import io.druid.db.DatabaseRuleManager; import io.druid.segment.IndexIO; -import io.druid.server.master.rules.IntervalDropRule; -import io.druid.server.master.rules.IntervalLoadRule; -import io.druid.server.master.rules.Rule; +import io.druid.server.coordinator.rules.IntervalLoadRule; +import io.druid.server.coordinator.rules.IntervalDropRule; +import io.druid.server.coordinator.rules.Rule; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -49,19 +49,19 @@ import java.util.List; /** */ -public class DruidMasterRuleRunnerTest +public class DruidCoordinatorRuleRunnerTest { - private DruidMaster master; + private DruidCoordinator coordinator; private LoadQueuePeon mockPeon; private List availableSegments; - private DruidMasterRuleRunner ruleRunner; + private DruidCoordinatorRuleRunner ruleRunner; private ServiceEmitter emitter; private DatabaseRuleManager databaseRuleManager; @Before public void setUp() { - master = EasyMock.createMock(DruidMaster.class); + coordinator = EasyMock.createMock(DruidCoordinator.class); mockPeon = EasyMock.createMock(LoadQueuePeon.class); emitter = EasyMock.createMock(ServiceEmitter.class); EmittingLogger.registerEmitter(emitter); @@ -86,7 +86,7 @@ public class DruidMasterRuleRunnerTest start = start.plusHours(1); } - ruleRunner = new DruidMasterRuleRunner(new ReplicationThrottler(24, 1), master); + ruleRunner = new DruidCoordinatorRuleRunner(new ReplicationThrottler(24, 1), coordinator); } @After @@ -171,17 +171,17 @@ public class DruidMasterRuleRunnerTest ) ); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMaxSegmentsToMove(5).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(5).build()) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 6); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("normal").get() == 6); @@ -261,8 +261,8 @@ public class DruidMasterRuleRunnerTest ) ); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) @@ -270,8 +270,8 @@ public class DruidMasterRuleRunnerTest .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 12); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("cold").get() == 18); @@ -347,8 +347,8 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) @@ -356,8 +356,8 @@ public class DruidMasterRuleRunnerTest .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 12); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("normal").get() == 0); @@ -408,8 +408,8 @@ public class DruidMasterRuleRunnerTest ) ); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withEmitter(emitter) .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) @@ -458,8 +458,8 @@ public class DruidMasterRuleRunnerTest ) ); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withEmitter(emitter) .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) @@ -481,9 +481,9 @@ public class DruidMasterRuleRunnerTest EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).atLeastOnce(); EasyMock.replay(mockPeon); - master.removeSegment(EasyMock.anyObject()); + coordinator.removeSegment(EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); - EasyMock.replay(master); + EasyMock.replay(coordinator); EasyMock.expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())).andReturn( Lists.newArrayList( @@ -520,21 +520,21 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); - EasyMock.verify(master); + EasyMock.verify(coordinator); } @Test @@ -594,17 +594,17 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); @@ -675,17 +675,17 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); @@ -752,17 +752,17 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("droppedCount") == null); Assert.assertTrue(stats.getGlobalStats().get("deletedCount").get() == 12); @@ -841,17 +841,17 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 1); @@ -911,16 +911,16 @@ public class DruidMasterRuleRunnerTest ) ); - DruidMasterRuntimeParams params = - new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = + new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 48); Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); @@ -939,7 +939,7 @@ public class DruidMasterRuleRunnerTest ); afterParams = ruleRunner.run( - new DruidMasterRuntimeParams.Builder() + new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withEmitter(emitter) .withAvailableSegments(Arrays.asList(overFlowSegment)) @@ -947,7 +947,7 @@ public class DruidMasterRuleRunnerTest .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .build() ); - stats = afterParams.getMasterStats(); + stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("assignedCount").get("hot").get() == 1); Assert.assertTrue(stats.getPerTierStats().get("unassignedCount") == null); @@ -1027,17 +1027,17 @@ public class DruidMasterRuleRunnerTest SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - DruidMasterRuntimeParams params = new DruidMasterRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new MasterDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(longerAvailableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(new DateTime("2013-01-01")) .build(); - DruidMasterRuntimeParams afterParams = ruleRunner.run(params); - MasterStats stats = afterParams.getMasterStats(); + DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); + CoordinatorStats stats = afterParams.getCoordinatorStats(); Assert.assertTrue(stats.getPerTierStats().get("droppedCount").get("normal").get() == 24); EasyMock.verify(mockPeon); diff --git a/server/src/test/java/io/druid/server/master/DruidMasterSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java similarity index 97% rename from server/src/test/java/io/druid/server/master/DruidMasterSegmentMergerTest.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index 204003f7dd7..72aeb40ab46 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -33,7 +33,7 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.atomic.AtomicReference; -public class DruidMasterSegmentMergerTest +public class DruidCoordinatorSegmentMergerTest { private static final long mergeBytesLimit = 100; private static final int mergeSegmentsLimit = 8; @@ -441,7 +441,7 @@ public class DruidMasterSegmentMergerTest } /** - * Runs DruidMasterSegmentMerger on a particular set of segments and returns the list of requested merges. + * Runs DruidCoordinatorSegmentMerger on a particular set of segments and returns the list of requested merges. */ private static List> merge(final Collection segments) { @@ -456,11 +456,11 @@ public class DruidMasterSegmentMergerTest }; final AtomicReference whitelistRef = new AtomicReference(null); - final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef); - final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() + final DruidCoordinatorSegmentMerger merger = new DruidCoordinatorSegmentMerger(indexingServiceClient, whitelistRef); + final DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams.newBuilder() .withAvailableSegments(ImmutableSet.copyOf(segments)) .withDynamicConfigs( - new MasterDynamicConfig.Builder().withMergeBytesLimit( + new CoordinatorDynamicConfig.Builder().withMergeBytesLimit( mergeBytesLimit ).withMergeSegmentsLimit( mergeSegmentsLimit diff --git a/server/src/test/java/io/druid/server/master/DruidMasterTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java similarity index 91% rename from server/src/test/java/io/druid/server/master/DruidMasterTest.java rename to server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index 44773da97d8..62dfb9979d0 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.MapMaker; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -39,9 +39,9 @@ import java.util.concurrent.ConcurrentMap; /** */ -public class DruidMasterTest +public class DruidCoordinatorTest { - private DruidMaster master; + private DruidCoordinator coordinator; private CuratorFramework curator; private LoadQueueTaskMaster taskMaster; private DatabaseSegmentManager databaseSegmentManager; @@ -67,8 +67,8 @@ public class DruidMasterTest scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(scheduledExecutorFactory); - master = new DruidMaster( - new DruidMasterConfig() + coordinator = new DruidCoordinator( + new DruidCoordinatorConfig() { @Override public String getHost() @@ -77,19 +77,19 @@ public class DruidMasterTest } @Override - public Duration getMasterStartDelay() + public Duration getCoordinatorStartDelay() { return null; } @Override - public Duration getMasterPeriod() + public Duration getCoordinatorPeriod() { return null; } @Override - public Duration getMasterSegmentMergerPeriod() + public Duration getCoordinatorSegmentMergerPeriod() { return null; } @@ -156,6 +156,6 @@ public class DruidMasterTest }); EasyMock.replay(serverInventoryView); - master.moveSegment("from", "to", "dummySegment", null); + coordinator.moveSegment("from", "to", "dummySegment", null); } } diff --git a/server/src/test/java/io/druid/server/master/LoadQueuePeonTester.java b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java similarity index 97% rename from server/src/test/java/io/druid/server/master/LoadQueuePeonTester.java rename to server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java index a8e53d1504d..46348b9ff8b 100644 --- a/server/src/test/java/io/druid/server/master/LoadQueuePeonTester.java +++ b/server/src/test/java/io/druid/server/coordinator/LoadQueuePeonTester.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import io.druid.timeline.DataSegment; diff --git a/server/src/test/java/io/druid/server/master/ReservoirSegmentSamplerTest.java b/server/src/test/java/io/druid/server/coordinator/ReservoirSegmentSamplerTest.java similarity index 99% rename from server/src/test/java/io/druid/server/master/ReservoirSegmentSamplerTest.java rename to server/src/test/java/io/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 9561e5feda7..1c1af1c7a86 100644 --- a/server/src/test/java/io/druid/server/master/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master; +package io.druid.server.coordinator; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; diff --git a/server/src/test/java/io/druid/server/master/rules/PeriodDropRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/PeriodDropRuleTest.java similarity index 98% rename from server/src/test/java/io/druid/server/master/rules/PeriodDropRuleTest.java rename to server/src/test/java/io/druid/server/coordinator/rules/PeriodDropRuleTest.java index 3b4967a1c7d..91d098bd66a 100644 --- a/server/src/test/java/io/druid/server/master/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/PeriodDropRuleTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; diff --git a/server/src/test/java/io/druid/server/master/rules/PeriodLoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/PeriodLoadRuleTest.java similarity index 98% rename from server/src/test/java/io/druid/server/master/rules/PeriodLoadRuleTest.java rename to server/src/test/java/io/druid/server/coordinator/rules/PeriodLoadRuleTest.java index de10902c6f2..6c172af1081 100644 --- a/server/src/test/java/io/druid/server/master/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/PeriodLoadRuleTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.master.rules; +package io.druid.server.coordinator.rules; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 9afe9ef02d0..202b63d6771 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -42,18 +42,18 @@ import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; +import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorConfig; +import io.druid.server.coordinator.LoadQueueTaskMaster; import io.druid.server.http.BackwardsCompatiableInfoResource; +import io.druid.server.http.CoordinatorDynamicConfigsResource; +import io.druid.server.http.CoordinatorRedirectInfo; +import io.druid.server.http.CoordinatorResource; import io.druid.server.http.InfoResource; -import io.druid.server.http.MasterDynamicConfigsResource; -import io.druid.server.http.MasterRedirectInfo; -import io.druid.server.http.MasterResource; import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectInfo; import io.druid.server.http.RedirectServlet; import io.druid.server.initialization.JettyServerInitializer; -import io.druid.server.master.DruidMaster; -import io.druid.server.master.DruidMasterConfig; -import io.druid.server.master.LoadQueueTaskMaster; import org.apache.curator.framework.CuratorFramework; import org.eclipse.jetty.server.Server; @@ -63,7 +63,7 @@ import java.util.List; */ @Command( name = "coordinator", - description = "Runs the Coordinator, see https://github.com/metamx/druid/wiki/Master for a description." + description = "Runs the Coordinator, see http://druid.io/docs/0.6.0/Coordinator.html for a description." ) public class CliCoordinator extends ServerRunnable { @@ -83,7 +83,7 @@ public class CliCoordinator extends ServerRunnable @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, DruidMasterConfig.class); + ConfigProvider.bind(binder, DruidCoordinatorConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); @@ -101,18 +101,18 @@ public class CliCoordinator extends ServerRunnable binder.bind(IndexingServiceClient.class).in(LazySingleton.class); - binder.bind(RedirectInfo.class).to(MasterRedirectInfo.class).in(LazySingleton.class); + binder.bind(RedirectInfo.class).to(CoordinatorRedirectInfo.class).in(LazySingleton.class); - binder.bind(DruidMaster.class); + binder.bind(DruidCoordinator.class); - LifecycleModule.register(binder, DruidMaster.class); + LifecycleModule.register(binder, DruidCoordinator.class); DiscoveryModule.register(binder, Self.class); binder.bind(JettyServerInitializer.class).toInstance(new CoordinatorJettyServerInitializer()); Jerseys.addResource(binder, BackwardsCompatiableInfoResource.class); Jerseys.addResource(binder, InfoResource.class); - Jerseys.addResource(binder, MasterResource.class); - Jerseys.addResource(binder, MasterDynamicConfigsResource.class); + Jerseys.addResource(binder, CoordinatorResource.class); + Jerseys.addResource(binder, CoordinatorDynamicConfigsResource.class); LifecycleModule.register(binder, Server.class); } @@ -120,7 +120,7 @@ public class CliCoordinator extends ServerRunnable @Provides @LazySingleton public LoadQueueTaskMaster getLoadQueueTaskMaster( - CuratorFramework curator, ObjectMapper jsonMapper, ScheduledExecutorFactory factory, DruidMasterConfig config + CuratorFramework curator, ObjectMapper jsonMapper, ScheduledExecutorFactory factory, DruidCoordinatorConfig config ) { return new LoadQueueTaskMaster(curator, jsonMapper, factory.create(1, "Master-PeonExec--%d"), config); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index e7206b70351..2e32855f1e1 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -42,7 +42,7 @@ import java.util.List; */ @Command( name = "historical", - description = "Runs a Historical node, see https://github.com/metamx/druid/wiki/Compute for a description" + description = "Runs a Historical node, see http://druid.io/docs/0.6.0/Historical.html for a description" ) public class CliHistorical extends ServerRunnable { diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index dd3103a5b81..d4fcb5cf0fa 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -34,8 +34,8 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; import io.druid.indexing.common.index.ChatHandlerProvider; -import io.druid.indexing.coordinator.ForkingTaskRunner; -import io.druid.indexing.coordinator.TaskRunner; +import io.druid.indexing.overlord.ForkingTaskRunner; +import io.druid.indexing.overlord.TaskRunner; import io.druid.indexing.worker.Worker; import io.druid.indexing.worker.WorkerCuratorCoordinator; import io.druid.indexing.worker.WorkerTaskMonitor; diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index a617b53d4c0..065e6c59039 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -47,30 +47,30 @@ import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import io.druid.indexing.common.tasklogs.TaskLogStreamer; import io.druid.indexing.common.tasklogs.TaskLogs; import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; -import io.druid.indexing.coordinator.DbTaskStorage; -import io.druid.indexing.coordinator.ForkingTaskRunnerFactory; -import io.druid.indexing.coordinator.HeapMemoryTaskStorage; -import io.druid.indexing.coordinator.IndexerDBCoordinator; -import io.druid.indexing.coordinator.RemoteTaskRunnerFactory; -import io.druid.indexing.coordinator.TaskLockbox; -import io.druid.indexing.coordinator.TaskMaster; -import io.druid.indexing.coordinator.TaskQueue; -import io.druid.indexing.coordinator.TaskRunnerFactory; -import io.druid.indexing.coordinator.TaskStorage; -import io.druid.indexing.coordinator.TaskStorageQueryAdapter; -import io.druid.indexing.coordinator.http.IndexerCoordinatorResource; -import io.druid.indexing.coordinator.http.OldIndexerCoordinatorResource; -import io.druid.indexing.coordinator.http.OverlordRedirectInfo; -import io.druid.indexing.coordinator.scaling.AutoScalingStrategy; -import io.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy; -import io.druid.indexing.coordinator.scaling.NoopAutoScalingStrategy; -import io.druid.indexing.coordinator.scaling.ResourceManagementSchedulerConfig; -import io.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; -import io.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactoryImpl; -import io.druid.indexing.coordinator.scaling.ResourceManagementStrategy; -import io.druid.indexing.coordinator.scaling.SimpleResourceManagementConfig; -import io.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; -import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.overlord.DbTaskStorage; +import io.druid.indexing.overlord.ForkingTaskRunnerFactory; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.IndexerDBCoordinator; +import io.druid.indexing.overlord.RemoteTaskRunnerFactory; +import io.druid.indexing.overlord.TaskLockbox; +import io.druid.indexing.overlord.TaskMaster; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunnerFactory; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.TaskStorageQueryAdapter; +import io.druid.indexing.overlord.http.OldOverlordResource; +import io.druid.indexing.overlord.http.OverlordResource; +import io.druid.indexing.overlord.http.OverlordRedirectInfo; +import io.druid.indexing.overlord.scaling.AutoScalingStrategy; +import io.druid.indexing.overlord.scaling.EC2AutoScalingStrategy; +import io.druid.indexing.overlord.scaling.NoopAutoScalingStrategy; +import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerConfig; +import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerFactory; +import io.druid.indexing.overlord.scaling.ResourceManagementSchedulerFactoryImpl; +import io.druid.indexing.overlord.scaling.ResourceManagementStrategy; +import io.druid.indexing.overlord.scaling.SimpleResourceManagementConfig; +import io.druid.indexing.overlord.scaling.SimpleResourceManagementStrategy; +import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectInfo; import io.druid.server.initialization.JettyServerInitializer; @@ -143,8 +143,8 @@ public class CliOverlord extends ServerRunnable binder.bind(RedirectInfo.class).to(OverlordRedirectInfo.class).in(LazySingleton.class); binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer()); - Jerseys.addResource(binder, IndexerCoordinatorResource.class); - Jerseys.addResource(binder, OldIndexerCoordinatorResource.class); + Jerseys.addResource(binder, OverlordResource.class); + Jerseys.addResource(binder, OldOverlordResource.class); LifecycleModule.register(binder, Server.class); } diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 4efa7e96e50..b874ec69a7b 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -49,12 +49,12 @@ import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.index.ChatHandlerProvider; import io.druid.indexing.common.index.NoopChatHandlerProvider; import io.druid.indexing.common.index.ServiceAnnouncingChatHandlerProvider; -import io.druid.indexing.coordinator.HeapMemoryTaskStorage; -import io.druid.indexing.coordinator.IndexerDBCoordinator; -import io.druid.indexing.coordinator.TaskQueue; -import io.druid.indexing.coordinator.TaskRunner; -import io.druid.indexing.coordinator.TaskStorage; -import io.druid.indexing.coordinator.ThreadPoolTaskRunner; +import io.druid.indexing.overlord.HeapMemoryTaskStorage; +import io.druid.indexing.overlord.IndexerDBCoordinator; +import io.druid.indexing.overlord.TaskQueue; +import io.druid.indexing.overlord.TaskRunner; +import io.druid.indexing.overlord.TaskStorage; +import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.executor.ChatHandlerResource; import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 391ad699594..bf5fcd14cc8 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -21,9 +21,9 @@ package io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import io.druid.server.coordinator.DruidCoordinator; import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.JettyServerInitializer; -import io.druid.server.master.DruidMaster; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -43,7 +43,7 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer public void initialize(Server server, Injector injector) { ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setResourceBase(DruidMaster.class.getClassLoader().getResource("static").toExternalForm()); + resourceHandler.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.setContextPath("/"); diff --git a/services/src/main/java/io/druid/cli/convert/ConvertProperties.java b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java index e19bb5b35df..613db784e2a 100644 --- a/services/src/main/java/io/druid/cli/convert/ConvertProperties.java +++ b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java @@ -63,6 +63,8 @@ public class ConvertProperties implements Runnable new Rename("com.metamx.emitter.http", "druid.emitter.http"), new Rename("com.metamx.emitter.http.url", "druid.emitter.http.url"), new Rename("com.metamx.druid.emitter.period", "druid.emitter.emissionPeriod"), + new PrefixRename("com.metamx.emitter", "druid.emitter"), + new PrefixRename("com.metamx.druid.emitter", "druid.emitter"), new IndexCacheConverter(), new Rename("druid.paths.segmentInfoCache", "druid.segmentCache.infoPath"), new Rename("com.metamx.aws.accessKey", "druid.s3.accessKey"), @@ -106,7 +108,8 @@ public class ConvertProperties implements Runnable new Rename("druid.worker.taskActionClient.retry.minWaitMillis", "druid.worker.taskActionClient.retry.minWait"), new Rename("druid.worker.taskActionClient.retry.maxWaitMillis", "druid.worker.taskActionClient.retry.maxWait"), new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName"), - new Rename("druid.master.merger.on", "druid.master.merge.on"), + new Rename("druid.master.merger.on", "druid.coordinator.merge.on"), + new PrefixRename("druid.master", "druid.coordinator"), new PrefixRename("druid.pusher", "druid.storage"), new DataSegmentPusherDefaultConverter(), new Rename("druid.pusher.hdfs.storageDirectory", "druid.storage.storageDirectory"),