diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/client/src/main/java/com/metamx/druid/client/DataSegment.java index b915f7680cd..e8b7e55a495 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -261,6 +261,7 @@ public class DataSegment implements Comparable ", loadSpec=" + loadSpec + ", interval=" + interval + ", dataSource='" + dataSource + '\'' + + ", binaryVersion='" + binaryVersion + '\'' + '}'; } diff --git a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java similarity index 92% rename from client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java index 5a8e3bdb50f..5f744918a0a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientAppendQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -29,7 +29,7 @@ import java.util.List; /** */ -public class ClientAppendQuery implements ClientMergeQuery +public class ClientAppendQuery { private final String dataSource; private final List segments; @@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java new file mode 100644 index 00000000000..c0b96bb80b4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java @@ -0,0 +1,57 @@ +package com.metamx.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.client.DataSegment; +import org.joda.time.Interval; + +/** + */ +public class ClientConversionQuery +{ + private final String dataSource; + private final Interval interval; + private final DataSegment segment; + + public ClientConversionQuery( + DataSegment segment + ) + { + this.dataSource = segment.getDataSource(); + this.interval = segment.getInterval(); + this.segment = segment; + } + + public ClientConversionQuery( + String dataSource, + Interval interval + ) + { + this.dataSource = dataSource; + this.interval = interval; + this.segment = null; + } + + @JsonProperty + public String getType() + { + return "version_converter"; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public DataSegment getSegment() + { + return segment; + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java similarity index 83% rename from client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java index 3acf20a0a43..3ae8dffb225 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDeleteQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; - - +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; -public class ClientDeleteQuery +/** + */ +public class ClientKillQuery { private final String dataSource; private final Interval interval; @JsonCreator - public ClientDeleteQuery( + public ClientKillQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) @@ -40,22 +40,21 @@ public class ClientDeleteQuery this.interval = interval; } + @JsonProperty + public String getType() + { + return "kill"; + } + + @JsonProperty public String getDataSource() { return dataSource; } + @JsonProperty public Interval getInterval() { return interval; } - - @Override - public String toString() - { - return "ClientDeleteQuery{" + - "dataSource='" + dataSource + '\'' + - ", interval=" + interval + - '}'; - } } diff --git a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java similarity index 90% rename from client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java rename to client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java index 4286cd211cb..e000826ff9a 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientDefaultMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -30,14 +30,14 @@ import java.util.List; /** */ -public class ClientDefaultMergeQuery implements ClientMergeQuery +public class ClientMergeQuery { private final String dataSource; private final List segments; private final List aggregators; @JsonCreator - public ClientDefaultMergeQuery( + public ClientMergeQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, @JsonProperty("aggregations") List aggregators @@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery } @JsonProperty - @Override + public String getType() + { + return "append"; + } + + @JsonProperty public String getDataSource() { return dataSource; } @JsonProperty - @Override public List getSegments() { return segments; @@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery @Override public String toString() { - return "ClientDefaultMergeQuery{" + + return "ClientMergeQuery{" + "dataSource='" + dataSource + '\'' + ", segments=" + segments + ", aggregators=" + aggregators + diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java new file mode 100644 index 00000000000..b659148d338 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -0,0 +1,115 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.client.indexing; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.metamx.common.IAE; +import com.metamx.druid.client.DataSegment; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.InputStreamResponseHandler; +import com.netflix.curator.x.discovery.ServiceInstance; +import com.netflix.curator.x.discovery.ServiceProvider; +import org.joda.time.Interval; + + +import java.io.InputStream; +import java.net.URL; +import java.util.Iterator; +import java.util.List; + +public class IndexingServiceClient +{ + private static final InputStreamResponseHandler RESPONSE_HANDLER = new InputStreamResponseHandler(); + + private final HttpClient client; + private final ObjectMapper jsonMapper; + private final ServiceProvider serviceProvider; + + public IndexingServiceClient( + HttpClient client, + ObjectMapper jsonMapper, + ServiceProvider serviceProvider + ) + { + this.client = client; + this.jsonMapper = jsonMapper; + this.serviceProvider = serviceProvider; + } + + public void mergeSegments(List segments) + { + final Iterator segmentsIter = segments.iterator(); + if (!segmentsIter.hasNext()) { + return; + } + + final String dataSource = segmentsIter.next().getDataSource(); + while (segmentsIter.hasNext()) { + DataSegment next = segmentsIter.next(); + if (!dataSource.equals(next.getDataSource())) { + throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource()); + } + } + + runQuery("merge", new ClientAppendQuery(dataSource, segments)); + } + + public void killSegments(String dataSource, Interval interval) + { + runQuery("index", new ClientKillQuery(dataSource, interval)); + } + + public void upgradeSegment(DataSegment dataSegment) + { + runQuery("task", new ClientConversionQuery(dataSegment)); + } + + public void upgradeSegments(String dataSource, Interval interval) + { + runQuery("task", new ClientConversionQuery(dataSource, interval)); + } + + private InputStream runQuery(String endpoint, Object queryObject) + { + try { + return client.post(new URL(String.format("%s/%s", baseUrl(), endpoint))) + .setContent("application/json", jsonMapper.writeValueAsBytes(queryObject)) + .go(RESPONSE_HANDLER) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + + private String baseUrl() + { + try { + final ServiceInstance instance = serviceProvider.getInstance(); + return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java deleted file mode 100644 index 02e0a7bd141..00000000000 --- a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.metamx.druid.merge; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** - */ -public class ClientKillQuery -{ - private final String dataSource; - private final Interval interval; - - @JsonCreator - public ClientKillQuery( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - this.dataSource = dataSource; - this.interval = interval; - } - - @JsonProperty - public String getType() - { - return "kill"; - } - - @JsonProperty - public String getDataSource() - { - return dataSource; - } - - @JsonProperty - public Interval getInterval() - { - return interval; - } -} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java new file mode 100644 index 00000000000..1ecfd24482c --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -0,0 +1,274 @@ +package com.metamx.druid.config; + +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +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 org.joda.time.Duration; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.tweak.ResultSetMapper; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class ConfigManager +{ + private static final Logger log = new Logger(ConfigManager.class); + + private final Object lock = new Object(); + private boolean started = false; + + private final IDBI dbi; + private final ConfigManagerConfig config; + + private final ScheduledExecutorService exec; + private final ConcurrentMap watchedConfigs; + private final String selectStatement; + + private volatile ConfigManager.PollingCallable poller; + + public ConfigManager(IDBI dbi, ConfigManagerConfig config) + { + this.dbi = dbi; + this.config = config; + + this.exec = ScheduledExecutors.fixed(1, "config-manager-%s"); + this.watchedConfigs = Maps.newConcurrentMap(); + this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", config.getConfigTable()); + } + + @LifecycleStart + public void start() + { + synchronized (lock) { + if (started) { + return; + } + + poller = new PollingCallable(); + ScheduledExecutors.scheduleWithFixedDelay(exec, new Duration(0), config.getPollDuration(), poller); + + started = true; + } + } + + @LifecycleStop + public void stop() + { + synchronized (lock) { + if (!started) { + return; + } + + poller.stop(); + poller = null; + + started = false; + } + } + + private void poll() + { + for (Map.Entry entry : watchedConfigs.entrySet()) { + try { + if (entry.getValue().swapIfNew(lookup(entry.getKey()))) { + log.info("New value for key[%s] seen.", entry.getKey()); + } + } + catch (Exception e) { + log.warn(e, "Exception when checking property[%s]", entry.getKey()); + } + } + } + + @SuppressWarnings("unchecked") + public AtomicReference watchConfig(final String key, final ConfigSerde serde) + { + ConfigHolder holder = watchedConfigs.get(key); + if (holder == null) { + try { + log.info("Creating watch for key[%s]", key); + + holder = exec.submit( + new Callable>() + { + @Override + @SuppressWarnings("unchecked") + public ConfigHolder call() throws Exception + { + if (!started) { + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } else { + try { + // Multiple of these callables can be submitted at the same time, but the callables themselves + // are executed serially, so double check that it hasn't already been populated. + if (!watchedConfigs.containsKey(key)) { + byte[] value = lookup(key); + ConfigHolder holder = new ConfigHolder(value, serde); + watchedConfigs.put(key, holder); + } + } + catch (Exception e) { + log.warn(e, "Failed loading config for key[%s]", key); + watchedConfigs.put(key, new ConfigHolder(null, serde)); + } + } + + return watchedConfigs.get(key); + } + } + ).get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e); + } + } + + return holder.getReference(); + } + + public byte[] lookup(final String key) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public byte[] withHandle(Handle handle) throws Exception + { + return handle.createQuery(selectStatement) + .bind("name", key) + .map( + new ResultSetMapper() + { + @Override + public byte[] map(int index, ResultSet r, StatementContext ctx) throws SQLException + { + return r.getBytes("payload"); + } + } + ) + .first(); + } + } + ); + } + + public boolean set(final String key, final ConfigSerde serde, final T obj) + { + if (obj == null) { + return false; + } + + final byte[] newBytes = serde.serialize(obj); + + try { + return exec.submit( + new Callable() + { + @Override + public Boolean call() throws Exception + { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement( + "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload" + ) + .bind("name", key) + .bind("payload", newBytes) + .execute(); + return null; + } + } + ); + + final ConfigHolder configHolder = watchedConfigs.get(key); + if (configHolder != null) { + configHolder.swapIfNew(newBytes); + } + + return true; + } + } + ).get(); + } + catch (Exception e) { + log.warn(e, "Failed to set[%s]", key); + return false; + } + } + + private static class ConfigHolder + { + private final AtomicReference rawBytes; + private final ConfigSerde serde; + private final AtomicReference reference; + + ConfigHolder( + byte[] rawBytes, + ConfigSerde serde + ) + { + this.rawBytes = new AtomicReference(rawBytes); + this.serde = serde; + this.reference = new AtomicReference(serde.deserialize(rawBytes)); + } + + public AtomicReference getReference() + { + return reference; + } + + public boolean swapIfNew(byte[] newBytes) + { + if (!Arrays.equals(newBytes, rawBytes.get())) { + reference.set(serde.deserialize(newBytes)); + rawBytes.set(newBytes); + return true; + } + return false; + } + } + + private class PollingCallable implements Callable + { + private volatile boolean stop = false; + + void stop() + { + stop = true; + } + + @Override + public ScheduledExecutors.Signal call() throws Exception + { + if (stop) { + return ScheduledExecutors.Signal.STOP; + } + + poll(); + return ScheduledExecutors.Signal.REPEAT; + } + } +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java new file mode 100644 index 00000000000..24706a83fb7 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java @@ -0,0 +1,18 @@ +package com.metamx.druid.config; + +import org.joda.time.Duration; +import org.skife.config.Config; +import org.skife.config.Default; + +/** + */ +public abstract class ConfigManagerConfig +{ + @Config("druid.indexer.configTable") + public abstract String getConfigTable(); + + @Config("druid.indexer.poll.duration") + @Default("PT1M") + public abstract Duration getPollDuration(); + +} diff --git a/common/src/main/java/com/metamx/druid/config/ConfigSerde.java b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java new file mode 100644 index 00000000000..95f0a1ee7d3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigSerde.java @@ -0,0 +1,9 @@ +package com.metamx.druid.config; + +/** +*/ +public interface ConfigSerde +{ + public byte[] serialize(T obj); + public T deserialize(byte[] bytes); +} diff --git a/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java new file mode 100644 index 00000000000..8e322f3ee80 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java @@ -0,0 +1,134 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class JacksonConfigManager +{ + private final ConfigManager configManager; + private final ObjectMapper jsonMapper; + + public JacksonConfigManager( + ConfigManager configManager, + ObjectMapper jsonMapper + ) + { + this.configManager = configManager; + this.jsonMapper = jsonMapper; + } + + public AtomicReference watch(String key, Class clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, Class clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public AtomicReference watch(String key, TypeReference clazz) + { + return watch(key, clazz, null); + } + + public AtomicReference watch(String key, TypeReference clazz, T defaultVal) + { + return configManager.watchConfig(key, create(clazz, defaultVal)); + } + + public boolean set(String key, T val) + { + return configManager.set(key, create(val.getClass(), null), val); + } + + private ConfigSerde create(final Class clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } + + private ConfigSerde create(final TypeReference clazz, final T defaultVal) + { + return new ConfigSerde() + { + @Override + public byte[] serialize(T obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public T deserialize(byte[] bytes) + { + if (bytes == null) { + return defaultVal; + } + + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/db/DbConnector.java b/common/src/main/java/com/metamx/druid/db/DbConnector.java index 73013ce6aa2..b8ab7a4747e 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -65,7 +65,7 @@ public class DbConnector dbi, configTableName, String.format( - "CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))", + "CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))", configTableName ) ); @@ -84,12 +84,7 @@ public class DbConnector @Override public Void withHandle(Handle handle) throws Exception { - List> table = handle.select( - String.format( - "SHOW tables LIKE '%s'", - tableName - ) - ); + List> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName)); if (table.isEmpty()) { log.info("Creating table[%s]", tableName); diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index d26e73f5b3c..621989b0d08 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -199,6 +199,19 @@ public class IndexIO } } + public static boolean convertSegment(File toConvert, File converted) throws IOException + { + final int version = getVersionFromDir(toConvert); + + switch (version) { + case 8: + DefaultIndexIOHandler.convertV8toV9(toConvert, converted); + return true; + default: + return false; + } + } + public static interface IndexIOHandler { /** @@ -229,7 +242,7 @@ public class IndexIO public void storeLatest(Index index, File file) throws IOException; } - static class DefaultIndexIOHandler implements IndexIOHandler + public static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); @Override diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 58ad3c5cc43..32cb188273a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.service.ServiceEmitter; @@ -45,29 +46,32 @@ import java.util.Map; public class TaskToolbox { private final TaskConfig config; - private final TaskActionClient taskActionClient; + private final Task task; + private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; - private final SegmentKiller segmentKiller; + private final DataSegmentKiller dataSegmentKiller; private final ObjectMapper objectMapper; public TaskToolbox( TaskConfig config, - TaskActionClient taskActionClient, + Task task, + TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, RestS3Service s3Client, DataSegmentPusher segmentPusher, - SegmentKiller segmentKiller, + DataSegmentKiller dataSegmentKiller, ObjectMapper objectMapper ) { this.config = config; - this.taskActionClient = taskActionClient; + this.task = task; + this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; this.s3Client = s3Client; this.segmentPusher = segmentPusher; - this.segmentKiller = segmentKiller; + this.dataSegmentKiller = dataSegmentKiller; this.objectMapper = objectMapper; } @@ -76,9 +80,9 @@ public class TaskToolbox return config; } - public TaskActionClient getTaskActionClient() + public TaskActionClient getTaskActionClientFactory() { - return taskActionClient; + return taskActionClientFactory.create(task); } public ServiceEmitter getEmitter() @@ -91,9 +95,9 @@ public class TaskToolbox return segmentPusher; } - public SegmentKiller getSegmentKiller() + public DataSegmentKiller getDataSegmentKiller() { - return segmentKiller; + return dataSegmentKiller; } public ObjectMapper getObjectMapper() @@ -101,7 +105,7 @@ public class TaskToolbox return objectMapper; } - public Map getSegments(final Task task, List segments) + public Map getSegments(List segments) throws SegmentLoadingException { final SingleSegmentLoader loader = new SingleSegmentLoader( @@ -112,7 +116,7 @@ public class TaskToolbox @Override public File getCacheDirectory() { - return new File(config.getTaskDir(task), "fetched_segments"); + return new File(getTaskDir(), "fetched_segments"); } } ); @@ -124,4 +128,9 @@ public class TaskToolbox return retVal; } + + public File getTaskDir() { + return new File(config.getBaseTaskDir(), task.getId()); + } + } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java new file mode 100644 index 00000000000..2266860ea86 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.emitter.service.ServiceEmitter; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +/** + * Stuff that may be needed by a Task in order to conduct its business. + */ +public class TaskToolboxFactory +{ + private final TaskConfig config; + private final TaskActionClientFactory taskActionClientFactory; + private final ServiceEmitter emitter; + private final RestS3Service s3Client; + private final DataSegmentPusher segmentPusher; + private final DataSegmentKiller dataSegmentKiller; + private final ObjectMapper objectMapper; + + public TaskToolboxFactory( + TaskConfig config, + TaskActionClientFactory taskActionClientFactory, + ServiceEmitter emitter, + RestS3Service s3Client, + DataSegmentPusher segmentPusher, + DataSegmentKiller dataSegmentKiller, + ObjectMapper objectMapper + ) + { + this.config = config; + this.taskActionClientFactory = taskActionClientFactory; + this.emitter = emitter; + this.s3Client = s3Client; + this.segmentPusher = segmentPusher; + this.dataSegmentKiller = dataSegmentKiller; + this.objectMapper = objectMapper; + } + + public ObjectMapper getObjectMapper() + { + return objectMapper; + } + + public TaskToolbox build(Task task) + { + return new TaskToolbox( + config, + task, + taskActionClientFactory, + emitter, + s3Client, + segmentPusher, + dataSegmentKiller, + objectMapper + ); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java index f6740064f52..e36dbf65a6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java @@ -1,17 +1,20 @@ package com.metamx.druid.merger.common.actions; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskStorage; import com.metamx.emitter.EmittingLogger; public class LocalTaskActionClient implements TaskActionClient { + private final Task task; private final TaskStorage storage; private final TaskActionToolbox toolbox; private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); - public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox) + public LocalTaskActionClient(Task task, TaskStorage storage, TaskActionToolbox toolbox) { + this.task = task; this.storage = storage; this.toolbox = toolbox; } @@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient @Override public RetType submit(TaskAction taskAction) { - final RetType ret = taskAction.perform(toolbox); + final RetType ret = taskAction.perform(task, toolbox); // Add audit log try { - storage.addAuditLog(taskAction); + storage.addAuditLog(task, taskAction); } catch (Exception e) { log.makeAlert(e, "Failed to record action in audit log") - .addData("task", taskAction.getTask().getId()) + .addData("task", task.getId()) .addData("actionClass", taskAction.getClass().getName()) .emit(); } diff --git a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java similarity index 58% rename from client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java rename to merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java index 3956991ed6b..89e37ceafab 100644 --- a/client/src/main/java/com/metamx/druid/merge/ClientMergeQuery.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java @@ -17,25 +17,27 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merge; +package com.metamx.druid.merger.common.actions; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.client.DataSegment; - - - -import java.util.List; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskStorage; /** */ -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="append", value=ClientAppendQuery.class) -}) -public interface ClientMergeQuery +public class LocalTaskActionClientFactory implements TaskActionClientFactory { - public String getDataSource(); + private final TaskStorage storage; + private final TaskActionToolbox toolbox; - public List getSegments(); + public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox) + { + this.storage = storage; + this.toolbox = toolbox; + } + + @Override + public TaskActionClient create(Task task) + { + return new LocalTaskActionClient(task, storage, toolbox); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java index f669af33625..de325ba274f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java @@ -11,25 +11,16 @@ import org.joda.time.Interval; public class LockAcquireAction implements TaskAction> { - private final Task task; private final Interval interval; @JsonCreator public LockAcquireAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction> } @Override - public Optional perform(TaskActionToolbox toolbox) + public Optional perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().tryLock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java index e0e3eddb71f..06a2879ec47 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java @@ -12,29 +12,13 @@ import java.util.List; public class LockListAction implements TaskAction> { - private final Task task; - - @JsonCreator - public LockListAction( - @JsonProperty("task") Task task - ) - { - this.task = task; - } - - @JsonProperty - public Task getTask() - { - return task; - } - public TypeReference> getReturnTypeReference() { return new TypeReference>() {}; } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getTaskLockbox().findLocksForTask(task); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java index 5c84d024a50..b932e748ed1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java @@ -12,25 +12,16 @@ import java.util.List; public class LockReleaseAction implements TaskAction { - private final Task task; private final Interval interval; @JsonCreator public LockReleaseAction( - @JsonProperty("task") Task task, @JsonProperty("interval") Interval interval ) { - this.task = task; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Interval getInterval() { @@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { toolbox.getTaskLockbox().unlock(task, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java index 26900e29942..5cebc6ee1ec 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; +import com.metamx.druid.merger.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; @@ -16,14 +17,16 @@ import java.util.Map; public class RemoteTaskActionClient implements TaskActionClient { + private final Task task; private final HttpClient httpClient; private final ServiceProvider serviceProvider; private final ObjectMapper jsonMapper; private static final Logger log = new Logger(RemoteTaskActionClient.class); - public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) { + this.task = task; this.httpClient = httpClient; this.serviceProvider = serviceProvider; this.jsonMapper = jsonMapper; @@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient public RetType submit(TaskAction taskAction) { try { - byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction); + byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction)); final String response = httpClient.post(getServiceUri().toURL()) .setContent("application/json", dataToSend) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java new file mode 100644 index 00000000000..659042bb592 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common.actions; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.http.client.HttpClient; +import com.netflix.curator.x.discovery.ServiceProvider; + +/** + */ +public class RemoteTaskActionClientFactory implements TaskActionClientFactory +{ + private final HttpClient httpClient; + private final ServiceProvider serviceProvider; + private final ObjectMapper jsonMapper; + + public RemoteTaskActionClientFactory(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper) + { + this.httpClient = httpClient; + this.serviceProvider = serviceProvider; + this.jsonMapper = jsonMapper; + } + + @Override + public TaskActionClient create(Task task) + { + return new RemoteTaskActionClient(task, httpClient, serviceProvider, jsonMapper); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java index 75ad4a9161f..5354e14878c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentInsertAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentInsertAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, false)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java index b20d130064e..56304533a68 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUnusedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUnusedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java index 0395057fe83..a776ed641cc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java @@ -12,28 +12,19 @@ import java.util.List; public class SegmentListUsedAction implements TaskAction> { - private final Task task; private final String dataSource; private final Interval interval; @JsonCreator public SegmentListUsedAction( - @JsonProperty("task") Task task, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) { - this.task = task; this.dataSource = dataSource; this.interval = interval; } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public String getDataSource() { @@ -52,7 +43,7 @@ public class SegmentListUsedAction implements TaskAction> } @Override - public List perform(TaskActionToolbox toolbox) + public List perform(Task task, TaskActionToolbox toolbox) { try { return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java index f1b61c58d9f..2ebedec0daf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -18,25 +18,16 @@ import java.util.Set; public class SegmentNukeAction implements TaskAction { - private final Task task; private final Set segments; @JsonCreator public SegmentNukeAction( - @JsonProperty("task") Task task, @JsonProperty("segments") Set segments ) { - this.task = task; this.segments = ImmutableSet.copyOf(segments); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public Set getSegments() { @@ -49,7 +40,7 @@ public class SegmentNukeAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { if(!toolbox.taskLockCoversSegments(task, segments, true)) { throw new ISE("Segments not covered by locks for task: %s", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java index a7a73d8eac7..ec48430c49a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java @@ -11,25 +11,16 @@ import java.util.List; public class SpawnTasksAction implements TaskAction { - private final Task task; private final List newTasks; @JsonCreator public SpawnTasksAction( - @JsonProperty("task") Task task, @JsonProperty("newTasks") List newTasks ) { - this.task = task; this.newTasks = ImmutableList.copyOf(newTasks); } - @JsonProperty - public Task getTask() - { - return task; - } - @JsonProperty public List getNewTasks() { @@ -42,7 +33,7 @@ public class SpawnTasksAction implements TaskAction } @Override - public Void perform(TaskActionToolbox toolbox) + public Void perform(Task task, TaskActionToolbox toolbox) { try { for(final Task newTask : newTasks) { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java index 7dedf50aad9..019b14a3b62 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java @@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference; }) public interface TaskAction { - public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller public TypeReference getReturnTypeReference(); // T_T - public RetType perform(TaskActionToolbox toolbox); + public RetType perform(Task task, TaskActionToolbox toolbox); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java similarity index 62% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java rename to merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java index 16eeb1c3439..2784a442f31 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/WorkerSetupManagerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java @@ -17,23 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.merger.common.actions; -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; +import com.metamx.druid.merger.common.task.Task; /** */ -public abstract class WorkerSetupManagerConfig +public interface TaskActionClientFactory { - @Config("druid.indexer.configTable") - public abstract String getConfigTable(); - - @Config("druid.indexer.workerSetupConfigName") - public abstract String getWorkerSetupConfigName(); - - @Config("druid.indexer.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); + public TaskActionClient create(Task task); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java new file mode 100644 index 00000000000..a440447a226 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common.actions; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.merger.common.task.Task; + +/** + */ +public class TaskActionHolder +{ + private final Task task; + private final TaskAction action; + + @JsonCreator + public TaskActionHolder( + @JsonProperty("task") Task task, + @JsonProperty("action") TaskAction action + ) + { + this.task = task; + this.action = action; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public TaskAction getAction() + { + return action; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java index 5918f0627c6..2bd27667514 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.common.config; import com.metamx.druid.merger.common.task.Task; @@ -17,8 +36,4 @@ public abstract class TaskConfig @Config("druid.merger.hadoopWorkingPath") public abstract String getHadoopWorkingPath(); - - public File getTaskDir(final Task task) { - return new File(getBaseTaskDir(), task.getId()); - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index 899e2066fb0..518fb04ab37 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -20,15 +20,19 @@ package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentListUsedAction; import org.joda.time.Interval; public abstract class AbstractTask implements Task { + private static final Joiner ID_JOINER = Joiner.on("_"); + private final String id; private final String groupId; private final String dataSource; @@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task .add("interval", getImplicitLockInterval()) .toString(); } + + /** Start helper methods **/ + public static String joinId(Object... objects) + { + return ID_JOINER.join(objects); + } + + public SegmentListUsedAction defaultListUsedAction() + { + return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get()); + } + + public TaskStatus success() + { + return TaskStatus.success(getId()); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index 48ccdda13de..5d15269677a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -44,7 +44,7 @@ import java.util.Map; /** */ -public class AppendTask extends MergeTask +public class AppendTask extends MergeTaskBase { @JsonCreator public AppendTask( diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java deleted file mode 100644 index a46c24e91f3..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.common.task; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Function; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.IndexMerger; - -import javax.annotation.Nullable; -import java.io.File; -import java.util.List; -import java.util.Map; - -/** - */ -public class DefaultMergeTask extends MergeTask -{ - private final List aggregators; - - @JsonCreator - public DefaultMergeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators - ) - { - super(dataSource, segments); - this.aggregators = aggregators; - } - - @Override - public File merge(final Map segments, final File outDir) - throws Exception - { - return IndexMerger.mergeQueryableIndex( - Lists.transform( - ImmutableList.copyOf(segments.values()), - new Function() - { - @Override - public QueryableIndex apply(@Nullable File input) - { - try { - return IndexIO.loadIndex(input); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ), - aggregators.toArray(new AggregatorFactory[aggregators.size()]), - outDir - ); - } - - @Override - public String getType() - { - return "merge"; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 991d76228dd..67754ee00d7 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Strategy: Create an empty segment covering the interval to be deleted - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final Interval interval = this.getImplicitLockInterval().get(); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); @@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask .shardSpec(new NoneShardSpec()) .build(); - final File outDir = new File(toolbox.getConfig().getTaskDir(this), segment.getIdentifier()); + final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier()); final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir); // Upload the segment @@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask segment.getVersion() ); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java index 29c0c517f17..f0d09d5137d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonCreator; @@ -74,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask ); // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); log.info("Setting version to: %s", myLock.getVersion()); configCopy.setVersion(myLock.getVersion()); @@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask List publishedSegments = job.getPublishedSegments(); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 2886c8cc77a..3dfe99a68f1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask } ); - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index 7dc65797ea8..1790ddc6aa5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -100,24 +100,21 @@ public class IndexGeneratorTask extends AbstractTask public TaskStatus run(final TaskToolbox toolbox) throws Exception { // We should have a lock from before we started running - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); // We know this exists final Interval interval = getImplicitLockInterval().get(); // Set up temporary directory for indexing final File tmpDir = new File( + toolbox.getTaskDir(), String.format( - "%s/%s", - toolbox.getConfig().getTaskDir(this).toString(), - String.format( - "%s_%s_%s_%s_%s", - this.getDataSource(), - interval.getStart(), - interval.getEnd(), - myLock.getVersion(), - schema.getShardSpec().getPartitionNum() - ) + "%s_%s_%s_%s_%s", + this.getDataSource(), + interval.getStart(), + interval.getEnd(), + myLock.getVersion(), + schema.getShardSpec().getPartitionNum() ) ); @@ -193,7 +190,7 @@ public class IndexGeneratorTask extends AbstractTask ); // Request segment pushes - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments))); // Done return TaskStatus.success(getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index 4e2f34a5171..d1bfc6d77fc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask @Override public TaskStatus preflight(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks())); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks())); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java index 8c2b5af8e6e..e652ab69151 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.common.task; import com.fasterxml.jackson.annotation.JsonCreator; @@ -53,7 +72,7 @@ public class KillTask extends AbstractTask public TaskStatus run(TaskToolbox toolbox) throws Exception { // Confirm we have a lock (will throw if there isn't exactly one element) - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); if(!myLock.getDataSource().equals(getDataSource())) { throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource()); @@ -64,14 +83,9 @@ public class KillTask extends AbstractTask } // List unused segments - final List unusedSegments = toolbox.getTaskActionClient() - .submit( - new SegmentListUnusedAction( - this, - myLock.getDataSource(), - myLock.getInterval() - ) - ); + final List unusedSegments = toolbox + .getTaskActionClientFactory() + .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for(final DataSegment unusedSegment : unusedSegments) { @@ -88,10 +102,12 @@ public class KillTask extends AbstractTask } // Kill segments - toolbox.getSegmentKiller().kill(unusedSegments); + for (DataSegment segment : unusedSegments) { + toolbox.getDataSegmentKiller().kill(segment); + } // Remove metadata for these segments - toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments))); + toolbox.getTaskActionClientFactory().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments))); return TaskStatus.success(getId()); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 1791c2a097d..4e6102f666b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -19,318 +19,71 @@ package com.metamx.druid.merger.common.task; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.common.base.Charsets; import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; -import com.google.common.hash.Hashing; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; +import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.actions.SegmentListUsedAction; -import com.metamx.druid.shard.NoneShardSpec; -import com.metamx.emitter.service.AlertEvent; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; -import org.joda.time.Interval; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.index.v1.IndexMerger; + + import javax.annotation.Nullable; import java.io.File; import java.util.List; import java.util.Map; -import java.util.Set; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "append", value = AppendTask.class) -}) -public abstract class MergeTask extends AbstractTask +public class MergeTask extends MergeTaskBase { - private final List segments; + private final List aggregators; - private static final Logger log = new Logger(MergeTask.class); - - protected MergeTask(final String dataSource, final List segments) - { - super( - // _not_ the version, just something uniqueish - String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), - dataSource, - computeMergedInterval(segments) - ); - - // Verify segment list is nonempty - Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); - // Verify segments are all in the correct datasource - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); - } - } - ) - ) == 0, "segments in the wrong datasource" - ); - // Verify segments are all unsharded - Preconditions.checkArgument( - Iterables.size( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(@Nullable DataSegment segment) - { - return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); - } - } - ) - ) == 0, "segments without NoneShardSpec" - ); - - this.segments = segments; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this))); - final ServiceEmitter emitter = toolbox.getEmitter(); - final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); - final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); - final File taskDir = toolbox.getConfig().getTaskDir(this); - - try { - - final long startTime = System.currentTimeMillis(); - - log.info( - "Starting merge of id[%s], segments: %s", - getId(), - Lists.transform( - segments, - new Function() - { - @Override - public String apply(@Nullable DataSegment input) - { - return input.getIdentifier(); - } - } - ) - ); - - - // download segments to merge - final Map gettedSegments = toolbox.getSegments(this, segments); - - // merge files together - final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); - - emitter.emit(builder.build("merger/numMerged", segments.size())); - emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); - - log.info( - "[%s] : Merged %d segments in %,d millis", - mergedSegment.getDataSource(), - segments.size(), - System.currentTimeMillis() - startTime - ); - - long uploadStart = System.currentTimeMillis(); - - // Upload file - final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); - - emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); - emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); - - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); - - return TaskStatus.success(getId()); - } - catch (Exception e) { - log.error( - e, - String.format( - "Exception merging %s[%s] segments", - mergedSegment.getDataSource(), - mergedSegment.getInterval() - ) - ); - emitter.emit( - new AlertEvent.Builder().build( - "Exception merging", - ImmutableMap.builder() - .put("exception", e.toString()) - .build() - ) - ); - - return TaskStatus.failure(getId()); - } - } - - /** - * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that - * we are operating on every segment that overlaps the chosen interval. - */ - @Override - public TaskStatus preflight(TaskToolbox toolbox) - { - final Function toIdentifier = new Function() - { - @Override - public String apply(DataSegment dataSegment) - { - return dataSegment.getIdentifier(); - } - }; - - final Set current = ImmutableSet.copyOf( - Iterables.transform( - toolbox.getTaskActionClient() - .submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())), - toIdentifier - ) - ); - final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); - - final Set missingFromRequested = Sets.difference(current, requested); - if (!missingFromRequested.isEmpty()) { - throw new ISE( - "Merge is invalid: current segment(s) are not in the requested set: %s", - Joiner.on(", ").join(missingFromRequested) - ); - } - - final Set missingFromCurrent = Sets.difference(requested, current); - if (!missingFromCurrent.isEmpty()) { - throw new ISE( - "Merge is invalid: requested segment(s) are not in the current set: %s", - Joiner.on(", ").join(missingFromCurrent) - ); - } - - return TaskStatus.running(getId()); - - } - - protected abstract File merge(Map segments, File outDir) - throws Exception; - - @JsonProperty - public List getSegments() - { - return segments; - } - - @Override - public String toString() - { - return Objects.toStringHelper(this) - .add("id", getId()) - .add("dataSource", getDataSource()) - .add("interval", getImplicitLockInterval()) - .add("segments", segments) - .toString(); - } - - private static String computeProcessingID(final String dataSource, final List segments) - { - final String segmentIDs = Joiner.on("_").join( - Iterables.transform( - Ordering.natural().sortedCopy(segments), new Function() - { - @Override - public String apply(DataSegment x) - { - return String.format( - "%s_%s_%s_%s", - x.getInterval().getStart(), - x.getInterval().getEnd(), - x.getVersion(), - x.getShardSpec().getPartitionNum() - ); - } - } - ) - ); - - return String.format( - "%s_%s", - dataSource, - Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() - ); - } - - private static Interval computeMergedInterval(final List segments) - { - Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); - - DateTime start = null; - DateTime end = null; - - for(final DataSegment segment : segments) { - if(start == null || segment.getInterval().getStart().isBefore(start)) { - start = segment.getInterval().getStart(); - } - - if(end == null || segment.getInterval().getEnd().isAfter(end)) { - end = segment.getInterval().getEnd(); - } - } - - return new Interval(start, end); - } - - private static DataSegment computeMergedSegment( - final String dataSource, - final String version, - final List segments + @JsonCreator + public MergeTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators ) { - final Interval mergedInterval = computeMergedInterval(segments); - final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + super(dataSource, segments); + this.aggregators = aggregators; + } - for (DataSegment segment : segments) { - mergedDimensions.addAll(segment.getDimensions()); - mergedMetrics.addAll(segment.getMetrics()); - } + @Override + public File merge(final Map segments, final File outDir) + throws Exception + { + return IndexMerger.mergeQueryableIndex( + Lists.transform( + ImmutableList.copyOf(segments.values()), + new Function() + { + @Override + public QueryableIndex apply(@Nullable File input) + { + try { + return IndexIO.loadIndex(input); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ), + aggregators.toArray(new AggregatorFactory[aggregators.size()]), + outDir + ); + } - return DataSegment.builder() - .dataSource(dataSource) - .interval(mergedInterval) - .version(version) - .shardSpec(new NoneShardSpec()) - .dimensions(Lists.newArrayList(mergedDimensions)) - .metrics(Lists.newArrayList(mergedMetrics)) - .build(); + @Override + public String getType() + { + return "merge"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java new file mode 100644 index 00000000000..e0b3dd6ff17 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java @@ -0,0 +1,315 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Charsets; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.hash.Hashing; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.shard.NoneShardSpec; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.AlertEvent; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +public abstract class MergeTaskBase extends AbstractTask +{ + private final List segments; + + private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class); + + protected MergeTaskBase(final String dataSource, final List segments) + { + super( + // _not_ the version, just something uniqueish + String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()), + dataSource, + computeMergedInterval(segments) + ); + + // Verify segment list is nonempty + Preconditions.checkArgument(segments.size() > 0, "segments nonempty"); + // Verify segments are all in the correct datasource + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource); + } + } + ) + ) == 0, "segments in the wrong datasource" + ); + // Verify segments are all unsharded + Preconditions.checkArgument( + Iterables.size( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(@Nullable DataSegment segment) + { + return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec); + } + } + ) + ) == 0, "segments without NoneShardSpec" + ); + + this.segments = segments; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); + final ServiceEmitter emitter = toolbox.getEmitter(); + final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); + final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments); + final File taskDir = toolbox.getTaskDir(); + + try { + + final long startTime = System.currentTimeMillis(); + + log.info( + "Starting merge of id[%s], segments: %s", + getId(), + Lists.transform( + segments, + new Function() + { + @Override + public String apply(@Nullable DataSegment input) + { + return input.getIdentifier(); + } + } + ) + ); + + + // download segments to merge + final Map gettedSegments = toolbox.getSegments(segments); + + // merge files together + final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged")); + + emitter.emit(builder.build("merger/numMerged", segments.size())); + emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime)); + + log.info( + "[%s] : Merged %d segments in %,d millis", + mergedSegment.getDataSource(), + segments.size(), + System.currentTimeMillis() - startTime + ); + + long uploadStart = System.currentTimeMillis(); + + // Upload file + final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment); + + emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart)); + emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize())); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment))); + + return TaskStatus.success(getId()); + } + catch (Exception e) { + log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource()) + .addData("interval", mergedSegment.getInterval()) + .emit(); + + return TaskStatus.failure(getId()); + } + } + + /** + * Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that + * we are operating on every segment that overlaps the chosen interval. + */ + @Override + public TaskStatus preflight(TaskToolbox toolbox) + { + final Function toIdentifier = new Function() + { + @Override + public String apply(DataSegment dataSegment) + { + return dataSegment.getIdentifier(); + } + }; + + final Set current = ImmutableSet.copyOf( + Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier) + ); + final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); + + final Set missingFromRequested = Sets.difference(current, requested); + if (!missingFromRequested.isEmpty()) { + throw new ISE( + "Merge is invalid: current segment(s) are not in the requested set: %s", + Joiner.on(", ").join(missingFromRequested) + ); + } + + final Set missingFromCurrent = Sets.difference(requested, current); + if (!missingFromCurrent.isEmpty()) { + throw new ISE( + "Merge is invalid: requested segment(s) are not in the current set: %s", + Joiner.on(", ").join(missingFromCurrent) + ); + } + + return TaskStatus.running(getId()); + + } + + protected abstract File merge(Map segments, File outDir) + throws Exception; + + @JsonProperty + public List getSegments() + { + return segments; + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", getId()) + .add("dataSource", getDataSource()) + .add("interval", getImplicitLockInterval()) + .add("segments", segments) + .toString(); + } + + private static String computeProcessingID(final String dataSource, final List segments) + { + final String segmentIDs = Joiner.on("_").join( + Iterables.transform( + Ordering.natural().sortedCopy(segments), new Function() + { + @Override + public String apply(DataSegment x) + { + return String.format( + "%s_%s_%s_%s", + x.getInterval().getStart(), + x.getInterval().getEnd(), + x.getVersion(), + x.getShardSpec().getPartitionNum() + ); + } + } + ) + ); + + return String.format( + "%s_%s", + dataSource, + Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase() + ); + } + + private static Interval computeMergedInterval(final List segments) + { + Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0"); + + DateTime start = null; + DateTime end = null; + + for(final DataSegment segment : segments) { + if(start == null || segment.getInterval().getStart().isBefore(start)) { + start = segment.getInterval().getStart(); + } + + if(end == null || segment.getInterval().getEnd().isAfter(end)) { + end = segment.getInterval().getEnd(); + } + } + + return new Interval(start, end); + } + + private static DataSegment computeMergedSegment( + final String dataSource, + final String version, + final List segments + ) + { + final Interval mergedInterval = computeMergedInterval(segments); + final Set mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + final Set mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + + for (DataSegment segment : segments) { + mergedDimensions.addAll(segment.getDimensions()); + mergedMetrics.addAll(segment.getMetrics()); + } + + return DataSegment.builder() + .dataSource(dataSource) + .interval(mergedInterval) + .version(version) + .shardSpec(new NoneShardSpec()) + .dimensions(Lists.newArrayList(mergedDimensions)) + .metrics(Lists.newArrayList(mergedMetrics)) + .build(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 8d93486ff6b..5f288be99dc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -41,16 +41,18 @@ import org.joda.time.Interval; * to release locks early if they desire. * */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "append", value = AppendTask.class), - @JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class), + @JsonSubTypes.Type(name = "merge", value = MergeTask.class), @JsonSubTypes.Type(name = "delete", value = DeleteTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), - @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class) + @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), + @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) }) public interface Task { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java deleted file mode 100644 index 28d3ab1fec1..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.metamx.druid.merger.common.task; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -/** - */ -public class V8toV9UpgradeTask extends AbstractTask -{ - public V8toV9UpgradeTask( - @JsonProperty("dataSource") String dataSource, - @JsonProperty("interval") Interval interval - ) - { - super( - String.format("v8tov9_%s_%s_%s", dataSource, interval.toString().replace("/", "_"), new DateTime()), - dataSource, - interval - ); - } - - @Override - public String getType() - { - return "8to9"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - throw new UnsupportedOperationException(); - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java new file mode 100644 index 00000000000..c8c0e2cbf42 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java @@ -0,0 +1,184 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger.common.task; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.loading.SegmentLoadingException; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; +import com.metamx.druid.merger.common.actions.TaskActionClient; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + */ +public class VersionConverterTask extends AbstractTask +{ + private static final String TYPE = "version_converter"; + private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID); + + private static final Logger log = new Logger(VersionConverterTask.class); + private final DataSegment segment; + + public VersionConverterTask( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId(TYPE, dataSource, interval.getStart(), interval.getEnd(), new DateTime()), + dataSource, + interval + ); + + this.segment = segment; + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + if (segment == null) { + throw new ISE("Segment was null, this should never run.", this.getClass().getSimpleName()); + } + + log.info("I'm in a subless mood."); + convertSegment(toolbox, segment); + return success(); + } + + @Override + public TaskStatus preflight(TaskToolbox toolbox) throws Exception + { + if (segment != null) { + return super.preflight(toolbox); + } + + final TaskActionClient taskClient = toolbox.getTaskActionClientFactory(); + + List segments = taskClient.submit(defaultListUsedAction()); + + final FunctionalIterable tasks = FunctionalIterable + .create(segments) + .keep( + new Function() + { + @Override + public Task apply(DataSegment segment) + { + final Integer segmentVersion = segment.getBinaryVersion(); + if (!CURR_VERSION_INTEGER.equals(segmentVersion)) { + return new SubTask(getGroupId(), segment); + } + + log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion); + return null; + } + } + ); + + taskClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks))); + + return TaskStatus.success(getId()); + } + + public static class SubTask extends AbstractTask + { + private final DataSegment segment; + + protected SubTask( + @JsonProperty("groupId") String groupId, + @JsonProperty("segment") DataSegment segment + ) + { + super( + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), + groupId, + segment.getDataSource(), + segment.getInterval() + ); + this.segment = segment; + } + + @Override + public String getType() + { + return "version_converter_sub"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Subs are good! Italian BMT and Meatball are probably my favorite."); + convertSegment(toolbox, segment); + return success(); + } + } + + private static void convertSegment(TaskToolbox toolbox, final DataSegment segment) + throws SegmentLoadingException, IOException + { + log.info("Converting segment[%s]", segment); + final Map localSegments = toolbox.getSegments(Arrays.asList(segment)); + + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + if (IndexIO.convertSegment(location, outLocation)) { + final int outVersion = IndexIO.getVersionFromDir(outLocation); + + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } else { + log.info("Conversion failed."); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index 5f39efae0bb..b878885dd4a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -327,11 +327,11 @@ public class DbTaskStorage implements TaskStorage } @Override - public void addAuditLog(final TaskAction taskAction) + public void addAuditLog(final Task task, final TaskAction taskAction) { Preconditions.checkNotNull(taskAction, "taskAction"); - log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction); + log.info("Logging action for task[%s]: %s", task.getId(), taskAction); dbi.withHandle( new HandleCallback() @@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage dbConnectorConfig.getTaskLogTable() ) ) - .bind("task_id", taskAction.getTask().getId()) + .bind("task_id", task.getId()) .bind("log_payload", jsonMapper.writeValueAsString(taskAction)) .execute(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java index 042f0b8196f..895804bc7fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java @@ -40,14 +40,14 @@ import java.util.concurrent.locks.ReentrantLock; * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not * thread safe. */ -public class LocalTaskStorage implements TaskStorage +public class HeapMemoryTaskStorage implements TaskStorage { private final ReentrantLock giant = new ReentrantLock(); private final Map tasks = Maps.newHashMap(); private final Multimap taskLocks = HashMultimap.create(); private final Multimap taskActions = ArrayListMultimap.create(); - private static final Logger log = new Logger(LocalTaskStorage.class); + private static final Logger log = new Logger(HeapMemoryTaskStorage.class); @Override public void insert(Task task, TaskStatus status) @@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage } @Override - public void addAuditLog(TaskAction taskAction) + public void addAuditLog(Task task, TaskAction taskAction) { giant.lock(); try { - taskActions.put(taskAction.getTask().getId(), taskAction); + taskActions.put(task.getId(), taskAction); } finally { giant.unlock(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index 573152fce09..5dbe8273d6c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import org.apache.commons.io.FileUtils; @@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService; */ public class LocalTaskRunner implements TaskRunner { - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; private static final Logger log = new Logger(LocalTaskRunner.class); public LocalTaskRunner( - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner @Override public void run(final Task task, final TaskCallback callback) { + final TaskToolbox toolbox = toolboxFactory.build(task); + exec.submit( new Runnable() { @@ -89,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner } try { - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); if (taskDir.exists()) { log.info("Removing task directory: %s", taskDir); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index e4a13136152..b1ed92087bc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -35,7 +35,7 @@ import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; /** * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure @@ -82,7 +83,7 @@ public class RemoteTaskRunner implements TaskRunner private final PathChildrenCache workerPathCache; private final ScheduledExecutorService scheduledExec; private final RetryPolicyFactory retryPolicyFactory; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupData; // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); @@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner PathChildrenCache workerPathCache, ScheduledExecutorService scheduledExec, RetryPolicyFactory retryPolicyFactory, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupData ) { this.jsonMapper = jsonMapper; @@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner this.workerPathCache = workerPathCache; this.scheduledExec = scheduledExec; this.retryPolicyFactory = retryPolicyFactory; - this.workerSetupManager = workerSetupManager; + this.workerSetupData = workerSetupData; } @LifecycleStart @@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner return (!input.isAtCapacity() && input.getWorker() .getVersion() - .compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0); + .compareTo(workerSetupData.get().getMinVersion()) >= 0); } } ) diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java index 07abd0594ac..0a4bd925d4d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator; import com.google.common.base.Function; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 0a38e5efa37..69f77b3526c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -26,6 +26,8 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.exec.TaskConsumer; import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; @@ -49,7 +51,7 @@ public class TaskMasterLifecycle private final ReentrantLock giant = new ReentrantLock(); private final Condition mayBeStopped = giant.newCondition(); private final TaskQueue taskQueue; - private final TaskToolbox taskToolbox; + private final TaskToolboxFactory taskToolboxFactory; private volatile boolean leading = false; private volatile TaskRunner taskRunner; @@ -59,7 +61,7 @@ public class TaskMasterLifecycle public TaskMasterLifecycle( final TaskQueue taskQueue, - final TaskToolbox taskToolbox, + final TaskToolboxFactory taskToolboxFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, final ServiceDiscoveryConfig serviceDiscoveryConfig, final TaskRunnerFactory runnerFactory, @@ -69,7 +71,7 @@ public class TaskMasterLifecycle ) { this.taskQueue = taskQueue; - this.taskToolbox = taskToolbox; + this.taskToolboxFactory = taskToolboxFactory; this.leaderSelector = new LeaderSelector( curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener() @@ -87,7 +89,7 @@ public class TaskMasterLifecycle final TaskConsumer taskConsumer = new TaskConsumer( taskQueue, taskRunner, - taskToolbox, + taskToolboxFactory, emitter ); @@ -221,9 +223,4 @@ public class TaskMasterLifecycle { return taskToolbox; } - - public ResourceManagementScheduler getResourceManagementScheduler() - { - return resourceManagementScheduler; - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 2e21f52876b..d6bfbfd889e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -69,7 +69,7 @@ public interface TaskStorage /** * Add an action taken by a task to the audit log. */ - public void addAuditLog(TaskAction taskAction); + public void addAuditLog(Task task, TaskAction taskAction); /** * Returns all actions taken by a task. diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index f1e31a9e5c1..48d3ecdc471 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -24,7 +24,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskRunner; @@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable { private final TaskQueue queue; private final TaskRunner runner; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ServiceEmitter emitter; private final Thread thready; @@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable public TaskConsumer( TaskQueue queue, TaskRunner runner, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ServiceEmitter emitter ) { this.queue = queue; this.runner = runner; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.emitter = emitter; this.thready = new Thread(this); } @@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable // Run preflight checks TaskStatus preflightStatus; try { - preflightStatus = task.preflight(toolbox); + preflightStatus = task.preflight(toolboxFactory.build(task)); log.info("Preflight done for task: %s", task.getId()); } catch (Exception e) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 5da9c557936..5452acac163 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -39,6 +39,9 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.GuiceServletConfig; @@ -48,22 +51,21 @@ import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.DbTaskStorage; +import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage; import com.metamx.druid.merger.coordinator.LocalTaskRunner; -import com.metamx.druid.merger.coordinator.LocalTaskStorage; import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.RemoteTaskRunner; import com.metamx.druid.merger.coordinator.RetryPolicyFactory; @@ -79,7 +81,6 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; -import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; @@ -88,7 +89,7 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerCo import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -123,6 +124,7 @@ import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -147,7 +149,8 @@ public class IndexerCoordinatorNode extends RegisteringNode private RestS3Service s3Service = null; private IndexerCoordinatorConfig config = null; private TaskConfig taskConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private MergerDBCoordinator mergerDBCoordinator = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; @@ -155,7 +158,6 @@ public class IndexerCoordinatorNode extends RegisteringNode private CuratorFramework curatorFramework = null; private ScheduledExecutorFactory scheduledExecutorFactory = null; private IndexerZkConfig indexerZkConfig; - private WorkerSetupManager workerSetupManager = null; private TaskRunnerFactory taskRunnerFactory = null; private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; private TaskMasterLifecycle taskMasterLifecycle = null; @@ -208,6 +210,12 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } + public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher) + { + this.segmentPusher = segmentPusher; + return this; + } + public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) { this.mergerDBCoordinator = mergeDbCoordinator; @@ -220,12 +228,6 @@ public class IndexerCoordinatorNode extends RegisteringNode return this; } - public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager) - { - this.workerSetupManager = workerSetupManager; - return this; - } - public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) { this.taskRunnerFactory = taskRunnerFactory; @@ -242,6 +244,10 @@ public class IndexerCoordinatorNode extends RegisteringNode { scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); + final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class); + DbConnector.createConfigTable(dbi, managerConfig.getConfigTable()); + JacksonConfigManager configManager = new JacksonConfigManager(new ConfigManager(dbi, managerConfig), jsonMapper); + initializeEmitter(); initializeMonitors(); initializeDB(); @@ -252,14 +258,14 @@ public class IndexerCoordinatorNode extends RegisteringNode initializeTaskStorage(); initializeTaskLockbox(); initializeTaskQueue(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); initializeCurator(); initializeIndexerZkConfig(); - initializeWorkerSetupManager(); - initializeTaskRunnerFactory(); - initializeResourceManagement(); + initializeTaskRunnerFactory(configManager); + initializeResourceManagement(configManager); initializeTaskMasterLifecycle(); initializeServer(); @@ -279,7 +285,7 @@ public class IndexerCoordinatorNode extends RegisteringNode emitter, taskMasterLifecycle, new TaskStorageQueryAdapter(taskStorage), - workerSetupManager + configManager ) ); @@ -339,7 +345,7 @@ public class IndexerCoordinatorNode extends RegisteringNode final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); taskMasterLifecycle = new TaskMasterLifecycle( taskQueue, - taskToolbox, + taskToolboxFactory, config, serviceDiscoveryConfig, taskRunnerFactory, @@ -403,7 +409,7 @@ public class IndexerCoordinatorNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -472,27 +478,27 @@ public class IndexerCoordinatorNode extends RegisteringNode ); } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new LocalTaskActionClient( + new LocalTaskActionClientFactory( taskStorage, new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter) ), emitter, s3Service, - dataSegmentPusher, - segmentKiller, + segmentPusher, + dataSegmentKiller, jsonMapper ); } @@ -546,7 +552,7 @@ public class IndexerCoordinatorNode extends RegisteringNode { if (taskStorage == null) { if (config.getStorageImpl().equals("local")) { - taskStorage = new LocalTaskStorage(); + taskStorage = new HeapMemoryTaskStorage(); } else if (config.getStorageImpl().equals("db")) { final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI()); @@ -556,26 +562,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - public void initializeWorkerSetupManager() - { - if (workerSetupManager == null) { - final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class); - - DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable()); - workerSetupManager = new WorkerSetupManager( - dbi, Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("WorkerSetupManagerExec--%d") - .build() - ), jsonMapper, workerSetupManagerConfig - ); - } - lifecycle.addManagedInstance(workerSetupManager); - } - - public void initializeTaskRunnerFactory() + private void initializeTaskRunnerFactory(final JacksonConfigManager configManager) { if (taskRunnerFactory == null) { if (config.getRunnerImpl().equals("remote")) { @@ -601,7 +588,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), retryScheduledExec, new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), - workerSetupManager + configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class) ); return remoteTaskRunner; @@ -615,7 +602,7 @@ public class IndexerCoordinatorNode extends RegisteringNode public TaskRunner build() { final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads()); - return new LocalTaskRunner(taskToolbox, runnerExec); + return new LocalTaskRunner(taskToolboxFactory, runnerExec); } }; } else { @@ -624,7 +611,7 @@ public class IndexerCoordinatorNode extends RegisteringNode } } - private void initializeResourceManagement() + private void initializeResourceManagement(final JacksonConfigManager configManager) { if (resourceManagementSchedulerFactory == null) { resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory() @@ -639,6 +626,9 @@ public class IndexerCoordinatorNode extends RegisteringNode .setNameFormat("ScalingExec--%d") .build() ); + final AtomicReference workerSetupData = configManager.watch( + WorkerSetupData.CONFIG_KEY, WorkerSetupData.class + ); AutoScalingStrategy strategy; if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { @@ -651,7 +641,7 @@ public class IndexerCoordinatorNode extends RegisteringNode ) ), configFactory.build(EC2AutoScalingStrategyConfig.class), - workerSetupManager + workerSetupData ); } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { strategy = new NoopAutoScalingStrategy(); @@ -664,7 +654,7 @@ public class IndexerCoordinatorNode extends RegisteringNode new SimpleResourceManagementStrategy( strategy, configFactory.build(SimpleResourceManagmentConfig.class), - workerSetupManager + workerSetupData ), configFactory.build(ResourceManagementSchedulerConfig.class), scalingScheduledExec diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index bde7bd6a2fd..b5afa1dceef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -19,6 +19,8 @@ package com.metamx.druid.merger.coordinator.http; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -26,8 +28,9 @@ import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; +import com.metamx.druid.merger.common.actions.TaskActionHolder; import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; @@ -35,10 +38,7 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; import com.metamx.druid.merger.coordinator.scaling.ScalingStats; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.core.type.TypeReference; import javax.ws.rs.Consumes; import javax.ws.rs.GET; @@ -49,6 +49,7 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.Response; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -61,16 +62,18 @@ public class IndexerCoordinatorResource private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; private final ObjectMapper jsonMapper; + private AtomicReference workerSetupDataRef = null; + @Inject public IndexerCoordinatorResource( IndexerCoordinatorConfig config, ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager, + JacksonConfigManager configManager, ObjectMapper jsonMapper ) throws Exception { @@ -78,7 +81,7 @@ public class IndexerCoordinatorResource this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; this.jsonMapper = jsonMapper; } @@ -98,18 +101,15 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response doIndex(final Task task) { - // verify against whitelist - if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - ImmutableMap.of( - "error", - String.format("dataSource[%s] is not whitelisted", task.getDataSource()) - ) - ) - .build(); - } + return taskPost(task); + } + @POST + @Path("/task") + @Consumes("application/json") + @Produces("application/json") + public Response taskPost(final Task task) + { taskMasterLifecycle.getTaskQueue().add(task); return Response.ok(ImmutableMap.of("task", task.getId())).build(); } @@ -162,7 +162,11 @@ public class IndexerCoordinatorResource @Produces("application/json") public Response getWorkerSetupData() { - return Response.ok(workerSetupManager.getWorkerSetupData()).build(); + if (workerSetupDataRef == null) { + workerSetupDataRef = configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class); + } + + return Response.ok(workerSetupDataRef.get()).build(); } @POST @@ -172,7 +176,7 @@ public class IndexerCoordinatorResource final WorkerSetupData workerSetupData ) { - if (!workerSetupManager.setWorkerSetupData(workerSetupData)) { + if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) { return Response.status(Response.Status.BAD_REQUEST).build(); } return Response.ok().build(); @@ -181,9 +185,12 @@ public class IndexerCoordinatorResource @POST @Path("/action") @Produces("application/json") - public Response doAction(final TaskAction action) + public Response doAction(final TaskActionHolder holder) { - final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); + final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask()) + .getTaskActionClientFactory() + .submit(holder.getAction()); + final Map retMap = Maps.newHashMap(); retMap.put("result", ret); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 2b26e08de6a..97f04dd6bad 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final WorkerSetupManager workerSetupManager; + private final JacksonConfigManager configManager; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, @@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, - WorkerSetupManager workerSetupManager + JacksonConfigManager configManager ) { this.jsonMapper = jsonMapper; @@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.workerSetupManager = workerSetupManager; + this.configManager = configManager; } @Override @@ -69,7 +69,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(ServiceEmitter.class).toInstance(emitter); bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); - bind(WorkerSetupManager.class).toInstance(workerSetupManager); + bind(JacksonConfigManager.class).toInstance(configManager); serve("/*").with(GuiceContainer.class); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 47ef22152dd..f9ecbb06f70 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -34,13 +34,13 @@ import com.google.common.collect.Lists; import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -51,26 +51,26 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy private final ObjectMapper jsonMapper; private final AmazonEC2Client amazonEC2Client; private final EC2AutoScalingStrategyConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupDataRef; public EC2AutoScalingStrategy( ObjectMapper jsonMapper, AmazonEC2Client amazonEC2Client, EC2AutoScalingStrategyConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupDataRef ) { this.jsonMapper = jsonMapper; this.amazonEC2Client = amazonEC2Client; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupDataRef = workerSetupDataRef; } @Override public AutoScalingData provision() { try { - WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); + WorkerSetupData setupData = workerSetupDataRef.get(); EC2NodeData workerConfig = setupData.getNodeData(); RunInstancesResult result = amazonEC2Client.runInstances( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java index 666bbaa998c..70c15bcada0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator.scaling; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java index 05c2f2f0b26..082870c83c8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -27,16 +27,16 @@ import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Duration; import java.util.Collection; -import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -46,7 +46,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private final AutoScalingStrategy autoScalingStrategy; private final SimpleResourceManagmentConfig config; - private final WorkerSetupManager workerSetupManager; + private final AtomicReference workerSetupdDataRef; private final ScalingStats scalingStats; private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); @@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat public SimpleResourceManagementStrategy( AutoScalingStrategy autoScalingStrategy, SimpleResourceManagmentConfig config, - WorkerSetupManager workerSetupManager + AtomicReference workerSetupdDataRef ) { this.autoScalingStrategy = autoScalingStrategy; this.config = config; - this.workerSetupManager = workerSetupManager; + this.workerSetupdDataRef = workerSetupdDataRef; this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); } @@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat boolean nothingTerminating = currentlyTerminating.isEmpty(); if (nothingTerminating) { - final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); + final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers(); if (zkWorkers.size() <= minNumWorkers) { log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); return false; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java index fada73cb40e..18cd85e6962 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java @@ -31,6 +31,8 @@ import java.util.List; */ public class WorkerSetupData { + public static final String CONFIG_KEY = "worker.setup"; + private final String minVersion; private final int minNumWorkers; private final EC2NodeData nodeData; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java deleted file mode 100644 index 89a0dd2d5c1..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupManager.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.merger.coordinator.setup; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.metamx.common.ISE; -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 com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig; -import org.apache.commons.collections.MapUtils; - -import org.joda.time.Duration; -import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; -import org.skife.jdbi.v2.Handle; -import org.skife.jdbi.v2.StatementContext; -import org.skife.jdbi.v2.tweak.HandleCallback; - -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicReference; - -/** - */ -public class WorkerSetupManager -{ - private static final Logger log = new Logger(WorkerSetupManager.class); - - private final DBI dbi; - private final ObjectMapper jsonMapper; - private final ScheduledExecutorService exec; - private final WorkerSetupManagerConfig config; - - private final Object lock = new Object(); - - private volatile AtomicReference workerSetupData = new AtomicReference(null); - private volatile boolean started = false; - - public WorkerSetupManager( - DBI dbi, - ScheduledExecutorService exec, - ObjectMapper jsonMapper, - WorkerSetupManagerConfig config - ) - { - this.dbi = dbi; - this.exec = exec; - this.jsonMapper = jsonMapper; - this.config = config; - } - - @LifecycleStart - public void start() - { - synchronized (lock) { - if (started) { - return; - } - - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(0), - config.getPollDuration(), - new Runnable() - { - @Override - public void run() - { - poll(); - } - } - ); - - started = true; - } - } - - @LifecycleStop - public void stop() - { - synchronized (lock) { - if (!started) { - return; - } - - started = false; - } - } - - public void poll() - { - try { - List setupDataList = dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) throws Exception - { - return handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE name = :name", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .fold( - Lists.newArrayList(), - new Folder3, Map>() - { - @Override - public ArrayList fold( - ArrayList workerNodeConfigurations, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) throws SQLException - { - try { - // stringObjectMap lowercases and jackson may fail serde - workerNodeConfigurations.add( - jsonMapper.readValue( - MapUtils.getString(stringObjectMap, "payload"), - WorkerSetupData.class - ) - ); - return workerNodeConfigurations; - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - } - } - ); - - if (setupDataList.isEmpty()) { - throw new ISE("WTF?! No configuration found for worker nodes!"); - } else if (setupDataList.size() != 1) { - throw new ISE("WTF?! Found more than one configuration for worker nodes"); - } - - workerSetupData.set(setupDataList.get(0)); - } - catch (Exception e) { - log.error(e, "Exception while polling for worker setup data!"); - } - } - - @SuppressWarnings("unchecked") - public WorkerSetupData getWorkerSetupData() - { - synchronized (lock) { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - return workerSetupData.get(); - } - } - - public boolean setWorkerSetupData(final WorkerSetupData value) - { - synchronized (lock) { - try { - if (!started) { - throw new ISE("Must start WorkerSetupManager first!"); - } - - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception - { - handle.createStatement( - String.format( - "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload", - config.getConfigTable() - ) - ) - .bind("name", config.getWorkerSetupConfigName()) - .bind("payload", jsonMapper.writeValueAsString(value)) - .execute(); - - return null; - } - } - ); - - workerSetupData.set(value); - } - catch (Exception e) { - log.error(e, "Exception updating worker config"); - return false; - } - } - - return true; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index d7836d46faa..867b8dd9cde 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -23,6 +23,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.task.Task; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; @@ -45,21 +46,21 @@ public class TaskMonitor private final PathChildrenCache pathChildrenCache; private final CuratorFramework cf; private final WorkerCuratorCoordinator workerCuratorCoordinator; - private final TaskToolbox toolbox; + private final TaskToolboxFactory toolboxFactory; private final ExecutorService exec; public TaskMonitor( PathChildrenCache pathChildrenCache, CuratorFramework cf, WorkerCuratorCoordinator workerCuratorCoordinator, - TaskToolbox toolbox, + TaskToolboxFactory toolboxFactory, ExecutorService exec ) { this.pathChildrenCache = pathChildrenCache; this.cf = cf; this.workerCuratorCoordinator = workerCuratorCoordinator; - this.toolbox = toolbox; + this.toolboxFactory = toolboxFactory; this.exec = exec; } @@ -81,10 +82,11 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Task task = toolbox.getObjectMapper().readValue( + final Task task = toolboxFactory.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), Task.class ); + final TaskToolbox toolbox = toolboxFactory.build(task); if (workerCuratorCoordinator.statusExists(task.getId())) { log.warn("Got task %s that I am already running...", task.getId()); @@ -99,7 +101,7 @@ public class TaskMonitor public void run() { final long startTime = System.currentTimeMillis(); - final File taskDir = toolbox.getConfig().getTaskDir(task); + final File taskDir = toolbox.getTaskDir(); log.info("Running task [%s]", task.getId()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index d02ffa5d9e3..caef5bd2935 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,15 +35,14 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; -import com.metamx.druid.loading.S3SegmentKiller; -import com.metamx.druid.loading.SegmentKiller; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; +import com.metamx.druid.loading.S3DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; @@ -106,7 +105,8 @@ public class WorkerNode extends RegisteringNode private ServiceEmitter emitter = null; private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; - private TaskToolbox taskToolbox = null; + private DataSegmentPusher segmentPusher = null; + private TaskToolboxFactory taskToolboxFactory = null; private CuratorFramework curatorFramework = null; private ServiceDiscovery serviceDiscovery = null; private ServiceProvider coordinatorServiceProvider = null; @@ -149,9 +149,15 @@ public class WorkerNode extends RegisteringNode return this; } - public WorkerNode setTaskToolbox(TaskToolbox taskToolbox) + public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher) { - this.taskToolbox = taskToolbox; + this.segmentPusher = segmentPusher; + return this; + } + + public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory) + { + this.taskToolboxFactory = taskToolboxFactory; return this; } @@ -195,6 +201,7 @@ public class WorkerNode extends RegisteringNode initializeCuratorFramework(); initializeServiceDiscovery(); initializeCoordinatorServiceProvider(); + initializeDataSegmentPusher(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); @@ -271,7 +278,7 @@ public class WorkerNode extends RegisteringNode InjectableValues.Std injectables = new InjectableValues.Std(); injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", taskToolbox.getSegmentPusher()); + .addValue("segmentPusher", segmentPusher); jsonMapper.setInjectableValues(injectables); } @@ -334,24 +341,24 @@ public class WorkerNode extends RegisteringNode } } + public void initializeDataSegmentPusher() + { + if (segmentPusher == null) { + segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper); + } + } + public void initializeTaskToolbox() throws S3ServiceException { - if (taskToolbox == null) { - final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( - s3Service, - configFactory.build(S3DataSegmentPusherConfig.class), - jsonMapper - ); - final SegmentKiller segmentKiller = new S3SegmentKiller( - s3Service - ); - taskToolbox = new TaskToolbox( + if (taskToolboxFactory == null) { + final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); + taskToolboxFactory = new TaskToolboxFactory( taskConfig, - new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), + new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper), emitter, s3Service, - dataSegmentPusher, - segmentKiller, + segmentPusher, + dataSegmentKiller, jsonMapper ); } @@ -417,7 +424,7 @@ public class WorkerNode extends RegisteringNode pathChildrenCache, curatorFramework, workerCuratorCoordinator, - taskToolbox, + taskToolboxFactory, workerExec ); lifecycle.addManagedInstance(taskMonitor); diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java index c23b498f739..d0a77cff447 100644 --- a/merger/src/test/java/com/metamx/druid/merger/TestTask.java +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -26,14 +26,14 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.task.DefaultMergeTask; +import com.metamx.druid.merger.common.task.MergeTask; import java.util.List; /** */ @JsonTypeName("test") -public class TestTask extends DefaultMergeTask +public class TestTask extends MergeTask { private final String id; private final TaskStatus status; diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java similarity index 89% rename from merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java rename to merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java index 7c779f6a74c..a2f6e8175fb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java @@ -31,7 +31,7 @@ import java.io.File; import java.util.List; import java.util.Map; -public class MergeTaskTest +public class MergeTaskBaseTest { private final DataSegment.Builder segmentBuilder = DataSegment.builder() .dataSource("foo") @@ -43,7 +43,7 @@ public class MergeTaskTest .add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build()) .build(); - final MergeTask testMergeTask = new MergeTask("foo", segments) + final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments) { @Override protected File merge(Map segments, File outDir) throws Exception @@ -61,13 +61,13 @@ public class MergeTaskTest @Test public void testDataSource() { - Assert.assertEquals("foo", testMergeTask.getDataSource()); + Assert.assertEquals("foo", testMergeTaskBase.getDataSource()); } @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get()); } @Test @@ -81,7 +81,7 @@ public class MergeTaskTest ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, - testMergeTask.getId().substring(0, desiredPrefix.length()) + testMergeTaskBase.getId().substring(0, desiredPrefix.length()) ); } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index 9242e19f355..d88ac044aed 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -11,13 +11,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.TestTask; import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; @@ -42,6 +41,7 @@ import org.junit.Test; import java.io.File; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; import static junit.framework.Assert.fail; @@ -60,7 +60,6 @@ public class RemoteTaskRunnerTest private PathChildrenCache pathChildrenCache; private RemoteTaskRunner remoteTaskRunner; private TaskMonitor taskMonitor; - private WorkerSetupManager workerSetupManager; private ScheduledExecutorService scheduledExec; @@ -280,7 +279,7 @@ public class RemoteTaskRunnerTest new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), cf, workerCuratorCoordinator, - new TaskToolbox( + new TaskToolboxFactory( new TaskConfig() { @Override @@ -316,17 +315,6 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); - - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( - new WorkerSetupData( - "0", - 0, - null, - null - ) - ).atLeastOnce(); - EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( jsonMapper, @@ -335,7 +323,7 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - workerSetupManager + new AtomicReference(new WorkerSetupData("0", 0, null, null)) ); // Create a single worker and wait for things for be ready diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java index ae5b46fdfc1..4fac2c5eded 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.merger.coordinator; import com.google.common.base.Optional; @@ -21,12 +40,13 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.LockAcquireAction; import com.metamx.druid.merger.common.actions.LockListAction; import com.metamx.druid.merger.common.actions.LockReleaseAction; @@ -56,10 +76,8 @@ import org.junit.Test; import java.io.File; import java.io.IOException; -import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.concurrent.Executors; @@ -71,7 +89,7 @@ public class TaskLifecycleTest private TaskQueue tq = null; private TaskRunner tr = null; private MockMergerDBCoordinator mdc = null; - private TaskToolbox tb = null; + private TaskToolboxFactory tb = null; private TaskConsumer tc = null; TaskStorageQueryAdapter tsqa = null; @@ -91,12 +109,12 @@ public class TaskLifecycleTest tmp = Files.createTempDir(); - ts = new LocalTaskStorage(); + ts = new HeapMemoryTaskStorage(); tl = new TaskLockbox(ts); tq = new TaskQueue(ts, tl); mdc = newMockMDC(); - tb = new TaskToolbox( + tb = new TaskToolboxFactory( new TaskConfig() { @Override @@ -117,7 +135,7 @@ public class TaskLifecycleTest return null; } }, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), newMockEmitter(), null, // s3 client new DataSegmentPusher() @@ -128,25 +146,16 @@ public class TaskLifecycleTest return segment; } }, - new SegmentKiller() + new DataSegmentKiller() { @Override - public void kill(Collection segments) throws ServiceException + public void kill(DataSegment segments) throws SegmentLoadingException { } }, new DefaultObjectMapper() - ) - { - @Override - public Map getSegments( - Task task, List segments - ) throws SegmentLoadingException - { - return ImmutableMap.of(); - } - }; + ); tr = new LocalTaskRunner( tb, @@ -239,11 +248,12 @@ public class TaskLifecycleTest @Test public void testKillTask() throws Exception { - // TODO: Worst test ever + // This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator + // Such that this test can test things... final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D")); - final TaskStatus mergedStatus = runTask(killTask); - Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + final TaskStatus status = runTask(killTask); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); } @@ -273,8 +283,8 @@ public class TaskLifecycleTest // Sort of similar to what realtime tasks do: // Acquire lock for first interval - final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock1 present", lock1.isPresent()); @@ -282,8 +292,8 @@ public class TaskLifecycleTest Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); // Acquire lock for second interval - final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + final Optional lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertTrue("lock2 present", lock2.isPresent()); @@ -291,10 +301,9 @@ public class TaskLifecycleTest Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2); // Push first segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -306,17 +315,16 @@ public class TaskLifecycleTest ); // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); // Push second segment - toolbox.getTaskActionClient() + toolbox.getTaskActionClientFactory() .submit( new SegmentInsertAction( - this, ImmutableSet.of( DataSegment.builder() .dataSource("foo") @@ -328,8 +336,8 @@ public class TaskLifecycleTest ); // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction()); // (Confirm lock sanity) Assert.assertEquals("locks4", ImmutableList.of(), locks4); @@ -362,8 +370,8 @@ public class TaskLifecycleTest public TaskStatus run(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) + toolbox.getTaskActionClientFactory() + .submit(new LockListAction()) ); final DataSegment segment = DataSegment.builder() @@ -372,7 +380,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -398,10 +406,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -409,7 +414,7 @@ public class TaskLifecycleTest .version(myLock.getVersion()) .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; @@ -435,10 +440,7 @@ public class TaskLifecycleTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - final TaskLock myLock = Iterables.getOnlyElement( - toolbox.getTaskActionClient() - .submit(new LockListAction(this)) - ); + final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction())); final DataSegment segment = DataSegment.builder() .dataSource("ds") @@ -446,7 +448,7 @@ public class TaskLifecycleTest .version(myLock.getVersion() + "1!!!1!!") .build(); - toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 0d003551ea9..dfb0d959d1a 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -26,7 +26,8 @@ import com.google.common.collect.Sets; import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.merger.common.actions.SpawnTasksAction; import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.task.AbstractTask; @@ -43,7 +44,7 @@ public class TaskQueueTest @Test public void testEmptyQueue() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -65,7 +66,7 @@ public class TaskQueueTest @Test public void testAddRemove() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); @@ -154,12 +155,12 @@ public class TaskQueueTest @Test public void testContinues() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -181,7 +182,7 @@ public class TaskQueueTest Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.notify(t1, t1.run(tb)); + tq.notify(t1, t1.run(tb.build(t1))); Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); @@ -195,7 +196,7 @@ public class TaskQueueTest Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); @@ -211,12 +212,12 @@ public class TaskQueueTest @Test public void testConcurrency() throws Exception { - final TaskStorage ts = new LocalTaskStorage(); + final TaskStorage ts = new HeapMemoryTaskStorage(); final TaskLockbox tl = new TaskLockbox(ts); final TaskQueue tq = newTaskQueue(ts, tl); - final TaskToolbox tb = new TaskToolbox( + final TaskToolboxFactory tb = new TaskToolboxFactory( null, - new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)), null, null, null, @@ -248,7 +249,7 @@ public class TaskQueueTest Thread.sleep(5); // Finish t0 - tq.notify(t0, t0.run(tb)); + tq.notify(t0, t0.run(tb.build(t0))); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -280,7 +281,7 @@ public class TaskQueueTest Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.notify(t3, t3.run(tb)); + tq.notify(t3, t3.run(tb.build(t3))); // We should be able to get t2 now final Task wt2 = tq.poll(); @@ -291,7 +292,7 @@ public class TaskQueueTest Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.notify(t2, t2.run(tb)); + tq.notify(t2, t2.run(tb.build(t2))); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time @@ -305,14 +306,14 @@ public class TaskQueueTest Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.notify(t4, t4.run(tb)); + tq.notify(t4, t4.run(tb.build(t4))); Assert.assertNull("null poll #4", tq.poll()); } @Test public void testBootstrap() throws Exception { - final TaskStorage storage = new LocalTaskStorage(); + final TaskStorage storage = new HeapMemoryTaskStorage(); final TaskLockbox lockbox = new TaskLockbox(storage); storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); @@ -374,7 +375,7 @@ public class TaskQueueTest @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks)); return TaskStatus.success(id); } }; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java index c3aa8378b07..cd569cb77e8 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -33,7 +33,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -42,6 +41,7 @@ import org.junit.Test; import java.util.Arrays; import java.util.Date; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest private Reservation reservation; private Instance instance; private EC2AutoScalingStrategy strategy; - private WorkerSetupManager workerSetupManager; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception @@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest runInstancesResult = EasyMock.createMock(RunInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); reservation = EasyMock.createMock(Reservation.class); - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + workerSetupData = new AtomicReference(null); instance = new Instance() .withInstanceId(INSTANCE_ID) @@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest return "8080"; } }, - workerSetupManager + workerSetupData ); } @@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest EasyMock.verify(runInstancesResult); EasyMock.verify(describeInstancesResult); EasyMock.verify(reservation); - EasyMock.verify(workerSetupManager); } @Test public void testScale() { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( + workerSetupData.set( new WorkerSetupData( "0", 0, @@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest new GalaxyUserData("env", "version", "type") ) ); - EasyMock.replay(workerSetupManager); EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( runInstancesResult diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java index 742525d38a7..2052ae014bb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -29,7 +29,6 @@ import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.Worker; import junit.framework.Assert; import org.easymock.EasyMock; @@ -42,21 +41,22 @@ import org.junit.Test; import java.util.Arrays; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; /** */ public class SimpleResourceManagementStrategyTest { private AutoScalingStrategy autoScalingStrategy; - private WorkerSetupManager workerSetupManager; private Task testTask; private SimpleResourceManagementStrategy simpleResourceManagementStrategy; + private AtomicReference workerSetupData; @Before public void setUp() throws Exception { - workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); + workerSetupData = new AtomicReference(null); testTask = new TestTask( "task1", @@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest return new Duration(0); } }, - workerSetupManager + workerSetupData ); } @@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest @Test public void testDoSuccessfulTerminate() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); @@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } @Test public void testSomethingTerminating() throws Exception { - EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); - EasyMock.replay(workerSetupManager); + workerSetupData.set(new WorkerSetupData("0", 0, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList("ip")).times(2); @@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE ); - EasyMock.verify(workerSetupManager); EasyMock.verify(autoScalingStrategy); } diff --git a/pom.xml b/pom.xml index 3c8f6e76b18..27e639e7c18 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.20.0 + 0.21.0 @@ -165,17 +165,17 @@ com.fasterxml.jackson.core jackson-annotations - 2.1.2 + 2.1.4 com.fasterxml.jackson.core jackson-core - 2.1.3 + 2.1.4 com.fasterxml.jackson.core jackson-databind - 2.1.4-mmx-2 + 2.1.4 com.fasterxml.jackson.datatype @@ -190,12 +190,12 @@ com.fasterxml.jackson.dataformat jackson-dataformat-smile - 2.1.3 + 2.1.4 com.fasterxml.jackson.jaxrs jackson-jaxrs-json-provider - 2.1.3 + 2.1.4 org.codehaus.jackson diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index 087e87d107f..fb47abab945 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -45,25 +45,15 @@ import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusher; -import com.metamx.druid.loading.LocalDataSegmentPusherConfig; -import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; - - - - -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; @@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { if (dataSegmentPusher == null) { - final Properties props = getProps(); - if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { - dataSegmentPusher = new LocalDataSegmentPusher( - getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() - ); - } - else { - - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); - } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } - - dataSegmentPusher = new S3DataSegmentPusher( - s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() - ); - } + dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper()); } } diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index e434bff995d..acd94438d84 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.rules.Rule; -import com.metamx.druid.merge.ClientKillQuery; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -66,6 +66,7 @@ public class InfoResource private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; private final DruidClusterInfo druidClusterInfo; + private final IndexingServiceClient indexingServiceClient; @Inject public InfoResource( @@ -73,7 +74,8 @@ public class InfoResource ServerInventoryManager serverInventoryManager, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, - DruidClusterInfo druidClusterInfo + DruidClusterInfo druidClusterInfo, + IndexingServiceClient indexingServiceClient ) { this.master = master; @@ -81,6 +83,7 @@ public class InfoResource this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; this.druidClusterInfo = druidClusterInfo; + this.indexingServiceClient = indexingServiceClient; } @GET @@ -374,7 +377,7 @@ public class InfoResource ) { if (kill != null && Boolean.valueOf(kill)) { - master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); + indexingServiceClient.killSegments(dataSourceName, new Interval(interval)); } else { if (!databaseSegmentManager.removeDatasource(dataSourceName)) { return Response.status(Response.Status.NOT_FOUND).build(); diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 64e63176d0c..32e03e7ba63 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -33,6 +33,9 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.client.ServerInventoryManagerConfig; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.coordination.DruidClusterInfoConfig; import com.metamx.druid.db.DatabaseRuleManager; @@ -49,6 +52,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.master.LoadQueuePeon; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService; */ public class MasterMain { - private static final Logger log = new Logger(ServerMain.class); + private static final Logger log = new Logger(MasterMain.class); public static void main(String[] args) throws Exception { @@ -166,13 +170,14 @@ public class MasterMain lifecycle ); - ServiceProvider serviceProvider = null; + IndexingServiceClient indexingServiceClient = null; if (druidMasterConfig.getMergerServiceName() != null) { - serviceProvider = Initialization.makeServiceProvider( + ServiceProvider serviceProvider = Initialization.makeServiceProvider( druidMasterConfig.getMergerServiceName(), serviceDiscovery, lifecycle ); + indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); } final DruidClusterInfo druidClusterInfo = new DruidClusterInfo( @@ -180,10 +185,14 @@ public class MasterMain masterYp ); + JacksonConfigManager configManager = new JacksonConfigManager( + new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper + ); + final DruidMaster master = new DruidMaster( druidMasterConfig, druidClusterInfo, - jsonMapper, + configManager, databaseSegmentManager, serverInventoryManager, databaseRuleManager, @@ -191,9 +200,7 @@ public class MasterMain emitter, scheduledExecutorFactory, new ConcurrentHashMap(), - serviceProvider, - httpClient, - new ToStringResponseHandler(Charsets.UTF_8) + indexingServiceClient ); lifecycle.addManagedInstance(master); @@ -226,7 +233,8 @@ public class MasterMain databaseRuleManager, druidClusterInfo, master, - jsonMapper + jsonMapper, + indexingServiceClient ) ); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index b725ed7f358..9bb59d79d43 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -21,7 +21,6 @@ package com.metamx.druid.http; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; -import com.metamx.druid.merge.ClientKillQuery; import javax.inject.Inject; import javax.ws.rs.Consumes; diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java index 47395f73eeb..64d0c98afa3 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java @@ -27,6 +27,7 @@ import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule private final DruidClusterInfo druidClusterInfo; private final DruidMaster master; private final ObjectMapper jsonMapper; + private final IndexingServiceClient indexingServiceClient; public MasterServletModule( ServerInventoryManager serverInventoryManager, @@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule DatabaseRuleManager databaseRuleManager, DruidClusterInfo druidClusterInfo, DruidMaster master, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + IndexingServiceClient indexingServiceClient ) { this.serverInventoryManager = serverInventoryManager; @@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule this.druidClusterInfo = druidClusterInfo; this.master = master; this.jsonMapper = jsonMapper; + this.indexingServiceClient = indexingServiceClient; } @Override @@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule bind(DatabaseRuleManager.class).toInstance(databaseRuleManager); bind(DruidMaster.class).toInstance(master); bind(DruidClusterInfo.class).toInstance(druidClusterInfo); + bind(IndexingServiceClient.class).toInstance(indexingServiceClient); serve("/*").with(GuiceContainer.class); } diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 7cd6caf3c1b..cca4f910b77 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -19,17 +19,24 @@ package com.metamx.druid.initialization; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.LocalDataSegmentPuller; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; @@ -48,12 +55,16 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import com.metamx.druid.utils.PropUtils; +import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; import org.skife.config.ConfigurationObjectFactory; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.util.Map; +import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; /** @@ -145,6 +156,34 @@ public class ServerInit return queryRunners; } + public static DataSegmentPusher getSegmentPusher( + final Properties props, + final ConfigurationObjectFactory configFactory, + final ObjectMapper jsonMapper + ) + { + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + return new LocalDataSegmentPusher(configFactory.build(LocalDataSegmentPusherConfig.class), jsonMapper); + } + else { + + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + return new S3DataSegmentPusher(s3Client, configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper); + } + } + private static class ComputeScratchPool extends StupidPool { private static final Logger log = new Logger(ComputeScratchPool.class); diff --git a/server/src/main/java/com/metamx/druid/master/MergerClient.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java similarity index 84% rename from server/src/main/java/com/metamx/druid/master/MergerClient.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java index c2556ccaea0..85483eaa505 100644 --- a/server/src/main/java/com/metamx/druid/master/MergerClient.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java @@ -17,15 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.master; +package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; -import java.util.List; - /** */ -public interface MergerClient +public interface DataSegmentKiller { - public void runRequest(String dataSource, List segments); + public void kill(DataSegment segments) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java new file mode 100644 index 00000000000..47c39c1857e --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java @@ -0,0 +1,70 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.util.Map; + +/** + */ +public class S3DataSegmentKiller implements DataSegmentKiller +{ + private static final Logger log = new Logger(S3DataSegmentKiller.class); + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentKiller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + try { + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); + } + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "Couldn't kill segment[%s]", segment.getIdentifier()); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index dd5ce951695..9dacbe8b546 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -87,11 +87,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher segment = segment.withSize(indexSize) .withLoadSpec( - ImmutableMap.of( - "type", "s3_zip", - "bucket", outputBucket, - "key", toPush.getKey() - ) + ImmutableMap.of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey()) ) .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java deleted file mode 100644 index 46f6acfc629..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.metamx.druid.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -import java.util.Collection; -import java.util.Map; - -/** - */ -public class S3SegmentKiller implements SegmentKiller -{ - private static final Logger log = new Logger(S3SegmentKiller.class); - - private final RestS3Service s3Client; - - @Inject - public S3SegmentKiller( - RestS3Service s3Client - ) - { - this.s3Client = s3Client; - } - - - @Override - public void kill(Collection segments) throws ServiceException - { - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; - - if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { - log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); - s3Client.deleteObject(s3Bucket, s3Path); - } - if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { - log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); - s3Client.deleteObject(s3Bucket, s3DescriptorPath); - } - } - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java deleted file mode 100644 index 8f8746d5324..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.loading; - -import com.metamx.druid.client.DataSegment; -import org.jets3t.service.ServiceException; - -import java.util.Collection; -import java.util.List; - -/** - */ -public interface SegmentKiller -{ - public void kill(Collection segments) throws ServiceException; -} diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index a21408144b3..74b9d17d57b 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -19,10 +19,8 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -41,32 +39,30 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerInventoryManager; +import com.metamx.druid.client.indexing.IndexingServiceClient; +import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.merge.ClientKillQuery; +import com.metamx.druid.index.v1.IndexIO; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; -import com.netflix.curator.x.discovery.ServiceProvider; import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; -import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -83,26 +79,22 @@ public class DruidMaster private final DruidMasterConfig config; private final DruidClusterInfo clusterInfo; + private final JacksonConfigManager configManager; private final DatabaseSegmentManager databaseSegmentManager; private final ServerInventoryManager serverInventoryManager; private final DatabaseRuleManager databaseRuleManager; private final PhoneBook yp; private final ServiceEmitter emitter; + private final IndexingServiceClient indexingServiceClient; private final ScheduledExecutorService exec; private final ScheduledExecutorService peonExec; private final PhoneBookPeon masterPeon; private final Map loadManagementPeons; - private final ServiceProvider serviceProvider; - - private final HttpClient httpClient; - private final HttpResponseHandler responseHandler; - - private final ObjectMapper jsonMapper; public DruidMaster( DruidMasterConfig config, DruidClusterInfo clusterInfo, - ObjectMapper jsonMapper, + JacksonConfigManager configManager, DatabaseSegmentManager databaseSegmentManager, ServerInventoryManager serverInventoryManager, DatabaseRuleManager databaseRuleManager, @@ -110,31 +102,25 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider, - HttpClient httpClient, - HttpResponseHandler responseHandler + IndexingServiceClient indexingServiceClient ) { this.config = config; this.clusterInfo = clusterInfo; - - this.jsonMapper = jsonMapper; + this.configManager = configManager; this.databaseSegmentManager = databaseSegmentManager; this.serverInventoryManager = serverInventoryManager; this.databaseRuleManager = databaseRuleManager; this.yp = zkPhoneBook; this.emitter = emitter; + this.indexingServiceClient = indexingServiceClient; this.masterPeon = new MasterListeningPeon(); this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d"); this.loadManagementPeons = loadManagementPeons; - - this.serviceProvider = serviceProvider; - this.httpClient = httpClient; - this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -349,27 +335,6 @@ public class DruidMaster } } - public void killSegments(ClientKillQuery killQuery) - { - try { - httpClient.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/index", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - public Set getAvailableDataSegments() { Set availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); @@ -390,7 +355,9 @@ public class DruidMaster for (DataSegment dataSegment : dataSegments) { if (dataSegment.getSize() < 0) { - log.warn("No size on Segment[%s], wtf?", dataSegment); + log.makeAlert("No size on Segment, wtf?") + .addData("segment", dataSegment) + .emit(); } availableSegments.add(dataSegment); } @@ -466,8 +433,14 @@ public class DruidMaster final List> masterRunnables = Lists.newArrayList(); masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod())); - if (config.isMergeSegments() && serviceProvider != null) { - masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod())); + if (config.isMergeSegments() && indexingServiceClient != null) { + + masterRunnables.add( + Pair.of( + new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)), + config.getMasterSegmentMergerPeriod() + ) + ); } for (final Pair masterRunnable : masterRunnables) { @@ -529,6 +502,39 @@ public class DruidMaster } } + public static class DruidMasterVersionConverter implements DruidMasterHelper + { + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whitelistRef; + + public DruidMasterVersionConverter( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) + { + this.indexingServiceClient = indexingServiceClient; + this.whitelistRef = whitelistRef; + } + + @Override + public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) + { + MergerWhitelist whitelist = whitelistRef.get(); + + for (DataSegment dataSegment : params.getAvailableSegments()) { + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + final Integer binaryVersion = dataSegment.getBinaryVersion(); + + if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) { + indexingServiceClient.upgradeSegment(dataSegment); + } + } + } + + return params; + } + } + private class MasterListeningPeon implements PhoneBookPeon { @Override @@ -723,12 +729,13 @@ public class DruidMaster private class MasterSegmentMergerRunnable extends MasterRunnable { - private MasterSegmentMergerRunnable() + private MasterSegmentMergerRunnable(final AtomicReference whitelistRef) { super( ImmutableList.of( new DruidMasterSegmentInfoLoader(DruidMaster.this), - new DruidMasterSegmentMerger(jsonMapper, serviceProvider), + new DruidMasterVersionConverter(indexingServiceClient, whitelistRef), + new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef), new DruidMasterHelper() { @Override @@ -739,8 +746,7 @@ public class DruidMaster params.getEmitter().emit( new ServiceMetricEvent.Builder().build( - "master/merge/count", - stats.getGlobalStats().get("mergedCount") + "master/merge/count", stats.getGlobalStats().get("mergedCount") ) ); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index f3d93d0e6fd..ef4f9d1e6d3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -19,8 +19,6 @@ package com.metamx.druid.master; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultiset; @@ -32,22 +30,19 @@ import com.google.common.collect.Multiset; import com.google.common.collect.Ordering; import com.metamx.common.Pair; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.http.client.response.ToStringResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; /** */ @@ -55,44 +50,40 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper { private static final Logger log = new Logger(DruidMasterSegmentMerger.class); - private final MergerClient mergerClient; + private final IndexingServiceClient indexingServiceClient; + private final AtomicReference whiteListRef; - public DruidMasterSegmentMerger(MergerClient mergerClient) + public DruidMasterSegmentMerger( + IndexingServiceClient indexingServiceClient, + AtomicReference whitelistRef + ) { - this.mergerClient = mergerClient; - } - - public DruidMasterSegmentMerger(ObjectMapper jsonMapper, ServiceProvider serviceProvider) - { - this.mergerClient = new HttpMergerClient( - HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), - new Lifecycle() - ), - new ToStringResponseHandler(Charsets.UTF_8), - jsonMapper, - serviceProvider - ); + this.indexingServiceClient = indexingServiceClient; + this.whiteListRef = whitelistRef; } @Override public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { + MergerWhitelist whitelist = whiteListRef.get(); + MasterStats stats = new MasterStats(); Map> dataSources = Maps.newHashMap(); // Find serviced segments by using a timeline for (DataSegment dataSegment : params.getAvailableSegments()) { - VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline(Ordering.natural()); - dataSources.put(dataSegment.getDataSource(), timeline); + if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { + VersionedIntervalTimeline timeline = dataSources.get(dataSegment.getDataSource()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline(Ordering.natural()); + dataSources.put(dataSegment.getDataSource(), timeline); + } + timeline.add( + dataSegment.getInterval(), + dataSegment.getVersion(), + dataSegment.getShardSpec().createChunk(dataSegment) + ); } - timeline.add( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(dataSegment) - ); } // Find segments to merge @@ -161,7 +152,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames); try { - mergerClient.runRequest(dataSource, segments); + indexingServiceClient.mergeSegments(segments); } catch (Exception e) { log.error( diff --git a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java b/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java deleted file mode 100644 index 07a0f8dba6e..00000000000 --- a/server/src/main/java/com/metamx/druid/master/HttpMergerClient.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. - * - * This program is free software; you can redistribute it and/or - * modify it under the terms of the GNU General Public License - * as published by the Free Software Foundation; either version 2 - * of the License, or (at your option) any later version. - * - * This program is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * GNU General Public License for more details. - * - * You should have received a copy of the GNU General Public License - * along with this program; if not, write to the Free Software - * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. - */ - -package com.metamx.druid.master; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merge.ClientAppendQuery; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; -import com.netflix.curator.x.discovery.ServiceProvider; - - -import java.net.URL; -import java.util.List; - -public class HttpMergerClient implements MergerClient -{ - private final HttpClient client; - private final HttpResponseHandler responseHandler; - private final ObjectMapper jsonMapper; - private final ServiceProvider serviceProvider; - - public HttpMergerClient( - HttpClient client, - HttpResponseHandler responseHandler, - ObjectMapper jsonMapper, - ServiceProvider serviceProvider - ) - { - this.client = client; - this.responseHandler = responseHandler; - this.jsonMapper = jsonMapper; - this.serviceProvider = serviceProvider; - } - - public void runRequest(String dataSource, List segments) - { - try { - byte[] dataToSend = jsonMapper.writeValueAsBytes( - new ClientAppendQuery(dataSource, segments) - ); - - client.post( - new URL( - String.format( - "http://%s:%s/mmx/merger/v1/merge", - serviceProvider.getInstance().getAddress(), - serviceProvider.getInstance().getPort() - ) - ) - ) - .setContent("application/json", dataToSend) - .go(responseHandler) - .get(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java new file mode 100644 index 00000000000..bd55a0cf057 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/master/MergerWhitelist.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.master; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.Sets; + +import java.util.List; +import java.util.Set; + +/** + */ +public class MergerWhitelist +{ + public static final String CONFIG_KEY = "merger.whitelist"; + + private final Set dataSources; + + @JsonCreator + public MergerWhitelist(Set dataSources) + { + this.dataSources = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + this.dataSources.addAll(dataSources); + } + + @JsonValue + public Set getDataSources() + { + return dataSources; + } + + public boolean contains(String val) + { + return dataSources.contains(val); + } +} diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java index 8e19ed5d330..1ad1f96d163 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java @@ -23,12 +23,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.indexing.IndexingServiceClient; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; import java.util.Collection; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; public class DruidMasterSegmentMergerTest { @@ -367,11 +369,7 @@ public class DruidMasterSegmentMergerTest DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build() ); - Assert.assertEquals( - ImmutableList.of( - ImmutableList.of(segments.get(4), segments.get(5)) - ), merge(segments) - ); + Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments)); } /** @@ -380,16 +378,17 @@ public class DruidMasterSegmentMergerTest private static List> merge(final Collection segments) { final List> retVal = Lists.newArrayList(); - final MergerClient mergerClient = new MergerClient() + final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null) { @Override - public void runRequest(String dataSource, List segmentsToMerge) + public void mergeSegments(List segmentsToMerge) { retVal.add(segmentsToMerge); } }; - final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(mergerClient); + final AtomicReference whitelistRef = new AtomicReference(null); + final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef); final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder() .withAvailableSegments(ImmutableSet.copyOf(segments)) .withMergeBytesLimit(mergeBytesLimit) diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index 895659586ec..584ae31de47 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -152,8 +152,6 @@ public class DruidMasterTest new NoopServiceEmitter(), scheduledExecutorFactory, loadManagementPeons, - null, - null, null ); }