From 75a464abe218f3b68dfc772ebf4456620ace83d8 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 14 May 2013 11:25:57 -0500 Subject: [PATCH 01/92] 1) Initial commit of conversion to using Guice modules for bootstrapping. Things don't actually completely work yet. --- .../java/com/metamx/druid/QueryableNode.java | 9 +- .../druid/client/ServerInventoryView.java | 9 +- .../CuratorConfig.java | 2 +- .../metamx/druid/curator/CuratorModule.java | 29 +++ .../discovery/CuratorServiceAnnouncer.java | 18 +- .../curator/discovery/DiscoveryModule.java | 43 ++++ .../com/metamx/druid/http/BrokerNode.java | 18 +- .../initialization/ConfigFactoryModule.java | 46 ++++ ...onfig.java => CuratorDiscoveryConfig.java} | 23 +- .../druid/initialization/DruidNodeConfig.java | 102 ++++++++ .../druid/initialization/EmitterModule.java | 128 ++++++++++ .../initialization/HttpEmitterConfig.java | 36 +++ .../initialization/HttpEmitterModule.java | 62 +++++ .../druid/initialization/Initialization.java | 22 +- .../initialization/LogEmitterModule.java | 48 ++++ .../initialization/PropertiesModule.java | 135 ++++++++++ common/pom.xml | 12 + .../com/metamx/druid/concurrent/Execs.java | 16 +- .../java/com/metamx/druid/db/DbConnector.java | 31 ++- .../metamx/druid/db/DbConnectorConfig.java | 63 +++-- .../com/metamx/druid/db/DbTablesConfig.java | 69 ++++++ .../metamx/druid/guice/ConfigProvider.java | 45 ++++ .../druid/guice/DruidGuiceExtensions.java | 15 ++ .../com/metamx/druid/guice/DruidScopes.java | 71 ++++++ .../druid/guice/DruidSecondaryModule.java | 45 ++++ .../druid/guice/JsonConfigProvider.java | 121 +++++++++ .../com/metamx/druid/guice/LazySingleton.java | 18 ++ .../metamx/druid/guice/LifecycleModule.java | 74 ++++++ .../metamx/druid/guice/LifecycleScope.java | 67 +++++ .../metamx/druid/guice/ManageLifecycle.java | 19 ++ .../metamx/druid/jackson/JacksonModule.java | 53 ++++ .../java/com/metamx/druid/jackson/Json.java | 17 ++ .../java/com/metamx/druid/jackson/Smile.java | 17 ++ .../com/metamx/druid/utils/SocketUtil.java | 39 +++ .../druid/guice/LifecycleScopeTest.java | 231 ++++++++++++++++++ .../coordinator/TaskMasterLifecycle.java | 6 +- .../http/IndexerCoordinatorNode.java | 19 +- .../merger/worker/executor/ExecutorNode.java | 12 +- .../druid/merger/worker/http/WorkerNode.java | 6 +- pom.xml | 7 +- .../metamx/druid/db/DatabaseRuleManager.java | 38 ++- .../druid/db/DatabaseRuleManagerConfig.java | 7 +- .../druid/db/DatabaseSegmentManager.java | 38 ++- .../db/DatabaseSegmentManagerConfig.java | 16 +- .../com/metamx/druid/guice/MasterModule.java | 54 ++++ .../com/metamx/druid/guice/ServerModule.java | 16 ++ .../com/metamx/druid/http/InfoResource.java | 2 +- .../com/metamx/druid/http/MasterMain.java | 183 +++++++------- .../com/metamx/druid/http/MasterResource.java | 2 +- .../loading/DelegatingSegmentLoader.java | 2 +- .../metamx/druid/metrics/MetricsModule.java | 77 ++++++ .../metamx/druid/metrics/ServerMonitor.java | 1 - .../druid/db/DatabaseSegmentManagerTest.java | 20 +- 53 files changed, 2002 insertions(+), 257 deletions(-) rename client/src/main/java/com/metamx/druid/{initialization => curator}/CuratorConfig.java (96%) create mode 100644 client/src/main/java/com/metamx/druid/curator/CuratorModule.java create mode 100644 client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java rename client/src/main/java/com/metamx/druid/initialization/{ServiceDiscoveryConfig.java => CuratorDiscoveryConfig.java} (69%) create mode 100644 client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/EmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java create mode 100644 common/src/main/java/com/metamx/druid/db/DbTablesConfig.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ConfigProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidScopes.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LazySingleton.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LifecycleModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LifecycleScope.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/JacksonModule.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/Json.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/Smile.java create mode 100644 common/src/main/java/com/metamx/druid/utils/SocketUtil.java create mode 100644 common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java create mode 100644 server/src/main/java/com/metamx/druid/guice/MasterModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/ServerModule.java create mode 100644 server/src/main/java/com/metamx/druid/metrics/MetricsModule.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 68f978929d6..ef8a995295b 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -42,9 +41,9 @@ import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.http.RequestLogger; -import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkPathsConfig; @@ -70,8 +69,6 @@ import java.lang.reflect.Field; import java.util.Arrays; import java.util.List; import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -349,14 +346,10 @@ public abstract class QueryableNode extends Registering private void initializeServerInventoryView() { if (serverInventoryView == null) { - final ExecutorService exec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() - ); serverInventoryView = new ServerInventoryView( getConfigFactory().build(ServerInventoryViewConfig.class), getZkPaths(), getCuratorFramework(), - exec, getJsonMapper() ); lifecycle.addManagedInstance(serverInventoryView); diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java index b15b547a5f2..5da48dcea3e 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java @@ -24,11 +24,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.MapMaker; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.concurrent.Execs; import com.metamx.druid.curator.inventory.CuratorInventoryManager; import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy; import com.metamx.druid.curator.inventory.InventoryManagerConfig; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; @@ -38,11 +41,11 @@ import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; /** */ +@ManageLifecycle public class ServerInventoryView implements ServerView, InventoryView { private static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class); @@ -55,11 +58,11 @@ public class ServerInventoryView implements ServerView, InventoryView private static final Map removedSegments = new MapMaker().makeMap(); + @Inject public ServerInventoryView( final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, - final ExecutorService exec, final ObjectMapper jsonMapper ) { @@ -79,7 +82,7 @@ public class ServerInventoryView implements ServerView, InventoryView return zkPaths.getServedSegmentsPath(); } }, - exec, + Execs.singleThreaded("ServerInventoryView-%s"), new CuratorInventoryManagerStrategy() { @Override diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java b/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java similarity index 96% rename from client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java rename to client/src/main/java/com/metamx/druid/curator/CuratorConfig.java index d09c36f369e..10e7e381278 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java +++ b/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package com.metamx.druid.curator; import org.skife.config.Config; import org.skife.config.Default; diff --git a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java new file mode 100644 index 00000000000..94fcee12b42 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java @@ -0,0 +1,29 @@ +package com.metamx.druid.curator; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.initialization.Initialization; +import org.apache.curator.framework.CuratorFramework; + +import java.io.IOException; + +/** + */ +public class CuratorModule implements Module +{ + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, CuratorConfig.class); + } + + @Provides @LazySingleton + public CuratorFramework makeCurator(CuratorConfig config, Lifecycle lifecycle) throws IOException + { + return Initialization.makeCuratorFramework(config, lifecycle); + } +} diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java index a3156b3205d..035924a6e2b 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java @@ -1,6 +1,7 @@ package com.metamx.druid.curator.discovery; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; @@ -10,18 +11,19 @@ import java.util.Map; /** * Uses the Curator Service Discovery recipe to announce services. */ -public class CuratorServiceAnnouncer implements ServiceAnnouncer +public class CuratorServiceAnnouncer implements ServiceAnnouncer { private static final Logger log = new Logger(CuratorServiceAnnouncer.class); - private final ServiceDiscovery discovery; - private final ServiceInstanceFactory instanceFactory; - private final Map> instanceMap = Maps.newHashMap(); + private final ServiceDiscovery discovery; + private final ServiceInstanceFactory instanceFactory; + private final Map> instanceMap = Maps.newHashMap(); private final Object monitor = new Object(); + @Inject public CuratorServiceAnnouncer( - ServiceDiscovery discovery, - ServiceInstanceFactory instanceFactory + ServiceDiscovery discovery, + ServiceInstanceFactory instanceFactory ) { this.discovery = discovery; @@ -31,7 +33,7 @@ public class CuratorServiceAnnouncer implements ServiceAnnouncer @Override public void announce(String service) throws Exception { - final ServiceInstance instance; + final ServiceInstance instance; synchronized (monitor) { if (instanceMap.containsKey(service)) { @@ -57,7 +59,7 @@ public class CuratorServiceAnnouncer implements ServiceAnnouncer @Override public void unannounce(String service) throws Exception { - final ServiceInstance instance; + final ServiceInstance instance; synchronized (monitor) { instance = instanceMap.get(service); diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java new file mode 100644 index 00000000000..c5a2c701bf8 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java @@ -0,0 +1,43 @@ +package com.metamx.druid.curator.discovery; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.Initialization; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.x.discovery.ServiceDiscovery; + +/** + */ +public class DiscoveryModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.discovery.curator", CuratorDiscoveryConfig.class); + binder.bind(ServiceAnnouncer.class).to(CuratorServiceAnnouncer.class).in(LazySingleton.class); + } + + @Provides @LazySingleton + public ServiceDiscovery getServiceDiscovery( + CuratorFramework curator, + CuratorDiscoveryConfig config, + Lifecycle lifecycle + ) throws Exception + { + return Initialization.makeServiceDiscoveryClient(curator, config, lifecycle); + } + + @Provides @LazySingleton + public ServiceInstanceFactory getServiceInstanceFactory( + DruidNodeConfig nodeConfig + ) + { + return Initialization.makeServiceInstanceFactory(nodeConfig); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 191c2ea4ad8..815ffbe1b9b 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -42,10 +42,11 @@ import com.metamx.druid.client.cache.MapCache; import com.metamx.druid.client.cache.MapCacheConfig; import com.metamx.druid.client.cache.MemcachedCache; import com.metamx.druid.client.cache.MemcachedCacheConfig; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.ReflectionQueryToolChestWarehouse; @@ -227,17 +228,18 @@ public class BrokerNode extends QueryableNode { if (useDiscovery) { final Lifecycle lifecycle = getLifecycle(); - final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig curatorDiscoveryConfig = getConfigFactory().build(CuratorDiscoveryConfig.class); + final DruidNodeConfig nodeConfig = getConfigFactory().build(DruidNodeConfig.class); final CuratorFramework curatorFramework = Initialization.makeCuratorFramework( - serviceDiscoveryConfig, lifecycle + getConfigFactory().build(CuratorConfig.class), lifecycle ); - final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, serviceDiscoveryConfig, lifecycle + final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, curatorDiscoveryConfig, lifecycle ); final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer( - serviceDiscoveryConfig, serviceDiscovery + nodeConfig, serviceDiscovery ); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, lifecycle); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); } } diff --git a/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java b/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java new file mode 100644 index 00000000000..6d136c53da4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.config.Config; +import com.metamx.druid.guice.LazySingleton; +import org.skife.config.ConfigurationObjectFactory; + +import java.util.Properties; + +/** + */ +public class ConfigFactoryModule implements Module +{ + @Override + public void configure(Binder binder) + { + + } + + @Provides @LazySingleton + public ConfigurationObjectFactory makeFactory(Properties props) + { + return Config.createFactory(props); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java similarity index 69% rename from client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java rename to client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java index 04776d6545a..ee0f7ced76e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java @@ -19,21 +19,22 @@ package com.metamx.druid.initialization; -import org.skife.config.Config; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public abstract class ServiceDiscoveryConfig extends CuratorConfig +public abstract class CuratorDiscoveryConfig { - @Config("druid.service") - public abstract String getServiceName(); + @JsonProperty + private String path = null; - @Config("druid.host") - public abstract String getHost(); + public String getPath() + { + return path; + } - @Config("druid.port") - public abstract int getPort(); - - @Config("druid.zk.paths.discoveryPath") - public abstract String getDiscoveryPath(); + public boolean useDiscovery() + { + return path != null; + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java new file mode 100644 index 00000000000..94f02c5edd5 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java @@ -0,0 +1,102 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.utils.SocketUtil; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +/** + */ +public abstract class DruidNodeConfig +{ + @NotNull + private String serviceName = null; + + @NotNull + private String host = null; + + @Min(0) @Max(0xffff) + private int port = -1; + + @JsonCreator + public DruidNodeConfig( + @JsonProperty("service") String serviceName, + @JsonProperty("host") String host, + @JsonProperty("port") Integer port + ) + { + this.serviceName = serviceName; + + if (port == null) { + if (host == null) { + setHostAndPort(null, -1); + } + else if (host.contains(":")) { + try { + setHostAndPort(host, Integer.parseInt(host.split(":")[1])); + } + catch (Exception e) { + setHostAndPort(host, -1); + } + } + else { + final int openPort = SocketUtil.findOpenPort(8080); + setHostAndPort(String.format("%s:%d", host, openPort), openPort); + } + } + else { + if (host == null || host.contains(":")) { + setHostAndPort(host, port); + } + else { + setHostAndPort(String.format("%s:%d", host, port), port); + } + } + } + + private void setHostAndPort(String host, int port) + { + this.host = host; + this.port = port; + } + + @JsonProperty("service") + public String getServiceName() + { + return serviceName; + } + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public int getPort() + { + return port; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java new file mode 100644 index 00000000000..edc01e65a26 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -0,0 +1,128 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Binding; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Provider; +import com.google.inject.Provides; +import com.google.inject.TypeLiteral; +import com.google.inject.name.Named; +import com.google.inject.name.Names; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.guice.DruidScopes; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.service.ServiceEmitter; + +import java.lang.annotation.Annotation; +import java.util.List; +import java.util.Properties; + +/** + */ +public class EmitterModule implements Module +{ + private static final Logger log = new Logger(EmitterModule.class); + + private final Properties props; + + @Inject + public EmitterModule( + Properties props + ) + { + this.props = props; + } + + @Override + public void configure(Binder binder) + { + String emitterType = props.getProperty("druid.emitter", ""); + + binder.install(new LogEmitterModule()); + binder.install(new HttpEmitterModule()); + + binder.bind(Emitter.class).toProvider(new EmitterProvider(emitterType)).in(DruidScopes.SINGLETON); + } + + @Provides + @LazySingleton + public ServiceEmitter getServiceEmitter(DruidNodeConfig config, Emitter emitter) + { + final ServiceEmitter retVal = new ServiceEmitter(config.getServiceName(), config.getHost(), emitter); + EmittingLogger.registerEmitter(retVal); + return retVal; + } + + private static class EmitterProvider implements Provider + { + private final String emitterType; + + private Emitter emitter = null; + + EmitterProvider( + String emitterType + ) + { + this.emitterType = emitterType; + } + + @Inject + public void inject(Injector injector) + { + final List> emitterBindings = injector.findBindingsByType(new TypeLiteral(){}); + + for (Binding binding : emitterBindings) { + if (Names.named(emitterType).equals(binding.getKey().getAnnotation())) { + emitter = binding.getProvider().get(); + break; + } + } + + if (emitter == null) { + List knownTypes = Lists.newArrayList(); + for (Binding binding : emitterBindings) { + final Annotation annotation = binding.getKey().getAnnotation(); + if (annotation != null) { + knownTypes.add(((Named) annotation).value()); + } + } + throw new ISE("Uknown emitter type, known types[%s]", knownTypes); + } + } + + + @Override + public Emitter get() + { + if (emitter == null) { + throw new ISE("Emitter was null, that's bad!"); + } + return emitter; + } + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java new file mode 100644 index 00000000000..fd6955d3579 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Duration; + +/** + */ +public abstract class HttpEmitterConfig extends com.metamx.emitter.core.HttpEmitterConfig +{ + @JsonProperty + private Duration timeOut = new Duration("PT5m"); + + public Duration getReadTimeout() + { + return timeOut; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java new file mode 100644 index 00000000000..8f7cc45018f --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.base.Supplier; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.HttpPostEmitter; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; + +import javax.annotation.Nullable; +import javax.net.ssl.SSLContext; + +/** + */ +public class HttpEmitterModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter.http", HttpEmitterConfig.class); + } + + @Provides @LazySingleton @Named("http") + public Emitter getEmitter(Supplier config, @Nullable SSLContext sslContext, Lifecycle lifecycle) + { + final HttpClientConfig.Builder builder = HttpClientConfig + .builder() + .withNumConnections(1) + .withReadTimeout(config.get().getReadTimeout()); + + if (sslContext != null) { + builder.withSslContext(sslContext); + } + + return new HttpPostEmitter(config.get(), HttpClientInit.createClient(builder.build(), lifecycle)); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 0d49ff0cefe..66366a3e739 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -27,6 +27,7 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.discovery.AddressPortServiceInstanceFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; @@ -42,7 +43,6 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.BoundedExponentialBackoffRetry; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; -import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; import org.apache.zookeeper.data.Stat; import org.mortbay.jetty.Connector; @@ -221,16 +221,16 @@ public class Initialization return framework; } - public static ServiceDiscovery makeServiceDiscoveryClient( + public static ServiceDiscovery makeServiceDiscoveryClient( CuratorFramework discoveryClient, - ServiceDiscoveryConfig config, + CuratorDiscoveryConfig config, Lifecycle lifecycle ) throws Exception { - final ServiceDiscovery serviceDiscovery = + final ServiceDiscovery serviceDiscovery = ServiceDiscoveryBuilder.builder(Void.class) - .basePath(config.getDiscoveryPath()) + .basePath(config.getPath()) .client(discoveryClient) .build(); @@ -260,21 +260,21 @@ public class Initialization } public static ServiceAnnouncer makeServiceAnnouncer( - ServiceDiscoveryConfig config, - ServiceDiscovery serviceDiscovery + DruidNodeConfig config, + ServiceDiscovery serviceDiscovery ) { - final ServiceInstanceFactory serviceInstanceFactory = makeServiceInstanceFactory(config); + final ServiceInstanceFactory serviceInstanceFactory = makeServiceInstanceFactory(config); return new CuratorServiceAnnouncer(serviceDiscovery, serviceInstanceFactory); } public static void announceDefaultService( - final ServiceDiscoveryConfig config, + final DruidNodeConfig nodeConfig, final ServiceAnnouncer serviceAnnouncer, final Lifecycle lifecycle ) throws Exception { - final String service = config.getServiceName().replace('/', ':'); + final String service = nodeConfig.getServiceName().replace('/', ':'); lifecycle.addHandler( new Lifecycle.Handler() @@ -357,7 +357,7 @@ public class Initialization ); } - public static ServiceInstanceFactory makeServiceInstanceFactory(ServiceDiscoveryConfig config) + public static ServiceInstanceFactory makeServiceInstanceFactory(DruidNodeConfig config) { final String host = config.getHost(); final String address; diff --git a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java new file mode 100644 index 00000000000..237a87ec1b6 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.LoggingEmitterConfig; + +/** + */ +public class LogEmitterModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter.logging", LoggingEmitterConfig.class); + } + + @Provides @LazySingleton @Named("logging") + public Emitter makeEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) + { + return new LoggingEmitter(config, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java new file mode 100644 index 00000000000..3e1eb3efa1a --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java @@ -0,0 +1,135 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.io.Closeables; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.common.config.Config; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorConfig; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.data.Stat; +import org.skife.config.ConfigurationObjectFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.Properties; + +/** + */ +public class PropertiesModule implements Module +{ + private static final Logger log = new Logger(PropertiesModule.class); + + private final String propertiesFile; + + public PropertiesModule(String propertiesFile) + { + this.propertiesFile = propertiesFile; + } + + @Override + public void configure(Binder binder) + { + final Properties zkProps = new Properties(); + final Properties fileProps = new Properties(zkProps); + + // Note that zookeeper coordinates must be either in cmdLine or in runtime.properties + Properties sp = System.getProperties(); + + Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain + tmp_props.putAll(sp); + + final InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); + if (stream == null) { + log.info("%s not found on classpath, relying only on system properties and zookeeper.", propertiesFile); + } else { + log.info("Loading properties from %s", propertiesFile); + try { + try { + fileProps.load(stream); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + finally { + Closeables.closeQuietly(stream); + } + } + + // log properties from file; stringPropertyNames() would normally cascade down into the sub Properties objects, but + // zkProps (the parent level) is empty at this point so it will only log properties from runtime.properties + for (String prop : fileProps.stringPropertyNames()) { + log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop)); + } + + final String zkHostsProperty = "druid.zk.service.host"; + + if (tmp_props.getProperty(zkHostsProperty) != null) { + final ConfigurationObjectFactory factory = Config.createFactory(tmp_props); + + ZkPathsConfig config; + try { + config = factory.build(ZkPathsConfig.class); + } + catch (IllegalArgumentException e) { + log.warn(e, "Unable to build ZkPathsConfig. Cannot load properties from ZK."); + config = null; + } + + if (config != null) { + Lifecycle lifecycle = new Lifecycle(); + try { + CuratorFramework curator = Initialization.makeCuratorFramework(factory.build(CuratorConfig.class), lifecycle); + + lifecycle.start(); + + final Stat stat = curator.checkExists().forPath(config.getPropertiesPath()); + if (stat != null) { + final byte[] data = curator.getData().forPath(config.getPropertiesPath()); + zkProps.load(new InputStreamReader(new ByteArrayInputStream(data), Charsets.UTF_8)); + } + + // log properties from zk + for (String prop : zkProps.stringPropertyNames()) { + log.info("Loaded(zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + lifecycle.stop(); + } + } + } else { + log.warn("property[%s] not set, skipping ZK-specified properties.", zkHostsProperty); + } + + binder.bind(Properties.class).toInstance(tmp_props); + } +} diff --git a/common/pom.xml b/common/pom.xml index 31831cf71fa..9bc588bff7e 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -63,6 +63,10 @@ org.apache.curator curator-x-discovery + + org.hibernate + hibernate-validator + it.uniroma3.mat extendedset @@ -87,6 +91,14 @@ com.fasterxml.jackson.datatype jackson-datatype-joda + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + + + com.google.inject + guice + org.jdbi jdbi diff --git a/common/src/main/java/com/metamx/druid/concurrent/Execs.java b/common/src/main/java/com/metamx/druid/concurrent/Execs.java index 7b86ca9e76c..619a8199b00 100644 --- a/common/src/main/java/com/metamx/druid/concurrent/Execs.java +++ b/common/src/main/java/com/metamx/druid/concurrent/Execs.java @@ -23,6 +23,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; /** */ @@ -31,7 +33,19 @@ public class Execs public static ExecutorService singleThreaded(String nameFormat) { return Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build() + makeThreadFactory(nameFormat) ); } + + public static ScheduledExecutorService scheduledSingleThreaded(String nameFormat) + { + return Executors.newSingleThreadScheduledExecutor( + makeThreadFactory(nameFormat) + ); + } + + public static ThreadFactory makeThreadFactory(String nameFormat) + { + return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build(); + } } 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 1e73b731353..d6b0b3e47f2 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -19,6 +19,7 @@ package com.metamx.druid.db; +import com.google.common.base.Supplier; import com.metamx.common.logger.Logger; import org.apache.commons.dbcp.BasicDataSource; import org.skife.jdbi.v2.DBI; @@ -159,12 +160,14 @@ public class DbConnector } } - private final DbConnectorConfig config; + private final Supplier config; + private final Supplier dbTables; private final DBI dbi; - public DbConnector(DbConnectorConfig config) + public DbConnector(Supplier config, Supplier dbTables) { this.config = config; + this.dbTables = dbTables; this.dbi = new DBI(getDatasource()); } @@ -176,16 +179,28 @@ public class DbConnector private DataSource getDatasource() { - BasicDataSource dataSource = new BasicDataSource(); - dataSource.setUsername(config.getDatabaseUser()); - dataSource.setPassword(config.getDatabasePassword()); - dataSource.setUrl(config.getDatabaseConnectURI()); + DbConnectorConfig connectorConfig = config.get(); - if (config.isValidationQuery()) { - dataSource.setValidationQuery(config.getValidationQuery()); + BasicDataSource dataSource = new BasicDataSource(); + dataSource.setUsername(connectorConfig.getUser()); + dataSource.setPassword(connectorConfig.getPassword()); + dataSource.setUrl(connectorConfig.getConnectURI()); + + if (connectorConfig.isUseValidationQuery()) { + dataSource.setValidationQuery(connectorConfig.getValidationQuery()); dataSource.setTestOnBorrow(true); } return dataSource; } + + public void createSegmentTable() + { + createSegmentTable(dbi, dbTables.get().getSegmentsTable()); + } + + public void createRulesTable() + { + createRuleTable(dbi, dbTables.get().getRulesTable()); + } } diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index fb7d1a99916..f6e15bbcc48 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -20,37 +20,60 @@ package com.metamx.druid.db; import com.fasterxml.jackson.annotation.JsonProperty; -import org.skife.config.Config; + +import javax.validation.constraints.NotNull; /** */ public abstract class DbConnectorConfig { - @JsonProperty("connectURI") - @Config("druid.database.connectURI") - public abstract String getDatabaseConnectURI(); + @JsonProperty + private boolean createTables = true; - @JsonProperty("user") - @Config("druid.database.user") - public abstract String getDatabaseUser(); + @JsonProperty + @NotNull + private String connectURI = null; - @JsonProperty("password") - @Config("druid.database.password") - public abstract String getDatabasePassword(); + @JsonProperty + @NotNull + private String user = null; - @JsonProperty("segmentTable") - @Config("druid.database.segmentTable") - public abstract String getSegmentTable(); + @JsonProperty + @NotNull + private String password = null; - @JsonProperty("useValidationQuery") - @Config("druid.database.validation") - public boolean isValidationQuery() { - return false; + @JsonProperty + private boolean useValidationQuery = false; + + @JsonProperty + private String validationQuery = "SELECT 1"; + + public boolean isCreateTables() + { + return createTables; + } + + public String getConnectURI() + { + return connectURI; + } + + public String getUser() + { + return user; + } + + public String getPassword() + { + return password; + } + + public boolean isUseValidationQuery() + { + return useValidationQuery; } - @JsonProperty("validationQuery") - @Config("druid.database.validationQuery") public String getValidationQuery() { - return "SELECT 1"; + return validationQuery; } } diff --git a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java new file mode 100644 index 00000000000..96d7170f6fb --- /dev/null +++ b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java @@ -0,0 +1,69 @@ +package com.metamx.druid.db; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.ISE; + +import javax.validation.constraints.NotNull; + +/** + */ +public class DbTablesConfig +{ + public static DbTablesConfig fromBase(String base) + { + return new DbTablesConfig(base, null, null); + } + + @NotNull + private final String base; + + @NotNull + private final String segmentsTable; + + @NotNull + private final String ruleTable; + + @JsonCreator + public DbTablesConfig( + @JsonProperty("base") String base, + @JsonProperty("segments") String segmentsTable, + @JsonProperty("rules") String rulesTable + ) + { + + this.base = base; + this.segmentsTable = makeTableName(segmentsTable, "segments"); + this.ruleTable = makeTableName(rulesTable, "rules"); + } + + private String makeTableName(String explicitTableName, String defaultSuffix) + { + if (explicitTableName == null) { + if (base == null) { + throw new ISE("table[%s] unknown! Both base and %s were null!", defaultSuffix, defaultSuffix); + } + return String.format("%s_%s", base, defaultSuffix); + } + + return explicitTableName; + } + + @JsonProperty + public String getBase() + { + return base; + } + + @JsonProperty("segments") + public String getSegmentsTable() + { + return segmentsTable; + } + + @JsonProperty("rules") + public String getRulesTable() + { + return ruleTable; + } +} \ No newline at end of file diff --git a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java new file mode 100644 index 00000000000..79eff1b41ff --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java @@ -0,0 +1,45 @@ +package com.metamx.druid.guice; + +import com.google.common.base.Preconditions; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Provider; +import org.skife.config.ConfigurationObjectFactory; + +/** + */ +public class ConfigProvider implements Provider +{ + public static void bind(Binder binder, Class clazz) + { + binder.bind(clazz).toProvider(of(clazz)).in(DruidScopes.SINGLETON); + } + + public static Provider of(Class clazz) + { + return new ConfigProvider(clazz); + } + + private final Class clazz; + + private T object = null; + + public ConfigProvider( + Class clazz + ) + { + this.clazz = clazz; + } + + @Inject + public void inject(ConfigurationObjectFactory factory) + { + object = factory.build(clazz); + } + + @Override + public T get() + { + return Preconditions.checkNotNull(object, "WTF!? Code misconfigured, inject() didn't get called."); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java new file mode 100644 index 00000000000..e999b725aa2 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java @@ -0,0 +1,15 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; + +/** + */ +public class DruidGuiceExtensions implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidScopes.java b/common/src/main/java/com/metamx/druid/guice/DruidScopes.java new file mode 100644 index 00000000000..151325e587b --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidScopes.java @@ -0,0 +1,71 @@ +package com.metamx.druid.guice; + +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.Scope; +import com.google.inject.Scopes; +import com.metamx.common.lifecycle.Lifecycle; + +/** + */ +public class DruidScopes +{ + public static final Scope SINGLETON = new Scope() + { + @Override + public Provider scope(Key key, Provider unscoped) + { + return Scopes.SINGLETON.scope(key, unscoped); + } + + @Override + public String toString() + { + return "DruidScopes.SINGLETON"; + } + }; + + public static final Scope LIFECYCLE = new Scope() + { + @Override + public Provider scope(final Key key, final Provider unscoped) + { + return new Provider() + { + + private Provider provider; + + @Inject + public void inject(final Lifecycle lifecycle) + { + provider = Scopes.SINGLETON.scope( + key, + new Provider() + { + + @Override + public T get() + { + return lifecycle.addManagedInstance(unscoped.get()); + } + } + ); + } + + @Override + public T get() + { + System.out.println(provider); + return provider.get(); + } + }; + } + + @Override + public String toString() + { + return "DruidScopes.LIFECYCLE"; + } + }; +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java new file mode 100644 index 00000000000..da8d2ce59c1 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -0,0 +1,45 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Module; +import com.metamx.druid.jackson.Json; +import com.metamx.druid.jackson.Smile; +import org.skife.config.ConfigurationObjectFactory; + +import java.util.Properties; + +/** + */ +public class DruidSecondaryModule implements Module +{ + private final Properties properties; + private final ConfigurationObjectFactory factory; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + + @Inject + public DruidSecondaryModule( + Properties properties, + ConfigurationObjectFactory factory, + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper + ) + { + this.properties = properties; + this.factory = factory; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + } + + @Override + public void configure(Binder binder) + { + binder.install(new DruidGuiceExtensions()); + binder.bind(Properties.class).toInstance(properties); + binder.bind(ConfigurationObjectFactory.class).toInstance(factory); + binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMapper); + binder.bind(ObjectMapper.class).annotatedWith(Smile.class).toInstance(smileMapper); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java new file mode 100644 index 00000000000..64dd107d0f0 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -0,0 +1,121 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Supplier; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.google.inject.TypeLiteral; +import com.metamx.common.IAE; +import com.metamx.common.ISE; + +import javax.validation.ConstraintViolation; +import javax.validation.Validator; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** + */ +public class JsonConfigProvider implements Provider> +{ + + private static final Joiner JOINER = Joiner.on(", "); + + public static void bind(Binder binder, String propertyBase, Class classToProvide) + { + binder.bind(new TypeLiteral>(){}).toProvider(of(propertyBase, classToProvide)).in(DruidScopes.SINGLETON); + } + + public static JsonConfigProvider of(String propertyBase, Class classToProvide) + { + return new JsonConfigProvider(propertyBase, classToProvide); + } + + private final String propertyBase; + private final Class classToProvide; + + private Supplier supplier; + + public JsonConfigProvider( + String propertyBase, + Class classToProvide + ) + { + this.propertyBase = propertyBase; + this.classToProvide = classToProvide; + } + + @Inject + public void inject( + Properties props, + ObjectMapper jsonMapper, + Validator validator + ) + { + Map jsonMap = Maps.newHashMap(); + for (String prop : props.stringPropertyNames()) { + if (prop.startsWith(propertyBase)) { + final String propValue = props.getProperty(prop); + try { + jsonMap.put(prop.substring(propertyBase.length()), jsonMapper.readValue(propValue, Object.class)); + } + catch (IOException e) { + throw new IAE("Unable to parse an object out of prop[%s]=[%s]", prop, propValue); + } + } + } + + final T config = jsonMapper.convertValue(jsonMap, classToProvide); + + final Set> violations = validator.validate(config); + if (!violations.isEmpty()) { + List messages = Lists.newArrayList(); + + for (ConstraintViolation violation : violations) { + messages.add(String.format("%s - %s", violation.getPropertyPath().toString(), violation.getMessage())); + } + + throw new ISE("Configuration violations[%s]", JOINER.join(messages)); + } + + this.supplier = new Supplier() + { + @Override + public T get() + { + return config; + } + }; + } + + @Override + public Supplier get() + { + return supplier; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/LazySingleton.java b/common/src/main/java/com/metamx/druid/guice/LazySingleton.java new file mode 100644 index 00000000000..005e9795de4 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LazySingleton.java @@ -0,0 +1,18 @@ +package com.metamx.druid.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface LazySingleton +{ +} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java new file mode 100644 index 00000000000..b31bbaddd11 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -0,0 +1,74 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; + +/** + * A Module to add lifecycle management to the injector. {@link DruidGuiceExtensions} must also be included. + */ +public class LifecycleModule implements Module +{ + private final LifecycleScope scope = new LifecycleScope(); + private final Key[] eagerClasses; + + /** + * A constructor that takes a list of classes to instantiate eagerly. Class {@link Key}s mentioned here will + * be pulled out of the injector with an injector.getInstance() call when the lifecycle is created. + * + * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper + * scope. + * + * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to + * auto-register things with the {@link Lifecycle} + * + * @param eagerClasses set of classes to instantiate eagerly + */ + public LifecycleModule( + Key... eagerClasses + ) + { + this.eagerClasses = eagerClasses; + } + + @Override + public void configure(Binder binder) + { + binder.bindScope(ManageLifecycle.class, scope); + } + + @Provides @LazySingleton + public Lifecycle getLifecycle(Injector injector) + { + Lifecycle lifecycle = new Lifecycle(); + scope.setLifecycle(lifecycle); + + for (Key key : eagerClasses) { + injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. + } + + return lifecycle; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java new file mode 100644 index 00000000000..223113728e3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java @@ -0,0 +1,67 @@ +package com.metamx.druid.guice; + +import com.google.common.collect.Lists; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.Scope; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; + +import java.util.List; + +/** + * A scope that adds objects to the Lifecycle. This is by definition also a lazy singleton scope. + */ +public class LifecycleScope implements Scope +{ + private static final Logger log = new Logger(LifecycleScope.class); + + private Lifecycle lifecycle; + private List instances = Lists.newLinkedList(); + + public void setLifecycle(Lifecycle lifecycle) + { + this.lifecycle = lifecycle; + synchronized (instances) { + for (Object instance : instances) { + lifecycle.addManagedInstance(instance); + } + } + } + + @Override + public Provider scope(final Key key, final Provider unscoped) + { + return new Provider() + { + private T value = null; + + @Override + public synchronized T get() + { + if (value == null) { + final T retVal = unscoped.get(); + + synchronized (instances) { + if (lifecycle == null) { + instances.add(retVal); + } + else { + try { + lifecycle.addMaybeStartManagedInstance(retVal); + } + catch (Exception e) { + log.warn(e, "Caught exception when trying to create a[%s]", key); + return null; + } + } + } + + value = retVal; + } + + return value; + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java new file mode 100644 index 00000000000..35752a64b0d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java @@ -0,0 +1,19 @@ +package com.metamx.druid.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface ManageLifecycle +{ +} diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java new file mode 100644 index 00000000000..aca59fd8bc3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -0,0 +1,53 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.guice.LazySingleton; + +/** + */ +public class JacksonModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); + } + + @Provides @LazySingleton @Json + public ObjectMapper jsonMapper() + { + return new DefaultObjectMapper(); + } + + @Provides @LazySingleton @Smile + public ObjectMapper smileMapper() + { + ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); + retVal.getJsonFactory().setCodec(retVal); + return retVal; + } +} diff --git a/common/src/main/java/com/metamx/druid/jackson/Json.java b/common/src/main/java/com/metamx/druid/jackson/Json.java new file mode 100644 index 00000000000..b0cc6401613 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/Json.java @@ -0,0 +1,17 @@ +package com.metamx.druid.jackson; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Json +{ +} diff --git a/common/src/main/java/com/metamx/druid/jackson/Smile.java b/common/src/main/java/com/metamx/druid/jackson/Smile.java new file mode 100644 index 00000000000..cc1c5b376e5 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/Smile.java @@ -0,0 +1,17 @@ +package com.metamx.druid.jackson; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Smile +{ +} diff --git a/common/src/main/java/com/metamx/druid/utils/SocketUtil.java b/common/src/main/java/com/metamx/druid/utils/SocketUtil.java new file mode 100644 index 00000000000..efea5f3003f --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/SocketUtil.java @@ -0,0 +1,39 @@ +package com.metamx.druid.utils; + +import com.metamx.common.ISE; + +import java.io.IOException; +import java.net.ServerSocket; + +/** + */ +public class SocketUtil +{ + public static int findOpenPort(int startPort) + { + int currPort = startPort; + + while (currPort < 0xffff) { + ServerSocket socket = null; + try { + socket = new ServerSocket(currPort); + return currPort; + } + catch (IOException e) { + ++currPort; + } + finally { + if (socket != null) { + try { + socket.close(); + } + catch (IOException e) { + + } + } + } + } + + throw new ISE("Unable to find open port between[%d] and [%d]", startPort, currPort); + } +} diff --git a/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java b/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java new file mode 100644 index 00000000000..9948dfe1470 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java @@ -0,0 +1,231 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import junit.framework.Assert; +import org.junit.Test; + +/** + */ +public class LifecycleScopeTest +{ + @Test + public void testAnnotation() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(AnnotatedClass.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + testIt(injector, lifecycle, instance); + } + + @Test + public void testExplicit() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(ExplicitClass.class).in(ManageLifecycle.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + testIt(injector, lifecycle, instance); + } + + private void testIt(Injector injector, Lifecycle lifecycle, TestInterface instance) + throws Exception + { + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(0, instance.getRan()); + + instance.run(); + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + lifecycle.start(); + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + injector.getInstance(TestInterface.class).run(); // It's a singleton + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + + lifecycle.stop(); + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(1, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + } + + /** + * This is a test for documentation purposes. It's there to show what weird things Guice will do when + * it sees both the annotation and an explicit binding. + * + * @throws Exception + */ + @Test + public void testAnnotatedAndExplicit() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(AnnotatedClass.class).in(ManageLifecycle.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(0, instance.getRan()); + + instance.run(); + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + lifecycle.start(); + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + injector.getInstance(TestInterface.class).run(); // It's a singleton + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + + lifecycle.stop(); + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(2, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + } + + private static interface TestInterface + { + public void run(); + public int getStarted(); + public int getStopped(); + public int getRan(); + } + + @ManageLifecycle + public static class AnnotatedClass implements TestInterface + { + int started = 0; + int stopped = 0; + int ran = 0; + + @LifecycleStart + public void start() + { + ++started; + } + + @LifecycleStop + public void stop() + { + ++stopped; + } + + @Override + public void run() + { + ++ran; + } + + public int getStarted() + { + return started; + } + + public int getStopped() + { + return stopped; + } + + public int getRan() + { + return ran; + } + } + + public static class ExplicitClass implements TestInterface + { + int started = 0; + int stopped = 0; + int ran = 0; + + @LifecycleStart + public void start() + { + ++started; + } + + @LifecycleStop + public void stop() + { + ++stopped; + } + + @Override + public void run() + { + ++ran; + } + + public int getStarted() + { + return started; + } + + public int getStopped() + { + return stopped; + } + + public int getRan() + { + return ran; + } + } + +} 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 c00df6bf252..63af3b31993 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 @@ -25,8 +25,8 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.merger.common.actions.TaskActionClient; import com.metamx.druid.merger.common.actions.TaskActionClientFactory; import com.metamx.druid.merger.common.task.Task; @@ -65,7 +65,7 @@ public class TaskMasterLifecycle final TaskQueue taskQueue, final TaskActionClientFactory taskActionClientFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, - final ServiceDiscoveryConfig serviceDiscoveryConfig, + final DruidNodeConfig nodeConfig, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, @@ -103,7 +103,7 @@ public class TaskMasterLifecycle final Lifecycle leaderLifecycle = new Lifecycle(); leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(taskRunner); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); leaderLifecycle.addManagedInstance(resourceManagementScheduler); 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 8a2923431ec..1e613cb6675 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 @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Charsets; import com.google.common.base.Optional; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -55,9 +56,10 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.common.RetryPolicyFactory; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; @@ -156,7 +158,7 @@ public class IndexerCoordinatorNode extends QueryableNode serviceDiscovery = null; private ServiceAnnouncer serviceAnnouncer = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; @@ -363,12 +365,12 @@ public class IndexerCoordinatorNode extends QueryableNode instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); } } @@ -614,7 +617,7 @@ public class IndexerCoordinatorNode extends QueryableNodeofInstance(dbConnectorConfig), null).getDBI() // TODO ); } else { throw new ISE("Invalid storage implementation: %s", config.getStorageImpl()); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java index 66ee23eb0df..f53de4c0b93 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java @@ -43,10 +43,11 @@ import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; 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; @@ -56,8 +57,8 @@ import com.metamx.druid.merger.common.TaskToolboxFactory; import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; import com.metamx.druid.merger.common.config.RetryPolicyConfig; import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.merger.common.index.ChatHandlerProvider; +import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.coordinator.ExecutorServiceTaskRunner; import com.metamx.druid.merger.worker.config.ChatHandlerProviderConfig; @@ -114,7 +115,7 @@ public class ExecutorNode extends BaseServerNode private WorkerConfig workerConfig = null; private DataSegmentPusher segmentPusher = null; private TaskToolboxFactory taskToolboxFactory = null; - private ServiceDiscovery serviceDiscovery = null; + private ServiceDiscovery serviceDiscovery = null; private ServiceAnnouncer serviceAnnouncer = null; private ServiceProvider coordinatorServiceProvider = null; private Server server = null; @@ -391,14 +392,15 @@ public class ExecutorNode extends BaseServerNode public void initializeServiceDiscovery() throws Exception { - final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig config = configFactory.build(CuratorDiscoveryConfig.class); if (serviceDiscovery == null) { this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( getCuratorFramework(), config, lifecycle ); } if (serviceAnnouncer == null) { - final ServiceInstanceFactory instanceFactory = Initialization.makeServiceInstanceFactory(config); + DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); + final ServiceInstanceFactory instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); } if (coordinatorServiceProvider == null) { 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 9ef0a900de6..a6da4b30da7 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 @@ -34,14 +34,12 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.QueryableNode; -import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.TaskLogConfig; @@ -332,7 +330,7 @@ public class WorkerNode extends QueryableNode public void initializeServiceDiscovery() throws Exception { if (serviceDiscovery == null) { - final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig config = getConfigFactory().build(CuratorDiscoveryConfig.class); this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( getCuratorFramework(), config, diff --git a/pom.xml b/pom.xml index d4ffa9c7ce3..3cb7eb0ffbc 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.22.3 + 0.23.0-SNAPSHOT 2.0.1-21-22 @@ -219,6 +219,11 @@ jackson-mapper-asl 1.9.11 + + org.hibernate + hibernate-validator + 5.0.1.Final + javax.inject javax.inject diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 708c61d9e24..4d51df1a8d5 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -21,17 +21,20 @@ package com.metamx.druid.db; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.MapUtils; 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.concurrent.Execs; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; - import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; @@ -52,12 +55,13 @@ import java.util.concurrent.atomic.AtomicReference; /** */ +@ManageLifecycle public class DatabaseRuleManager { public static void createDefaultRule( final DBI dbi, final String ruleTable, - final String defaultDatasource, + final String defaultTier, final ObjectMapper jsonMapper ) { @@ -72,7 +76,7 @@ public class DatabaseRuleManager String.format( "SELECT id from %s where datasource='%s';", ruleTable, - defaultDatasource + defaultTier ) ); @@ -94,8 +98,8 @@ public class DatabaseRuleManager ruleTable ) ) - .bind("id", String.format("%s_%s", defaultDatasource, version)) - .bind("dataSource", defaultDatasource) + .bind("id", String.format("%s_%s", defaultTier, version)) + .bind("dataSource", defaultTier) .bind("version", version) .bind("payload", jsonMapper.writeValueAsString(defaultRules)) .execute(); @@ -115,6 +119,7 @@ public class DatabaseRuleManager private final ObjectMapper jsonMapper; private final ScheduledExecutorService exec; private final DatabaseRuleManagerConfig config; + private final Supplier dbTables; private final DBI dbi; private final AtomicReference>> rules; @@ -122,18 +127,21 @@ public class DatabaseRuleManager private volatile boolean started = false; + @Inject public DatabaseRuleManager( ObjectMapper jsonMapper, - ScheduledExecutorService exec, DatabaseRuleManagerConfig config, + Supplier dbTables, DBI dbi ) { this.jsonMapper = jsonMapper; - this.exec = exec; this.config = config; + this.dbTables = dbTables; this.dbi = dbi; + this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"); + this.rules = new AtomicReference>>( new ConcurrentHashMap>() ); @@ -147,6 +155,7 @@ public class DatabaseRuleManager return; } + createDefaultRule(dbi, getRulesTable(), config.getDefaultTier(), jsonMapper); ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), @@ -192,8 +201,11 @@ public class DatabaseRuleManager return handle.createQuery( // Return latest version rule by dataSource String.format( - "SELECT %1$s.dataSource, %1$s.payload FROM %1$s INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds ON %1$s.datasource = ds.datasource and %1$s.version = ds.version", - config.getRuleTable() + "SELECT r.dataSource, r.payload " + + "FROM %1$s r " + + "INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds " + + "ON r.datasource = ds.datasource and r.version = ds.version", + getRulesTable() ) ).fold( Maps.>newHashMap(), @@ -255,8 +267,8 @@ public class DatabaseRuleManager if (theRules.get(dataSource) != null) { retVal.addAll(theRules.get(dataSource)); } - if (theRules.get(config.getDefaultDatasource()) != null) { - retVal.addAll(theRules.get(config.getDefaultDatasource())); + if (theRules.get(config.getDefaultTier()) != null) { + retVal.addAll(theRules.get(config.getDefaultTier())); } return retVal; } @@ -275,7 +287,7 @@ public class DatabaseRuleManager handle.createStatement( String.format( "INSERT INTO %s (id, dataSource, version, payload) VALUES (:id, :dataSource, :version, :payload)", - config.getRuleTable() + getRulesTable() ) ) .bind("id", String.format("%s_%s", dataSource, version)) @@ -303,4 +315,6 @@ public class DatabaseRuleManager return true; } + + private String getRulesTable() {return dbTables.get().getRulesTable();} } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java index 41e21a0943b..5f41945df58 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java @@ -27,12 +27,9 @@ import org.skife.config.Default; */ public abstract class DatabaseRuleManagerConfig { - @Config("druid.database.ruleTable") - public abstract String getRuleTable(); - - @Config("druid.database.defaultDatasource") + @Config("druid.database.rules.defaultTier") @Default("_default") - public abstract String getDefaultDatasource(); + public abstract String getDefaultTier(); @Config("druid.database.rules.poll.duration") @Default("PT1M") diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index 2ea9056627f..fce933e6294 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -21,9 +21,11 @@ package com.metamx.druid.db; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; @@ -33,7 +35,8 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; - +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.ManageLifecycle; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -57,6 +60,7 @@ import java.util.concurrent.atomic.AtomicReference; /** */ +@ManageLifecycle public class DatabaseSegmentManager { private static final Logger log = new Logger(DatabaseSegmentManager.class); @@ -65,26 +69,30 @@ public class DatabaseSegmentManager private final ObjectMapper jsonMapper; private final ScheduledExecutorService exec; - private final DatabaseSegmentManagerConfig config; + private final Supplier config; + private final Supplier dbTables; private final AtomicReference> dataSources; private final DBI dbi; private volatile boolean started = false; + @Inject public DatabaseSegmentManager( ObjectMapper jsonMapper, - ScheduledExecutorService exec, - DatabaseSegmentManagerConfig config, + Supplier config, + Supplier dbTables, DBI dbi ) { this.jsonMapper = jsonMapper; - this.exec = exec; this.config = config; + this.dbTables = dbTables; this.dataSources = new AtomicReference>( new ConcurrentHashMap() ); this.dbi = dbi; + + this.exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); } @LifecycleStart @@ -98,7 +106,7 @@ public class DatabaseSegmentManager ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), - config.getPollDuration(), + config.get().getPollDuration(), new Runnable() { @Override @@ -136,7 +144,7 @@ public class DatabaseSegmentManager public VersionedIntervalTimeline withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", config.getSegmentTable()) + String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", getSegmentsTable()) ) .bind("dataSource", ds) .fold( @@ -204,7 +212,7 @@ public class DatabaseSegmentManager batch.add( String.format( "UPDATE %s SET used=1 WHERE id = '%s'", - config.getSegmentTable(), + getSegmentsTable(), segment.getIdentifier() ) ); @@ -234,7 +242,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=1 WHERE id = :id", config.getSegmentTable()) + String.format("UPDATE %s SET used=1 WHERE id = :id", getSegmentsTable()) ) .bind("id", segmentId) .execute(); @@ -268,7 +276,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE dataSource = :dataSource", config.getSegmentTable()) + String.format("UPDATE %s SET used=0 WHERE dataSource = :dataSource", getSegmentsTable()) ) .bind("dataSource", ds) .execute(); @@ -298,7 +306,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE id = :segmentID", config.getSegmentTable()) + String.format("UPDATE %s SET used=0 WHERE id = :segmentID", getSegmentsTable()) ).bind("segmentID", segmentID) .execute(); @@ -354,7 +362,7 @@ public class DatabaseSegmentManager public List withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT DISTINCT(datasource) FROM %s", config.getSegmentTable()) + String.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable()) ) .fold( Lists.newArrayList(), @@ -398,7 +406,7 @@ public class DatabaseSegmentManager public List> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT payload FROM %s WHERE used=1", config.getSegmentTable()) + String.format("SELECT payload FROM %s WHERE used=1", getSegmentsTable()) ).list(); } } @@ -451,4 +459,8 @@ public class DatabaseSegmentManager log.error(e, "Problem polling DB."); } } + + private String getSegmentsTable() { + return dbTables.get().getSegmentsTable(); + } } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java index 25e90cd16bd..85532089243 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java @@ -19,18 +19,18 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; /** */ -public abstract class DatabaseSegmentManagerConfig +public class DatabaseSegmentManagerConfig { - @Config("druid.database.segmentTable") - public abstract String getSegmentTable(); + @JsonProperty + private Duration pollDuration = new Duration("PT1M"); - @Config("druid.database.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); + public Duration getPollDuration() + { + return pollDuration; + } } diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java new file mode 100644 index 00000000000..39e6fe10e8c --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -0,0 +1,54 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; +import com.metamx.druid.initialization.ZkPathsConfig; +import org.skife.jdbi.v2.DBI; + +/** + */ +public class MasterModule implements Module +{ + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, ZkPathsConfig.class); + ConfigProvider.bind(binder, ServerInventoryViewConfig.class); + ConfigProvider.bind(binder, DbConnectorConfig.class); + + JsonConfigProvider.bind(binder, "druid.database.tables", DbTablesConfig.class); + + } + + @Provides @LazySingleton + public DBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) + { + if (config.isCreateTables()) { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createSegmentTable(); + dbConnector.createRulesTable(); + } + + @Override + public void stop() + { + + } + } + ); + } + + return dbConnector.getDBI(); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java new file mode 100644 index 00000000000..8440df3042b --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.java @@ -0,0 +1,16 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.druid.initialization.DruidNodeConfig; + +/** + */ +public class ServerModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid", DruidNodeConfig.class); + } +} 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 ff5b5ae58d7..3a17b71c7a4 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -26,6 +26,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; @@ -38,7 +39,6 @@ import com.metamx.druid.master.rules.Rule; import org.joda.time.Interval; import javax.annotation.Nullable; -import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; 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 0eb9f53a360..87ef7dc64f1 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -21,57 +21,56 @@ package com.metamx.druid.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; +import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.common.collect.Iterables; import com.google.inject.Guice; import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.config.ConfigManager; import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.curator.discovery.DiscoveryModule; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.guice.DruidGuiceExtensions; +import com.metamx.druid.guice.DruidSecondaryModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.MasterModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.initialization.ConfigFactoryModule; +import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.PropertiesModule; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.initialization.ZkPathsConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.jackson.JacksonModule; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; +import com.metamx.druid.metrics.MetricsModule; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.ToStringResponseHandler; -import com.metamx.metrics.JvmMonitor; -import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; -import org.joda.time.Duration; import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; @@ -80,11 +79,9 @@ import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; import org.skife.jdbi.v2.DBI; +import javax.annotation.Nullable; import java.net.URL; -import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; +import java.util.Arrays; /** */ @@ -96,89 +93,41 @@ public class MasterMain { LogLevelAdjuster.register(); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); - final Properties props = Initialization.loadProperties(); - final ConfigurationObjectFactory configFactory = Config.createFactory(props); - final Lifecycle lifecycle = new Lifecycle(); - - final HttpClientConfig.Builder httpClientConfigBuilder = HttpClientConfig.builder().withNumConnections(1); - - final String emitterTimeout = props.getProperty("druid.emitter.timeOut"); - if (emitterTimeout != null) { - httpClientConfigBuilder.withReadTimeout(new Duration(emitterTimeout)); - } - final HttpClient httpClient = HttpClientInit.createClient(httpClientConfigBuilder.build(), lifecycle); - - final ServiceEmitter emitter = new ServiceEmitter( - PropUtils.getProperty(props, "druid.service"), - PropUtils.getProperty(props, "druid.host"), - Emitters.create(props, httpClient, jsonMapper, lifecycle) + Injector injector = makeInjector( + DruidSecondaryModule.class, + new LifecycleModule(Key.get(MonitorScheduler.class)), + EmitterModule.class, + CuratorModule.class, + MetricsModule.class, + DiscoveryModule.class, + ServerModule.class, + MasterModule.class ); - EmittingLogger.registerEmitter(emitter); + + final ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); + final ConfigurationObjectFactory configFactory = injector.getInstance(ConfigurationObjectFactory.class); + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final ServiceEmitter emitter = injector.getInstance(ServiceEmitter.class); final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); - final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); - CuratorFramework curatorFramework = Initialization.makeCuratorFramework( - serviceDiscoveryConfig, - lifecycle - ); + CuratorFramework curatorFramework = injector.getInstance(CuratorFramework.class); final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class); - final ExecutorService exec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() - ); - ServerInventoryView serverInventoryView = new ServerInventoryView( - configFactory.build(ServerInventoryViewConfig.class), zkPaths, curatorFramework, exec, jsonMapper - ); - lifecycle.addManagedInstance(serverInventoryView); + ServerInventoryView serverInventoryView = injector.getInstance(ServerInventoryView.class); - final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); - final DatabaseRuleManagerConfig databaseRuleManagerConfig = configFactory.build(DatabaseRuleManagerConfig.class); - final DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); - DbConnector.createSegmentTable(dbi, PropUtils.getProperty(props, "druid.database.segmentTable")); - DbConnector.createRuleTable(dbi, PropUtils.getProperty(props, "druid.database.ruleTable")); - DatabaseRuleManager.createDefaultRule( - dbi, databaseRuleManagerConfig.getRuleTable(), databaseRuleManagerConfig.getDefaultDatasource(), jsonMapper - ); - final DatabaseSegmentManager databaseSegmentManager = new DatabaseSegmentManager( - jsonMapper, - scheduledExecutorFactory.create(1, "DatabaseSegmentManager-Exec--%d"), - configFactory.build(DatabaseSegmentManagerConfig.class), - dbi - ); - final DatabaseRuleManager databaseRuleManager = new DatabaseRuleManager( - jsonMapper, - scheduledExecutorFactory.create(1, "DatabaseRuleManager-Exec--%d"), - databaseRuleManagerConfig, - dbi - ); - - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - final MonitorScheduler healthMonitor = new MonitorScheduler( - configFactory.build(MonitorSchedulerConfig.class), - globalScheduledExec, - emitter, - ImmutableList.of( - new JvmMonitor(), - new SysMonitor() - ) - ); - lifecycle.addManagedInstance(healthMonitor); + final DatabaseSegmentManager databaseSegmentManager = injector.getInstance(DatabaseSegmentManager.class); + final DatabaseRuleManager databaseRuleManager = injector.getInstance(DatabaseRuleManager.class); final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class); + final DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); - final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, - serviceDiscoveryConfig, - lifecycle - ); - final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer( - serviceDiscoveryConfig, serviceDiscovery - ); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, lifecycle); + final ServiceDiscovery serviceDiscovery = injector.getInstance(Key.get(new TypeLiteral>(){})); + final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); IndexingServiceClient indexingServiceClient = null; if (druidMasterConfig.getMergerServiceName() != null) { @@ -187,9 +136,10 @@ public class MasterMain serviceDiscovery, lifecycle ); - indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); +// indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); TODO } + DBI dbi = injector.getInstance(DBI.class); final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); JacksonConfigManager configManager = new JacksonConfigManager( @@ -237,7 +187,7 @@ public class MasterMain ) ); - final Injector injector = Guice.createInjector( + final Injector injector2 = Guice.createInjector( new MasterServletModule( serverInventoryView, databaseSegmentManager, @@ -289,7 +239,7 @@ public class MasterMain final Context root = new Context(server, "/", Context.SESSIONS); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addEventListener(new GuiceServletConfig(injector)); + root.addEventListener(new GuiceServletConfig(injector2)); root.addFilter( new FilterHolder( new RedirectFilter( @@ -304,4 +254,41 @@ public class MasterMain server.start(); server.join(); } + + private static Injector makeInjector(final Object... modules) + { + final Injector baseInjector = Guice.createInjector( + new DruidGuiceExtensions(), + new JacksonModule(), + new PropertiesModule("runtime.properties"), + new ConfigFactoryModule() + ); + + return Guice.createInjector( + Iterables.transform( + Arrays.asList(modules), + new Function() + { + @Override + @SuppressWarnings("unchecked") + public Module apply(@Nullable Object input) + { + if (input instanceof Module) { + baseInjector.injectMembers(input); + return (Module) input; + } + if (input instanceof Class) { + if (Module.class.isAssignableFrom((Class) input)) { + return baseInjector.getInstance((Class) input); + } + else { + throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); + } + } + throw new ISE("Unknown module type[%s]", input.getClass()); + } + } + ) + ); + } } 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 9bb59d79d43..5934fdd5f6e 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -19,10 +19,10 @@ package com.metamx.druid.http; +import com.google.inject.Inject; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; -import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.GET; import javax.ws.rs.POST; diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index f113051d506..089f137db1a 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -19,12 +19,12 @@ 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 com.metamx.druid.index.Segment; -import javax.inject.Inject; import java.util.Map; /** diff --git a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java new file mode 100644 index 00000000000..610514bcc39 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java @@ -0,0 +1,77 @@ +package com.metamx.druid.metrics; + +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.logger.Logger; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.JvmMonitor; +import com.metamx.metrics.Monitor; +import com.metamx.metrics.MonitorScheduler; +import com.metamx.metrics.MonitorSchedulerConfig; +import com.metamx.metrics.SysMonitor; + +import java.util.List; + +/** + * Sets up the {@link MonitorScheduler} to monitor things on a regular schedule. {@link Monitor}s must be explicitly + * bound in order to be loaded. + */ +public class MetricsModule implements Module +{ + private static final Logger log = new Logger(MetricsModule.class); + + private final Class[] monitors; + + /** + * A constructor that takes a list of {@link Monitor} classes to explicitly bind so that they will be instantiated + * + * @param monitors list of {@link Monitor} classes to explicitly bind + */ + public MetricsModule( + Class... monitors + ) + { + this.monitors = monitors; + } + + @Override + public void configure(Binder binder) + { + binder.bind(JvmMonitor.class).in(LazySingleton.class); + binder.bind(SysMonitor.class).in(LazySingleton.class); + + for (Class monitor : monitors) { + binder.bind(monitor).in(LazySingleton.class); + } + } + + @Provides @ManageLifecycle + public MonitorScheduler getMonitorScheduler(MonitorSchedulerConfig config, ServiceEmitter emitter, Injector injector) + { + List monitors = Lists.newArrayList(); + + for (Key key: injector.getBindings().keySet()) { + if (Monitor.class.isAssignableFrom(key.getClass())) { + final Monitor monitor = (Monitor) injector.getInstance(key); + + log.info("Adding monitor[%s]", monitor); + + monitors.add(monitor); + } + } + + return new MonitorScheduler( + config, + Execs.scheduledSingleThreaded("MonitorScheduler-%s"), + emitter, + monitors + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java index 7db7e3ba922..9898d85a76e 100644 --- a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java +++ b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java @@ -19,7 +19,6 @@ package com.metamx.druid.metrics; -import com.metamx.druid.client.DruidServer; import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.coordination.ServerManager; import com.metamx.emitter.service.ServiceEmitter; diff --git a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java index 751cad29a13..223e55faa9d 100644 --- a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java +++ b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java @@ -19,10 +19,10 @@ package com.metamx.druid.db; +import com.google.common.base.Suppliers; import com.google.common.collect.Maps; import com.metamx.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; -import org.joda.time.Duration; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -32,7 +32,6 @@ import org.skife.jdbi.v2.tweak.HandleCallback; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** */ @@ -48,21 +47,8 @@ public class DatabaseSegmentManagerTest dbi = EasyMock.createMock(DBI.class); manager = new DatabaseSegmentManager( new DefaultObjectMapper(), - EasyMock.createMock(ScheduledExecutorService.class), - new DatabaseSegmentManagerConfig() - { - @Override - public String getSegmentTable() - { - return null; - } - - @Override - public Duration getPollDuration() - { - return null; - } - }, + Suppliers.ofInstance(new DatabaseSegmentManagerConfig()), + Suppliers.ofInstance(DbTablesConfig.fromBase("test")), dbi ); From 9df458a0651774f52163f3054732ef591e59f9e0 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 14 May 2013 11:25:57 -0500 Subject: [PATCH 02/92] 1) Initial commit of conversion to using Guice modules for bootstrapping. Things don't actually completely work yet. --- .../java/com/metamx/druid/QueryableNode.java | 9 +- .../druid/client/ServerInventoryView.java | 9 +- .../CuratorConfig.java | 2 +- .../metamx/druid/curator/CuratorModule.java | 29 +++ .../discovery/CuratorServiceAnnouncer.java | 18 +- .../curator/discovery/DiscoveryModule.java | 43 ++++ .../com/metamx/druid/http/BrokerNode.java | 18 +- .../initialization/ConfigFactoryModule.java | 46 ++++ ...onfig.java => CuratorDiscoveryConfig.java} | 23 +- .../druid/initialization/DruidNodeConfig.java | 102 ++++++++ .../druid/initialization/EmitterModule.java | 128 ++++++++++ .../initialization/HttpEmitterConfig.java | 36 +++ .../initialization/HttpEmitterModule.java | 62 +++++ .../druid/initialization/Initialization.java | 22 +- .../initialization/LogEmitterModule.java | 48 ++++ .../initialization/PropertiesModule.java | 135 ++++++++++ common/pom.xml | 12 + .../com/metamx/druid/concurrent/Execs.java | 16 +- .../java/com/metamx/druid/db/DbConnector.java | 31 ++- .../metamx/druid/db/DbConnectorConfig.java | 63 +++-- .../com/metamx/druid/db/DbTablesConfig.java | 69 ++++++ .../metamx/druid/guice/ConfigProvider.java | 45 ++++ .../druid/guice/DruidGuiceExtensions.java | 15 ++ .../com/metamx/druid/guice/DruidScopes.java | 71 ++++++ .../druid/guice/DruidSecondaryModule.java | 45 ++++ .../druid/guice/JsonConfigProvider.java | 121 +++++++++ .../com/metamx/druid/guice/LazySingleton.java | 18 ++ .../metamx/druid/guice/LifecycleModule.java | 74 ++++++ .../metamx/druid/guice/LifecycleScope.java | 67 +++++ .../metamx/druid/guice/ManageLifecycle.java | 19 ++ .../metamx/druid/jackson/JacksonModule.java | 53 ++++ .../java/com/metamx/druid/jackson/Json.java | 17 ++ .../java/com/metamx/druid/jackson/Smile.java | 17 ++ .../com/metamx/druid/utils/SocketUtil.java | 39 +++ .../druid/guice/LifecycleScopeTest.java | 231 ++++++++++++++++++ .../coordinator/TaskMasterLifecycle.java | 20 +- .../http/IndexerCoordinatorNode.java | 19 +- .../worker/executor/ExecutorNode.java | 32 +-- .../indexing/worker/http/WorkerNode.java | 4 +- pom.xml | 17 +- .../metamx/druid/db/DatabaseRuleManager.java | 38 ++- .../druid/db/DatabaseRuleManagerConfig.java | 7 +- .../druid/db/DatabaseSegmentManager.java | 38 ++- .../db/DatabaseSegmentManagerConfig.java | 16 +- .../com/metamx/druid/guice/MasterModule.java | 54 ++++ .../com/metamx/druid/guice/ServerModule.java | 16 ++ .../com/metamx/druid/http/InfoResource.java | 2 +- .../com/metamx/druid/http/MasterMain.java | 184 +++++++------- .../com/metamx/druid/http/MasterResource.java | 2 +- .../loading/DelegatingSegmentLoader.java | 2 +- .../metamx/druid/metrics/MetricsModule.java | 77 ++++++ .../metamx/druid/metrics/ServerMonitor.java | 1 - .../druid/db/DatabaseSegmentManagerTest.java | 20 +- 53 files changed, 2028 insertions(+), 274 deletions(-) rename client/src/main/java/com/metamx/druid/{initialization => curator}/CuratorConfig.java (96%) create mode 100644 client/src/main/java/com/metamx/druid/curator/CuratorModule.java create mode 100644 client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java rename client/src/main/java/com/metamx/druid/initialization/{ServiceDiscoveryConfig.java => CuratorDiscoveryConfig.java} (69%) create mode 100644 client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/EmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java create mode 100644 common/src/main/java/com/metamx/druid/db/DbTablesConfig.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ConfigProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidScopes.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LazySingleton.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LifecycleModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/LifecycleScope.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/JacksonModule.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/Json.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/Smile.java create mode 100644 common/src/main/java/com/metamx/druid/utils/SocketUtil.java create mode 100644 common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java create mode 100644 server/src/main/java/com/metamx/druid/guice/MasterModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/ServerModule.java create mode 100644 server/src/main/java/com/metamx/druid/metrics/MetricsModule.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 68f978929d6..ef8a995295b 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -42,9 +41,9 @@ import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.http.RequestLogger; -import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkPathsConfig; @@ -70,8 +69,6 @@ import java.lang.reflect.Field; import java.util.Arrays; import java.util.List; import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -349,14 +346,10 @@ public abstract class QueryableNode extends Registering private void initializeServerInventoryView() { if (serverInventoryView == null) { - final ExecutorService exec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() - ); serverInventoryView = new ServerInventoryView( getConfigFactory().build(ServerInventoryViewConfig.class), getZkPaths(), getCuratorFramework(), - exec, getJsonMapper() ); lifecycle.addManagedInstance(serverInventoryView); diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java index c98091e0560..dc4d0f79460 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java @@ -24,11 +24,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.MapMaker; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.concurrent.Execs; import com.metamx.druid.curator.inventory.CuratorInventoryManager; import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy; import com.metamx.druid.curator.inventory.InventoryManagerConfig; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; @@ -38,11 +41,11 @@ import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; /** */ +@ManageLifecycle public class ServerInventoryView implements ServerView, InventoryView { private static final EmittingLogger log = new EmittingLogger(ServerInventoryView.class); @@ -55,11 +58,11 @@ public class ServerInventoryView implements ServerView, InventoryView private static final Map removedSegments = new MapMaker().makeMap(); + @Inject public ServerInventoryView( final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, - final ExecutorService exec, final ObjectMapper jsonMapper ) { @@ -79,7 +82,7 @@ public class ServerInventoryView implements ServerView, InventoryView return zkPaths.getServedSegmentsPath(); } }, - exec, + Execs.singleThreaded("ServerInventoryView-%s"), new CuratorInventoryManagerStrategy() { @Override diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java b/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java similarity index 96% rename from client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java rename to client/src/main/java/com/metamx/druid/curator/CuratorConfig.java index d09c36f369e..10e7e381278 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java +++ b/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package com.metamx.druid.curator; import org.skife.config.Config; import org.skife.config.Default; diff --git a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java new file mode 100644 index 00000000000..94fcee12b42 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java @@ -0,0 +1,29 @@ +package com.metamx.druid.curator; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.initialization.Initialization; +import org.apache.curator.framework.CuratorFramework; + +import java.io.IOException; + +/** + */ +public class CuratorModule implements Module +{ + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, CuratorConfig.class); + } + + @Provides @LazySingleton + public CuratorFramework makeCurator(CuratorConfig config, Lifecycle lifecycle) throws IOException + { + return Initialization.makeCuratorFramework(config, lifecycle); + } +} diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java index a3156b3205d..035924a6e2b 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java @@ -1,6 +1,7 @@ package com.metamx.druid.curator.discovery; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; @@ -10,18 +11,19 @@ import java.util.Map; /** * Uses the Curator Service Discovery recipe to announce services. */ -public class CuratorServiceAnnouncer implements ServiceAnnouncer +public class CuratorServiceAnnouncer implements ServiceAnnouncer { private static final Logger log = new Logger(CuratorServiceAnnouncer.class); - private final ServiceDiscovery discovery; - private final ServiceInstanceFactory instanceFactory; - private final Map> instanceMap = Maps.newHashMap(); + private final ServiceDiscovery discovery; + private final ServiceInstanceFactory instanceFactory; + private final Map> instanceMap = Maps.newHashMap(); private final Object monitor = new Object(); + @Inject public CuratorServiceAnnouncer( - ServiceDiscovery discovery, - ServiceInstanceFactory instanceFactory + ServiceDiscovery discovery, + ServiceInstanceFactory instanceFactory ) { this.discovery = discovery; @@ -31,7 +33,7 @@ public class CuratorServiceAnnouncer implements ServiceAnnouncer @Override public void announce(String service) throws Exception { - final ServiceInstance instance; + final ServiceInstance instance; synchronized (monitor) { if (instanceMap.containsKey(service)) { @@ -57,7 +59,7 @@ public class CuratorServiceAnnouncer implements ServiceAnnouncer @Override public void unannounce(String service) throws Exception { - final ServiceInstance instance; + final ServiceInstance instance; synchronized (monitor) { instance = instanceMap.get(service); diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java new file mode 100644 index 00000000000..c5a2c701bf8 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java @@ -0,0 +1,43 @@ +package com.metamx.druid.curator.discovery; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.Initialization; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.x.discovery.ServiceDiscovery; + +/** + */ +public class DiscoveryModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.discovery.curator", CuratorDiscoveryConfig.class); + binder.bind(ServiceAnnouncer.class).to(CuratorServiceAnnouncer.class).in(LazySingleton.class); + } + + @Provides @LazySingleton + public ServiceDiscovery getServiceDiscovery( + CuratorFramework curator, + CuratorDiscoveryConfig config, + Lifecycle lifecycle + ) throws Exception + { + return Initialization.makeServiceDiscoveryClient(curator, config, lifecycle); + } + + @Provides @LazySingleton + public ServiceInstanceFactory getServiceInstanceFactory( + DruidNodeConfig nodeConfig + ) + { + return Initialization.makeServiceInstanceFactory(nodeConfig); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 75b69fe5d94..6fd5b2e8004 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -42,10 +42,11 @@ import com.metamx.druid.client.cache.MapCache; import com.metamx.druid.client.cache.MapCacheConfig; import com.metamx.druid.client.cache.MemcachedCache; import com.metamx.druid.client.cache.MemcachedCacheConfig; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.ReflectionQueryToolChestWarehouse; @@ -229,17 +230,18 @@ public class BrokerNode extends QueryableNode { if (useDiscovery) { final Lifecycle lifecycle = getLifecycle(); - final ServiceDiscoveryConfig serviceDiscoveryConfig = getConfigFactory().build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig curatorDiscoveryConfig = getConfigFactory().build(CuratorDiscoveryConfig.class); + final DruidNodeConfig nodeConfig = getConfigFactory().build(DruidNodeConfig.class); final CuratorFramework curatorFramework = Initialization.makeCuratorFramework( - serviceDiscoveryConfig, lifecycle + getConfigFactory().build(CuratorConfig.class), lifecycle ); - final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, serviceDiscoveryConfig, lifecycle + final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, curatorDiscoveryConfig, lifecycle ); final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer( - serviceDiscoveryConfig, serviceDiscovery + nodeConfig, serviceDiscovery ); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, lifecycle); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); } } diff --git a/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java b/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java new file mode 100644 index 00000000000..6d136c53da4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.config.Config; +import com.metamx.druid.guice.LazySingleton; +import org.skife.config.ConfigurationObjectFactory; + +import java.util.Properties; + +/** + */ +public class ConfigFactoryModule implements Module +{ + @Override + public void configure(Binder binder) + { + + } + + @Provides @LazySingleton + public ConfigurationObjectFactory makeFactory(Properties props) + { + return Config.createFactory(props); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java similarity index 69% rename from client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java rename to client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java index 04776d6545a..ee0f7ced76e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java @@ -19,21 +19,22 @@ package com.metamx.druid.initialization; -import org.skife.config.Config; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public abstract class ServiceDiscoveryConfig extends CuratorConfig +public abstract class CuratorDiscoveryConfig { - @Config("druid.service") - public abstract String getServiceName(); + @JsonProperty + private String path = null; - @Config("druid.host") - public abstract String getHost(); + public String getPath() + { + return path; + } - @Config("druid.port") - public abstract int getPort(); - - @Config("druid.zk.paths.discoveryPath") - public abstract String getDiscoveryPath(); + public boolean useDiscovery() + { + return path != null; + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java new file mode 100644 index 00000000000..94f02c5edd5 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java @@ -0,0 +1,102 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.utils.SocketUtil; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +/** + */ +public abstract class DruidNodeConfig +{ + @NotNull + private String serviceName = null; + + @NotNull + private String host = null; + + @Min(0) @Max(0xffff) + private int port = -1; + + @JsonCreator + public DruidNodeConfig( + @JsonProperty("service") String serviceName, + @JsonProperty("host") String host, + @JsonProperty("port") Integer port + ) + { + this.serviceName = serviceName; + + if (port == null) { + if (host == null) { + setHostAndPort(null, -1); + } + else if (host.contains(":")) { + try { + setHostAndPort(host, Integer.parseInt(host.split(":")[1])); + } + catch (Exception e) { + setHostAndPort(host, -1); + } + } + else { + final int openPort = SocketUtil.findOpenPort(8080); + setHostAndPort(String.format("%s:%d", host, openPort), openPort); + } + } + else { + if (host == null || host.contains(":")) { + setHostAndPort(host, port); + } + else { + setHostAndPort(String.format("%s:%d", host, port), port); + } + } + } + + private void setHostAndPort(String host, int port) + { + this.host = host; + this.port = port; + } + + @JsonProperty("service") + public String getServiceName() + { + return serviceName; + } + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public int getPort() + { + return port; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java new file mode 100644 index 00000000000..edc01e65a26 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -0,0 +1,128 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Binding; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Provider; +import com.google.inject.Provides; +import com.google.inject.TypeLiteral; +import com.google.inject.name.Named; +import com.google.inject.name.Names; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.guice.DruidScopes; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.service.ServiceEmitter; + +import java.lang.annotation.Annotation; +import java.util.List; +import java.util.Properties; + +/** + */ +public class EmitterModule implements Module +{ + private static final Logger log = new Logger(EmitterModule.class); + + private final Properties props; + + @Inject + public EmitterModule( + Properties props + ) + { + this.props = props; + } + + @Override + public void configure(Binder binder) + { + String emitterType = props.getProperty("druid.emitter", ""); + + binder.install(new LogEmitterModule()); + binder.install(new HttpEmitterModule()); + + binder.bind(Emitter.class).toProvider(new EmitterProvider(emitterType)).in(DruidScopes.SINGLETON); + } + + @Provides + @LazySingleton + public ServiceEmitter getServiceEmitter(DruidNodeConfig config, Emitter emitter) + { + final ServiceEmitter retVal = new ServiceEmitter(config.getServiceName(), config.getHost(), emitter); + EmittingLogger.registerEmitter(retVal); + return retVal; + } + + private static class EmitterProvider implements Provider + { + private final String emitterType; + + private Emitter emitter = null; + + EmitterProvider( + String emitterType + ) + { + this.emitterType = emitterType; + } + + @Inject + public void inject(Injector injector) + { + final List> emitterBindings = injector.findBindingsByType(new TypeLiteral(){}); + + for (Binding binding : emitterBindings) { + if (Names.named(emitterType).equals(binding.getKey().getAnnotation())) { + emitter = binding.getProvider().get(); + break; + } + } + + if (emitter == null) { + List knownTypes = Lists.newArrayList(); + for (Binding binding : emitterBindings) { + final Annotation annotation = binding.getKey().getAnnotation(); + if (annotation != null) { + knownTypes.add(((Named) annotation).value()); + } + } + throw new ISE("Uknown emitter type, known types[%s]", knownTypes); + } + } + + + @Override + public Emitter get() + { + if (emitter == null) { + throw new ISE("Emitter was null, that's bad!"); + } + return emitter; + } + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java new file mode 100644 index 00000000000..fd6955d3579 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Duration; + +/** + */ +public abstract class HttpEmitterConfig extends com.metamx.emitter.core.HttpEmitterConfig +{ + @JsonProperty + private Duration timeOut = new Duration("PT5m"); + + public Duration getReadTimeout() + { + return timeOut; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java new file mode 100644 index 00000000000..8f7cc45018f --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.base.Supplier; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.HttpPostEmitter; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; + +import javax.annotation.Nullable; +import javax.net.ssl.SSLContext; + +/** + */ +public class HttpEmitterModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter.http", HttpEmitterConfig.class); + } + + @Provides @LazySingleton @Named("http") + public Emitter getEmitter(Supplier config, @Nullable SSLContext sslContext, Lifecycle lifecycle) + { + final HttpClientConfig.Builder builder = HttpClientConfig + .builder() + .withNumConnections(1) + .withReadTimeout(config.get().getReadTimeout()); + + if (sslContext != null) { + builder.withSslContext(sslContext); + } + + return new HttpPostEmitter(config.get(), HttpClientInit.createClient(builder.build(), lifecycle)); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 0d49ff0cefe..66366a3e739 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -27,6 +27,7 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.discovery.AddressPortServiceInstanceFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; @@ -42,7 +43,6 @@ import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.BoundedExponentialBackoffRetry; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; -import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; import org.apache.zookeeper.data.Stat; import org.mortbay.jetty.Connector; @@ -221,16 +221,16 @@ public class Initialization return framework; } - public static ServiceDiscovery makeServiceDiscoveryClient( + public static ServiceDiscovery makeServiceDiscoveryClient( CuratorFramework discoveryClient, - ServiceDiscoveryConfig config, + CuratorDiscoveryConfig config, Lifecycle lifecycle ) throws Exception { - final ServiceDiscovery serviceDiscovery = + final ServiceDiscovery serviceDiscovery = ServiceDiscoveryBuilder.builder(Void.class) - .basePath(config.getDiscoveryPath()) + .basePath(config.getPath()) .client(discoveryClient) .build(); @@ -260,21 +260,21 @@ public class Initialization } public static ServiceAnnouncer makeServiceAnnouncer( - ServiceDiscoveryConfig config, - ServiceDiscovery serviceDiscovery + DruidNodeConfig config, + ServiceDiscovery serviceDiscovery ) { - final ServiceInstanceFactory serviceInstanceFactory = makeServiceInstanceFactory(config); + final ServiceInstanceFactory serviceInstanceFactory = makeServiceInstanceFactory(config); return new CuratorServiceAnnouncer(serviceDiscovery, serviceInstanceFactory); } public static void announceDefaultService( - final ServiceDiscoveryConfig config, + final DruidNodeConfig nodeConfig, final ServiceAnnouncer serviceAnnouncer, final Lifecycle lifecycle ) throws Exception { - final String service = config.getServiceName().replace('/', ':'); + final String service = nodeConfig.getServiceName().replace('/', ':'); lifecycle.addHandler( new Lifecycle.Handler() @@ -357,7 +357,7 @@ public class Initialization ); } - public static ServiceInstanceFactory makeServiceInstanceFactory(ServiceDiscoveryConfig config) + public static ServiceInstanceFactory makeServiceInstanceFactory(DruidNodeConfig config) { final String host = config.getHost(); final String address; diff --git a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java new file mode 100644 index 00000000000..237a87ec1b6 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.LoggingEmitterConfig; + +/** + */ +public class LogEmitterModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter.logging", LoggingEmitterConfig.class); + } + + @Provides @LazySingleton @Named("logging") + public Emitter makeEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) + { + return new LoggingEmitter(config, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java new file mode 100644 index 00000000000..3e1eb3efa1a --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java @@ -0,0 +1,135 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.initialization; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.io.Closeables; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.common.config.Config; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorConfig; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.data.Stat; +import org.skife.config.ConfigurationObjectFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.Properties; + +/** + */ +public class PropertiesModule implements Module +{ + private static final Logger log = new Logger(PropertiesModule.class); + + private final String propertiesFile; + + public PropertiesModule(String propertiesFile) + { + this.propertiesFile = propertiesFile; + } + + @Override + public void configure(Binder binder) + { + final Properties zkProps = new Properties(); + final Properties fileProps = new Properties(zkProps); + + // Note that zookeeper coordinates must be either in cmdLine or in runtime.properties + Properties sp = System.getProperties(); + + Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain + tmp_props.putAll(sp); + + final InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); + if (stream == null) { + log.info("%s not found on classpath, relying only on system properties and zookeeper.", propertiesFile); + } else { + log.info("Loading properties from %s", propertiesFile); + try { + try { + fileProps.load(stream); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + finally { + Closeables.closeQuietly(stream); + } + } + + // log properties from file; stringPropertyNames() would normally cascade down into the sub Properties objects, but + // zkProps (the parent level) is empty at this point so it will only log properties from runtime.properties + for (String prop : fileProps.stringPropertyNames()) { + log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop)); + } + + final String zkHostsProperty = "druid.zk.service.host"; + + if (tmp_props.getProperty(zkHostsProperty) != null) { + final ConfigurationObjectFactory factory = Config.createFactory(tmp_props); + + ZkPathsConfig config; + try { + config = factory.build(ZkPathsConfig.class); + } + catch (IllegalArgumentException e) { + log.warn(e, "Unable to build ZkPathsConfig. Cannot load properties from ZK."); + config = null; + } + + if (config != null) { + Lifecycle lifecycle = new Lifecycle(); + try { + CuratorFramework curator = Initialization.makeCuratorFramework(factory.build(CuratorConfig.class), lifecycle); + + lifecycle.start(); + + final Stat stat = curator.checkExists().forPath(config.getPropertiesPath()); + if (stat != null) { + final byte[] data = curator.getData().forPath(config.getPropertiesPath()); + zkProps.load(new InputStreamReader(new ByteArrayInputStream(data), Charsets.UTF_8)); + } + + // log properties from zk + for (String prop : zkProps.stringPropertyNames()) { + log.info("Loaded(zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + finally { + lifecycle.stop(); + } + } + } else { + log.warn("property[%s] not set, skipping ZK-specified properties.", zkHostsProperty); + } + + binder.bind(Properties.class).toInstance(tmp_props); + } +} diff --git a/common/pom.xml b/common/pom.xml index f8f6a5c6ffa..cfb8620e574 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -63,6 +63,10 @@ org.apache.curator curator-x-discovery + + org.hibernate + hibernate-validator + it.uniroma3.mat extendedset @@ -87,6 +91,14 @@ com.fasterxml.jackson.datatype jackson-datatype-joda + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + + + com.google.inject + guice + org.jdbi jdbi diff --git a/common/src/main/java/com/metamx/druid/concurrent/Execs.java b/common/src/main/java/com/metamx/druid/concurrent/Execs.java index 7b86ca9e76c..619a8199b00 100644 --- a/common/src/main/java/com/metamx/druid/concurrent/Execs.java +++ b/common/src/main/java/com/metamx/druid/concurrent/Execs.java @@ -23,6 +23,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; /** */ @@ -31,7 +33,19 @@ public class Execs public static ExecutorService singleThreaded(String nameFormat) { return Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build() + makeThreadFactory(nameFormat) ); } + + public static ScheduledExecutorService scheduledSingleThreaded(String nameFormat) + { + return Executors.newSingleThreadScheduledExecutor( + makeThreadFactory(nameFormat) + ); + } + + public static ThreadFactory makeThreadFactory(String nameFormat) + { + return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(nameFormat).build(); + } } 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 cb202b1a8f2..d6b0b3e47f2 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -19,6 +19,7 @@ package com.metamx.druid.db; +import com.google.common.base.Supplier; import com.metamx.common.logger.Logger; import org.apache.commons.dbcp.BasicDataSource; import org.skife.jdbi.v2.DBI; @@ -159,12 +160,14 @@ public class DbConnector } } - private final DbConnectorConfig config; + private final Supplier config; + private final Supplier dbTables; private final DBI dbi; - public DbConnector(DbConnectorConfig config) + public DbConnector(Supplier config, Supplier dbTables) { this.config = config; + this.dbTables = dbTables; this.dbi = new DBI(getDatasource()); } @@ -176,16 +179,28 @@ public class DbConnector private DataSource getDatasource() { - BasicDataSource dataSource = new BasicDataSource(); - dataSource.setUsername(config.getDatabaseUser()); - dataSource.setPassword(config.getDatabasePassword()); - dataSource.setUrl(config.getDatabaseConnectURI()); + DbConnectorConfig connectorConfig = config.get(); - if (config.useValidationQuery()) { - dataSource.setValidationQuery(config.getValidationQuery()); + BasicDataSource dataSource = new BasicDataSource(); + dataSource.setUsername(connectorConfig.getUser()); + dataSource.setPassword(connectorConfig.getPassword()); + dataSource.setUrl(connectorConfig.getConnectURI()); + + if (connectorConfig.isUseValidationQuery()) { + dataSource.setValidationQuery(connectorConfig.getValidationQuery()); dataSource.setTestOnBorrow(true); } return dataSource; } + + public void createSegmentTable() + { + createSegmentTable(dbi, dbTables.get().getSegmentsTable()); + } + + public void createRulesTable() + { + createRuleTable(dbi, dbTables.get().getRulesTable()); + } } diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index e17302a3183..f6e15bbcc48 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -20,37 +20,60 @@ package com.metamx.druid.db; import com.fasterxml.jackson.annotation.JsonProperty; -import org.skife.config.Config; + +import javax.validation.constraints.NotNull; /** */ public abstract class DbConnectorConfig { - @JsonProperty("connectURI") - @Config("druid.database.connectURI") - public abstract String getDatabaseConnectURI(); + @JsonProperty + private boolean createTables = true; - @JsonProperty("user") - @Config("druid.database.user") - public abstract String getDatabaseUser(); + @JsonProperty + @NotNull + private String connectURI = null; - @JsonProperty("password") - @Config("druid.database.password") - public abstract String getDatabasePassword(); + @JsonProperty + @NotNull + private String user = null; - @JsonProperty("segmentTable") - @Config("druid.database.segmentTable") - public abstract String getSegmentTable(); + @JsonProperty + @NotNull + private String password = null; - @JsonProperty("useValidationQuery") - @Config("druid.database.validation") - public boolean useValidationQuery() { - return false; + @JsonProperty + private boolean useValidationQuery = false; + + @JsonProperty + private String validationQuery = "SELECT 1"; + + public boolean isCreateTables() + { + return createTables; + } + + public String getConnectURI() + { + return connectURI; + } + + public String getUser() + { + return user; + } + + public String getPassword() + { + return password; + } + + public boolean isUseValidationQuery() + { + return useValidationQuery; } - @JsonProperty("validationQuery") - @Config("druid.database.validationQuery") public String getValidationQuery() { - return "SELECT 1"; + return validationQuery; } } diff --git a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java new file mode 100644 index 00000000000..96d7170f6fb --- /dev/null +++ b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java @@ -0,0 +1,69 @@ +package com.metamx.druid.db; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.ISE; + +import javax.validation.constraints.NotNull; + +/** + */ +public class DbTablesConfig +{ + public static DbTablesConfig fromBase(String base) + { + return new DbTablesConfig(base, null, null); + } + + @NotNull + private final String base; + + @NotNull + private final String segmentsTable; + + @NotNull + private final String ruleTable; + + @JsonCreator + public DbTablesConfig( + @JsonProperty("base") String base, + @JsonProperty("segments") String segmentsTable, + @JsonProperty("rules") String rulesTable + ) + { + + this.base = base; + this.segmentsTable = makeTableName(segmentsTable, "segments"); + this.ruleTable = makeTableName(rulesTable, "rules"); + } + + private String makeTableName(String explicitTableName, String defaultSuffix) + { + if (explicitTableName == null) { + if (base == null) { + throw new ISE("table[%s] unknown! Both base and %s were null!", defaultSuffix, defaultSuffix); + } + return String.format("%s_%s", base, defaultSuffix); + } + + return explicitTableName; + } + + @JsonProperty + public String getBase() + { + return base; + } + + @JsonProperty("segments") + public String getSegmentsTable() + { + return segmentsTable; + } + + @JsonProperty("rules") + public String getRulesTable() + { + return ruleTable; + } +} \ No newline at end of file diff --git a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java new file mode 100644 index 00000000000..79eff1b41ff --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java @@ -0,0 +1,45 @@ +package com.metamx.druid.guice; + +import com.google.common.base.Preconditions; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Provider; +import org.skife.config.ConfigurationObjectFactory; + +/** + */ +public class ConfigProvider implements Provider +{ + public static void bind(Binder binder, Class clazz) + { + binder.bind(clazz).toProvider(of(clazz)).in(DruidScopes.SINGLETON); + } + + public static Provider of(Class clazz) + { + return new ConfigProvider(clazz); + } + + private final Class clazz; + + private T object = null; + + public ConfigProvider( + Class clazz + ) + { + this.clazz = clazz; + } + + @Inject + public void inject(ConfigurationObjectFactory factory) + { + object = factory.build(clazz); + } + + @Override + public T get() + { + return Preconditions.checkNotNull(object, "WTF!? Code misconfigured, inject() didn't get called."); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java new file mode 100644 index 00000000000..e999b725aa2 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java @@ -0,0 +1,15 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; + +/** + */ +public class DruidGuiceExtensions implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidScopes.java b/common/src/main/java/com/metamx/druid/guice/DruidScopes.java new file mode 100644 index 00000000000..151325e587b --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidScopes.java @@ -0,0 +1,71 @@ +package com.metamx.druid.guice; + +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.Scope; +import com.google.inject.Scopes; +import com.metamx.common.lifecycle.Lifecycle; + +/** + */ +public class DruidScopes +{ + public static final Scope SINGLETON = new Scope() + { + @Override + public Provider scope(Key key, Provider unscoped) + { + return Scopes.SINGLETON.scope(key, unscoped); + } + + @Override + public String toString() + { + return "DruidScopes.SINGLETON"; + } + }; + + public static final Scope LIFECYCLE = new Scope() + { + @Override + public Provider scope(final Key key, final Provider unscoped) + { + return new Provider() + { + + private Provider provider; + + @Inject + public void inject(final Lifecycle lifecycle) + { + provider = Scopes.SINGLETON.scope( + key, + new Provider() + { + + @Override + public T get() + { + return lifecycle.addManagedInstance(unscoped.get()); + } + } + ); + } + + @Override + public T get() + { + System.out.println(provider); + return provider.get(); + } + }; + } + + @Override + public String toString() + { + return "DruidScopes.LIFECYCLE"; + } + }; +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java new file mode 100644 index 00000000000..da8d2ce59c1 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -0,0 +1,45 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Module; +import com.metamx.druid.jackson.Json; +import com.metamx.druid.jackson.Smile; +import org.skife.config.ConfigurationObjectFactory; + +import java.util.Properties; + +/** + */ +public class DruidSecondaryModule implements Module +{ + private final Properties properties; + private final ConfigurationObjectFactory factory; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + + @Inject + public DruidSecondaryModule( + Properties properties, + ConfigurationObjectFactory factory, + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper + ) + { + this.properties = properties; + this.factory = factory; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + } + + @Override + public void configure(Binder binder) + { + binder.install(new DruidGuiceExtensions()); + binder.bind(Properties.class).toInstance(properties); + binder.bind(ConfigurationObjectFactory.class).toInstance(factory); + binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMapper); + binder.bind(ObjectMapper.class).annotatedWith(Smile.class).toInstance(smileMapper); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java new file mode 100644 index 00000000000..64dd107d0f0 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -0,0 +1,121 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.google.common.base.Supplier; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.google.inject.TypeLiteral; +import com.metamx.common.IAE; +import com.metamx.common.ISE; + +import javax.validation.ConstraintViolation; +import javax.validation.Validator; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** + */ +public class JsonConfigProvider implements Provider> +{ + + private static final Joiner JOINER = Joiner.on(", "); + + public static void bind(Binder binder, String propertyBase, Class classToProvide) + { + binder.bind(new TypeLiteral>(){}).toProvider(of(propertyBase, classToProvide)).in(DruidScopes.SINGLETON); + } + + public static JsonConfigProvider of(String propertyBase, Class classToProvide) + { + return new JsonConfigProvider(propertyBase, classToProvide); + } + + private final String propertyBase; + private final Class classToProvide; + + private Supplier supplier; + + public JsonConfigProvider( + String propertyBase, + Class classToProvide + ) + { + this.propertyBase = propertyBase; + this.classToProvide = classToProvide; + } + + @Inject + public void inject( + Properties props, + ObjectMapper jsonMapper, + Validator validator + ) + { + Map jsonMap = Maps.newHashMap(); + for (String prop : props.stringPropertyNames()) { + if (prop.startsWith(propertyBase)) { + final String propValue = props.getProperty(prop); + try { + jsonMap.put(prop.substring(propertyBase.length()), jsonMapper.readValue(propValue, Object.class)); + } + catch (IOException e) { + throw new IAE("Unable to parse an object out of prop[%s]=[%s]", prop, propValue); + } + } + } + + final T config = jsonMapper.convertValue(jsonMap, classToProvide); + + final Set> violations = validator.validate(config); + if (!violations.isEmpty()) { + List messages = Lists.newArrayList(); + + for (ConstraintViolation violation : violations) { + messages.add(String.format("%s - %s", violation.getPropertyPath().toString(), violation.getMessage())); + } + + throw new ISE("Configuration violations[%s]", JOINER.join(messages)); + } + + this.supplier = new Supplier() + { + @Override + public T get() + { + return config; + } + }; + } + + @Override + public Supplier get() + { + return supplier; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/LazySingleton.java b/common/src/main/java/com/metamx/druid/guice/LazySingleton.java new file mode 100644 index 00000000000..005e9795de4 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LazySingleton.java @@ -0,0 +1,18 @@ +package com.metamx.druid.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface LazySingleton +{ +} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java new file mode 100644 index 00000000000..b31bbaddd11 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -0,0 +1,74 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; + +/** + * A Module to add lifecycle management to the injector. {@link DruidGuiceExtensions} must also be included. + */ +public class LifecycleModule implements Module +{ + private final LifecycleScope scope = new LifecycleScope(); + private final Key[] eagerClasses; + + /** + * A constructor that takes a list of classes to instantiate eagerly. Class {@link Key}s mentioned here will + * be pulled out of the injector with an injector.getInstance() call when the lifecycle is created. + * + * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper + * scope. + * + * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to + * auto-register things with the {@link Lifecycle} + * + * @param eagerClasses set of classes to instantiate eagerly + */ + public LifecycleModule( + Key... eagerClasses + ) + { + this.eagerClasses = eagerClasses; + } + + @Override + public void configure(Binder binder) + { + binder.bindScope(ManageLifecycle.class, scope); + } + + @Provides @LazySingleton + public Lifecycle getLifecycle(Injector injector) + { + Lifecycle lifecycle = new Lifecycle(); + scope.setLifecycle(lifecycle); + + for (Key key : eagerClasses) { + injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. + } + + return lifecycle; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java new file mode 100644 index 00000000000..223113728e3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java @@ -0,0 +1,67 @@ +package com.metamx.druid.guice; + +import com.google.common.collect.Lists; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.Scope; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; + +import java.util.List; + +/** + * A scope that adds objects to the Lifecycle. This is by definition also a lazy singleton scope. + */ +public class LifecycleScope implements Scope +{ + private static final Logger log = new Logger(LifecycleScope.class); + + private Lifecycle lifecycle; + private List instances = Lists.newLinkedList(); + + public void setLifecycle(Lifecycle lifecycle) + { + this.lifecycle = lifecycle; + synchronized (instances) { + for (Object instance : instances) { + lifecycle.addManagedInstance(instance); + } + } + } + + @Override + public Provider scope(final Key key, final Provider unscoped) + { + return new Provider() + { + private T value = null; + + @Override + public synchronized T get() + { + if (value == null) { + final T retVal = unscoped.get(); + + synchronized (instances) { + if (lifecycle == null) { + instances.add(retVal); + } + else { + try { + lifecycle.addMaybeStartManagedInstance(retVal); + } + catch (Exception e) { + log.warn(e, "Caught exception when trying to create a[%s]", key); + return null; + } + } + } + + value = retVal; + } + + return value; + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java new file mode 100644 index 00000000000..35752a64b0d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java @@ -0,0 +1,19 @@ +package com.metamx.druid.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface ManageLifecycle +{ +} diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java new file mode 100644 index 00000000000..aca59fd8bc3 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -0,0 +1,53 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.guice.LazySingleton; + +/** + */ +public class JacksonModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); + } + + @Provides @LazySingleton @Json + public ObjectMapper jsonMapper() + { + return new DefaultObjectMapper(); + } + + @Provides @LazySingleton @Smile + public ObjectMapper smileMapper() + { + ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); + retVal.getJsonFactory().setCodec(retVal); + return retVal; + } +} diff --git a/common/src/main/java/com/metamx/druid/jackson/Json.java b/common/src/main/java/com/metamx/druid/jackson/Json.java new file mode 100644 index 00000000000..b0cc6401613 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/Json.java @@ -0,0 +1,17 @@ +package com.metamx.druid.jackson; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Json +{ +} diff --git a/common/src/main/java/com/metamx/druid/jackson/Smile.java b/common/src/main/java/com/metamx/druid/jackson/Smile.java new file mode 100644 index 00000000000..cc1c5b376e5 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/Smile.java @@ -0,0 +1,17 @@ +package com.metamx.druid.jackson; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Smile +{ +} diff --git a/common/src/main/java/com/metamx/druid/utils/SocketUtil.java b/common/src/main/java/com/metamx/druid/utils/SocketUtil.java new file mode 100644 index 00000000000..efea5f3003f --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/SocketUtil.java @@ -0,0 +1,39 @@ +package com.metamx.druid.utils; + +import com.metamx.common.ISE; + +import java.io.IOException; +import java.net.ServerSocket; + +/** + */ +public class SocketUtil +{ + public static int findOpenPort(int startPort) + { + int currPort = startPort; + + while (currPort < 0xffff) { + ServerSocket socket = null; + try { + socket = new ServerSocket(currPort); + return currPort; + } + catch (IOException e) { + ++currPort; + } + finally { + if (socket != null) { + try { + socket.close(); + } + catch (IOException e) { + + } + } + } + } + + throw new ISE("Unable to find open port between[%d] and [%d]", startPort, currPort); + } +} diff --git a/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java b/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java new file mode 100644 index 00000000000..9948dfe1470 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java @@ -0,0 +1,231 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import junit.framework.Assert; +import org.junit.Test; + +/** + */ +public class LifecycleScopeTest +{ + @Test + public void testAnnotation() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(AnnotatedClass.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + testIt(injector, lifecycle, instance); + } + + @Test + public void testExplicit() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(ExplicitClass.class).in(ManageLifecycle.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + testIt(injector, lifecycle, instance); + } + + private void testIt(Injector injector, Lifecycle lifecycle, TestInterface instance) + throws Exception + { + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(0, instance.getRan()); + + instance.run(); + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + lifecycle.start(); + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + injector.getInstance(TestInterface.class).run(); // It's a singleton + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + + lifecycle.stop(); + Assert.assertEquals(1, instance.getStarted()); + Assert.assertEquals(1, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + } + + /** + * This is a test for documentation purposes. It's there to show what weird things Guice will do when + * it sees both the annotation and an explicit binding. + * + * @throws Exception + */ + @Test + public void testAnnotatedAndExplicit() throws Exception + { + final Injector injector = Guice.createInjector( + new DruidGuiceExtensions(), + new LifecycleModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(TestInterface.class).to(AnnotatedClass.class).in(ManageLifecycle.class); + } + } + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final TestInterface instance = injector.getInstance(TestInterface.class); + + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(0, instance.getRan()); + + instance.run(); + Assert.assertEquals(0, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + lifecycle.start(); + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(1, instance.getRan()); + + injector.getInstance(TestInterface.class).run(); // It's a singleton + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(0, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + + lifecycle.stop(); + Assert.assertEquals(2, instance.getStarted()); + Assert.assertEquals(2, instance.getStopped()); + Assert.assertEquals(2, instance.getRan()); + } + + private static interface TestInterface + { + public void run(); + public int getStarted(); + public int getStopped(); + public int getRan(); + } + + @ManageLifecycle + public static class AnnotatedClass implements TestInterface + { + int started = 0; + int stopped = 0; + int ran = 0; + + @LifecycleStart + public void start() + { + ++started; + } + + @LifecycleStop + public void stop() + { + ++stopped; + } + + @Override + public void run() + { + ++ran; + } + + public int getStarted() + { + return started; + } + + public int getStopped() + { + return stopped; + } + + public int getRan() + { + return ran; + } + } + + public static class ExplicitClass implements TestInterface + { + int started = 0; + int stopped = 0; + int ran = 0; + + @LifecycleStart + public void start() + { + ++started; + } + + @LifecycleStop + public void stop() + { + ++stopped; + } + + @Override + public void run() + { + ++ran; + } + + public int getStarted() + { + return started; + } + + public int getStopped() + { + return stopped; + } + + public int getRan() + { + return ran; + } + } + +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index d83ecf2f747..8ffc2b521bd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -25,15 +25,15 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; -import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; -import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; -import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; -import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.merger.common.actions.TaskActionClientFactory; +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; +import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -65,7 +65,7 @@ public class TaskMasterLifecycle final TaskQueue taskQueue, final TaskActionClientFactory taskActionClientFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, - final ServiceDiscoveryConfig serviceDiscoveryConfig, + final DruidNodeConfig nodeConfig, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, @@ -103,7 +103,7 @@ public class TaskMasterLifecycle final Lifecycle leaderLifecycle = new Lifecycle(); leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(taskRunner); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); if (indexerCoordinatorConfig.isAutoScalingEnabled()) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 69a9d4dac88..d0d19f7552a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Charsets; import com.google.common.base.Optional; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -55,9 +56,10 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; @@ -156,7 +158,7 @@ public class IndexerCoordinatorNode extends QueryableNode serviceDiscovery = null; private ServiceAnnouncer serviceAnnouncer = null; private TaskStorage taskStorage = null; private TaskQueue taskQueue = null; @@ -365,12 +367,12 @@ public class IndexerCoordinatorNode extends QueryableNode instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); } } @@ -616,7 +619,7 @@ public class IndexerCoordinatorNode extends QueryableNodeofInstance(dbConnectorConfig), null).getDBI() // TODO ); } else { throw new ISE("Invalid storage implementation: %s", config.getStorageImpl()); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 06c160106d5..852eb002cc1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -43,25 +43,26 @@ import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; 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.S3DataSegmentKiller; -import com.metamx.druid.indexing.common.RetryPolicyFactory; -import com.metamx.druid.indexing.common.TaskToolboxFactory; -import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; -import com.metamx.druid.indexing.common.config.TaskConfig; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.ChatHandlerProvider; -import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; -import com.metamx.druid.indexing.worker.config.WorkerConfig; +import com.metamx.druid.merger.common.RetryPolicyFactory; +import com.metamx.druid.merger.common.TaskToolboxFactory; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; +import com.metamx.druid.merger.common.config.RetryPolicyConfig; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.index.ChatHandlerProvider; +import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; +import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; +import com.metamx.druid.merger.coordinator.ExecutorServiceTaskRunner; +import com.metamx.druid.merger.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.merger.worker.config.WorkerConfig; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -114,7 +115,7 @@ public class ExecutorNode extends BaseServerNode private WorkerConfig workerConfig = null; private DataSegmentPusher segmentPusher = null; private TaskToolboxFactory taskToolboxFactory = null; - private ServiceDiscovery serviceDiscovery = null; + private ServiceDiscovery serviceDiscovery = null; private ServiceAnnouncer serviceAnnouncer = null; private ServiceProvider coordinatorServiceProvider = null; private Server server = null; @@ -391,14 +392,15 @@ public class ExecutorNode extends BaseServerNode public void initializeServiceDiscovery() throws Exception { - final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig config = configFactory.build(CuratorDiscoveryConfig.class); if (serviceDiscovery == null) { this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( getCuratorFramework(), config, lifecycle ); } if (serviceAnnouncer == null) { - final ServiceInstanceFactory instanceFactory = Initialization.makeServiceInstanceFactory(config); + DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); + final ServiceInstanceFactory instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); } if (coordinatorServiceProvider == null) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 1ccf32e0795..2d69208cfd8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -37,9 +37,9 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; @@ -330,7 +330,7 @@ public class WorkerNode extends QueryableNode public void initializeServiceDiscovery() throws Exception { if (serviceDiscovery == null) { - final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class); + final CuratorDiscoveryConfig config = getConfigFactory().build(CuratorDiscoveryConfig.class); this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( getCuratorFramework(), config, diff --git a/pom.xml b/pom.xml index b9483992da0..cae6b4b4c3c 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ UTF-8 - 0.22.3 + 0.23.0-SNAPSHOT 2.0.2-21-22 @@ -215,6 +215,21 @@ jackson-jaxrs-json-provider 2.1.4 + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + + + org.hibernate + hibernate-validator + 5.0.1.Final + javax.inject javax.inject diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 708c61d9e24..4d51df1a8d5 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -21,17 +21,20 @@ package com.metamx.druid.db; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.MapUtils; 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.concurrent.Execs; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; - import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; @@ -52,12 +55,13 @@ import java.util.concurrent.atomic.AtomicReference; /** */ +@ManageLifecycle public class DatabaseRuleManager { public static void createDefaultRule( final DBI dbi, final String ruleTable, - final String defaultDatasource, + final String defaultTier, final ObjectMapper jsonMapper ) { @@ -72,7 +76,7 @@ public class DatabaseRuleManager String.format( "SELECT id from %s where datasource='%s';", ruleTable, - defaultDatasource + defaultTier ) ); @@ -94,8 +98,8 @@ public class DatabaseRuleManager ruleTable ) ) - .bind("id", String.format("%s_%s", defaultDatasource, version)) - .bind("dataSource", defaultDatasource) + .bind("id", String.format("%s_%s", defaultTier, version)) + .bind("dataSource", defaultTier) .bind("version", version) .bind("payload", jsonMapper.writeValueAsString(defaultRules)) .execute(); @@ -115,6 +119,7 @@ public class DatabaseRuleManager private final ObjectMapper jsonMapper; private final ScheduledExecutorService exec; private final DatabaseRuleManagerConfig config; + private final Supplier dbTables; private final DBI dbi; private final AtomicReference>> rules; @@ -122,18 +127,21 @@ public class DatabaseRuleManager private volatile boolean started = false; + @Inject public DatabaseRuleManager( ObjectMapper jsonMapper, - ScheduledExecutorService exec, DatabaseRuleManagerConfig config, + Supplier dbTables, DBI dbi ) { this.jsonMapper = jsonMapper; - this.exec = exec; this.config = config; + this.dbTables = dbTables; this.dbi = dbi; + this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"); + this.rules = new AtomicReference>>( new ConcurrentHashMap>() ); @@ -147,6 +155,7 @@ public class DatabaseRuleManager return; } + createDefaultRule(dbi, getRulesTable(), config.getDefaultTier(), jsonMapper); ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), @@ -192,8 +201,11 @@ public class DatabaseRuleManager return handle.createQuery( // Return latest version rule by dataSource String.format( - "SELECT %1$s.dataSource, %1$s.payload FROM %1$s INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds ON %1$s.datasource = ds.datasource and %1$s.version = ds.version", - config.getRuleTable() + "SELECT r.dataSource, r.payload " + + "FROM %1$s r " + + "INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds " + + "ON r.datasource = ds.datasource and r.version = ds.version", + getRulesTable() ) ).fold( Maps.>newHashMap(), @@ -255,8 +267,8 @@ public class DatabaseRuleManager if (theRules.get(dataSource) != null) { retVal.addAll(theRules.get(dataSource)); } - if (theRules.get(config.getDefaultDatasource()) != null) { - retVal.addAll(theRules.get(config.getDefaultDatasource())); + if (theRules.get(config.getDefaultTier()) != null) { + retVal.addAll(theRules.get(config.getDefaultTier())); } return retVal; } @@ -275,7 +287,7 @@ public class DatabaseRuleManager handle.createStatement( String.format( "INSERT INTO %s (id, dataSource, version, payload) VALUES (:id, :dataSource, :version, :payload)", - config.getRuleTable() + getRulesTable() ) ) .bind("id", String.format("%s_%s", dataSource, version)) @@ -303,4 +315,6 @@ public class DatabaseRuleManager return true; } + + private String getRulesTable() {return dbTables.get().getRulesTable();} } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java index 41e21a0943b..5f41945df58 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java @@ -27,12 +27,9 @@ import org.skife.config.Default; */ public abstract class DatabaseRuleManagerConfig { - @Config("druid.database.ruleTable") - public abstract String getRuleTable(); - - @Config("druid.database.defaultDatasource") + @Config("druid.database.rules.defaultTier") @Default("_default") - public abstract String getDefaultDatasource(); + public abstract String getDefaultTier(); @Config("druid.database.rules.poll.duration") @Default("PT1M") diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index 2ea9056627f..fce933e6294 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -21,9 +21,11 @@ package com.metamx.druid.db; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; @@ -33,7 +35,8 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; - +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.ManageLifecycle; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; @@ -57,6 +60,7 @@ import java.util.concurrent.atomic.AtomicReference; /** */ +@ManageLifecycle public class DatabaseSegmentManager { private static final Logger log = new Logger(DatabaseSegmentManager.class); @@ -65,26 +69,30 @@ public class DatabaseSegmentManager private final ObjectMapper jsonMapper; private final ScheduledExecutorService exec; - private final DatabaseSegmentManagerConfig config; + private final Supplier config; + private final Supplier dbTables; private final AtomicReference> dataSources; private final DBI dbi; private volatile boolean started = false; + @Inject public DatabaseSegmentManager( ObjectMapper jsonMapper, - ScheduledExecutorService exec, - DatabaseSegmentManagerConfig config, + Supplier config, + Supplier dbTables, DBI dbi ) { this.jsonMapper = jsonMapper; - this.exec = exec; this.config = config; + this.dbTables = dbTables; this.dataSources = new AtomicReference>( new ConcurrentHashMap() ); this.dbi = dbi; + + this.exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); } @LifecycleStart @@ -98,7 +106,7 @@ public class DatabaseSegmentManager ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), - config.getPollDuration(), + config.get().getPollDuration(), new Runnable() { @Override @@ -136,7 +144,7 @@ public class DatabaseSegmentManager public VersionedIntervalTimeline withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", config.getSegmentTable()) + String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", getSegmentsTable()) ) .bind("dataSource", ds) .fold( @@ -204,7 +212,7 @@ public class DatabaseSegmentManager batch.add( String.format( "UPDATE %s SET used=1 WHERE id = '%s'", - config.getSegmentTable(), + getSegmentsTable(), segment.getIdentifier() ) ); @@ -234,7 +242,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=1 WHERE id = :id", config.getSegmentTable()) + String.format("UPDATE %s SET used=1 WHERE id = :id", getSegmentsTable()) ) .bind("id", segmentId) .execute(); @@ -268,7 +276,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE dataSource = :dataSource", config.getSegmentTable()) + String.format("UPDATE %s SET used=0 WHERE dataSource = :dataSource", getSegmentsTable()) ) .bind("dataSource", ds) .execute(); @@ -298,7 +306,7 @@ public class DatabaseSegmentManager public Void withHandle(Handle handle) throws Exception { handle.createStatement( - String.format("UPDATE %s SET used=0 WHERE id = :segmentID", config.getSegmentTable()) + String.format("UPDATE %s SET used=0 WHERE id = :segmentID", getSegmentsTable()) ).bind("segmentID", segmentID) .execute(); @@ -354,7 +362,7 @@ public class DatabaseSegmentManager public List withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT DISTINCT(datasource) FROM %s", config.getSegmentTable()) + String.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable()) ) .fold( Lists.newArrayList(), @@ -398,7 +406,7 @@ public class DatabaseSegmentManager public List> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT payload FROM %s WHERE used=1", config.getSegmentTable()) + String.format("SELECT payload FROM %s WHERE used=1", getSegmentsTable()) ).list(); } } @@ -451,4 +459,8 @@ public class DatabaseSegmentManager log.error(e, "Problem polling DB."); } } + + private String getSegmentsTable() { + return dbTables.get().getSegmentsTable(); + } } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java index 25e90cd16bd..85532089243 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java @@ -19,18 +19,18 @@ package com.metamx.druid.db; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; /** */ -public abstract class DatabaseSegmentManagerConfig +public class DatabaseSegmentManagerConfig { - @Config("druid.database.segmentTable") - public abstract String getSegmentTable(); + @JsonProperty + private Duration pollDuration = new Duration("PT1M"); - @Config("druid.database.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); + public Duration getPollDuration() + { + return pollDuration; + } } diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java new file mode 100644 index 00000000000..39e6fe10e8c --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -0,0 +1,54 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; +import com.metamx.druid.initialization.ZkPathsConfig; +import org.skife.jdbi.v2.DBI; + +/** + */ +public class MasterModule implements Module +{ + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, ZkPathsConfig.class); + ConfigProvider.bind(binder, ServerInventoryViewConfig.class); + ConfigProvider.bind(binder, DbConnectorConfig.class); + + JsonConfigProvider.bind(binder, "druid.database.tables", DbTablesConfig.class); + + } + + @Provides @LazySingleton + public DBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) + { + if (config.isCreateTables()) { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createSegmentTable(); + dbConnector.createRulesTable(); + } + + @Override + public void stop() + { + + } + } + ); + } + + return dbConnector.getDBI(); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java new file mode 100644 index 00000000000..8440df3042b --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.java @@ -0,0 +1,16 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.druid.initialization.DruidNodeConfig; + +/** + */ +public class ServerModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid", DruidNodeConfig.class); + } +} 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 d858c72467a..e62cabb3075 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -26,6 +26,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.client.DruidServer; @@ -38,7 +39,6 @@ import com.metamx.druid.master.rules.Rule; import org.joda.time.Interval; import javax.annotation.Nullable; -import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; 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 6c48aba9021..71b34d5c199 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -21,57 +21,56 @@ package com.metamx.druid.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; +import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.common.collect.Iterables; import com.google.inject.Guice; import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.config.ConfigManager; import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.curator.discovery.DiscoveryModule; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.guice.DruidGuiceExtensions; +import com.metamx.druid.guice.DruidSecondaryModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.MasterModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.initialization.ConfigFactoryModule; +import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.PropertiesModule; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.initialization.ZkPathsConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.jackson.JacksonModule; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; +import com.metamx.druid.metrics.MetricsModule; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; import com.metamx.http.client.response.ToStringResponseHandler; -import com.metamx.metrics.JvmMonitor; -import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; -import org.joda.time.Duration; import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; @@ -81,12 +80,9 @@ import org.mortbay.servlet.GzipFilter; import org.skife.config.ConfigurationObjectFactory; import org.skife.jdbi.v2.DBI; +import javax.annotation.Nullable; import java.net.URL; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; +import java.util.Arrays; /** */ @@ -98,92 +94,41 @@ public class MasterMain { LogLevelAdjuster.register(); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); - final Properties props = Initialization.loadProperties(); - final ConfigurationObjectFactory configFactory = Config.createFactory(props); - final Lifecycle lifecycle = new Lifecycle(); - - final HttpClientConfig.Builder httpClientConfigBuilder = HttpClientConfig.builder().withNumConnections(1); - - final String emitterTimeout = props.getProperty("druid.emitter.timeOut"); - if (emitterTimeout != null) { - httpClientConfigBuilder.withReadTimeout(new Duration(emitterTimeout)); - } - final HttpClient httpClient = HttpClientInit.createClient(httpClientConfigBuilder.build(), lifecycle); - - final ServiceEmitter emitter = new ServiceEmitter( - PropUtils.getProperty(props, "druid.service"), - PropUtils.getProperty(props, "druid.host"), - Emitters.create(props, httpClient, jsonMapper, lifecycle) + Injector injector = makeInjector( + DruidSecondaryModule.class, + new LifecycleModule(Key.get(MonitorScheduler.class)), + EmitterModule.class, + CuratorModule.class, + MetricsModule.class, + DiscoveryModule.class, + ServerModule.class, + MasterModule.class ); - EmittingLogger.registerEmitter(emitter); + + final ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); + final ConfigurationObjectFactory configFactory = injector.getInstance(ConfigurationObjectFactory.class); + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + final ServiceEmitter emitter = injector.getInstance(ServiceEmitter.class); final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); - final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); - CuratorFramework curatorFramework = Initialization.makeCuratorFramework( - serviceDiscoveryConfig, - lifecycle - ); + CuratorFramework curatorFramework = injector.getInstance(CuratorFramework.class); final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class); - final ExecutorService exec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() - ); - ServerInventoryView serverInventoryView = new ServerInventoryView( - configFactory.build(ServerInventoryViewConfig.class), zkPaths, curatorFramework, exec, jsonMapper - ); - lifecycle.addManagedInstance(serverInventoryView); + ServerInventoryView serverInventoryView = injector.getInstance(ServerInventoryView.class); - final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class); - final DatabaseRuleManagerConfig databaseRuleManagerConfig = configFactory.build(DatabaseRuleManagerConfig.class); - final DBI dbi = new DbConnector(dbConnectorConfig).getDBI(); - DbConnector.createSegmentTable(dbi, PropUtils.getProperty(props, "druid.database.segmentTable")); - DbConnector.createRuleTable(dbi, PropUtils.getProperty(props, "druid.database.ruleTable")); - DatabaseRuleManager.createDefaultRule( - dbi, databaseRuleManagerConfig.getRuleTable(), databaseRuleManagerConfig.getDefaultDatasource(), jsonMapper - ); - final DatabaseSegmentManager databaseSegmentManager = new DatabaseSegmentManager( - jsonMapper, - scheduledExecutorFactory.create(1, "DatabaseSegmentManager-Exec--%d"), - configFactory.build(DatabaseSegmentManagerConfig.class), - dbi - ); - final DatabaseRuleManager databaseRuleManager = new DatabaseRuleManager( - jsonMapper, - scheduledExecutorFactory.create(1, "DatabaseRuleManager-Exec--%d"), - databaseRuleManagerConfig, - dbi - ); - - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - final List monitors = Lists.newArrayList(); - monitors.add(new JvmMonitor()); - if (Boolean.parseBoolean(props.getProperty("druid.monitoring.monitorSystem", "false"))) { - monitors.add(new SysMonitor()); - } - - final MonitorScheduler healthMonitor = new MonitorScheduler( - configFactory.build(MonitorSchedulerConfig.class), - globalScheduledExec, - emitter, - monitors - ); - lifecycle.addManagedInstance(healthMonitor); + final DatabaseSegmentManager databaseSegmentManager = injector.getInstance(DatabaseSegmentManager.class); + final DatabaseRuleManager databaseRuleManager = injector.getInstance(DatabaseRuleManager.class); final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class); + final DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); - final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, - serviceDiscoveryConfig, - lifecycle - ); - final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer( - serviceDiscoveryConfig, serviceDiscovery - ); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, lifecycle); + final ServiceDiscovery serviceDiscovery = injector.getInstance(Key.get(new TypeLiteral>(){})); + final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); ServiceProvider serviceProvider = null; if (druidMasterConfig.getMergerServiceName() != null) { @@ -192,9 +137,11 @@ public class MasterMain serviceDiscovery, lifecycle ); +// indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); TODO } IndexingServiceClient indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); + DBI dbi = injector.getInstance(DBI.class); final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); JacksonConfigManager configManager = new JacksonConfigManager( @@ -242,7 +189,7 @@ public class MasterMain ) ); - final Injector injector = Guice.createInjector( + final Injector injector2 = Guice.createInjector( new MasterServletModule( serverInventoryView, databaseSegmentManager, @@ -310,4 +257,41 @@ public class MasterMain server.start(); server.join(); } + + private static Injector makeInjector(final Object... modules) + { + final Injector baseInjector = Guice.createInjector( + new DruidGuiceExtensions(), + new JacksonModule(), + new PropertiesModule("runtime.properties"), + new ConfigFactoryModule() + ); + + return Guice.createInjector( + Iterables.transform( + Arrays.asList(modules), + new Function() + { + @Override + @SuppressWarnings("unchecked") + public Module apply(@Nullable Object input) + { + if (input instanceof Module) { + baseInjector.injectMembers(input); + return (Module) input; + } + if (input instanceof Class) { + if (Module.class.isAssignableFrom((Class) input)) { + return baseInjector.getInstance((Class) input); + } + else { + throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); + } + } + throw new ISE("Unknown module type[%s]", input.getClass()); + } + } + ) + ); + } } 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 9bb59d79d43..5934fdd5f6e 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -19,10 +19,10 @@ package com.metamx.druid.http; +import com.google.inject.Inject; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; -import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.GET; import javax.ws.rs.POST; diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index f113051d506..089f137db1a 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -19,12 +19,12 @@ 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 com.metamx.druid.index.Segment; -import javax.inject.Inject; import java.util.Map; /** diff --git a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java new file mode 100644 index 00000000000..610514bcc39 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java @@ -0,0 +1,77 @@ +package com.metamx.druid.metrics; + +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.logger.Logger; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.JvmMonitor; +import com.metamx.metrics.Monitor; +import com.metamx.metrics.MonitorScheduler; +import com.metamx.metrics.MonitorSchedulerConfig; +import com.metamx.metrics.SysMonitor; + +import java.util.List; + +/** + * Sets up the {@link MonitorScheduler} to monitor things on a regular schedule. {@link Monitor}s must be explicitly + * bound in order to be loaded. + */ +public class MetricsModule implements Module +{ + private static final Logger log = new Logger(MetricsModule.class); + + private final Class[] monitors; + + /** + * A constructor that takes a list of {@link Monitor} classes to explicitly bind so that they will be instantiated + * + * @param monitors list of {@link Monitor} classes to explicitly bind + */ + public MetricsModule( + Class... monitors + ) + { + this.monitors = monitors; + } + + @Override + public void configure(Binder binder) + { + binder.bind(JvmMonitor.class).in(LazySingleton.class); + binder.bind(SysMonitor.class).in(LazySingleton.class); + + for (Class monitor : monitors) { + binder.bind(monitor).in(LazySingleton.class); + } + } + + @Provides @ManageLifecycle + public MonitorScheduler getMonitorScheduler(MonitorSchedulerConfig config, ServiceEmitter emitter, Injector injector) + { + List monitors = Lists.newArrayList(); + + for (Key key: injector.getBindings().keySet()) { + if (Monitor.class.isAssignableFrom(key.getClass())) { + final Monitor monitor = (Monitor) injector.getInstance(key); + + log.info("Adding monitor[%s]", monitor); + + monitors.add(monitor); + } + } + + return new MonitorScheduler( + config, + Execs.scheduledSingleThreaded("MonitorScheduler-%s"), + emitter, + monitors + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java index 7db7e3ba922..9898d85a76e 100644 --- a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java +++ b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java @@ -19,7 +19,6 @@ package com.metamx.druid.metrics; -import com.metamx.druid.client.DruidServer; import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.coordination.ServerManager; import com.metamx.emitter.service.ServiceEmitter; diff --git a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java index 751cad29a13..223e55faa9d 100644 --- a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java +++ b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java @@ -19,10 +19,10 @@ package com.metamx.druid.db; +import com.google.common.base.Suppliers; import com.google.common.collect.Maps; import com.metamx.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; -import org.joda.time.Duration; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -32,7 +32,6 @@ import org.skife.jdbi.v2.tweak.HandleCallback; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; /** */ @@ -48,21 +47,8 @@ public class DatabaseSegmentManagerTest dbi = EasyMock.createMock(DBI.class); manager = new DatabaseSegmentManager( new DefaultObjectMapper(), - EasyMock.createMock(ScheduledExecutorService.class), - new DatabaseSegmentManagerConfig() - { - @Override - public String getSegmentTable() - { - return null; - } - - @Override - public Duration getPollDuration() - { - return null; - } - }, + Suppliers.ofInstance(new DatabaseSegmentManagerConfig()), + Suppliers.ofInstance(DbTablesConfig.fromBase("test")), dbi ); From f68df7ab692e09e7a878be02fcadeaaa4aafd35d Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 7 Jun 2013 12:01:46 -0700 Subject: [PATCH 03/92] 1) Make tests work and continue trying to make the DruidMaster start up with just Guice --- .../indexing/IndexingServiceClient.java | 2 + .../com/metamx/druid/guice/HttpModule.java | 40 +++++++++++++++ .../metamx/druid/indexer/DbUpdaterJob.java | 9 ++-- .../indexer/HadoopDruidIndexerConfig.java | 11 ++-- .../indexer/updater/DbUpdaterJobSpec.java | 46 ++++++++++------- .../druid/indexer/updater/UpdaterJobSpec.java | 34 ------------- .../indexer/updater/ZkUpdaterJobSpec.java | 50 ------------------- .../indexer/HadoopDruidIndexerConfigTest.java | 14 +++--- .../indexing/common/task/HadoopIndexTask.java | 4 +- .../coordinator/MergerDBCoordinator.java | 14 ++++-- .../coordinator/TaskMasterLifecycle.java | 14 +++--- .../http/IndexerCoordinatorNode.java | 14 +++--- .../worker/executor/ExecutorNode.java | 22 ++++---- .../coordinator/TaskLifecycleTest.java | 16 +++--- .../metamx/druid/realtime/RealtimeNode.java | 3 +- .../druid/db/DatabaseSegmentManager.java | 2 +- .../db/DatabaseSegmentManagerConfig.java | 6 +-- .../com/metamx/druid/guice/MasterModule.java | 9 ++++ .../com/metamx/druid/http/MasterMain.java | 5 +- .../com/metamx/druid/http/RedirectFilter.java | 4 -- 20 files changed, 143 insertions(+), 176 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/guice/HttpModule.java delete mode 100644 indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java delete mode 100644 indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java 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 index ea17b6b6838..f4be29681c3 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -21,6 +21,7 @@ package com.metamx.druid.client.indexing; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; +import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.druid.client.DataSegment; @@ -43,6 +44,7 @@ public class IndexingServiceClient private final ObjectMapper jsonMapper; private final ServiceProvider serviceProvider; + @Inject public IndexingServiceClient( HttpClient client, ObjectMapper jsonMapper, diff --git a/common/src/main/java/com/metamx/druid/guice/HttpModule.java b/common/src/main/java/com/metamx/druid/guice/HttpModule.java new file mode 100644 index 00000000000..214acdf28cf --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/HttpModule.java @@ -0,0 +1,40 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import sun.net.www.http.HttpClient; + +import javax.validation.constraints.Min; + +/** + */ +public class HttpModule implements Module +{ + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.global.http", DruidHttpClientConfig.class); + } + + public abstract static class DruidHttpClientConfig + { + @JsonProperty + @Min(0) + private int numConnections = 5; + + public int getNumConnections() + { + return numConnections; + } + } + + @Provides @LazySingleton @ManageLifecycle + public HttpClient makeHttpClient(DruidHttpClientConfig config) + { + return null; // TODO + } + +} diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 3025df982e8..7e2efe33bb3 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnector; -import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; import org.joda.time.DateTime; import org.skife.jdbi.v2.DBI; @@ -43,7 +42,6 @@ public class DbUpdaterJob implements Jobby private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); private final HadoopDruidIndexerConfig config; - private final DbUpdaterJobSpec spec; private final DBI dbi; public DbUpdaterJob( @@ -51,8 +49,7 @@ public class DbUpdaterJob implements Jobby ) { this.config = config; - this.spec = (DbUpdaterJobSpec) config.getUpdaterJobSpec(); - this.dbi = new DbConnector(spec).getDBI(); + this.dbi = new DbConnector(config.getUpdaterJobSpec(), null).getDBI(); } @Override @@ -70,13 +67,13 @@ public class DbUpdaterJob implements Jobby String.format( "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - spec.getSegmentTable() + config.getUpdaterJobSpec().getSegmentTable() ) ); for (final DataSegment segment : segments) { batch.add( - new ImmutableMap.Builder() + new ImmutableMap.Builder() .put("id", segment.getIdentifier()) .put("dataSource", segment.getDataSource()) .put("created_date", new DateTime().toString()) diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 6d0ea322313..290a64fa471 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -51,7 +51,7 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.path.PathSpec; import com.metamx.druid.indexer.rollup.DataRollupSpec; -import com.metamx.druid.indexer.updater.UpdaterJobSpec; +import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.input.InputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.ShardSpec; @@ -59,7 +59,6 @@ import com.metamx.druid.utils.JodaUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.Job; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -178,7 +177,7 @@ public class HadoopDruidIndexerConfig private volatile Map> shardSpecs = ImmutableMap.of(); private volatile boolean overwriteFiles = false; private volatile DataRollupSpec rollupSpec; - private volatile UpdaterJobSpec updaterJobSpec; + private volatile DbUpdaterJobSpec updaterJobSpec; private volatile boolean ignoreInvalidRows = false; private volatile List registererers = Lists.newArrayList(); @@ -203,7 +202,7 @@ public class HadoopDruidIndexerConfig final @JsonProperty("shardSpecs") Map> shardSpecs, final @JsonProperty("overwriteFiles") boolean overwriteFiles, final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec, - final @JsonProperty("updaterJobSpec") UpdaterJobSpec updaterJobSpec, + final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec, final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows, final @JsonProperty("registererers") List registererers ) @@ -497,12 +496,12 @@ public class HadoopDruidIndexerConfig } @JsonProperty - public UpdaterJobSpec getUpdaterJobSpec() + public DbUpdaterJobSpec getUpdaterJobSpec() { return updaterJobSpec; } - public void setUpdaterJobSpec(UpdaterJobSpec updaterJobSpec) + public void setUpdaterJobSpec(DbUpdaterJobSpec updaterJobSpec) { this.updaterJobSpec = updaterJobSpec; } diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java index 539c6860573..3d0a85c310b 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java @@ -20,11 +20,12 @@ package com.metamx.druid.indexer.updater; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Supplier; import com.metamx.druid.db.DbConnectorConfig; /** */ -public class DbUpdaterJobSpec extends DbConnectorConfig implements UpdaterJobSpec +public class DbUpdaterJobSpec implements Supplier { @JsonProperty("connectURI") public String connectURI; @@ -38,26 +39,33 @@ public class DbUpdaterJobSpec extends DbConnectorConfig implements UpdaterJobSpe @JsonProperty("segmentTable") public String segmentTable; - @Override - public String getDatabaseConnectURI() - { - return connectURI; - } - - @Override - public String getDatabaseUser() - { - return user; - } - - @Override - public String getDatabasePassword() - { - return password; - } - public String getSegmentTable() { return segmentTable; } + + @Override + public DbConnectorConfig get() + { + return new DbConnectorConfig() + { + @Override + public String getConnectURI() + { + return connectURI; + } + + @Override + public String getUser() + { + return user; + } + + @Override + public String getPassword() + { + return password; + } + }; + } } diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java deleted file mode 100644 index f4294869152..00000000000 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java +++ /dev/null @@ -1,34 +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.indexer.updater; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "zk", value = ZkUpdaterJobSpec.class), - @JsonSubTypes.Type(name = "db", value = DbUpdaterJobSpec.class) -}) -public interface UpdaterJobSpec -{ -} diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java deleted file mode 100644 index b79fab21747..00000000000 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java +++ /dev/null @@ -1,50 +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.indexer.updater; - -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - */ -public class ZkUpdaterJobSpec implements UpdaterJobSpec -{ - @JsonProperty("zkHosts") - public String zkQuorum; - - @JsonProperty("zkBasePath") - private String zkBasePath; - - public ZkUpdaterJobSpec() {} - - public String getZkQuorum() - { - return zkQuorum; - } - - public String getZkBasePath() - { - return zkBasePath; - } - - public boolean postToZk() - { - return !(zkQuorum == null || zkBasePath == null); - } -} diff --git a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index dcabe168e67..dc7c2df196c 100644 --- a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -22,11 +22,11 @@ package com.metamx.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; - import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -367,12 +367,14 @@ public class HadoopDruidIndexerConfigTest HadoopDruidIndexerConfig.class ); - final DbUpdaterJobSpec spec = (DbUpdaterJobSpec) cfg.getUpdaterJobSpec(); + final DbUpdaterJobSpec spec = cfg.getUpdaterJobSpec(); + final DbConnectorConfig connectorConfig = spec.get(); + Assert.assertEquals("segments", spec.getSegmentTable()); - Assert.assertEquals("jdbc:mysql://localhost/druid", spec.getDatabaseConnectURI()); - Assert.assertEquals("rofl", spec.getDatabaseUser()); - Assert.assertEquals("p4ssw0rd", spec.getDatabasePassword()); - Assert.assertEquals(false, spec.useValidationQuery()); + Assert.assertEquals("jdbc:mysql://localhost/druid", connectorConfig.getConnectURI()); + Assert.assertEquals("rofl", connectorConfig.getUser()); + Assert.assertEquals("p4ssw0rd", connectorConfig.getPassword()); + Assert.assertEquals(false, connectorConfig.isUseValidationQuery()); } @Test diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java index 0ce0b4c382c..b3c5f5b8a4e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java @@ -29,12 +29,12 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import com.metamx.druid.indexer.HadoopDruidIndexerJob; -import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.LockListAction; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.utils.JodaUtils; import org.joda.time.DateTime; @@ -51,7 +51,7 @@ public class HadoopIndexTask extends AbstractTask * @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters * for creating Druid index segments. It may be modified. *

- * Here, we will ensure that the UpdaterJobSpec field of the config is set to null, such that the + * Here, we will ensure that the DbConnectorConfig field of the config is set to null, such that the * job does not push a list of published segments the database. Instead, we will use the method * IndexGeneratorJob.getPublishedSegments() to simply return a list of the published * segments, and let the indexing service report these segments to the database. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java index 1b327f40a13..ea258269c84 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java @@ -31,6 +31,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; import org.joda.time.DateTime; import org.joda.time.Interval; import org.skife.jdbi.v2.DBI; @@ -57,16 +58,19 @@ public class MergerDBCoordinator private final ObjectMapper jsonMapper; private final DbConnectorConfig dbConnectorConfig; + private final DbTablesConfig dbTables; private final DBI dbi; public MergerDBCoordinator( ObjectMapper jsonMapper, DbConnectorConfig dbConnectorConfig, + DbTablesConfig dbTables, DBI dbi ) { this.jsonMapper = jsonMapper; this.dbConnectorConfig = dbConnectorConfig; + this.dbTables = dbTables; this.dbi = dbi; } @@ -87,7 +91,7 @@ public class MergerDBCoordinator handle.createQuery( String.format( "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", - dbConnectorConfig.getSegmentTable() + dbTables.getSegmentsTable() ) ) .bind("dataSource", dataSource) @@ -170,7 +174,7 @@ public class MergerDBCoordinator final List> exists = handle.createQuery( String.format( "SELECT id FROM %s WHERE id = :identifier", - dbConnectorConfig.getSegmentTable() + dbTables.getSegmentsTable() ) ).bind( "identifier", @@ -185,7 +189,7 @@ public class MergerDBCoordinator handle.createStatement( String.format( "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - dbConnectorConfig.getSegmentTable() + dbTables.getSegmentsTable() ) ) .bind("id", segment.getIdentifier()) @@ -230,7 +234,7 @@ public class MergerDBCoordinator private void deleteSegment(final Handle handle, final DataSegment segment) { handle.createStatement( - String.format("DELETE from %s WHERE id = :id", dbConnectorConfig.getSegmentTable()) + String.format("DELETE from %s WHERE id = :id", dbTables.getSegmentsTable()) ).bind("id", segment.getIdentifier()) .execute(); } @@ -246,7 +250,7 @@ public class MergerDBCoordinator return handle.createQuery( String.format( "SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0", - dbConnectorConfig.getSegmentTable() + dbTables.getSegmentsTable() ) ) .bind("dataSource", dataSource) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index 8ffc2b521bd..26a7c5210d3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -25,15 +25,15 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.merger.common.actions.TaskActionClient; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -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; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index d0d19f7552a..5510a3c19c8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -24,7 +24,6 @@ import com.amazonaws.services.ec2.AmazonEC2Client; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; @@ -56,11 +55,6 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNodeConfig; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; @@ -101,6 +95,11 @@ import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagmentConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -108,7 +107,6 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.http.client.HttpClient; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; -import com.metamx.http.client.response.ToStringResponseHandler; import com.metamx.metrics.JvmMonitor; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; @@ -319,7 +317,6 @@ public class IndexerCoordinatorNode extends QueryableNode segmentPublisher = new DbSegmentPublisher( getJsonMapper(), dbSegmentPublisherConfig, - new DbConnector(getConfigFactory().build(DbConnectorConfig.class)).getDBI() + new DbConnector(Suppliers.ofInstance(getConfigFactory().build(DbConnectorConfig.class)), null).getDBI() // TODO ); getLifecycle().addManagedInstance(segmentPublisher); } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index fce933e6294..b51976f79e3 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -106,7 +106,7 @@ public class DatabaseSegmentManager ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), - config.get().getPollDuration(), + config.get().getPollDuration().toStandardDuration(), new Runnable() { @Override diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java index 85532089243..1541f00cca3 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerConfig.java @@ -20,16 +20,16 @@ package com.metamx.druid.db; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Duration; +import org.joda.time.Period; /** */ public class DatabaseSegmentManagerConfig { @JsonProperty - private Duration pollDuration = new Duration("PT1M"); + private Period pollDuration = new Period("PT1M"); - public Duration getPollDuration() + public Period getPollDuration() { return pollDuration; } diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index 39e6fe10e8c..0d69a86d438 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -5,10 +5,12 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.http.client.HttpClient; import org.skife.jdbi.v2.DBI; /** @@ -26,6 +28,13 @@ public class MasterModule implements Module } + @Provides + public IndexingServiceClient getIndexingServiceClient(HttpClient client) + { + // TODO + return null; + } + @Provides @LazySingleton public DBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) { 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 71b34d5c199..74f7e52b465 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -20,7 +20,6 @@ package com.metamx.druid.http; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; @@ -66,7 +65,6 @@ import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; import com.metamx.druid.metrics.MetricsModule; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.response.ToStringResponseHandler; import com.metamx.metrics.MonitorScheduler; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; @@ -131,6 +129,7 @@ public class MasterMain Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); ServiceProvider serviceProvider = null; + IndexingServiceClient indexingServiceClient = null; if (druidMasterConfig.getMergerServiceName() != null) { serviceProvider = Initialization.makeServiceProvider( druidMasterConfig.getMergerServiceName(), @@ -139,7 +138,6 @@ public class MasterMain ); // indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); TODO } - IndexingServiceClient indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); DBI dbi = injector.getInstance(DBI.class); final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); @@ -246,7 +244,6 @@ public class MasterMain root.addFilter( new FilterHolder( new RedirectFilter( - new ToStringResponseHandler(Charsets.UTF_8), redirectInfo ) ), "/*", 0 diff --git a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java index f6f4c30e7cf..9a1a3c66bd3 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java @@ -20,7 +20,6 @@ package com.metamx.druid.http; import com.metamx.common.logger.Logger; -import com.metamx.http.client.response.HttpResponseHandler; import javax.servlet.Filter; import javax.servlet.FilterChain; @@ -39,15 +38,12 @@ public class RedirectFilter implements Filter { private static final Logger log = new Logger(RedirectFilter.class); - private final HttpResponseHandler responseHandler; private final RedirectInfo redirectInfo; public RedirectFilter( - HttpResponseHandler responseHandler, RedirectInfo redirectInfo ) { - this.responseHandler = responseHandler; this.redirectInfo = redirectInfo; } From 2f56c24259b0ea7743429956f0e90cdc8c067f79 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 7 Jun 2013 17:37:33 -0700 Subject: [PATCH 04/92] 1) Inject IndexingServiceClient 2) Switch all the DBI references to IDBI --- .../client/indexing/IndexingService.java | 17 +++++ .../indexing/IndexingServiceClient.java | 15 ++-- .../client/selector/DiscoverySelector.java | 8 +++ .../metamx/druid/client/selector/Server.java | 9 +++ .../druid/client/selector/ServerSelector.java | 3 +- .../druid/initialization/EmitterModule.java | 3 +- .../metamx/druid/config/ConfigManager.java | 4 +- .../druid/config/JacksonConfigManager.java | 2 + .../java/com/metamx/druid/db/DbConnector.java | 15 ++-- .../com/metamx/druid/guice/HttpModule.java | 40 ----------- .../metamx/druid/guice/LifecycleScope.java | 2 +- .../druid/guice/annotations/Global.java | 17 +++++ .../metamx/druid/indexer/DbUpdaterJob.java | 4 +- .../indexing/coordinator/DbTaskStorage.java | 9 ++- .../coordinator/MergerDBCoordinator.java | 6 +- .../http/IndexerCoordinatorNode.java | 4 +- .../druid/realtime/DbSegmentPublisher.java | 6 +- .../indexing/IndexingServiceSelector.java | 68 +++++++++++++++++++ .../metamx/druid/db/DatabaseRuleManager.java | 8 +-- .../druid/db/DatabaseSegmentManager.java | 6 +- .../metamx/druid/guice/HttpClientModule.java | 65 ++++++++++++++++++ .../com/metamx/druid/guice/MasterModule.java | 29 ++++++-- .../com/metamx/druid/http/MasterMain.java | 47 ++++--------- .../druid/db/DatabaseSegmentManagerTest.java | 6 +- 24 files changed, 268 insertions(+), 125 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java create mode 100644 client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java create mode 100644 client/src/main/java/com/metamx/druid/client/selector/Server.java delete mode 100644 common/src/main/java/com/metamx/druid/guice/HttpModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/annotations/Global.java create mode 100644 server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java create mode 100644 server/src/main/java/com/metamx/druid/guice/HttpClientModule.java diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java new file mode 100644 index 00000000000..4198703ee63 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java @@ -0,0 +1,17 @@ +package com.metamx.druid.client.indexing; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface IndexingService +{ +} 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 index f4be29681c3..e3de082f7fc 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -25,10 +25,11 @@ import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.selector.DiscoverySelector; +import com.metamx.druid.client.selector.Server; +import com.metamx.druid.guice.annotations.Global; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; -import org.apache.curator.x.discovery.ServiceInstance; -import org.apache.curator.x.discovery.ServiceProvider; import org.joda.time.Interval; import java.io.InputStream; @@ -42,13 +43,13 @@ public class IndexingServiceClient private final HttpClient client; private final ObjectMapper jsonMapper; - private final ServiceProvider serviceProvider; + private final DiscoverySelector serviceProvider; @Inject public IndexingServiceClient( - HttpClient client, + @Global HttpClient client, ObjectMapper jsonMapper, - ServiceProvider serviceProvider + @IndexingService DiscoverySelector serviceProvider ) { this.client = client; @@ -105,12 +106,12 @@ public class IndexingServiceClient private String baseUrl() { try { - final ServiceInstance instance = serviceProvider.getInstance(); + final Server instance = serviceProvider.pick(); if (instance == null) { throw new ISE("Cannot find instance of indexingService"); } - return String.format("http://%s:%s/druid/indexer/v1", instance.getAddress(), instance.getPort()); + return String.format("http://%s:%s/druid/indexer/v1", instance.getHost(), instance.getPort()); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java b/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java new file mode 100644 index 00000000000..1b55472e431 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java @@ -0,0 +1,8 @@ +package com.metamx.druid.client.selector; + +/** + */ +public interface DiscoverySelector +{ + public T pick(); +} diff --git a/client/src/main/java/com/metamx/druid/client/selector/Server.java b/client/src/main/java/com/metamx/druid/client/selector/Server.java new file mode 100644 index 00000000000..9859c888239 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/selector/Server.java @@ -0,0 +1,9 @@ +package com.metamx.druid.client.selector; + +/** + */ +public interface Server +{ + public String getHost(); + public int getPort(); +} diff --git a/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java b/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java index 388761aaaa2..d4876e5680f 100644 --- a/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java +++ b/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java @@ -22,7 +22,6 @@ package com.metamx.druid.client.selector; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; import java.util.Collections; import java.util.Comparator; @@ -30,7 +29,7 @@ import java.util.Set; /** */ -public class ServerSelector +public class ServerSelector implements DiscoverySelector { private static final Comparator comparator = new Comparator() { diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java index edc01e65a26..408358c2283 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -32,7 +32,6 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.DruidScopes; import com.metamx.druid.guice.LazySingleton; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitter; @@ -66,7 +65,7 @@ public class EmitterModule implements Module binder.install(new LogEmitterModule()); binder.install(new HttpEmitterModule()); - binder.bind(Emitter.class).toProvider(new EmitterProvider(emitterType)).in(DruidScopes.SINGLETON); + binder.bind(Emitter.class).toProvider(new EmitterProvider(emitterType)).in(LazySingleton.class); } @Provides diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java index 4aa97e7ffc7..14b504a60ec 100644 --- a/common/src/main/java/com/metamx/druid/config/ConfigManager.java +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -2,6 +2,7 @@ package com.metamx.druid.config; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -42,7 +43,8 @@ public class ConfigManager private volatile ConfigManager.PollingCallable poller; - public ConfigManager(IDBI dbi, ConfigManagerConfig config) + @Inject + public ConfigManager(IDBI dbi, ConfigManagerConfig config) // TODO: use DbTables and a different config { this.dbi = dbi; this.config = config; diff --git a/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java index 8e322f3ee80..13ed516fe62 100644 --- a/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java +++ b/common/src/main/java/com/metamx/druid/config/JacksonConfigManager.java @@ -23,6 +23,7 @@ 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 com.google.inject.Inject; import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; @@ -34,6 +35,7 @@ public class JacksonConfigManager private final ConfigManager configManager; private final ObjectMapper jsonMapper; + @Inject public JacksonConfigManager( ConfigManager configManager, ObjectMapper jsonMapper 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 d6b0b3e47f2..1cfc9904b09 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -24,6 +24,7 @@ import com.metamx.common.logger.Logger; import org.apache.commons.dbcp.BasicDataSource; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.tweak.HandleCallback; import javax.sql.DataSource; @@ -36,7 +37,7 @@ public class DbConnector { private static final Logger log = new Logger(DbConnector.class); - public static void createSegmentTable(final DBI dbi, final String segmentTableName) + public static void createSegmentTable(final IDBI dbi, final String segmentTableName) { createTable( dbi, @@ -48,7 +49,7 @@ public class DbConnector ); } - public static void createRuleTable(final DBI dbi, final String ruleTableName) + public static void createRuleTable(final IDBI dbi, final String ruleTableName) { createTable( dbi, @@ -60,7 +61,7 @@ public class DbConnector ); } - public static void createConfigTable(final DBI dbi, final String configTableName) + public static void createConfigTable(final IDBI dbi, final String configTableName) { createTable( dbi, @@ -72,7 +73,7 @@ public class DbConnector ); } - public static void createTaskTable(final DBI dbi, final String taskTableName) + public static void createTaskTable(final IDBI dbi, final String taskTableName) { createTable( dbi, @@ -92,7 +93,7 @@ public class DbConnector ); } - public static void createTaskLogTable(final DBI dbi, final String taskLogsTableName) + public static void createTaskLogTable(final IDBI dbi, final String taskLogsTableName) { createTable( dbi, @@ -110,7 +111,7 @@ public class DbConnector ); } - public static void createTaskLockTable(final DBI dbi, final String taskLocksTableName) + public static void createTaskLockTable(final IDBI dbi, final String taskLocksTableName) { createTable( dbi, @@ -129,7 +130,7 @@ public class DbConnector } public static void createTable( - final DBI dbi, + final IDBI dbi, final String tableName, final String sql ) diff --git a/common/src/main/java/com/metamx/druid/guice/HttpModule.java b/common/src/main/java/com/metamx/druid/guice/HttpModule.java deleted file mode 100644 index 214acdf28cf..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/HttpModule.java +++ /dev/null @@ -1,40 +0,0 @@ -package com.metamx.druid.guice; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Provides; -import sun.net.www.http.HttpClient; - -import javax.validation.constraints.Min; - -/** - */ -public class HttpModule implements Module -{ - - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.global.http", DruidHttpClientConfig.class); - } - - public abstract static class DruidHttpClientConfig - { - @JsonProperty - @Min(0) - private int numConnections = 5; - - public int getNumConnections() - { - return numConnections; - } - } - - @Provides @LazySingleton @ManageLifecycle - public HttpClient makeHttpClient(DruidHttpClientConfig config) - { - return null; // TODO - } - -} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java index 223113728e3..f9d8afad575 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java @@ -34,7 +34,7 @@ public class LifecycleScope implements Scope { return new Provider() { - private T value = null; + private volatile T value = null; @Override public synchronized T get() diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Global.java b/common/src/main/java/com/metamx/druid/guice/annotations/Global.java new file mode 100644 index 00000000000..497e1d17c6c --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/annotations/Global.java @@ -0,0 +1,17 @@ +package com.metamx.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Global +{ +} diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java index 7e2efe33bb3..5140fee9180 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java @@ -26,8 +26,8 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DbConnector; import com.metamx.druid.jackson.DefaultObjectMapper; import org.joda.time.DateTime; -import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.PreparedBatch; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -42,7 +42,7 @@ public class DbUpdaterJob implements Jobby private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); private final HadoopDruidIndexerConfig config; - private final DBI dbi; + private final IDBI dbi; public DbUpdaterJob( HadoopDruidIndexerConfig config diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java index a647770b33b..a9c4ad7c60c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java @@ -28,16 +28,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.actions.TaskAction; -import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; import com.metamx.emitter.EmittingLogger; - import org.joda.time.DateTime; -import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.exceptions.StatementException; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -48,11 +47,11 @@ public class DbTaskStorage implements TaskStorage { private final ObjectMapper jsonMapper; private final IndexerDbConnectorConfig dbConnectorConfig; - private final DBI dbi; + private final IDBI dbi; private static final EmittingLogger log = new EmittingLogger(DbTaskStorage.class); - public DbTaskStorage(ObjectMapper jsonMapper, IndexerDbConnectorConfig dbConnectorConfig, DBI dbi) + public DbTaskStorage(ObjectMapper jsonMapper, IndexerDbConnectorConfig dbConnectorConfig, IDBI dbi) { this.jsonMapper = jsonMapper; this.dbConnectorConfig = dbConnectorConfig; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java index ea258269c84..3f893d79cdd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java @@ -34,10 +34,10 @@ import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; import org.joda.time.DateTime; import org.joda.time.Interval; -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.IDBI; import org.skife.jdbi.v2.ResultIterator; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.TransactionCallback; @@ -59,13 +59,13 @@ public class MergerDBCoordinator private final ObjectMapper jsonMapper; private final DbConnectorConfig dbConnectorConfig; private final DbTablesConfig dbTables; - private final DBI dbi; + private final IDBI dbi; public MergerDBCoordinator( ObjectMapper jsonMapper, DbConnectorConfig dbConnectorConfig, DbTablesConfig dbTables, - DBI dbi + IDBI dbi ) { this.jsonMapper = jsonMapper; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 5510a3c19c8..c98497a3602 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -126,7 +126,7 @@ import org.mortbay.jetty.servlet.FilterHolder; import org.mortbay.jetty.servlet.ServletHolder; import org.mortbay.resource.ResourceCollection; import org.skife.config.ConfigurationObjectFactory; -import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.IDBI; import java.io.IOException; import java.io.InputStream; @@ -153,7 +153,7 @@ public class IndexerCoordinatorNode extends QueryableNode monitors = null; private ServiceEmitter emitter = null; private DbConnectorConfig dbConnectorConfig = null; - private DBI dbi = null; + private IDBI dbi = null; private IndexerCoordinatorConfig config = null; private MergerDBCoordinator mergerDBCoordinator = null; private ServiceDiscovery serviceDiscovery = null; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java index 7a7e0e8ed7f..660e4bc7599 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java @@ -4,8 +4,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import org.joda.time.DateTime; -import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.tweak.HandleCallback; import java.io.IOException; @@ -18,12 +18,12 @@ public class DbSegmentPublisher implements SegmentPublisher private final ObjectMapper jsonMapper; private final DbSegmentPublisherConfig config; - private final DBI dbi; + private final IDBI dbi; public DbSegmentPublisher( ObjectMapper jsonMapper, DbSegmentPublisherConfig config, - DBI dbi + IDBI dbi ) { this.jsonMapper = jsonMapper; diff --git a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java new file mode 100644 index 00000000000..d6d07dcafe8 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java @@ -0,0 +1,68 @@ +package com.metamx.druid.client.indexing; + +import com.google.inject.Inject; +import com.metamx.common.lifecycle.LifecycleStart; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.selector.DiscoverySelector; +import com.metamx.druid.client.selector.Server; +import org.apache.curator.x.discovery.ServiceInstance; +import org.apache.curator.x.discovery.ServiceProvider; + +import java.io.IOException; + +/** +*/ +public class IndexingServiceSelector implements DiscoverySelector +{ + private static final Logger log = new Logger(IndexingServiceSelector.class); + + private final ServiceProvider serviceProvider; + + @Inject + public IndexingServiceSelector( + @IndexingService ServiceProvider serviceProvider + ) { + this.serviceProvider = serviceProvider; + } + + @Override + public Server pick() + { + final ServiceInstance instance; + try { + instance = serviceProvider.getInstance(); + } + catch (Exception e) { + log.info(e, ""); + return null; + } + + return new Server() + { + @Override + public String getHost() + { + return instance.getAddress(); + } + + @Override + public int getPort() + { + return instance.getPort(); + } + }; + } + + @LifecycleStart + public void start() throws Exception + { + serviceProvider.start(); + } + + @LifecycleStop + public void stop() throws IOException + { + serviceProvider.close(); + } +} diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 4d51df1a8d5..5e8550b0165 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -38,10 +38,10 @@ import com.metamx.druid.master.rules.Rule; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; -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.IDBI; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -59,7 +59,7 @@ import java.util.concurrent.atomic.AtomicReference; public class DatabaseRuleManager { public static void createDefaultRule( - final DBI dbi, + final IDBI dbi, final String ruleTable, final String defaultTier, final ObjectMapper jsonMapper @@ -120,7 +120,7 @@ public class DatabaseRuleManager private final ScheduledExecutorService exec; private final DatabaseRuleManagerConfig config; private final Supplier dbTables; - private final DBI dbi; + private final IDBI dbi; private final AtomicReference>> rules; private final Object lock = new Object(); @@ -132,7 +132,7 @@ public class DatabaseRuleManager ObjectMapper jsonMapper, DatabaseRuleManagerConfig config, Supplier dbTables, - DBI dbi + IDBI dbi ) { this.jsonMapper = jsonMapper; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index b51976f79e3..7b34febefe5 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -41,10 +41,10 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.skife.jdbi.v2.Batch; -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.IDBI; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; @@ -72,7 +72,7 @@ public class DatabaseSegmentManager private final Supplier config; private final Supplier dbTables; private final AtomicReference> dataSources; - private final DBI dbi; + private final IDBI dbi; private volatile boolean started = false; @@ -81,7 +81,7 @@ public class DatabaseSegmentManager ObjectMapper jsonMapper, Supplier config, Supplier dbTables, - DBI dbi + IDBI dbi ) { this.jsonMapper = jsonMapper; diff --git a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java new file mode 100644 index 00000000000..00ea9d01a7f --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java @@ -0,0 +1,65 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.annotations.Global; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import javax.net.ssl.SSLContext; +import javax.validation.constraints.Min; + +/** + */ +public class HttpClientModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.global.http", DruidHttpClientConfig.class); + } + + public abstract static class DruidHttpClientConfig + { + @JsonProperty + @Min(0) + private int numConnections = 5; + + @JsonProperty + private Period readTimeout = null; + + public int getNumConnections() + { + return numConnections; + } + + public Duration getReadTimeout() + { + return readTimeout.toStandardDuration(); + } + } + + @Provides @LazySingleton @Global + public HttpClient makeHttpClient(DruidHttpClientConfig config, Lifecycle lifecycle, @Nullable SSLContext sslContext) + { + final HttpClientConfig.Builder builder = HttpClientConfig + .builder() + .withNumConnections(config.getNumConnections()) + .withReadTimeout(config.getReadTimeout()); + + if (sslContext != null) { + builder.withSslContext(sslContext); + } + + return HttpClientInit.createClient(builder.build(), lifecycle); + } + + +} diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index 0d69a86d438..60d6ccbad6d 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -3,15 +3,22 @@ package com.metamx.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.TypeLiteral; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceClient; +import com.metamx.druid.client.indexing.IndexingServiceSelector; +import com.metamx.druid.client.selector.DiscoverySelector; +import com.metamx.druid.client.selector.Server; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.initialization.ZkPathsConfig; -import com.metamx.http.client.HttpClient; -import org.skife.jdbi.v2.DBI; +import com.metamx.druid.master.DruidMasterConfig; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceProvider; +import org.skife.jdbi.v2.IDBI; /** */ @@ -26,17 +33,25 @@ public class MasterModule implements Module JsonConfigProvider.bind(binder, "druid.database.tables", DbTablesConfig.class); + binder.bind(new TypeLiteral>(){}) + .annotatedWith(IndexingService.class) + .to(IndexingServiceSelector.class) + .in(ManageLifecycle.class); + binder.bind(IndexingServiceClient.class).in(LazySingleton.class); } - @Provides - public IndexingServiceClient getIndexingServiceClient(HttpClient client) + @Provides @ManageLifecycle @IndexingService + public DiscoverySelector getIndexingServiceSelector(DruidMasterConfig config, ServiceDiscovery serviceDiscovery) { - // TODO - return null; + final ServiceProvider serviceProvider = serviceDiscovery.serviceProviderBuilder() + .serviceName(config.getMergerServiceName()) + .build(); + + return new IndexingServiceSelector(serviceProvider); } @Provides @LazySingleton - public DBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) + public IDBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) { if (config.isCreateTables()) { lifecycle.addHandler( 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 74f7e52b465..12c2a87dc7f 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -22,12 +22,11 @@ package com.metamx.druid.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.TypeLiteral; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -48,6 +47,7 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DbConnector; import com.metamx.druid.guice.DruidGuiceExtensions; import com.metamx.druid.guice.DruidSecondaryModule; +import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; @@ -67,8 +67,6 @@ import com.metamx.druid.metrics.MetricsModule; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceProvider; import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.DefaultServlet; @@ -76,11 +74,12 @@ import org.mortbay.jetty.servlet.FilterHolder; import org.mortbay.jetty.servlet.ServletHolder; import org.mortbay.servlet.GzipFilter; import org.skife.config.ConfigurationObjectFactory; -import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.IDBI; import javax.annotation.Nullable; import java.net.URL; import java.util.Arrays; +import java.util.List; /** */ @@ -93,9 +92,9 @@ public class MasterMain LogLevelAdjuster.register(); Injector injector = makeInjector( - DruidSecondaryModule.class, new LifecycleModule(Key.get(MonitorScheduler.class)), EmitterModule.class, + HttpClientModule.class, CuratorModule.class, MetricsModule.class, DiscoveryModule.class, @@ -124,22 +123,10 @@ public class MasterMain final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class); final DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); - final ServiceDiscovery serviceDiscovery = injector.getInstance(Key.get(new TypeLiteral>(){})); final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); - ServiceProvider serviceProvider = null; - IndexingServiceClient indexingServiceClient = null; - if (druidMasterConfig.getMergerServiceName() != null) { - serviceProvider = Initialization.makeServiceProvider( - druidMasterConfig.getMergerServiceName(), - serviceDiscovery, - lifecycle - ); -// indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider); TODO - } - - DBI dbi = injector.getInstance(DBI.class); + IDBI dbi = injector.getInstance(IDBI.class); // TODO: make tables and stuff final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); JacksonConfigManager configManager = new JacksonConfigManager( @@ -160,7 +147,7 @@ public class MasterMain curatorFramework, emitter, scheduledExecutorFactory, - indexingServiceClient, + injector.getInstance(IndexingServiceClient.class), taskMaster ); lifecycle.addManagedInstance(master); @@ -187,17 +174,6 @@ public class MasterMain ) ); - final Injector injector2 = Guice.createInjector( - new MasterServletModule( - serverInventoryView, - databaseSegmentManager, - databaseRuleManager, - master, - jsonMapper, - indexingServiceClient - ) - ); - final Server server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class)); final RedirectInfo redirectInfo = new RedirectInfo() @@ -264,9 +240,14 @@ public class MasterMain new ConfigFactoryModule() ); + List actualModules = Lists.newArrayList(); + + actualModules.add(DruidSecondaryModule.class); + actualModules.addAll(Arrays.asList(modules)); + return Guice.createInjector( - Iterables.transform( - Arrays.asList(modules), + Lists.transform( + actualModules, new Function() { @Override diff --git a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java index 223e55faa9d..0ccd2401c73 100644 --- a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java +++ b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java @@ -26,7 +26,7 @@ import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.tweak.HandleCallback; import java.util.Arrays; @@ -38,13 +38,13 @@ import java.util.Map; public class DatabaseSegmentManagerTest { private DatabaseSegmentManager manager; - private DBI dbi; + private IDBI dbi; private List> testRows; @Before public void setUp() throws Exception { - dbi = EasyMock.createMock(DBI.class); + dbi = EasyMock.createMock(IDBI.class); manager = new DatabaseSegmentManager( new DefaultObjectMapper(), Suppliers.ofInstance(new DatabaseSegmentManagerConfig()), From 06f7e7e665b6a9ba9f064d95ef1178da93a9165b Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 10 Jun 2013 14:13:28 -0700 Subject: [PATCH 05/92] 1) Fix DatabaseSegmentManager so that Unit Test passes --- .../java/com/metamx/druid/db/DatabaseSegmentManager.java | 6 ++++-- .../com/metamx/druid/db/DatabaseSegmentManagerTest.java | 6 ++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index 7b34febefe5..b5beabc9988 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -103,10 +103,11 @@ public class DatabaseSegmentManager return; } + final Duration delay = config.get().getPollDuration().toStandardDuration(); ScheduledExecutors.scheduleWithFixedDelay( exec, - new Duration(0), - config.get().getPollDuration().toStandardDuration(), + delay, + delay, new Runnable() { @Override @@ -131,6 +132,7 @@ public class DatabaseSegmentManager started = false; dataSources.set(new ConcurrentHashMap()); + exec.shutdownNow(); } } diff --git a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java index 0ccd2401c73..5c61217e430 100644 --- a/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java +++ b/server/src/test/java/com/metamx/druid/db/DatabaseSegmentManagerTest.java @@ -94,10 +94,7 @@ public class DatabaseSegmentManagerTest + "\"twitterstream_2012-01-05T00:00:00.000Z_2012-01-06T00:00:00.000Z_2012-01-06T22:19:12.565Z\"}" ); - testRows = Arrays.>asList( - map1, - map2 - ); + testRows = Arrays.>asList(map1, map2); } @After @@ -114,5 +111,6 @@ public class DatabaseSegmentManagerTest manager.start(); manager.poll(); + manager.stop(); } } From 11ea15fc1a90604c8edd71a26b9f9d9bc451d650 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 17 Jun 2013 10:53:50 -0700 Subject: [PATCH 06/92] 1) Introduce Jetty 8 2) Fix up modules based on exceptions at startup for MasterMain --- client/pom.xml | 12 +- .../java/com/metamx/druid/QueryableNode.java | 12 +- .../curator/discovery/DiscoveryModule.java | 9 +- .../com/metamx/druid/http/BrokerNode.java | 19 +- .../com/metamx/druid/http/QueryServlet.java | 3 +- ...igFactoryModule.java => ConfigModule.java} | 8 +- .../CuratorDiscoveryConfig.java | 2 +- .../druid/initialization/DruidNodeConfig.java | 4 +- .../druid/initialization/EmitterModule.java | 9 +- .../initialization/HttpEmitterConfig.java | 8 +- .../initialization/HttpEmitterModule.java | 6 +- .../druid/initialization/Initialization.java | 62 ++++- .../JettyServerInitializer.java | 11 + .../initialization/JettyServerModule.java | 62 +++++ common/pom.xml | 4 + .../metamx/druid/config/ConfigManager.java | 17 +- .../druid/config/ConfigManagerConfig.java | 22 +- .../druid/config/ConfigManagerProvider.java | 74 ++++++ .../java/com/metamx/druid/db/DbConnector.java | 27 ++- .../metamx/druid/db/DbConnectorConfig.java | 2 +- .../com/metamx/druid/db/DbTablesConfig.java | 55 ++++- .../metamx/druid/guice/ConfigProvider.java | 13 +- .../druid/guice/DruidSecondaryModule.java | 11 +- .../druid/guice/JsonConfigProvider.java | 69 ++---- .../metamx/druid/guice/JsonConfigurator.java | 94 ++++++++ examples/pom.xml | 105 -------- indexing-service/pom.xml | 4 +- .../http/IndexerCoordinatorNode.java | 61 ++--- .../http/IndexerCoordinatorResource.java | 8 - .../http/OldIndexerCoordinatorResource.java | 6 +- .../worker/executor/ExecutorNode.java | 12 +- .../indexing/worker/http/WorkerNode.java | 20 +- pom.xml | 30 ++- realtime/pom.xml | 4 +- .../metamx/druid/realtime/RealtimeNode.java | 6 +- server/pom.xml | 4 +- .../metamx/druid/db/DatabaseRuleManager.java | 12 +- .../druid/db/DatabaseRuleManagerConfig.java | 27 ++- .../druid/db/DatabaseRuleManagerProvider.java | 79 ++++++ .../db/DatabaseSegmentManagerProvider.java | 81 +++++++ .../metamx/druid/guice/HttpClientModule.java | 11 +- .../com/metamx/druid/guice/MasterModule.java | 83 ++++--- .../com/metamx/druid/http/ComputeNode.java | 10 +- .../com/metamx/druid/http/MasterMain.java | 228 +++--------------- .../metamx/druid/http/MasterRedirectInfo.java | 47 ++++ .../com/metamx/druid/http/RedirectFilter.java | 2 + .../metamx/druid/http/RedirectServlet.java | 2 + .../com/metamx/druid/master/DruidMaster.java | 4 + .../druid/master/LoadQueueTaskMaster.java | 2 + .../metrics/DruidMonitorSchedulerConfig.java | 26 ++ .../metamx/druid/metrics/MetricsModule.java | 15 +- 51 files changed, 948 insertions(+), 556 deletions(-) rename client/src/main/java/com/metamx/druid/initialization/{ConfigFactoryModule.java => ConfigModule.java} (80%) create mode 100644 client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java create mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java create mode 100644 server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java create mode 100644 server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerProvider.java create mode 100644 server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java create mode 100644 server/src/main/java/com/metamx/druid/metrics/DruidMonitorSchedulerConfig.java diff --git a/client/pom.xml b/client/pom.xml index 69e2999ec32..10268136193 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -86,12 +86,16 @@ curator-x-discovery - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server - org.mortbay.jetty - jetty-util + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-servlets com.google.guava diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index ef8a995295b..ce70dca4398 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -43,6 +43,7 @@ import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.announcement.Announcer; +import com.metamx.druid.guice.JsonConfigurator; import com.metamx.druid.http.RequestLogger; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; @@ -60,10 +61,11 @@ import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; +import org.eclipse.jetty.server.Server; import org.joda.time.Duration; -import org.mortbay.jetty.Server; import org.skife.config.ConfigurationObjectFactory; +import javax.validation.Validation; import java.io.IOException; import java.lang.reflect.Field; import java.util.Arrays; @@ -83,6 +85,7 @@ public abstract class QueryableNode extends Registering private final Properties props; private final ConfigurationObjectFactory configFactory; private final String nodeType; + private final JsonConfigurator jsonConfigurator; private DruidServerMetadata druidServerMetadata = null; private ServiceEmitter emitter = null; @@ -124,6 +127,8 @@ public abstract class QueryableNode extends Registering Preconditions.checkNotNull(smileMapper, "smileMapper"); Preconditions.checkNotNull(configFactory, "configFactory"); + this.jsonConfigurator = new JsonConfigurator(jsonMapper, Validation.buildDefaultValidatorFactory().getValidator()); + Preconditions.checkState(smileMapper.getJsonFactory() instanceof SmileFactory, "smileMapper should use smile."); this.nodeType = nodeType; } @@ -245,6 +250,11 @@ public abstract class QueryableNode extends Registering return configFactory; } + public JsonConfigurator getJsonConfigurator() + { + return jsonConfigurator; + } + public DruidServerMetadata getDruidServerMetadata() { initializeDruidServerMetadata(); diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java index c5a2c701bf8..aa386977403 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java @@ -1,5 +1,6 @@ package com.metamx.druid.curator.discovery; +import com.google.common.base.Supplier; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; @@ -26,18 +27,18 @@ public class DiscoveryModule implements Module @Provides @LazySingleton public ServiceDiscovery getServiceDiscovery( CuratorFramework curator, - CuratorDiscoveryConfig config, + Supplier config, Lifecycle lifecycle ) throws Exception { - return Initialization.makeServiceDiscoveryClient(curator, config, lifecycle); + return Initialization.makeServiceDiscoveryClient(curator, config.get(), lifecycle); } @Provides @LazySingleton public ServiceInstanceFactory getServiceInstanceFactory( - DruidNodeConfig nodeConfig + Supplier nodeConfig ) { - return Initialization.makeServiceInstanceFactory(nodeConfig); + return Initialization.makeServiceInstanceFactory(nodeConfig.get()); } } diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 6fd5b2e8004..4fbdf78f0f0 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -57,9 +57,9 @@ import com.metamx.http.client.HttpClientInit; import com.metamx.metrics.Monitor; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.ServletHolder; -import org.mortbay.servlet.GzipFilter; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; import org.skife.config.ConfigurationObjectFactory; import java.util.List; @@ -210,19 +210,22 @@ public class BrokerNode extends QueryableNode theModules.addAll(extraModules); final Injector injector = Guice.createInjector(theModules); - final Context root = new Context(getServer(), "/", Context.SESSIONS); + final ServletContextHandler root = new ServletContextHandler(); + root.setContextPath("/"); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet( - new ServletHolder(new QueryServlet(getJsonMapper(), getSmileMapper(), texasRanger, getEmitter(), getRequestLogger())), + new ServletHolder( + new QueryServlet(getJsonMapper(), getSmileMapper(), texasRanger, getEmitter(), getRequestLogger()) + ), "/druid/v2/*" ); - root.addFilter(GzipFilter.class, "/*", 0); + root.addFilter(GzipFilter.class, "/*", null); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/v2/datasources/*", 0); + root.addFilter(GuiceFilter.class, "/druid/v2/datasources/*", null); for (String path : pathsForGuiceFilter) { - root.addFilter(GuiceFilter.class, path, 0); + root.addFilter(GuiceFilter.class, path, null); } } diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 8b871a49bcc..78cb428998a 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -33,9 +33,8 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; - +import org.eclipse.jetty.server.Request; import org.joda.time.DateTime; -import org.mortbay.jetty.Request; import javax.servlet.ServletException; import javax.servlet.http.HttpServlet; diff --git a/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java b/client/src/main/java/com/metamx/druid/initialization/ConfigModule.java similarity index 80% rename from client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java rename to client/src/main/java/com/metamx/druid/initialization/ConfigModule.java index 6d136c53da4..5d1245c23c2 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ConfigFactoryModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/ConfigModule.java @@ -23,19 +23,23 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.config.Config; +import com.metamx.druid.guice.JsonConfigurator; import com.metamx.druid.guice.LazySingleton; import org.skife.config.ConfigurationObjectFactory; +import javax.validation.Validation; +import javax.validation.Validator; import java.util.Properties; /** */ -public class ConfigFactoryModule implements Module +public class ConfigModule implements Module { @Override public void configure(Binder binder) { - + binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator()); + binder.bind(JsonConfigurator.class).in(LazySingleton.class); } @Provides @LazySingleton diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java index ee0f7ced76e..4fc6c28360e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public abstract class CuratorDiscoveryConfig +public class CuratorDiscoveryConfig { @JsonProperty private String path = null; diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java index 94f02c5edd5..c0ab3491d4b 100644 --- a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java @@ -29,7 +29,7 @@ import javax.validation.constraints.NotNull; /** */ -public abstract class DruidNodeConfig +public class DruidNodeConfig { @NotNull private String serviceName = null; @@ -42,8 +42,8 @@ public abstract class DruidNodeConfig @JsonCreator public DruidNodeConfig( - @JsonProperty("service") String serviceName, @JsonProperty("host") String host, + @JsonProperty("service") String serviceName, @JsonProperty("port") Integer port ) { diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java index 408358c2283..578f8e3ea16 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -19,6 +19,7 @@ package com.metamx.druid.initialization; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Binding; @@ -46,6 +47,7 @@ import java.util.Properties; public class EmitterModule implements Module { private static final Logger log = new Logger(EmitterModule.class); + private static final String EMITTER_PROPERTY = "druid.emitter"; private final Properties props; @@ -60,7 +62,7 @@ public class EmitterModule implements Module @Override public void configure(Binder binder) { - String emitterType = props.getProperty("druid.emitter", ""); + String emitterType = props.getProperty(EMITTER_PROPERTY, ""); binder.install(new LogEmitterModule()); binder.install(new HttpEmitterModule()); @@ -70,8 +72,9 @@ public class EmitterModule implements Module @Provides @LazySingleton - public ServiceEmitter getServiceEmitter(DruidNodeConfig config, Emitter emitter) + public ServiceEmitter getServiceEmitter(Supplier configSupplier, Emitter emitter) { + final DruidNodeConfig config = configSupplier.get(); final ServiceEmitter retVal = new ServiceEmitter(config.getServiceName(), config.getHost(), emitter); EmittingLogger.registerEmitter(retVal); return retVal; @@ -110,7 +113,7 @@ public class EmitterModule implements Module knownTypes.add(((Named) annotation).value()); } } - throw new ISE("Uknown emitter type, known types[%s]", knownTypes); + throw new ISE("Uknown emitter type[%s]=[%s], known types[%s]", EMITTER_PROPERTY, emitterType, knownTypes); } } diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java index fd6955d3579..2a9467b3961 100644 --- a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java @@ -20,16 +20,16 @@ package com.metamx.druid.initialization; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Duration; +import org.joda.time.Period; /** */ -public abstract class HttpEmitterConfig extends com.metamx.emitter.core.HttpEmitterConfig +public class HttpEmitterConfig extends com.metamx.emitter.core.HttpEmitterConfig { @JsonProperty - private Duration timeOut = new Duration("PT5m"); + private Period timeOut = new Period("PT5M"); - public Duration getReadTimeout() + public Period getReadTimeout() { return timeOut; } diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java index 8f7cc45018f..9f9c5eb5de0 100644 --- a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java @@ -24,6 +24,7 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.name.Named; +import com.google.inject.util.Providers; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; @@ -43,6 +44,9 @@ public class HttpEmitterModule implements Module public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.emitter.http", HttpEmitterConfig.class); + + // Fix the injection of this if we want to enable ssl emission of events. + binder.bind(SSLContext.class).toProvider(Providers.of(null)).in(LazySingleton.class); } @Provides @LazySingleton @Named("http") @@ -51,7 +55,7 @@ public class HttpEmitterModule implements Module final HttpClientConfig.Builder builder = HttpClientConfig .builder() .withNumConnections(1) - .withReadTimeout(config.get().getReadTimeout()); + .withReadTimeout(config.get().getReadTimeout().toStandardDuration()); if (sslContext != null) { builder.withSslContext(sslContext); diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 66366a3e739..faf7e5dbb65 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -21,8 +21,14 @@ package com.metamx.druid.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; +import com.google.common.base.Function; import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; @@ -33,9 +39,12 @@ import com.metamx.druid.curator.discovery.AddressPortServiceInstanceFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceInstanceFactory; +import com.metamx.druid.guice.DruidGuiceExtensions; +import com.metamx.druid.guice.DruidSecondaryModule; import com.metamx.druid.http.EmittingRequestLogger; import com.metamx.druid.http.FileRequestLogger; import com.metamx.druid.http.RequestLogger; +import com.metamx.druid.jackson.JacksonModule; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.core.Emitter; import org.apache.curator.framework.CuratorFramework; @@ -45,17 +54,20 @@ import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; import org.apache.curator.x.discovery.ServiceProvider; import org.apache.zookeeper.data.Stat; -import org.mortbay.jetty.Connector; -import org.mortbay.jetty.Server; -import org.mortbay.jetty.nio.SelectChannelConnector; -import org.mortbay.thread.QueuedThreadPool; +import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.nio.SelectChannelConnector; +import org.eclipse.jetty.util.thread.QueuedThreadPool; import org.skife.config.ConfigurationObjectFactory; +import javax.annotation.Nullable; import java.io.ByteArrayInputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.util.Arrays; +import java.util.List; import java.util.Properties; /** @@ -370,4 +382,46 @@ public class Initialization return new AddressPortServiceInstanceFactory(address, config.getPort()); } + + public static Injector makeInjector(final Object... modules) + { + final Injector baseInjector = Guice.createInjector( + new DruidGuiceExtensions(), + new JacksonModule(), + new PropertiesModule("runtime.properties"), + new ConfigModule() + ); + + List actualModules = Lists.newArrayList(); + + actualModules.add(DruidSecondaryModule.class); + actualModules.addAll(Arrays.asList(modules)); + + return Guice.createInjector( + Lists.transform( + actualModules, + new Function() + { + @Override + @SuppressWarnings("unchecked") + public Module apply(@Nullable Object input) + { + if (input instanceof Module) { + baseInjector.injectMembers(input); + return (Module) input; + } + if (input instanceof Class) { + if (Module.class.isAssignableFrom((Class) input)) { + return baseInjector.getInstance((Class) input); + } + else { + throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); + } + } + throw new ISE("Unknown module type[%s]", input.getClass()); + } + } + ) + ); + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java new file mode 100644 index 00000000000..05b11c98e40 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java @@ -0,0 +1,11 @@ +package com.metamx.druid.initialization; + +import com.google.inject.Injector; +import org.eclipse.jetty.server.Server; + +/** + */ +public interface JettyServerInitializer +{ + public void initialize(Server server, Injector injector); +} diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java new file mode 100644 index 00000000000..170c50396aa --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -0,0 +1,62 @@ +package com.metamx.druid.initialization; + +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import org.eclipse.jetty.server.Server; + +/** + */ +public class JettyServerModule implements Module +{ + private static final Logger log = new Logger(JettyServerModule.class); + + private final JettyServerInitializer initializer; + + public JettyServerModule( + JettyServerInitializer initializer + ) + { + this.initializer = initializer; + } + + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, ServerConfig.class); + } + + @Provides @LazySingleton + public Server getServer(Injector injector, Lifecycle lifecycle, ServerConfig config) + { + final Server server = Initialization.makeJettyServer(config); + initializer.initialize(server, injector); + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + server.start(); + } + + @Override + public void stop() + { + try { + server.stop(); + } + catch (Exception e) { + log.warn(e, "Unable to stop Jetty server."); + } + } + } + ); + return server; + } +} diff --git a/common/pom.xml b/common/pom.xml index cfb8620e574..66538c62e6d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -67,6 +67,10 @@ org.hibernate hibernate-validator + + javax.validation + validation-api + it.uniroma3.mat extendedset diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManager.java b/common/src/main/java/com/metamx/druid/config/ConfigManager.java index 14b504a60ec..eb9bf67b476 100644 --- a/common/src/main/java/com/metamx/druid/config/ConfigManager.java +++ b/common/src/main/java/com/metamx/druid/config/ConfigManager.java @@ -1,5 +1,6 @@ package com.metamx.druid.config; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.inject.Inject; @@ -7,6 +8,7 @@ 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.db.DbTablesConfig; import org.joda.time.Duration; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; @@ -34,7 +36,7 @@ public class ConfigManager private boolean started = false; private final IDBI dbi; - private final ConfigManagerConfig config; + private final Supplier config; private final ScheduledExecutorService exec; private final ConcurrentMap watchedConfigs; @@ -44,17 +46,20 @@ public class ConfigManager private volatile ConfigManager.PollingCallable poller; @Inject - public ConfigManager(IDBI dbi, ConfigManagerConfig config) // TODO: use DbTables and a different config + public ConfigManager(IDBI dbi, Supplier dbTables, Supplier 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()); + + final String configTable = dbTables.get().getConfigTable(); + + this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", configTable); insertStatement = String.format( "INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload", - config.getConfigTable() + configTable ); } @@ -67,7 +72,9 @@ public class ConfigManager } poller = new PollingCallable(); - ScheduledExecutors.scheduleWithFixedDelay(exec, new Duration(0), config.getPollDuration(), poller); + ScheduledExecutors.scheduleWithFixedDelay( + exec, new Duration(0), config.get().getPollDuration().toStandardDuration(), poller + ); started = true; } diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java index 94ab2eabd2a..b0eb7f5aa60 100644 --- a/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java +++ b/common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java @@ -1,18 +1,20 @@ package com.metamx.druid.config; -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; + +import javax.validation.constraints.NotNull; /** */ -public abstract class ConfigManagerConfig +public class ConfigManagerConfig { - @Config("druid.database.configTable") - public abstract String getConfigTable(); - - @Config("druid.indexer.poll.duration") - @Default("PT1M") - public abstract Duration getPollDuration(); + @JsonProperty + @NotNull + private Period pollDuration = new Period("PT1M"); + public Period getPollDuration() + { + return pollDuration; + } } diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java new file mode 100644 index 00000000000..032d2668152 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java @@ -0,0 +1,74 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.google.common.base.Supplier; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbTablesConfig; + +/** + */ +public class ConfigManagerProvider implements Provider +{ + private final DbConnector dbConnector; + private final Supplier dbTables; + private final Supplier config; + private final Lifecycle lifecycle; + + @Inject + ConfigManagerProvider( + DbConnector dbConnector, + Supplier dbTables, + Supplier config, + Lifecycle lifecycle + ) + { + this.dbConnector = dbConnector; + this.dbTables = dbTables; + this.config = config; + this.lifecycle = lifecycle; + } + + @Override + public ConfigManager get() + { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createConfigTable(); + } + + @Override + public void stop() + { + + } + } + ); + + return new ConfigManager(dbConnector.getDBI(), dbTables, config); + } +} 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 1cfc9904b09..f0c60bac43c 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnector.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnector.java @@ -20,6 +20,7 @@ package com.metamx.druid.db; import com.google.common.base.Supplier; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import org.apache.commons.dbcp.BasicDataSource; import org.skife.jdbi.v2.DBI; @@ -165,6 +166,7 @@ public class DbConnector private final Supplier dbTables; private final DBI dbi; + @Inject public DbConnector(Supplier config, Supplier dbTables) { this.config = config; @@ -197,11 +199,32 @@ public class DbConnector public void createSegmentTable() { - createSegmentTable(dbi, dbTables.get().getSegmentsTable()); + if (config.get().isCreateTables()) { + createSegmentTable(dbi, dbTables.get().getSegmentsTable()); + } } public void createRulesTable() { - createRuleTable(dbi, dbTables.get().getRulesTable()); + if (config.get().isCreateTables()) { + createRuleTable(dbi, dbTables.get().getRulesTable()); + } + } + + public void createConfigTable() + { + if (config.get().isCreateTables()) { + createRuleTable(dbi, dbTables.get().getConfigTable()); + } + } + + public void createTaskTables() + { + if (config.get().isCreateTables()) { + final DbTablesConfig dbTablesConfig = dbTables.get(); + createTaskTable(dbi, dbTablesConfig.getTasksTable()); + createTaskLogTable(dbi, dbTablesConfig.getTaskLogTable()); + createTaskLockTable(dbi, dbTablesConfig.getTaskLockTable()); + } } } diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index f6e15bbcc48..bd9e15f669d 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -25,7 +25,7 @@ import javax.validation.constraints.NotNull; /** */ -public abstract class DbConnectorConfig +public class DbConnectorConfig { @JsonProperty private boolean createTables = true; diff --git a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java index 96d7170f6fb..54567b3fe74 100644 --- a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java @@ -12,7 +12,7 @@ public class DbTablesConfig { public static DbTablesConfig fromBase(String base) { - return new DbTablesConfig(base, null, null); + return new DbTablesConfig(base, null, null, null, null, null, null); } @NotNull @@ -22,19 +22,38 @@ public class DbTablesConfig private final String segmentsTable; @NotNull - private final String ruleTable; + private final String rulesTable; + + @NotNull + private final String configTable; + + @NotNull + private final String tasksTable; + + @NotNull + private final String taskLogTable; + + @NotNull + private final String taskLockTable; @JsonCreator public DbTablesConfig( @JsonProperty("base") String base, @JsonProperty("segments") String segmentsTable, - @JsonProperty("rules") String rulesTable + @JsonProperty("rules") String rulesTable, + @JsonProperty("config") String configTable, + @JsonProperty("tasks") String tasksTable, + @JsonProperty("taskLog") String taskLogTable, + @JsonProperty("taskLock") String taskLockTable ) { - this.base = base; this.segmentsTable = makeTableName(segmentsTable, "segments"); - this.ruleTable = makeTableName(rulesTable, "rules"); + this.rulesTable = makeTableName(rulesTable, "rules"); + this.configTable = makeTableName(configTable, "config"); + this.tasksTable = makeTableName(tasksTable, "tasks"); + this.taskLogTable = makeTableName(taskLogTable, "task_log"); + this.taskLockTable = makeTableName(taskLockTable, "task_lock"); } private String makeTableName(String explicitTableName, String defaultSuffix) @@ -64,6 +83,30 @@ public class DbTablesConfig @JsonProperty("rules") public String getRulesTable() { - return ruleTable; + return rulesTable; + } + + @JsonProperty("config") + public String getConfigTable() + { + return configTable; + } + + @JsonProperty("tasks") + public String getTasksTable() + { + return tasksTable; + } + + @JsonProperty("taskLog") + public String getTaskLogTable() + { + return taskLogTable; + } + + @JsonProperty("taskLock") + public String getTaskLockTable() + { + return taskLockTable; } } \ No newline at end of file diff --git a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java index 79eff1b41ff..d86bcbc04b7 100644 --- a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java @@ -4,15 +4,18 @@ import com.google.common.base.Preconditions; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Provider; +import com.metamx.common.logger.Logger; import org.skife.config.ConfigurationObjectFactory; /** */ public class ConfigProvider implements Provider { + private static final Logger log = new Logger(ConfigProvider.class); + public static void bind(Binder binder, Class clazz) { - binder.bind(clazz).toProvider(of(clazz)).in(DruidScopes.SINGLETON); + binder.bind(clazz).toProvider(of(clazz)).in(LazySingleton.class); } public static Provider of(Class clazz) @@ -34,7 +37,13 @@ public class ConfigProvider implements Provider @Inject public void inject(ConfigurationObjectFactory factory) { - object = factory.build(clazz); + try { + object = factory.build(clazz); + } + catch (IllegalArgumentException e) { + log.info("Unable to build instance of class[%s]", clazz); + throw e; + } } @Override diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java index da8d2ce59c1..43f54207652 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -8,6 +8,7 @@ import com.metamx.druid.jackson.Json; import com.metamx.druid.jackson.Smile; import org.skife.config.ConfigurationObjectFactory; +import javax.validation.Validator; import java.util.Properties; /** @@ -18,19 +19,25 @@ public class DruidSecondaryModule implements Module private final ConfigurationObjectFactory factory; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; + private final Validator validator; + private final JsonConfigurator jsonConfigurator; @Inject public DruidSecondaryModule( Properties properties, ConfigurationObjectFactory factory, @Json ObjectMapper jsonMapper, - @Smile ObjectMapper smileMapper + @Smile ObjectMapper smileMapper, + Validator validator, + JsonConfigurator jsonConfigurator ) { this.properties = properties; this.factory = factory; this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; + this.validator = validator; + this.jsonConfigurator = jsonConfigurator; } @Override @@ -41,5 +48,7 @@ public class DruidSecondaryModule implements Module binder.bind(ConfigurationObjectFactory.class).toInstance(factory); binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMapper); binder.bind(ObjectMapper.class).annotatedWith(Smile.class).toInstance(smileMapper); + binder.bind(Validator.class).toInstance(validator); + binder.bind(JsonConfigurator.class).toInstance(jsonConfigurator); } } diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java index 64dd107d0f0..8c633060333 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -19,36 +19,26 @@ package com.metamx.druid.guice; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Joiner; import com.google.common.base.Supplier; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import com.google.common.base.Suppliers; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Provider; -import com.google.inject.TypeLiteral; -import com.metamx.common.IAE; -import com.metamx.common.ISE; +import com.google.inject.util.Types; -import javax.validation.ConstraintViolation; -import javax.validation.Validator; -import java.io.IOException; -import java.util.List; -import java.util.Map; import java.util.Properties; -import java.util.Set; /** */ public class JsonConfigProvider implements Provider> { - - private static final Joiner JOINER = Joiner.on(", "); - + @SuppressWarnings("unchecked") public static void bind(Binder binder, String propertyBase, Class classToProvide) { - binder.bind(new TypeLiteral>(){}).toProvider(of(propertyBase, classToProvide)).in(DruidScopes.SINGLETON); + binder.bind(Key.get(Types.newParameterizedType(Supplier.class, classToProvide))) + .toProvider((Provider) of(propertyBase, classToProvide)) + .in(LazySingleton.class); } public static JsonConfigProvider of(String propertyBase, Class classToProvide) @@ -59,7 +49,8 @@ public class JsonConfigProvider implements Provider> private final String propertyBase; private final Class classToProvide; - private Supplier supplier; + private Properties props; + private JsonConfigurator configurator; public JsonConfigProvider( String propertyBase, @@ -73,49 +64,17 @@ public class JsonConfigProvider implements Provider> @Inject public void inject( Properties props, - ObjectMapper jsonMapper, - Validator validator + JsonConfigurator configurator ) { - Map jsonMap = Maps.newHashMap(); - for (String prop : props.stringPropertyNames()) { - if (prop.startsWith(propertyBase)) { - final String propValue = props.getProperty(prop); - try { - jsonMap.put(prop.substring(propertyBase.length()), jsonMapper.readValue(propValue, Object.class)); - } - catch (IOException e) { - throw new IAE("Unable to parse an object out of prop[%s]=[%s]", prop, propValue); - } - } - } - - final T config = jsonMapper.convertValue(jsonMap, classToProvide); - - final Set> violations = validator.validate(config); - if (!violations.isEmpty()) { - List messages = Lists.newArrayList(); - - for (ConstraintViolation violation : violations) { - messages.add(String.format("%s - %s", violation.getPropertyPath().toString(), violation.getMessage())); - } - - throw new ISE("Configuration violations[%s]", JOINER.join(messages)); - } - - this.supplier = new Supplier() - { - @Override - public T get() - { - return config; - } - }; + this.props = props; + this.configurator = configurator; } @Override public Supplier get() { - return supplier; + final T config = configurator.configurate(props, propertyBase, classToProvide); + return Suppliers.ofInstance(config); } } diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java new file mode 100644 index 00000000000..2f35bed160b --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -0,0 +1,94 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.inject.Inject; +import com.google.inject.ProvisionException; +import com.google.inject.spi.Message; +import com.metamx.common.logger.Logger; + +import javax.annotation.Nullable; +import javax.validation.ConstraintViolation; +import javax.validation.Validator; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +/** + */ +public class JsonConfigurator +{ + private static final Logger log = new Logger(JsonConfigurator.class); + + private static final Joiner JOINER = Joiner.on("; "); + + private final ObjectMapper jsonMapper; + private final Validator validator; + + @Inject + public JsonConfigurator( + ObjectMapper jsonMapper, + Validator validator + ) + { + this.jsonMapper = jsonMapper; + this.validator = validator; + } + + public T configurate(Properties props, String propertyPrefix, Class clazz) throws ProvisionException + { + // Make it end with a period so we only include properties with sub-object thingies. + final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + "."; + + Map jsonMap = Maps.newHashMap(); + for (String prop : props.stringPropertyNames()) { + if (prop.startsWith(propertyBase)) { + final String propValue = props.getProperty(prop); + Object value; + try { + value = jsonMapper.readValue(propValue, Object.class); + } + catch (IOException e) { + log.debug("Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue); + value = propValue; + } + + jsonMap.put(prop.substring(propertyBase.length()), value); + } + } + + final T config = jsonMapper.convertValue(jsonMap, clazz); + + final Set> violations = validator.validate(config); + if (!violations.isEmpty()) { + List messages = Lists.newArrayList(); + + for (ConstraintViolation violation : violations) { + messages.add(String.format("%s - %s", violation.getPropertyPath().toString(), violation.getMessage())); + } + + throw new ProvisionException( + Iterables.transform( + messages, + new Function() + { + @Nullable + @Override + public Message apply(@Nullable String input) + { + return new Message(String.format("%s%s", propertyBase, input)); + } + } + ) + ); + } + + return config; + } +} diff --git a/examples/pom.xml b/examples/pom.xml index 0c5e000dddc..ff53f37e46a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -29,99 +29,6 @@ ${project.parent.version} - - com.metamx - emitter - - - com.metamx - http-client - - - com.metamx - java-util - - - com.metamx - server-metrics - - - - com.davekoelle - alphanum - - - commons-codec - commons-codec - - - org.skife.config - config-magic - - - com.google.guava - guava - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - com.ibm.icu - icu4j - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - org.jdbi - jdbi - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.sun.jersey.contribs - jersey-guice - - - org.mortbay.jetty - jetty - - - joda-time - joda-time - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - org.twitter4j twitter4j-core @@ -137,18 +44,6 @@ twitter4j-stream 2.2.6 - - - - junit - junit - test - - - org.easymock - easymock - test - diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index e3ff3e37803..b7ab5798382 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -145,8 +145,8 @@ jets3t - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server joda-time diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index c98497a3602..5a6ac400d4c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Optional; +import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -51,6 +52,7 @@ import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; @@ -115,18 +117,17 @@ import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.x.discovery.ServiceDiscovery; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.resource.ResourceCollection; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; import org.joda.time.Duration; -import org.mortbay.jetty.Server; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.DefaultServlet; -import org.mortbay.jetty.servlet.FilterHolder; -import org.mortbay.jetty.servlet.ServletHolder; -import org.mortbay.resource.ResourceCollection; import org.skife.config.ConfigurationObjectFactory; -import org.skife.jdbi.v2.IDBI; import java.io.IOException; import java.io.InputStream; @@ -153,7 +154,8 @@ public class IndexerCoordinatorNode extends QueryableNode monitors = null; private ServiceEmitter emitter = null; private DbConnectorConfig dbConnectorConfig = null; - private IDBI dbi = null; + private DbConnector dbi = null; + private Supplier dbTables = null; private IndexerCoordinatorConfig config = null; private MergerDBCoordinator mergerDBCoordinator = null; private ServiceDiscovery serviceDiscovery = null; @@ -238,19 +240,30 @@ public class IndexerCoordinatorNode extends QueryableNode getDbTables() + { + if (dbTables == null) { + dbTables = Suppliers.ofInstance( + getJsonConfigurator().configurate(getProps(), "druid.database.tables", DbTablesConfig.class) + ); + } + return dbTables; + } + public void doInit() throws Exception { final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(getLifecycle()); initializeDB(); final ConfigManagerConfig managerConfig = getConfigFactory().build(ConfigManagerConfig.class); - DbConnector.createConfigTable(dbi, managerConfig.getConfigTable()); + dbi.createConfigTable(); JacksonConfigManager configManager = new JacksonConfigManager( getLifecycle().addManagedInstance( new ConfigManager( - dbi, - managerConfig + dbi.getDBI(), + getDbTables(), + Suppliers.ofInstance(managerConfig) ) ), getJsonMapper() ); @@ -296,7 +309,7 @@ public class IndexerCoordinatorNode extends QueryableNodeofInstance(dbConnectorConfig), null).getDBI() // TODO - ); + taskStorage = new DbTaskStorage(getJsonMapper(), dbConnectorConfig, dbi.getDBI()); } else { throw new ISE("Invalid storage implementation: %s", config.getStorageImpl()); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java index e6f49b0d544..90837729306 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java @@ -41,10 +41,8 @@ import com.metamx.druid.indexing.coordinator.TaskQueue; import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; -import com.metamx.emitter.service.ServiceEmitter; import javax.ws.rs.Consumes; import javax.ws.rs.DefaultValue; @@ -89,8 +87,6 @@ public class IndexerCoordinatorResource } }; - private final IndexerCoordinatorConfig config; - private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final TaskLogProvider taskLogProvider; @@ -101,8 +97,6 @@ public class IndexerCoordinatorResource @Inject public IndexerCoordinatorResource( - IndexerCoordinatorConfig config, - ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, TaskLogProvider taskLogProvider, @@ -110,8 +104,6 @@ public class IndexerCoordinatorResource ObjectMapper jsonMapper ) throws Exception { - this.config = config; - this.emitter = emitter; this.taskMasterLifecycle = taskMasterLifecycle; this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.taskLogProvider = taskLogProvider; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java index 36e7e948cd8..84897f424bd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java @@ -6,8 +6,6 @@ import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.emitter.service.ServiceEmitter; import javax.ws.rs.Path; @@ -19,8 +17,6 @@ public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource { @Inject public OldIndexerCoordinatorResource( - IndexerCoordinatorConfig config, - ServiceEmitter emitter, TaskMasterLifecycle taskMasterLifecycle, TaskStorageQueryAdapter taskStorageQueryAdapter, TaskLogProvider taskLogProvider, @@ -28,6 +24,6 @@ public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource ObjectMapper jsonMapper ) throws Exception { - super(config, emitter, taskMasterLifecycle, taskStorageQueryAdapter, taskLogProvider, configManager, jsonMapper); + super(taskMasterLifecycle, taskStorageQueryAdapter, taskLogProvider, configManager, jsonMapper); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 6e0fa8b6fc3..2f3c1fbf4bb 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -77,13 +77,13 @@ import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; -import org.mortbay.jetty.Server; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.DefaultServlet; -import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; import java.util.List; @@ -216,12 +216,12 @@ public class ExecutorNode extends BaseServerNode chatHandlerProvider ) ); - final Context root = new Context(server, "/", Context.SESSIONS); + final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", 0); + root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); root.addServlet( new ServletHolder( new QueryServlet(getJsonMapper(), getSmileMapper(), taskRunner, emitter, getRequestLogger()) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 2d69208cfd8..7b978d649e3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -37,10 +37,6 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; @@ -54,6 +50,10 @@ import com.metamx.druid.indexing.worker.Worker; import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; import com.metamx.druid.indexing.worker.WorkerTaskMonitor; import com.metamx.druid.indexing.worker.config.WorkerConfig; +import com.metamx.druid.initialization.CuratorDiscoveryConfig; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -70,14 +70,14 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; import org.joda.time.Duration; -import org.mortbay.jetty.Server; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.DefaultServlet; -import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; import java.util.List; @@ -204,12 +204,12 @@ public class WorkerNode extends QueryableNode forkingTaskRunner ) ); - final Context root = new Context(server, "/", Context.SESSIONS); + final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", 0); + root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); } @LifecycleStart diff --git a/pom.xml b/pom.xml index cae6b4b4c3c..b5767dcdbb6 100644 --- a/pom.xml +++ b/pom.xml @@ -230,6 +230,11 @@ hibernate-validator 5.0.1.Final + + javax.validation + validation-api + 1.1.0.Final + javax.inject javax.inject @@ -266,20 +271,19 @@ 0.8.1 - org.mortbay.jetty - jetty - 6.1.26 - - - org.mortbay.jetty - servlet-api - - + org.eclipse.jetty + jetty-server + 8.1.11.v20130520 - org.mortbay.jetty - jetty-util - 6.1.26 + org.eclipse.jetty + jetty-servlet + 8.1.11.v20130520 + + + org.eclipse.jetty + jetty-servlets + 8.1.11.v20130520 joda-time @@ -299,7 +303,7 @@ javax.servlet servlet-api - 2.5 + 3.0 org.slf4j diff --git a/realtime/pom.xml b/realtime/pom.xml index daf2dca271e..ca8b22f4de3 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -98,8 +98,8 @@ jets3t - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server joda-time 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 c8d492d8716..4357f39c2b5 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -48,8 +48,8 @@ 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.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; import java.io.File; @@ -148,7 +148,7 @@ public class RealtimeNode extends BaseServerNode startMonitoring(monitors); - final Context root = new Context(getServer(), "/", Context.SESSIONS); + final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet( new ServletHolder( diff --git a/server/pom.xml b/server/pom.xml index be9fc9a2e7d..0ae6ac0560f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -146,8 +146,8 @@ jets3t - org.mortbay.jetty - jetty + org.eclipse.jetty + jetty-server joda-time diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 5e8550b0165..b308296fb31 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -118,7 +118,7 @@ public class DatabaseRuleManager private final ObjectMapper jsonMapper; private final ScheduledExecutorService exec; - private final DatabaseRuleManagerConfig config; + private final Supplier config; private final Supplier dbTables; private final IDBI dbi; private final AtomicReference>> rules; @@ -130,7 +130,7 @@ public class DatabaseRuleManager @Inject public DatabaseRuleManager( ObjectMapper jsonMapper, - DatabaseRuleManagerConfig config, + Supplier config, Supplier dbTables, IDBI dbi ) @@ -155,11 +155,11 @@ public class DatabaseRuleManager return; } - createDefaultRule(dbi, getRulesTable(), config.getDefaultTier(), jsonMapper); + createDefaultRule(dbi, getRulesTable(), config.get().getDefaultTier(), jsonMapper); ScheduledExecutors.scheduleWithFixedDelay( exec, new Duration(0), - config.getRulesPollDuration(), + config.get().getPollDuration().toStandardDuration(), new Runnable() { @Override @@ -267,8 +267,8 @@ public class DatabaseRuleManager if (theRules.get(dataSource) != null) { retVal.addAll(theRules.get(dataSource)); } - if (theRules.get(config.getDefaultTier()) != null) { - retVal.addAll(theRules.get(config.getDefaultTier())); + if (theRules.get(config.get().getDefaultTier()) != null) { + retVal.addAll(theRules.get(config.get().getDefaultTier())); } return retVal; } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java index 5f41945df58..8dfc0b83be1 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerConfig.java @@ -19,19 +19,26 @@ package com.metamx.druid.db; -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; /** */ -public abstract class DatabaseRuleManagerConfig +public class DatabaseRuleManagerConfig { - @Config("druid.database.rules.defaultTier") - @Default("_default") - public abstract String getDefaultTier(); + @JsonProperty + private String defaultTier = "_default"; - @Config("druid.database.rules.poll.duration") - @Default("PT1M") - public abstract Duration getRulesPollDuration(); + @JsonProperty + private Period pollDuration = new Period("PT1M"); + + public String getDefaultTier() + { + return defaultTier; + } + + public Period getPollDuration() + { + return pollDuration; + } } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java new file mode 100644 index 00000000000..a2760068cc1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java @@ -0,0 +1,79 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.db; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.metamx.common.lifecycle.Lifecycle; + +/** + */ +public class DatabaseRuleManagerProvider implements Provider +{ + private final ObjectMapper jsonMapper; + private final Supplier config; + private final Supplier dbTables; + private final DbConnector dbConnector; + private final Lifecycle lifecycle; + + @Inject + public DatabaseRuleManagerProvider( + ObjectMapper jsonMapper, + Supplier config, + Supplier dbTables, + DbConnector dbConnector, + Lifecycle lifecycle + ) + { + this.jsonMapper = jsonMapper; + this.config = config; + this.dbTables = dbTables; + this.dbConnector = dbConnector; + this.lifecycle = lifecycle; + } + + @Override + public DatabaseRuleManager get() + { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createRulesTable(); + DatabaseRuleManager.createDefaultRule( + dbConnector.getDBI(), dbTables.get().getRulesTable(), config.get().getDefaultTier(), jsonMapper + ); + } + + @Override + public void stop() + { + + } + } + ); + + return new DatabaseRuleManager(jsonMapper, config, dbTables, dbConnector.getDBI()); + } +} diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerProvider.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerProvider.java new file mode 100644 index 00000000000..02abdbafb67 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManagerProvider.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.db; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.metamx.common.lifecycle.Lifecycle; + +/** + */ +public class DatabaseSegmentManagerProvider implements Provider +{ + private final ObjectMapper jsonMapper; + private final Supplier config; + private final Supplier dbTables; + private final DbConnector dbConnector; + private final Lifecycle lifecycle; + + @Inject + public DatabaseSegmentManagerProvider( + ObjectMapper jsonMapper, + Supplier config, + Supplier dbTables, + DbConnector dbConnector, + Lifecycle lifecycle + ) + { + this.jsonMapper = jsonMapper; + this.config = config; + this.dbTables = dbTables; + this.dbConnector = dbConnector; + this.lifecycle = lifecycle; + } + + @Override + public DatabaseSegmentManager get() + { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createSegmentTable(); + } + + @Override + public void stop() + { + + } + } + ); + + return new DatabaseSegmentManager( + jsonMapper, + config, + dbTables, + dbConnector.getDBI() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java index 00ea9d01a7f..ac5eb18df16 100644 --- a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java @@ -1,6 +1,7 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Supplier; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; @@ -26,7 +27,7 @@ public class HttpClientModule implements Module JsonConfigProvider.bind(binder, "druid.global.http", DruidHttpClientConfig.class); } - public abstract static class DruidHttpClientConfig + public static class DruidHttpClientConfig { @JsonProperty @Min(0) @@ -47,8 +48,14 @@ public class HttpClientModule implements Module } @Provides @LazySingleton @Global - public HttpClient makeHttpClient(DruidHttpClientConfig config, Lifecycle lifecycle, @Nullable SSLContext sslContext) + public HttpClient makeHttpClient( + Supplier configSupplier, + Lifecycle lifecycle, + @Nullable SSLContext sslContext + ) { + final DruidHttpClientConfig config = configSupplier.get(); + final HttpClientConfig.Builder builder = HttpClientConfig .builder() .withNumConnections(config.getNumConnections()) diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index 60d6ccbad6d..a791d2eb31e 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -1,9 +1,12 @@ package com.metamx.druid.guice; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.indexing.IndexingService; @@ -11,11 +14,25 @@ import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.client.indexing.IndexingServiceSelector; import com.metamx.druid.client.selector.DiscoverySelector; import com.metamx.druid.client.selector.Server; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.config.ConfigManager; +import com.metamx.druid.config.ConfigManagerConfig; +import com.metamx.druid.config.ConfigManagerProvider; +import com.metamx.druid.db.DatabaseRuleManager; +import com.metamx.druid.db.DatabaseRuleManagerConfig; +import com.metamx.druid.db.DatabaseRuleManagerProvider; +import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.db.DatabaseSegmentManagerConfig; +import com.metamx.druid.db.DatabaseSegmentManagerProvider; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; +import com.metamx.druid.http.MasterRedirectInfo; +import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.master.DruidMasterConfig; +import com.metamx.druid.master.LoadQueueTaskMaster; +import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; import org.skife.jdbi.v2.IDBI; @@ -27,52 +44,58 @@ public class MasterModule implements Module @Override public void configure(Binder binder) { + ConfigProvider.bind(binder, DruidMasterConfig.class); ConfigProvider.bind(binder, ZkPathsConfig.class); ConfigProvider.bind(binder, ServerInventoryViewConfig.class); - ConfigProvider.bind(binder, DbConnectorConfig.class); - JsonConfigProvider.bind(binder, "druid.database.tables", DbTablesConfig.class); + JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class); + JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.config", ConfigManagerConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); + + binder.bind(DatabaseSegmentManager.class) + .toProvider(DatabaseSegmentManagerProvider.class) + .in(ManageLifecycle.class); + + binder.bind(DatabaseRuleManager.class) + .toProvider(DatabaseRuleManagerProvider.class) + .in(ManageLifecycle.class); + + binder.bind(ConfigManager.class) + .toProvider(ConfigManagerProvider.class) + .in(ManageLifecycle.class); binder.bind(new TypeLiteral>(){}) .annotatedWith(IndexingService.class) .to(IndexingServiceSelector.class) .in(ManageLifecycle.class); binder.bind(IndexingServiceClient.class).in(LazySingleton.class); + + binder.bind(RedirectInfo.class).to(MasterRedirectInfo.class).in(LazySingleton.class); } - @Provides @ManageLifecycle @IndexingService - public DiscoverySelector getIndexingServiceSelector(DruidMasterConfig config, ServiceDiscovery serviceDiscovery) + @Provides @LazySingleton @IndexingService + public ServiceProvider getServiceProvider(DruidMasterConfig config, ServiceDiscovery serviceDiscovery) { - final ServiceProvider serviceProvider = serviceDiscovery.serviceProviderBuilder() - .serviceName(config.getMergerServiceName()) - .build(); - - return new IndexingServiceSelector(serviceProvider); + return serviceDiscovery.serviceProviderBuilder().serviceName(config.getMergerServiceName()).build(); } @Provides @LazySingleton - public IDBI getDbi(final DbConnector dbConnector, final DbConnectorConfig config, Lifecycle lifecycle) + public IDBI getDbi(final DbConnector dbConnector) { - if (config.isCreateTables()) { - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - dbConnector.createSegmentTable(); - dbConnector.createRulesTable(); - } - - @Override - public void stop() - { - - } - } - ); - } - return dbConnector.getDBI(); } + + @Provides @LazySingleton + public LoadQueueTaskMaster getLoadQueueTaskMaster(CuratorFramework curator, ObjectMapper jsonMapper) + { + return new LoadQueueTaskMaster(curator, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d")); + } + + @Provides @LazySingleton + public ScheduledExecutorFactory getScheduledExecutorFactory(Lifecycle lifecycle) + { + return ScheduledExecutors.createFactory(lifecycle); + } } diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index f538b69b4db..5ffc6c234bc 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -38,19 +38,15 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.Monitor; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; 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; import java.util.List; @@ -128,7 +124,7 @@ public class ComputeNode extends BaseServerNode monitors.add(new ServerMonitor(getDruidServerMetadata(), serverManager)); startMonitoring(monitors); - final Context root = new Context(getServer(), "/", Context.SESSIONS); + final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet( new ServletHolder( 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 12c2a87dc7f..f4c3841f1a1 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,67 +19,33 @@ package com.metamx.druid.http; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.indexing.IndexingServiceClient; -import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.config.ConfigManager; -import com.metamx.druid.config.ConfigManagerConfig; -import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.guice.DruidGuiceExtensions; -import com.metamx.druid.guice.DruidSecondaryModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.initialization.ConfigFactoryModule; import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.PropertiesModule; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; -import com.metamx.druid.jackson.JacksonModule; +import com.metamx.druid.initialization.JettyServerInitializer; +import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; -import com.metamx.druid.master.DruidMasterConfig; -import com.metamx.druid.master.LoadQueueTaskMaster; import com.metamx.druid.metrics.MetricsModule; -import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; -import org.apache.curator.framework.CuratorFramework; -import org.mortbay.jetty.Server; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.DefaultServlet; -import org.mortbay.jetty.servlet.FilterHolder; -import org.mortbay.jetty.servlet.ServletHolder; -import org.mortbay.servlet.GzipFilter; -import org.skife.config.ConfigurationObjectFactory; -import org.skife.jdbi.v2.IDBI; - -import javax.annotation.Nullable; -import java.net.URL; -import java.util.Arrays; -import java.util.List; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; /** */ @@ -91,185 +57,55 @@ public class MasterMain { LogLevelAdjuster.register(); - Injector injector = makeInjector( - new LifecycleModule(Key.get(MonitorScheduler.class)), + Injector injector = Initialization.makeInjector( + new LifecycleModule(Key.get(MonitorScheduler.class), Key.get(DruidMaster.class)), EmitterModule.class, HttpClientModule.class, CuratorModule.class, - MetricsModule.class, + new MetricsModule(), DiscoveryModule.class, ServerModule.class, + new JettyServerModule(new MasterJettyServerInitializer()), MasterModule.class ); - final ObjectMapper jsonMapper = injector.getInstance(ObjectMapper.class); - final ConfigurationObjectFactory configFactory = injector.getInstance(ConfigurationObjectFactory.class); final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - final ServiceEmitter emitter = injector.getInstance(ServiceEmitter.class); - - final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); - - CuratorFramework curatorFramework = injector.getInstance(CuratorFramework.class); - - final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class); - - ServerInventoryView serverInventoryView = injector.getInstance(ServerInventoryView.class); - - - final DatabaseSegmentManager databaseSegmentManager = injector.getInstance(DatabaseSegmentManager.class); - final DatabaseRuleManager databaseRuleManager = injector.getInstance(DatabaseRuleManager.class); - - final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class); - final DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); + final DruidNodeConfig nodeConfig = injector.getInstance(DruidNodeConfig.class); final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); - Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); - - IDBI dbi = injector.getInstance(IDBI.class); // TODO: make tables and stuff - final ConfigManagerConfig configManagerConfig = configFactory.build(ConfigManagerConfig.class); - DbConnector.createConfigTable(dbi, configManagerConfig.getConfigTable()); - JacksonConfigManager configManager = new JacksonConfigManager( - new ConfigManager(dbi, configManagerConfig), jsonMapper - ); - - final LoadQueueTaskMaster taskMaster = new LoadQueueTaskMaster( - curatorFramework, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d") - ); - - final DruidMaster master = new DruidMaster( - druidMasterConfig, - zkPaths, - configManager, - databaseSegmentManager, - serverInventoryView, - databaseRuleManager, - curatorFramework, - emitter, - scheduledExecutorFactory, - injector.getInstance(IndexingServiceClient.class), - taskMaster - ); - lifecycle.addManagedInstance(master); try { lifecycle.start(); + + Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); } catch (Throwable t) { log.error(t, "Error when starting up. Failing."); System.exit(1); } - Runtime.getRuntime().addShutdownHook( - new Thread( - new Runnable() - { - @Override - public void run() - { - log.info("Running shutdown hook"); - lifecycle.stop(); - } - } - ) - ); - - final Server server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class)); - - final RedirectInfo redirectInfo = new RedirectInfo() - { - @Override - public boolean doLocal() - { - return master.isClusterMaster(); - } - - @Override - public URL getRedirectURL(String queryString, String requestURI) - { - try { - final String currentMaster = master.getCurrentMaster(); - if (currentMaster == null) { - return null; - } - - String location = String.format("http://%s%s", currentMaster, requestURI); - - if (queryString != null) { - location = String.format("%s?%s", location, queryString); - } - - return new URL(location); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - }; - - final Context staticContext = new Context(server, "/static", Context.SESSIONS); - staticContext.addServlet(new ServletHolder(new RedirectServlet(redirectInfo)), "/*"); - - staticContext.setResourceBase(ComputeMain.class.getClassLoader().getResource("static").toExternalForm()); - - final Context root = new Context(server, "/", Context.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GzipFilter.class, "/*", 0); - root.addFilter( - new FilterHolder( - new RedirectFilter( - redirectInfo - ) - ), "/*", 0 - ); - root.addFilter(GuiceFilter.class, "/info/*", 0); - root.addFilter(GuiceFilter.class, "/master/*", 0); - - server.start(); - server.join(); + lifecycle.join(); } - private static Injector makeInjector(final Object... modules) + private static class MasterJettyServerInitializer implements JettyServerInitializer { - final Injector baseInjector = Guice.createInjector( - new DruidGuiceExtensions(), - new JacksonModule(), - new PropertiesModule("runtime.properties"), - new ConfigFactoryModule() - ); + @Override + public void initialize(Server server, Injector injector) + { + final ServletContextHandler staticContext = new ServletContextHandler(server, "/static", ServletContextHandler.SESSIONS); + staticContext.addServlet(new ServletHolder(injector.getInstance(RedirectServlet.class)), "/*"); - List actualModules = Lists.newArrayList(); + staticContext.setResourceBase(ComputeMain.class.getClassLoader().getResource("static").toExternalForm()); - actualModules.add(DruidSecondaryModule.class); - actualModules.addAll(Arrays.asList(modules)); - - return Guice.createInjector( - Lists.transform( - actualModules, - new Function() - { - @Override - @SuppressWarnings("unchecked") - public Module apply(@Nullable Object input) - { - if (input instanceof Module) { - baseInjector.injectMembers(input); - return (Module) input; - } - if (input instanceof Class) { - if (Module.class.isAssignableFrom((Class) input)) { - return baseInjector.getInstance((Class) input); - } - else { - throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); - } - } - throw new ISE("Unknown module type[%s]", input.getClass()); - } - } - ) - ); + final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new StatusServlet()), "/status"); + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addEventListener(new GuiceServletConfig(injector)); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GuiceFilter.class, "/info/*", null); + root.addFilter(GuiceFilter.class, "/master/*", null); + } } } diff --git a/server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java b/server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java new file mode 100644 index 00000000000..f3c263682df --- /dev/null +++ b/server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java @@ -0,0 +1,47 @@ +package com.metamx.druid.http; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.druid.master.DruidMaster; + +import java.net.URL; + +/** +*/ +public class MasterRedirectInfo implements RedirectInfo +{ + private final DruidMaster master; + + @Inject + public MasterRedirectInfo(DruidMaster master) { + this.master = master; + } + + @Override + public boolean doLocal() + { + return master.isClusterMaster(); + } + + @Override + public URL getRedirectURL(String queryString, String requestURI) + { + try { + final String currentMaster = master.getCurrentMaster(); + if (currentMaster == null) { + return null; + } + + String location = String.format("http://%s%s", currentMaster, requestURI); + + if (queryString != null) { + location = String.format("%s?%s", location, queryString); + } + + return new URL(location); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java index 9a1a3c66bd3..5ed2bff98ee 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java @@ -19,6 +19,7 @@ package com.metamx.druid.http; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import javax.servlet.Filter; @@ -40,6 +41,7 @@ public class RedirectFilter implements Filter private final RedirectInfo redirectInfo; + @Inject public RedirectFilter( RedirectInfo redirectInfo ) diff --git a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java b/server/src/main/java/com/metamx/druid/http/RedirectServlet.java index 2befa9cd458..7b4ad2aec53 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectServlet.java @@ -19,6 +19,7 @@ package com.metamx.druid.http; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import org.mortbay.jetty.servlet.DefaultServlet; @@ -38,6 +39,7 @@ public class RedirectServlet extends DefaultServlet private final RedirectInfo redirectInfo; + @Inject public RedirectServlet( RedirectInfo redirectInfo ) 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 c3800e87bba..2a5f5a77fa3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -28,6 +28,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.Closeables; +import com.google.inject.Inject; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -44,6 +45,7 @@ import com.metamx.druid.concurrent.Execs; import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; @@ -68,6 +70,7 @@ import java.util.concurrent.atomic.AtomicReference; /** */ +@ManageLifecycle public class DruidMaster { public static final String MASTER_OWNER_NODE = "_MASTER"; @@ -94,6 +97,7 @@ public class DruidMaster private final Map loadManagementPeons; private final AtomicReference leaderLatch; + @Inject public DruidMaster( DruidMasterConfig config, ZkPathsConfig zkPaths, diff --git a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java index 9ef5b61e5a0..eecdbacef30 100644 --- a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java +++ b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java @@ -20,6 +20,7 @@ package com.metamx.druid.master; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; import java.util.concurrent.ExecutorService; @@ -33,6 +34,7 @@ public class LoadQueueTaskMaster private final ObjectMapper jsonMapper; private final ExecutorService peonExec; + @Inject public LoadQueueTaskMaster( CuratorFramework curator, ObjectMapper jsonMapper, diff --git a/server/src/main/java/com/metamx/druid/metrics/DruidMonitorSchedulerConfig.java b/server/src/main/java/com/metamx/druid/metrics/DruidMonitorSchedulerConfig.java new file mode 100644 index 00000000000..68d70738cca --- /dev/null +++ b/server/src/main/java/com/metamx/druid/metrics/DruidMonitorSchedulerConfig.java @@ -0,0 +1,26 @@ +package com.metamx.druid.metrics; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.metrics.MonitorSchedulerConfig; +import org.joda.time.Duration; +import org.joda.time.Period; + +/** + */ +public class DruidMonitorSchedulerConfig extends MonitorSchedulerConfig +{ + @JsonProperty + private Period emissionPeriod = new Period("PT1M"); + + @JsonProperty + public Period getEmissionPeriod() + { + return emissionPeriod; + } + + @Override + public Duration getEmitterPeriod() + { + return emissionPeriod.toStandardDuration(); + } +} diff --git a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java index 610514bcc39..28e68c7c17b 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java +++ b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java @@ -1,5 +1,6 @@ package com.metamx.druid.metrics; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; @@ -8,13 +9,13 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.JvmMonitor; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import java.util.List; @@ -44,8 +45,10 @@ public class MetricsModule implements Module @Override public void configure(Binder binder) { + JsonConfigProvider.bind(binder, "druid.monitoring", DruidMonitorSchedulerConfig.class); + binder.bind(JvmMonitor.class).in(LazySingleton.class); - binder.bind(SysMonitor.class).in(LazySingleton.class); + binder.bind(SysMonitor.class).in(LazySingleton.class); // TODO: allow for disabling of this monitor for (Class monitor : monitors) { binder.bind(monitor).in(LazySingleton.class); @@ -53,7 +56,11 @@ public class MetricsModule implements Module } @Provides @ManageLifecycle - public MonitorScheduler getMonitorScheduler(MonitorSchedulerConfig config, ServiceEmitter emitter, Injector injector) + public MonitorScheduler getMonitorScheduler( + Supplier config, + ServiceEmitter emitter, + Injector injector + ) { List monitors = Lists.newArrayList(); @@ -68,7 +75,7 @@ public class MetricsModule implements Module } return new MonitorScheduler( - config, + config.get(), Execs.scheduledSingleThreaded("MonitorScheduler-%s"), emitter, monitors From 149c2e3560935fb5590da4f86b620012cff19e47 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 20 Jun 2013 17:19:51 -0700 Subject: [PATCH 07/92] 1) A bunch more changes to make things actually run. Not quite done, but close! --- client/pom.xml | 2 +- .../druid/initialization/EmitterModule.java | 19 +++++++++++---- .../initialization/LogEmitterModule.java | 4 +++- .../com/metamx/druid/db/DbTablesConfig.java | 9 +------- .../druid/guice/JsonConfigProvider.java | 21 +++++++++++++++-- .../metamx/druid/guice/LifecycleModule.java | 23 +++++++++++++++---- pom.xml | 4 ++-- server/pom.xml | 2 +- .../com/metamx/druid/http/MasterMain.java | 3 +-- 9 files changed, 61 insertions(+), 26 deletions(-) diff --git a/client/pom.xml b/client/pom.xml index 10268136193..14632df35df 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -164,7 +164,7 @@ javax.servlet - servlet-api + javax.servlet-api org.slf4j diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java index 578f8e3ea16..7b2602efb3a 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -98,11 +98,10 @@ public class EmitterModule implements Module { final List> emitterBindings = injector.findBindingsByType(new TypeLiteral(){}); - for (Binding binding : emitterBindings) { - if (Names.named(emitterType).equals(binding.getKey().getAnnotation())) { - emitter = binding.getProvider().get(); - break; - } + emitter = findEmitter(emitterType, emitterBindings); + + if (emitter == null) { + emitter = findEmitter(LogEmitterModule.EMITTER_TYPE, emitterBindings); } if (emitter == null) { @@ -117,6 +116,16 @@ public class EmitterModule implements Module } } + private Emitter findEmitter(String emitterType, List> emitterBindings) + { + for (Binding binding : emitterBindings) { + if (Names.named(emitterType).equals(binding.getKey().getAnnotation())) { + return binding.getProvider().get(); + } + } + return null; + } + @Override public Emitter get() diff --git a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java index 237a87ec1b6..e64c259098b 100644 --- a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java @@ -34,13 +34,15 @@ import com.metamx.emitter.core.LoggingEmitterConfig; */ public class LogEmitterModule implements Module { + public static final String EMITTER_TYPE = "logging"; + @Override public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.emitter.logging", LoggingEmitterConfig.class); } - @Provides @LazySingleton @Named("logging") + @Provides @LazySingleton @Named(EMITTER_TYPE) public Emitter makeEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) { return new LoggingEmitter(config, jsonMapper); diff --git a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java index 54567b3fe74..9065f0d1bf2 100644 --- a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java @@ -2,7 +2,6 @@ package com.metamx.druid.db; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.common.ISE; import javax.validation.constraints.NotNull; @@ -18,22 +17,16 @@ public class DbTablesConfig @NotNull private final String base; - @NotNull private final String segmentsTable; - @NotNull private final String rulesTable; - @NotNull private final String configTable; - @NotNull private final String tasksTable; - @NotNull private final String taskLogTable; - @NotNull private final String taskLockTable; @JsonCreator @@ -60,7 +53,7 @@ public class DbTablesConfig { if (explicitTableName == null) { if (base == null) { - throw new ISE("table[%s] unknown! Both base and %s were null!", defaultSuffix, defaultSuffix); + return null; } return String.format("%s_%s", base, defaultSuffix); } diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java index 8c633060333..855ab742ff3 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -52,6 +52,8 @@ public class JsonConfigProvider implements Provider> private Properties props; private JsonConfigurator configurator; + private Supplier retVal = null; + public JsonConfigProvider( String propertyBase, Class classToProvide @@ -74,7 +76,22 @@ public class JsonConfigProvider implements Provider> @Override public Supplier get() { - final T config = configurator.configurate(props, propertyBase, classToProvide); - return Suppliers.ofInstance(config); + if (retVal != null) { + return retVal; + } + + try { + final T config = configurator.configurate(props, propertyBase, classToProvide); + retVal = Suppliers.ofInstance(config); + } + catch (RuntimeException e) { + // When a runtime exception gets thrown out, this provider will get called again if the object is asked for again. + // This will have the same failed result, 'cause when it's called no parameters will have actually changed. For + // Guice will then report the same error multiple times, which is pretty annoying. Cache a null supplier and + // return that instead. This is technically enforcing a singleton, but such is life. + retVal = Suppliers.ofInstance(null); + throw e; + } + return retVal; } } diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java index b31bbaddd11..7e0588449ed 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -60,14 +60,29 @@ public class LifecycleModule implements Module } @Provides @LazySingleton - public Lifecycle getLifecycle(Injector injector) + public Lifecycle getLifecycle(final Injector injector) { Lifecycle lifecycle = new Lifecycle(); scope.setLifecycle(lifecycle); - for (Key key : eagerClasses) { - injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. - } + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + for (Key key : eagerClasses) { + injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. + } + } + + @Override + public void stop() + { + + } + } + ); return lifecycle; } diff --git a/pom.xml b/pom.xml index b5767dcdbb6..a5a570ff1d1 100644 --- a/pom.xml +++ b/pom.xml @@ -302,8 +302,8 @@ javax.servlet - servlet-api - 3.0 + javax.servlet-api + 3.0.1 org.slf4j diff --git a/server/pom.xml b/server/pom.xml index 0ae6ac0560f..48289cca972 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -163,7 +163,7 @@ javax.servlet - servlet-api + javax.servlet-api org.slf4j 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 f4c3841f1a1..9f43e482d69 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -76,9 +76,8 @@ public class MasterMain final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); try { - lifecycle.start(); - Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); + lifecycle.start(); } catch (Throwable t) { log.error(t, "Error when starting up. Failing."); From e892d14f772f47b28adbcf884e578afd4b123ccb Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 9 Jul 2013 16:16:56 -0700 Subject: [PATCH 08/92] 1) Configuration fixes, slowly gonna make the MasterMain actually start up! --- .../druid/initialization/CuratorDiscoveryConfig.java | 3 +++ .../main/java/com/metamx/druid/guice/HttpClientModule.java | 2 +- server/src/main/java/com/metamx/druid/http/MasterMain.java | 7 +++++-- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java index 4fc6c28360e..00221db1a1d 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java @@ -21,11 +21,14 @@ package com.metamx.druid.initialization; import com.fasterxml.jackson.annotation.JsonProperty; +import javax.validation.constraints.NotNull; + /** */ public class CuratorDiscoveryConfig { @JsonProperty + @NotNull private String path = null; public String getPath() diff --git a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java index ac5eb18df16..757c75b3a71 100644 --- a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java @@ -43,7 +43,7 @@ public class HttpClientModule implements Module public Duration getReadTimeout() { - return readTimeout.toStandardDuration(); + return readTimeout == null ? null : readTimeout.toStandardDuration(); } } 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 9f43e482d69..aef21c936b2 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,8 +19,10 @@ package com.metamx.druid.http; +import com.google.common.base.Supplier; import com.google.inject.Injector; import com.google.inject.Key; +import com.google.inject.TypeLiteral; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -71,12 +73,13 @@ public class MasterMain final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - final DruidNodeConfig nodeConfig = injector.getInstance(DruidNodeConfig.class); + final Supplier nodeConfig = injector.getInstance(Key.get(new TypeLiteral>(){})); final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); try { - Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); + // TODO: Make the announcement work through the lifecycle + Initialization.announceDefaultService(nodeConfig.get(), serviceAnnouncer, lifecycle); lifecycle.start(); } catch (Throwable t) { From e885cb5a4d200162325d7dc015f26155d8be1afc Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 18 Jul 2013 11:15:39 -0700 Subject: [PATCH 09/92] 1) Switch CuratorDiscoveryConfig to not be an abstract class 'cause Jackson cannot instantiate them --- .../com/metamx/druid/initialization/CuratorDiscoveryConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java index ee0f7ced76e..4fc6c28360e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public abstract class CuratorDiscoveryConfig +public class CuratorDiscoveryConfig { @JsonProperty private String path = null; From 85a35201b7f1ba7df6210dd19f6f4d1c4dd5086d Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 19 Jul 2013 16:29:54 -0700 Subject: [PATCH 10/92] 1) The Master compiles and runs with Guice bindings!!!! 2) The service discovery stuff really needs to be reworked. It's unhappy now. --- .../druid/initialization/Initialization.java | 3 +- .../metamx/druid/db/DbConnectorConfig.java | 13 +++++++ .../druid/guice/DruidSecondaryModule.java | 2 ++ .../druid/guice/JsonConfigProvider.java | 2 +- .../metamx/druid/guice/JsonConfigurator.java | 11 ++++-- .../metamx/druid/guice/LifecycleModule.java | 30 ++++++---------- pom.xml | 2 +- .../indexing/IndexingServiceSelector.java | 3 +- .../metamx/druid/db/DatabaseRuleManager.java | 3 +- .../druid/db/DatabaseRuleManagerProvider.java | 36 +++++++++++-------- .../com/metamx/druid/guice/MasterModule.java | 26 ++++++++++++++ .../com/metamx/druid/master/DruidMaster.java | 6 ++-- 12 files changed, 92 insertions(+), 45 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index faf7e5dbb65..aa82268290e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -265,7 +265,8 @@ public class Initialization throw Throwables.propagate(e); } } - } + }, + Lifecycle.Stage.LAST ); return serviceDiscovery; diff --git a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java index bd9e15f669d..074551541d0 100644 --- a/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbConnectorConfig.java @@ -76,4 +76,17 @@ public class DbConnectorConfig public String getValidationQuery() { return validationQuery; } + + @Override + public String toString() + { + return "DbConnectorConfig{" + + "createTables=" + createTables + + ", connectURI='" + connectURI + '\'' + + ", user='" + user + '\'' + + ", password=****" + + ", useValidationQuery=" + useValidationQuery + + ", validationQuery='" + validationQuery + '\'' + + '}'; + } } diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java index 43f54207652..fae107c92df 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -3,6 +3,7 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; import com.metamx.druid.jackson.Json; import com.metamx.druid.jackson.Smile; @@ -46,6 +47,7 @@ public class DruidSecondaryModule implements Module binder.install(new DruidGuiceExtensions()); binder.bind(Properties.class).toInstance(properties); binder.bind(ConfigurationObjectFactory.class).toInstance(factory); + binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMapper); binder.bind(ObjectMapper.class).annotatedWith(Smile.class).toInstance(smileMapper); binder.bind(Validator.class).toInstance(validator); diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java index 855ab742ff3..a171e76e40a 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -86,7 +86,7 @@ public class JsonConfigProvider implements Provider> } catch (RuntimeException e) { // When a runtime exception gets thrown out, this provider will get called again if the object is asked for again. - // This will have the same failed result, 'cause when it's called no parameters will have actually changed. For + // This will have the same failed result, 'cause when it's called no parameters will have actually changed. // Guice will then report the same error multiple times, which is pretty annoying. Cache a null supplier and // return that instead. This is technically enforcing a singleton, but such is life. retVal = Suppliers.ofInstance(null); diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java index 2f35bed160b..9db904ee946 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -52,10 +52,15 @@ public class JsonConfigurator final String propValue = props.getProperty(prop); Object value; try { - value = jsonMapper.readValue(propValue, Object.class); + // If it's a String Jackson wants it to be quoted, so check if it's not an object or array and quote. + String modifiedPropValue = propValue; + if (! (modifiedPropValue.startsWith("[") || modifiedPropValue.startsWith("{"))) { + modifiedPropValue = String.format("\"%s\"", modifiedPropValue); + } + value = jsonMapper.readValue(modifiedPropValue, Object.class); } catch (IOException e) { - log.debug("Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue); + log.info(e, "Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue); value = propValue; } @@ -89,6 +94,8 @@ public class JsonConfigurator ); } + log.info("Loaded class[%s] as [%s]", clazz, config); + return config; } } diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java index 7e0588449ed..be380212eee 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -62,27 +62,17 @@ public class LifecycleModule implements Module @Provides @LazySingleton public Lifecycle getLifecycle(final Injector injector) { - Lifecycle lifecycle = new Lifecycle(); - scope.setLifecycle(lifecycle); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - for (Key key : eagerClasses) { - injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. - } - } - - @Override - public void stop() - { - - } + Lifecycle lifecycle = new Lifecycle(){ + @Override + public void start() throws Exception + { + for (Key key : eagerClasses) { + injector.getInstance(key); // Pull the key so as to "eagerly" load up the class. } - ); + super.start(); + } + }; + scope.setLifecycle(lifecycle); return lifecycle; } diff --git a/pom.xml b/pom.xml index a5a570ff1d1..206dfd9d74c 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ UTF-8 0.23.0-SNAPSHOT - 2.0.2-21-22 + 2.1.0-incubating diff --git a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java index d6d07dcafe8..41259c1b28d 100644 --- a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java +++ b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java @@ -9,6 +9,7 @@ import com.metamx.druid.client.selector.Server; import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; +import javax.annotation.Nullable; import java.io.IOException; /** @@ -21,7 +22,7 @@ public class IndexingServiceSelector implements DiscoverySelector @Inject public IndexingServiceSelector( - @IndexingService ServiceProvider serviceProvider + @Nullable @IndexingService ServiceProvider serviceProvider ) { this.serviceProvider = serviceProvider; } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index b308296fb31..bc9347561eb 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -33,6 +33,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.guice.ManageLifecycle; +import com.metamx.druid.jackson.Json; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; import org.joda.time.DateTime; @@ -129,7 +130,7 @@ public class DatabaseRuleManager @Inject public DatabaseRuleManager( - ObjectMapper jsonMapper, + @Json ObjectMapper jsonMapper, Supplier config, Supplier dbTables, IDBI dbi diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java index a2760068cc1..3b42bf26340 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManagerProvider.java @@ -21,6 +21,7 @@ package com.metamx.druid.db; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.inject.Inject; import com.google.inject.Provider; import com.metamx.common.lifecycle.Lifecycle; @@ -54,25 +55,30 @@ public class DatabaseRuleManagerProvider implements Provider serviceDiscovery) { + // TODO: This service discovery stuff is really really janky. It needs to be reworked. + if (config.getMergerServiceName() == null) { + return new ServiceProvider() + { + @Override + public void start() throws Exception + { + + } + + @Override + public ServiceInstance getInstance() throws Exception + { + return null; + } + + @Override + public void close() throws IOException + { + + } + }; + } return serviceDiscovery.serviceProviderBuilder().serviceName(config.getMergerServiceName()).build(); } 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 2a5f5a77fa3..4561fe502ac 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -446,17 +446,17 @@ public class DruidMaster curator, ZKPaths.makePath(zkPaths.getMasterPath(), MASTER_OWNER_NODE), config.getHost() ); - newLeaderLatch.attachListener( + newLeaderLatch.addListener( new LeaderLatchListener() { @Override - public void becomeMaster() + public void isLeader() { DruidMaster.this.becomeMaster(); } @Override - public void stopBeingMaster() + public void notLeader() { DruidMaster.this.stopBeingMaster(); } From 02ffd805c8f3f90afa6e8ac3b790c29235ed8721 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 23 Jul 2013 14:55:36 -0700 Subject: [PATCH 11/92] 1) Move DbConnector into its own module 2) Move JacksonConfig stuff into its own module 3) Fix LogEmitterModule to inject properly with a Supplier<> 4) Set Guice to require explicit bindings, to protect from weirdness like was initially experienced with LoggingEmitterConfig 5) Master fires up and appears to all work with Guice! Whoot! --- .../druid/initialization/EmitterModule.java | 3 +- .../initialization/HttpEmitterModule.java | 3 +- .../druid/initialization/Initialization.java | 17 ++++- .../initialization/LogEmitterModule.java | 9 ++- .../druid/config/ConfigManagerProvider.java | 74 ------------------- .../metamx/druid/guice/DbConnectorModule.java | 29 ++++++++ .../druid/guice/DruidGuiceExtensions.java | 1 + .../guice/JacksonConfigManagerModule.java | 52 +++++++++++++ .../metamx/druid/guice/ManageLifecycle.java | 2 + pom.xml | 2 +- .../com/metamx/druid/guice/MasterModule.java | 25 ++----- .../com/metamx/druid/http/MasterMain.java | 4 + 12 files changed, 119 insertions(+), 102 deletions(-) delete mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java create mode 100644 common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java index 7b2602efb3a..c3b1259934e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -34,6 +34,7 @@ import com.google.inject.name.Names; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.service.ServiceEmitter; @@ -71,7 +72,7 @@ public class EmitterModule implements Module } @Provides - @LazySingleton + @ManageLifecycle public ServiceEmitter getServiceEmitter(Supplier configSupplier, Emitter emitter) { final DruidNodeConfig config = configSupplier.get(); diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java index 9f9c5eb5de0..7c2c9f2ca69 100644 --- a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java @@ -28,6 +28,7 @@ import com.google.inject.util.Providers; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.core.HttpPostEmitter; import com.metamx.http.client.HttpClientConfig; @@ -49,7 +50,7 @@ public class HttpEmitterModule implements Module binder.bind(SSLContext.class).toProvider(Providers.of(null)).in(LazySingleton.class); } - @Provides @LazySingleton @Named("http") + @Provides @ManageLifecycle @Named("http") public Emitter getEmitter(Supplier config, @Nullable SSLContext sslContext, Lifecycle lifecycle) { final HttpClientConfig.Builder builder = HttpClientConfig diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index aa82268290e..8d9e90a1069 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -25,6 +25,7 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Module; @@ -390,7 +391,21 @@ public class Initialization new DruidGuiceExtensions(), new JacksonModule(), new PropertiesModule("runtime.properties"), - new ConfigModule() + new ConfigModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(DruidSecondaryModule.class); + + for (Object module : modules) { + if (module instanceof Class) { + binder.bind((Class) module); + } + } + } + } ); List actualModules = Lists.newArrayList(); diff --git a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java index e64c259098b..5e9b7bd71a7 100644 --- a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java @@ -20,12 +20,13 @@ package com.metamx.druid.initialization; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.name.Named; import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.core.LoggingEmitter; import com.metamx.emitter.core.LoggingEmitterConfig; @@ -42,9 +43,9 @@ public class LogEmitterModule implements Module JsonConfigProvider.bind(binder, "druid.emitter.logging", LoggingEmitterConfig.class); } - @Provides @LazySingleton @Named(EMITTER_TYPE) - public Emitter makeEmitter(LoggingEmitterConfig config, ObjectMapper jsonMapper) + @Provides @ManageLifecycle @Named(EMITTER_TYPE) + public Emitter makeEmitter(Supplier config, ObjectMapper jsonMapper) { - return new LoggingEmitter(config, jsonMapper); + return new LoggingEmitter(config.get(), jsonMapper); } } diff --git a/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java b/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java deleted file mode 100644 index 032d2668152..00000000000 --- a/common/src/main/java/com/metamx/druid/config/ConfigManagerProvider.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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.google.common.base.Supplier; -import com.google.inject.Inject; -import com.google.inject.Provider; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbTablesConfig; - -/** - */ -public class ConfigManagerProvider implements Provider -{ - private final DbConnector dbConnector; - private final Supplier dbTables; - private final Supplier config; - private final Lifecycle lifecycle; - - @Inject - ConfigManagerProvider( - DbConnector dbConnector, - Supplier dbTables, - Supplier config, - Lifecycle lifecycle - ) - { - this.dbConnector = dbConnector; - this.dbTables = dbTables; - this.config = config; - this.lifecycle = lifecycle; - } - - @Override - public ConfigManager get() - { - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - dbConnector.createConfigTable(); - } - - @Override - public void stop() - { - - } - } - ); - - return new ConfigManager(dbConnector.getDBI(), dbTables, config); - } -} diff --git a/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java b/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java new file mode 100644 index 00000000000..624bd030175 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java @@ -0,0 +1,29 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; +import org.skife.jdbi.v2.IDBI; + +/** + */ +public class DbConnectorModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class); + JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class); + + binder.bind(DbConnector.class); + } + + @Provides @LazySingleton + public IDBI getDbi(final DbConnector dbConnector) + { + return dbConnector.getDBI(); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java index e999b725aa2..1010e56188c 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java @@ -10,6 +10,7 @@ public class DruidGuiceExtensions implements Module @Override public void configure(Binder binder) { + binder.requireExplicitBindings(); binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); } } diff --git a/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java b/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java new file mode 100644 index 00000000000..05838c04383 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java @@ -0,0 +1,52 @@ +package com.metamx.druid.guice; + +import com.google.common.base.Supplier; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.lifecycle.Lifecycle; +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.DbTablesConfig; + +/** + */ +public class JacksonConfigManagerModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.manager.config", ConfigManagerConfig.class); + binder.bind(JacksonConfigManager.class); + } + + @Provides @ManageLifecycle + public ConfigManager getConfigManager( + final DbConnector dbConnector, + final Supplier dbTables, + final Supplier config, + final Lifecycle lifecycle + ) + { + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + dbConnector.createConfigTable(); + } + + @Override + public void stop() + { + + } + } + ); + + return new ConfigManager(dbConnector.getDBI(), dbTables, config); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java index 35752a64b0d..24e269ae00b 100644 --- a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java +++ b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java @@ -10,6 +10,8 @@ import static java.lang.annotation.RetentionPolicy.RUNTIME; /** * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} + * + * This Scope gets defined by {@link com.metamx.druid.guice.LifecycleModule} */ @Target({ ElementType.TYPE, ElementType.METHOD }) @Retention(RUNTIME) diff --git a/pom.xml b/pom.xml index 206dfd9d74c..95436f18d3f 100644 --- a/pom.xml +++ b/pom.xml @@ -61,7 +61,7 @@ com.metamx emitter - 0.2.3 + 0.2.5-SNAPSHOT com.metamx diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index 1a3b693fc94..5a4f38d0917 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -8,6 +8,7 @@ import com.google.inject.TypeLiteral; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceClient; @@ -15,28 +16,22 @@ import com.metamx.druid.client.indexing.IndexingServiceSelector; import com.metamx.druid.client.selector.DiscoverySelector; import com.metamx.druid.client.selector.Server; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.config.ConfigManager; -import com.metamx.druid.config.ConfigManagerConfig; -import com.metamx.druid.config.ConfigManagerProvider; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseRuleManagerProvider; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DatabaseSegmentManagerProvider; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; -import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.http.MasterRedirectInfo; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; -import org.skife.jdbi.v2.IDBI; import java.io.IOException; @@ -51,12 +46,12 @@ public class MasterModule implements Module ConfigProvider.bind(binder, ZkPathsConfig.class); ConfigProvider.bind(binder, ServerInventoryViewConfig.class); - JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class); - JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class); - JsonConfigProvider.bind(binder, "druid.manager.config", ConfigManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); + binder.bind(DruidMaster.class); + binder.bind(ServerInventoryView.class); + binder.bind(DatabaseSegmentManager.class) .toProvider(DatabaseSegmentManagerProvider.class) .in(ManageLifecycle.class); @@ -65,10 +60,6 @@ public class MasterModule implements Module .toProvider(DatabaseRuleManagerProvider.class) .in(ManageLifecycle.class); - binder.bind(ConfigManager.class) - .toProvider(ConfigManagerProvider.class) - .in(ManageLifecycle.class); - binder.bind(new TypeLiteral>(){}) .annotatedWith(IndexingService.class) .to(IndexingServiceSelector.class) @@ -107,12 +98,6 @@ public class MasterModule implements Module return serviceDiscovery.serviceProviderBuilder().serviceName(config.getMergerServiceName()).build(); } - @Provides @LazySingleton - public IDBI getDbi(final DbConnector dbConnector) - { - return dbConnector.getDBI(); - } - @Provides @LazySingleton public LoadQueueTaskMaster getLoadQueueTaskMaster(CuratorFramework curator, ObjectMapper jsonMapper) { 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 aef21c936b2..bc4fc6189f6 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -29,7 +29,9 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; @@ -63,6 +65,8 @@ public class MasterMain new LifecycleModule(Key.get(MonitorScheduler.class), Key.get(DruidMaster.class)), EmitterModule.class, HttpClientModule.class, + DbConnectorModule.class, + JacksonConfigManagerModule.class, CuratorModule.class, new MetricsModule(), DiscoveryModule.class, From 6b9963b472f6a632090a4bff01201e3e7e052252 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 26 Jul 2013 14:03:51 -0700 Subject: [PATCH 12/92] 1) Move the @Json and @Smile annotations to com.metamx.druid.guice.annotations 2) Changed ServiceAnnouncer to take a DruidNode object 3) Remove ServiceInstanceFactory interface and implementations 4) Add registrations to DiscoveryModule so that you can register nodes that should be announced on startup 5) Make the set of default Monitors configurable --- .../AddressPortServiceInstanceFactory.java | 27 --- .../discovery/CuratorServiceAnnouncer.java | 58 +++++-- .../curator/discovery/DiscoveryModule.java | 162 ++++++++++++++++-- .../discovery/NoopServiceAnnouncer.java | 6 +- .../curator/discovery/ServiceAnnouncer.java | 7 +- .../discovery/ServiceInstanceFactory.java | 8 - .../com/metamx/druid/http/BrokerNode.java | 8 +- .../{DruidNodeConfig.java => DruidNode.java} | 16 +- .../druid/initialization/EmitterModule.java | 5 +- .../druid/initialization/Initialization.java | 30 +--- .../druid/guice/DruidSecondaryModule.java | 4 +- .../druid/guice/JsonConfigProvider.java | 43 ++++- .../metamx/druid/guice/JsonConfigurator.java | 2 +- .../metamx/druid/guice/LifecycleModule.java | 102 +++++++++-- .../{jackson => guice/annotations}/Json.java | 2 +- .../metamx/druid/guice/annotations/Self.java | 17 ++ .../{jackson => guice/annotations}/Smile.java | 2 +- .../metamx/druid/jackson/JacksonModule.java | 2 + .../common/index/ChatHandlerProvider.java | 35 ++-- .../coordinator/TaskMasterLifecycle.java | 4 +- .../http/IndexerCoordinatorNode.java | 9 +- .../config/ChatHandlerProviderConfig.java | 9 +- .../worker/executor/ExecutorNode.java | 17 +- .../metamx/druid/db/DatabaseRuleManager.java | 2 +- .../com/metamx/druid/guice/MasterModule.java | 2 +- .../com/metamx/druid/guice/ServerModule.java | 5 +- .../com/metamx/druid/http/MasterMain.java | 18 +- .../metamx/druid/metrics/MetricsModule.java | 55 ++++-- .../metamx/druid/metrics/MonitorsConfig.java | 35 ++++ 29 files changed, 489 insertions(+), 203 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/curator/discovery/AddressPortServiceInstanceFactory.java delete mode 100644 client/src/main/java/com/metamx/druid/curator/discovery/ServiceInstanceFactory.java rename client/src/main/java/com/metamx/druid/initialization/{DruidNodeConfig.java => DruidNode.java} (90%) rename common/src/main/java/com/metamx/druid/{jackson => guice/annotations}/Json.java (89%) create mode 100644 common/src/main/java/com/metamx/druid/guice/annotations/Self.java rename common/src/main/java/com/metamx/druid/{jackson => guice/annotations}/Smile.java (89%) create mode 100644 server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/AddressPortServiceInstanceFactory.java b/client/src/main/java/com/metamx/druid/curator/discovery/AddressPortServiceInstanceFactory.java deleted file mode 100644 index 3f4fa988655..00000000000 --- a/client/src/main/java/com/metamx/druid/curator/discovery/AddressPortServiceInstanceFactory.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.metamx.druid.curator.discovery; - -import com.google.common.base.Throwables; -import org.apache.curator.x.discovery.ServiceInstance; - -public class AddressPortServiceInstanceFactory implements ServiceInstanceFactory -{ - private final String address; - private final int port; - - public AddressPortServiceInstanceFactory(String address, int port) - { - this.address = address; - this.port = port; - } - - @Override - public ServiceInstance create(String service) - { - try { - return ServiceInstance.builder().name(service).address(address).port(port).build(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java index 035924a6e2b..dc277d927c6 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java @@ -1,8 +1,10 @@ package com.metamx.druid.curator.discovery; +import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.inject.Inject; -import com.metamx.common.logger.Logger; +import com.metamx.druid.initialization.DruidNode; +import com.metamx.emitter.EmittingLogger; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; @@ -13,56 +15,66 @@ import java.util.Map; */ public class CuratorServiceAnnouncer implements ServiceAnnouncer { - private static final Logger log = new Logger(CuratorServiceAnnouncer.class); + private static final EmittingLogger log = new EmittingLogger(CuratorServiceAnnouncer.class); private final ServiceDiscovery discovery; - private final ServiceInstanceFactory instanceFactory; private final Map> instanceMap = Maps.newHashMap(); private final Object monitor = new Object(); @Inject public CuratorServiceAnnouncer( - ServiceDiscovery discovery, - ServiceInstanceFactory instanceFactory + ServiceDiscovery discovery ) { this.discovery = discovery; - this.instanceFactory = instanceFactory; } @Override - public void announce(String service) throws Exception + public void announce(DruidNode service) { - final ServiceInstance instance; + final String serviceName = getServiceName(service); + final ServiceInstance instance; synchronized (monitor) { - if (instanceMap.containsKey(service)) { + if (instanceMap.containsKey(serviceName)) { log.warn("Ignoring request to announce service[%s]", service); return; } else { - instance = instanceFactory.create(service); - instanceMap.put(service, instance); + try { + instance = ServiceInstance.builder() + .name(serviceName) + .address(service.getHost()) + .port(service.getPort()) + .build(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + instanceMap.put(serviceName, instance); } } try { log.info("Announcing service[%s]", service); discovery.registerService(instance); - } catch (Exception e) { + } + catch (Exception e) { log.warn("Failed to announce service[%s]", service); synchronized (monitor) { - instanceMap.remove(service); + instanceMap.remove(serviceName); } } } @Override - public void unannounce(String service) throws Exception + public void unannounce(DruidNode service) { + final String serviceName = getServiceName(service); final ServiceInstance instance; synchronized (monitor) { - instance = instanceMap.get(service); + instance = instanceMap.get(serviceName); if (instance == null) { log.warn("Ignoring request to unannounce service[%s]", service); return; @@ -72,12 +84,20 @@ public class CuratorServiceAnnouncer implements ServiceAnnouncer log.info("Unannouncing service[%s]", service); try { discovery.unregisterService(instance); - } catch (Exception e) { - log.warn(e, "Failed to unannounce service[%s]", service); - } finally { + } + catch (Exception e) { + log.makeAlert(e, "Failed to unannounce service[%s], zombie znode perhaps in existence.", serviceName) + .addData("service", service) + .emit(); + } + finally { synchronized (monitor) { - instanceMap.remove(service); + instanceMap.remove(serviceName); } } } + + private String getServiceName(DruidNode service) { + return service.getServiceName().replaceAll("/", ":"); + } } diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java index aa386977403..16da46a1dd9 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java @@ -1,27 +1,175 @@ package com.metamx.druid.curator.discovery; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.collect.Lists; import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.TypeLiteral; +import com.google.inject.name.Named; +import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; +import javax.annotation.Nullable; +import java.lang.annotation.Annotation; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + /** + * The DiscoveryModule allows for the registration of Keys of DruidNode objects, which it intends to be + * automatically announced at the end of the lifecycle start. + * + * In order for this to work a ServiceAnnouncer instance *must* be injected and instantiated first. + * This can often be achieved by registering ServiceAnnouncer.class with the LifecycleModule. */ public class DiscoveryModule implements Module { + private static final String NAME = "DiscoveryModule:internal"; + + public final List>> nodesToAnnounce = new CopyOnWriteArrayList>>(); + public boolean configured = false; + + /** + * Requests that the un-annotated DruidNode instance be injected and published as part of the lifecycle. + * + * That is, this module will announce the DruidNode instance returned by + * injector.getInstance(Key.get(DruidNode.class)) automatically. + * Announcement will happen in the LAST stage of the Lifecycle + * + * @return this, for chaining. + */ + public DiscoveryModule registerDefault() + { + return registerKey(Key.get(new TypeLiteral>(){})); + } + + /** + * Requests that the annotated DruidNode instance be injected and published as part of the lifecycle. + * + * That is, this module will announce the DruidNode instance returned by + * injector.getInstance(Key.get(DruidNode.class, annotation)) automatically. + * Announcement will happen in the LAST stage of the Lifecycle + * + * @param annotation The annotation instance to use in finding the DruidNode instance, usually a Named annotation + * @return this, for chaining. + */ + public DiscoveryModule register(Annotation annotation) + { + return registerKey(Key.get(new TypeLiteral>(){}, annotation)); + } + + /** + * Requests that the annotated DruidNode instance be injected and published as part of the lifecycle. + * + * That is, this module will announce the DruidNode instance returned by + * injector.getInstance(Key.get(DruidNode.class, annotation)) automatically. + * Announcement will happen in the LAST stage of the Lifecycle + * + * @param annotation The annotation class to use in finding the DruidNode instance + * @return this, for chaining + */ + public DiscoveryModule register(Class annotation) + { + return registerKey(Key.get(new TypeLiteral>(){}, annotation)); + } + + /** + * Requests that the keyed DruidNode instance be injected and published as part of the lifecycle. + * + * That is, this module will announce the DruidNode instance returned by + * injector.getInstance(Key.get(DruidNode.class, annotation)) automatically. + * Announcement will happen in the LAST stage of the Lifecycle + * + * @param key The key to use in finding the DruidNode instance + * @return this, for chaining + */ + public DiscoveryModule registerKey(Key> key) + { + synchronized (nodesToAnnounce) { + Preconditions.checkState(!configured, "Cannot register key[%s] after configuration.", key); + } + nodesToAnnounce.add(key); + return this; + } + @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.discovery.curator", CuratorDiscoveryConfig.class); - binder.bind(ServiceAnnouncer.class).to(CuratorServiceAnnouncer.class).in(LazySingleton.class); + synchronized (nodesToAnnounce) { + configured = true; + JsonConfigProvider.bind(binder, "druid.discovery.curator", CuratorDiscoveryConfig.class); + + binder.bind(CuratorServiceAnnouncer.class).in(LazySingleton.class); + + // We bind this eagerly so that it gets instantiated and registers stuff with Lifecycle as a side-effect + binder.bind(ServiceAnnouncer.class) + .to(Key.get(CuratorServiceAnnouncer.class, Names.named(NAME))) + .asEagerSingleton(); + } + } + + @Provides @LazySingleton @Named(NAME) + public CuratorServiceAnnouncer getServiceAnnouncer( + final CuratorServiceAnnouncer announcer, + final Injector injector, + final Lifecycle lifecycle + ) + { + lifecycle.addHandler( + new Lifecycle.Handler() + { + private volatile List> nodes = null; + + @Override + public void start() throws Exception + { + if (nodes == null) { + nodes = Lists.transform( + nodesToAnnounce, + new Function>, Supplier>() + { + @Nullable + @Override + public Supplier apply( + @Nullable Key> input + ) + { + return injector.getInstance(input); + } + } + ); + } + + for (Supplier node : nodes) { + announcer.announce(node.get()); + } + } + + @Override + public void stop() + { + if (nodes != null) { + for (Supplier node : nodes) { + announcer.unannounce(node.get()); + } + } + } + }, + Lifecycle.Stage.LAST + ); + + return announcer; } @Provides @LazySingleton @@ -33,12 +181,4 @@ public class DiscoveryModule implements Module { return Initialization.makeServiceDiscoveryClient(curator, config.get(), lifecycle); } - - @Provides @LazySingleton - public ServiceInstanceFactory getServiceInstanceFactory( - Supplier nodeConfig - ) - { - return Initialization.makeServiceInstanceFactory(nodeConfig.get()); - } } diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java index 782739f7a24..87c65bc9837 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java @@ -1,18 +1,20 @@ package com.metamx.druid.curator.discovery; +import com.metamx.druid.initialization.DruidNode; + /** * Does nothing. */ public class NoopServiceAnnouncer implements ServiceAnnouncer { @Override - public void unannounce(String service) + public void announce(DruidNode node) { } @Override - public void announce(String service) + public void unannounce(DruidNode node) { } diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java index 4e91122423a..bdd9361fd5d 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java @@ -1,11 +1,14 @@ package com.metamx.druid.curator.discovery; +import com.metamx.druid.initialization.DruidNode; + /** * Announces our ability to serve a particular function. Multiple users may announce the same service, in which * case they are treated as interchangeable instances of that service. */ public interface ServiceAnnouncer { - public void announce(String service) throws Exception; - public void unannounce(String service) throws Exception; + public void announce(DruidNode node); + + public void unannounce(DruidNode node); } diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceInstanceFactory.java b/client/src/main/java/com/metamx/druid/curator/discovery/ServiceInstanceFactory.java deleted file mode 100644 index 8f8cc19c0bd..00000000000 --- a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceInstanceFactory.java +++ /dev/null @@ -1,8 +0,0 @@ -package com.metamx.druid.curator.discovery; - -import org.apache.curator.x.discovery.ServiceInstance; - -public interface ServiceInstanceFactory -{ - public ServiceInstance create(String service); -} diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java index 4fbdf78f0f0..a68ff4def5a 100644 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ b/client/src/main/java/com/metamx/druid/http/BrokerNode.java @@ -45,7 +45,7 @@ import com.metamx.druid.client.cache.MemcachedCacheConfig; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.query.QueryToolChestWarehouse; @@ -234,16 +234,14 @@ public class BrokerNode extends QueryableNode if (useDiscovery) { final Lifecycle lifecycle = getLifecycle(); final CuratorDiscoveryConfig curatorDiscoveryConfig = getConfigFactory().build(CuratorDiscoveryConfig.class); - final DruidNodeConfig nodeConfig = getConfigFactory().build(DruidNodeConfig.class); + final DruidNode nodeConfig = getConfigFactory().build(DruidNode.class); final CuratorFramework curatorFramework = Initialization.makeCuratorFramework( getConfigFactory().build(CuratorConfig.class), lifecycle ); final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( curatorFramework, curatorDiscoveryConfig, lifecycle ); - final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer( - nodeConfig, serviceDiscovery - ); + final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer(serviceDiscovery); Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); } } diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java similarity index 90% rename from client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java rename to client/src/main/java/com/metamx/druid/initialization/DruidNode.java index c0ab3491d4b..bd8be448e66 100644 --- a/client/src/main/java/com/metamx/druid/initialization/DruidNodeConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java @@ -29,7 +29,7 @@ import javax.validation.constraints.NotNull; /** */ -public class DruidNodeConfig +public class DruidNode { @NotNull private String serviceName = null; @@ -41,9 +41,9 @@ public class DruidNodeConfig private int port = -1; @JsonCreator - public DruidNodeConfig( - @JsonProperty("host") String host, + public DruidNode( @JsonProperty("service") String serviceName, + @JsonProperty("host") String host, @JsonProperty("port") Integer port ) { @@ -99,4 +99,14 @@ public class DruidNodeConfig { return port; } + + @Override + public String toString() + { + return "DruidNode{" + + "serviceName='" + serviceName + '\'' + + ", host='" + host + '\'' + + ", port=" + port + + '}'; + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java index c3b1259934e..889b440bf80 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java @@ -35,6 +35,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.guice.ManageLifecycle; +import com.metamx.druid.guice.annotations.Self; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.service.ServiceEmitter; @@ -73,9 +74,9 @@ public class EmitterModule implements Module @Provides @ManageLifecycle - public ServiceEmitter getServiceEmitter(Supplier configSupplier, Emitter emitter) + public ServiceEmitter getServiceEmitter(@Self Supplier configSupplier, Emitter emitter) { - final DruidNodeConfig config = configSupplier.get(); + final DruidNode config = configSupplier.get(); final ServiceEmitter retVal = new ServiceEmitter(config.getServiceName(), config.getHost(), emitter); EmittingLogger.registerEmitter(retVal); return retVal; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 8d9e90a1069..81bb6e2f750 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -36,10 +36,8 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; -import com.metamx.druid.curator.discovery.AddressPortServiceInstanceFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.guice.DruidGuiceExtensions; import com.metamx.druid.guice.DruidSecondaryModule; import com.metamx.druid.http.EmittingRequestLogger; @@ -274,39 +272,35 @@ public class Initialization } public static ServiceAnnouncer makeServiceAnnouncer( - DruidNodeConfig config, ServiceDiscovery serviceDiscovery ) { - final ServiceInstanceFactory serviceInstanceFactory = makeServiceInstanceFactory(config); - return new CuratorServiceAnnouncer(serviceDiscovery, serviceInstanceFactory); + return new CuratorServiceAnnouncer(serviceDiscovery); } public static void announceDefaultService( - final DruidNodeConfig nodeConfig, + final DruidNode nodeConfig, final ServiceAnnouncer serviceAnnouncer, final Lifecycle lifecycle ) throws Exception { - final String service = nodeConfig.getServiceName().replace('/', ':'); - lifecycle.addHandler( new Lifecycle.Handler() { @Override public void start() throws Exception { - serviceAnnouncer.announce(service); + serviceAnnouncer.announce(nodeConfig); } @Override public void stop() { try { - serviceAnnouncer.unannounce(service); + serviceAnnouncer.unannounce(nodeConfig); } catch (Exception e) { - log.warn(e, "Failed to unannouce default service[%s]", service); + log.warn(e, "Failed to unannouce default service[%s]", nodeConfig.getServiceName()); } } } @@ -371,20 +365,6 @@ public class Initialization ); } - public static ServiceInstanceFactory makeServiceInstanceFactory(DruidNodeConfig config) - { - final String host = config.getHost(); - final String address; - final int colon = host.indexOf(':'); - if (colon < 0) { - address = host; - } else { - address = host.substring(0, colon); - } - - return new AddressPortServiceInstanceFactory(address, config.getPort()); - } - public static Injector makeInjector(final Object... modules) { final Injector baseInjector = Guice.createInjector( diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java index fae107c92df..9d60f5efcc8 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -5,8 +5,8 @@ import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Key; import com.google.inject.Module; -import com.metamx.druid.jackson.Json; -import com.metamx.druid.jackson.Smile; +import com.metamx.druid.guice.annotations.Json; +import com.metamx.druid.guice.annotations.Smile; import org.skife.config.ConfigurationObjectFactory; import javax.validation.Validator; diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java index a171e76e40a..d85a99d02b3 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -27,6 +27,7 @@ import com.google.inject.Key; import com.google.inject.Provider; import com.google.inject.util.Types; +import java.lang.annotation.Annotation; import java.util.Properties; /** @@ -36,9 +37,45 @@ public class JsonConfigProvider implements Provider> @SuppressWarnings("unchecked") public static void bind(Binder binder, String propertyBase, Class classToProvide) { - binder.bind(Key.get(Types.newParameterizedType(Supplier.class, classToProvide))) - .toProvider((Provider) of(propertyBase, classToProvide)) - .in(LazySingleton.class); + bind( + binder, + propertyBase, + classToProvide, + (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide)) + ); + } + + @SuppressWarnings("unchecked") + public static void bind(Binder binder, String propertyBase, Class classToProvide, Annotation annotation) + { + bind( + binder, + propertyBase, + classToProvide, + (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation) + ); + } + + @SuppressWarnings("unchecked") + public static void bind( + Binder binder, + String propertyBase, + Class classToProvide, + Class annotation + ) + { + bind( + binder, + propertyBase, + classToProvide, + (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation) + ); + } + + @SuppressWarnings("unchecked") + public static void bind(Binder binder, String propertyBase, Class clazz, Key> key) + { + binder.bind(key).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class); } public static JsonConfigProvider of(String propertyBase, Class classToProvide) diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java index 9db904ee946..2d642260562 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -94,7 +94,7 @@ public class JsonConfigurator ); } - log.info("Loaded class[%s] as [%s]", clazz, config); + log.info("Loaded class[%s] from props[%s] as [%s]", clazz, propertyBase, config); return config; } diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java index be380212eee..3d5ee5c1dac 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -19,6 +19,7 @@ package com.metamx.druid.guice; +import com.google.common.base.Preconditions; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -26,37 +27,116 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.lifecycle.Lifecycle; +import java.lang.annotation.Annotation; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; + /** * A Module to add lifecycle management to the injector. {@link DruidGuiceExtensions} must also be included. */ public class LifecycleModule implements Module { private final LifecycleScope scope = new LifecycleScope(); - private final Key[] eagerClasses; + private final List> eagerClasses = new CopyOnWriteArrayList>(); + public boolean configured = false; /** - * A constructor that takes a list of classes to instantiate eagerly. Class {@link Key}s mentioned here will - * be pulled out of the injector with an injector.getInstance() call when the lifecycle is created. + * Registers a class to instantiate eagerly. Classes mentioned here will be pulled out of + * the injector with an injector.getInstance() call when the lifecycle is created. * * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper - * scope. + * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial + * side-effect even if nothing actually directly depends on the instance. * * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to - * auto-register things with the {@link Lifecycle} + * auto-register things with the {@link Lifecycle}. It is also possible to just bind things eagerly with Guice, + * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules + * is less error-prone. * - * @param eagerClasses set of classes to instantiate eagerly + * @param clazz, the class to instantiate + * @return this, for chaining. */ - public LifecycleModule( - Key... eagerClasses - ) + public LifecycleModule register(Class clazz) { - this.eagerClasses = eagerClasses; + return registerKey(Key.get(clazz)); + } + + /** + * Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of + * the injector with an injector.getInstance() call when the lifecycle is created. + * + * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper + * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial + * side-effect even if nothing actually directly depends on the instance. + * + * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to + * auto-register things with the {@link Lifecycle}. It is also possible to just bind things eagerly with Guice, + * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules + * is less error-prone. + * + * @param clazz, the class to instantiate + * @param annotation The annotation instance to register with Guice, usually a Named annotation + * @return this, for chaining. + */ + public LifecycleModule register(Class clazz, Annotation annotation) + { + return registerKey(Key.get(clazz, annotation)); + } + + /** + * Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of + * the injector with an injector.getInstance() call when the lifecycle is created. + * + * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper + * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial + * side-effect even if nothing actually directly depends on the instance. + * + * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to + * auto-register things with the {@link Lifecycle}. It is also possible to just bind things eagerly with Guice, + * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules + * is less error-prone. + * + * @param clazz, the class to instantiate + * @param annotation The annotation class to register with Guice + * @return this, for chaining + */ + public LifecycleModule register(Class clazz, Class annotation) + { + return registerKey(Key.get(clazz, annotation)); + } + + /** + * Registers a key to instantiate eagerly. {@link Key}s mentioned here will be pulled out of + * the injector with an injector.getInstance() call when the lifecycle is created. + * + * Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper + * scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial + * side-effect even if nothing actually directly depends on the instance. + * + * This mechanism exists to allow the {@link Lifecycle} to be the primary entry point from the injector, not to + * auto-register things with the {@link Lifecycle}. It is also possible to just bind things eagerly with Guice, + * it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules + * is less error-prone. + * + * @param key The key to use in finding the DruidNode instance + * @return this, for chaining + */ + public LifecycleModule registerKey(Key key) + { + synchronized (eagerClasses) { + Preconditions.checkState(!configured, "Cannot register key[%s] after configuration.", key); + } + eagerClasses.add(key); + return this; } @Override public void configure(Binder binder) { - binder.bindScope(ManageLifecycle.class, scope); + synchronized (eagerClasses) { + configured = true; + binder.bindScope(ManageLifecycle.class, scope); + } } @Provides @LazySingleton diff --git a/common/src/main/java/com/metamx/druid/jackson/Json.java b/common/src/main/java/com/metamx/druid/guice/annotations/Json.java similarity index 89% rename from common/src/main/java/com/metamx/druid/jackson/Json.java rename to common/src/main/java/com/metamx/druid/guice/annotations/Json.java index b0cc6401613..c20a3f5f2f0 100644 --- a/common/src/main/java/com/metamx/druid/jackson/Json.java +++ b/common/src/main/java/com/metamx/druid/guice/annotations/Json.java @@ -1,4 +1,4 @@ -package com.metamx.druid.jackson; +package com.metamx.druid.guice.annotations; import com.google.inject.BindingAnnotation; diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Self.java b/common/src/main/java/com/metamx/druid/guice/annotations/Self.java new file mode 100644 index 00000000000..f3e9073a7ca --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/annotations/Self.java @@ -0,0 +1,17 @@ +package com.metamx.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Self +{ +} diff --git a/common/src/main/java/com/metamx/druid/jackson/Smile.java b/common/src/main/java/com/metamx/druid/guice/annotations/Smile.java similarity index 89% rename from common/src/main/java/com/metamx/druid/jackson/Smile.java rename to common/src/main/java/com/metamx/druid/guice/annotations/Smile.java index cc1c5b376e5..c3bee835d6c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/Smile.java +++ b/common/src/main/java/com/metamx/druid/guice/annotations/Smile.java @@ -1,4 +1,4 @@ -package com.metamx.druid.jackson; +package com.metamx.druid.guice.annotations; import com.google.inject.BindingAnnotation; diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java index aca59fd8bc3..694f4d2bf4c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -26,6 +26,8 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.annotations.Json; +import com.metamx.druid.guice.annotations.Smile; /** */ diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java index 6d67a7a5fb1..590b99bcf5d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java @@ -6,6 +6,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.initialization.DruidNode; import java.util.concurrent.ConcurrentMap; @@ -32,43 +33,40 @@ public class ChatHandlerProvider this.handlers = Maps.newConcurrentMap(); } - public void register(final String key, ChatHandler handler) + public void register(final String service, ChatHandler handler) { - final String service = serviceName(key); - log.info("Registering Eventhandler: %s", key); + log.info("Registering Eventhandler: %s", service); - if (handlers.putIfAbsent(key, handler) != null) { - throw new ISE("handler already registered for key: %s", key); + if (handlers.putIfAbsent(service, handler) != null) { + throw new ISE("handler already registered for service[%s]", service); } try { - serviceAnnouncer.announce(service); + serviceAnnouncer.announce(makeDruidNode(service)); } catch (Exception e) { - log.warn(e, "Failed to register service: %s", service); - handlers.remove(key, handler); + log.warn(e, "Failed to register service[%s]", service); + handlers.remove(service, handler); } } - public void unregister(final String key) + public void unregister(final String service) { - final String service = serviceName(key); + log.info("Unregistering chat handler for service[%s]", service); - log.info("Unregistering chat handler: %s", key); - - final ChatHandler handler = handlers.get(key); + final ChatHandler handler = handlers.get(service); if (handler == null) { - log.warn("handler not currently registered, ignoring: %s", key); + log.info("handler not currently registered, ignoring: %s", service); } try { - serviceAnnouncer.unannounce(service); + serviceAnnouncer.unannounce(makeDruidNode(service)); } catch (Exception e) { log.warn(e, "Failed to unregister service: %s", service); } - handlers.remove(key, handler); + handlers.remove(service, handler); } public Optional get(final String key) @@ -76,8 +74,7 @@ public class ChatHandlerProvider return Optional.fromNullable(handlers.get(key)); } - private String serviceName(String key) - { - return String.format(config.getServiceFormat(), key); + private DruidNode makeDruidNode(String service) { + return new DruidNode(service, config.getHost(), config.getPort()); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index 26a7c5210d3..0f717e57ddd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -32,7 +32,7 @@ import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; @@ -65,7 +65,7 @@ public class TaskMasterLifecycle final TaskQueue taskQueue, final TaskActionClientFactory taskActionClientFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, - final DruidNodeConfig nodeConfig, + final DruidNode nodeConfig, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 5a6ac400d4c..7edc3044139 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -49,7 +49,6 @@ import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; @@ -98,7 +97,7 @@ import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStr import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagmentConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -377,7 +376,7 @@ public class IndexerCoordinatorNode extends QueryableNode instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); - this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); + this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java index da0ce3b67b3..76a1cd2a070 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java @@ -1,13 +1,14 @@ package com.metamx.druid.indexing.worker.config; import org.skife.config.Config; -import org.skife.config.DefaultNull; public abstract class ChatHandlerProviderConfig { - @Config("druid.indexer.chathandler.service") - @DefaultNull - public abstract String getServiceFormat(); + @Config("druid.indexer.chathandler.publishDiscovery") + public boolean isPublishDiscovery() + { + return false; + } @Config("druid.host") public abstract String getHost(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 2f3c1fbf4bb..7ff9e5d1297 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -39,7 +39,6 @@ import com.metamx.druid.BaseServerNode; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.NoopServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; @@ -55,7 +54,6 @@ import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNodeConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServerInit; @@ -399,9 +397,7 @@ public class ExecutorNode extends BaseServerNode ); } if (serviceAnnouncer == null) { - DruidNodeConfig nodeConfig = configFactory.build(DruidNodeConfig.class); - final ServiceInstanceFactory instanceFactory = Initialization.makeServiceInstanceFactory(nodeConfig); - this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery, instanceFactory); + this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery); } if (coordinatorServiceProvider == null) { this.coordinatorServiceProvider = Initialization.makeServiceProvider( @@ -433,16 +429,13 @@ public class ExecutorNode extends BaseServerNode if (chatHandlerProvider == null) { final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); final ServiceAnnouncer myServiceAnnouncer; - if (config.getServiceFormat() == null) { + if (config.isPublishDiscovery()) { + myServiceAnnouncer = serviceAnnouncer; + } else { log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); myServiceAnnouncer = new NoopServiceAnnouncer(); - } else { - myServiceAnnouncer = serviceAnnouncer; } - this.chatHandlerProvider = new ChatHandlerProvider( - config, - myServiceAnnouncer - ); + this.chatHandlerProvider = new ChatHandlerProvider(config, myServiceAnnouncer); } } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index bc9347561eb..c8cdcc7c14b 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -33,7 +33,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.jackson.Json; +import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; import org.joda.time.DateTime; diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index 5a4f38d0917..fdb7bb771c7 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -49,7 +49,7 @@ public class MasterModule implements Module JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); - binder.bind(DruidMaster.class); + binder.bind(DruidMaster.class).asEagerSingleton(); binder.bind(ServerInventoryView.class); binder.bind(DatabaseSegmentManager.class) diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java index 8440df3042b..57e98a3c7b4 100644 --- a/server/src/main/java/com/metamx/druid/guice/ServerModule.java +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.java @@ -2,7 +2,8 @@ package com.metamx.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.initialization.DruidNode; /** */ @@ -11,6 +12,6 @@ public class ServerModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid", DruidNodeConfig.class); + JsonConfigProvider.bind(binder, "druid", DruidNode.class, Self.class); } } 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 bc4fc6189f6..8fcbfb61baf 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,31 +19,25 @@ package com.metamx.druid.http; -import com.google.common.base.Supplier; import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.TypeLiteral; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.initialization.DruidNodeConfig; +import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerInitializer; import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.log.LogLevelAdjuster; -import com.metamx.druid.master.DruidMaster; import com.metamx.druid.metrics.MetricsModule; -import com.metamx.metrics.MonitorScheduler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; @@ -62,14 +56,14 @@ public class MasterMain LogLevelAdjuster.register(); Injector injector = Initialization.makeInjector( - new LifecycleModule(Key.get(MonitorScheduler.class), Key.get(DruidMaster.class)), + new LifecycleModule(), EmitterModule.class, HttpClientModule.class, DbConnectorModule.class, JacksonConfigManagerModule.class, CuratorModule.class, new MetricsModule(), - DiscoveryModule.class, + new DiscoveryModule().register(Self.class), ServerModule.class, new JettyServerModule(new MasterJettyServerInitializer()), MasterModule.class @@ -77,13 +71,7 @@ public class MasterMain final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - final Supplier nodeConfig = injector.getInstance(Key.get(new TypeLiteral>(){})); - - final ServiceAnnouncer serviceAnnouncer = injector.getInstance(ServiceAnnouncer.class); - try { - // TODO: Make the announcement work through the lifecycle - Initialization.announceDefaultService(nodeConfig.get(), serviceAnnouncer, lifecycle); lifecycle.start(); } catch (Throwable t) { diff --git a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java index 28e68c7c17b..bb0063872ac 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java +++ b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java @@ -1,24 +1,28 @@ package com.metamx.druid.metrics; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.JsonConfigurator; import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.metrics.JvmMonitor; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.SysMonitor; import java.util.List; +import java.util.Properties; +import java.util.concurrent.CopyOnWriteArrayList; /** * Sets up the {@link MonitorScheduler} to monitor things on a regular schedule. {@link Monitor}s must be explicitly @@ -28,18 +32,30 @@ public class MetricsModule implements Module { private static final Logger log = new Logger(MetricsModule.class); - private final Class[] monitors; + private final List> monitors = new CopyOnWriteArrayList>(); + public boolean configured = false; - /** - * A constructor that takes a list of {@link Monitor} classes to explicitly bind so that they will be instantiated - * - * @param monitors list of {@link Monitor} classes to explicitly bind - */ - public MetricsModule( - Class... monitors - ) + public MetricsModule register(Class monitorClazz) { - this.monitors = monitors; + synchronized (monitors) { + Preconditions.checkState(!configured, "Cannot register monitor[%s] after configuration.", monitorClazz); + } + monitors.add(monitorClazz); + return this; + } + + @Inject + public void setProperties(Properties props, JsonConfigurator configurator) + { + final MonitorsConfig config = configurator.configurate( + props, + "druid.monitoring", + MonitorsConfig.class + ); + + for (Class monitorClazz : config.getMonitors()) { + register(monitorClazz); + } } @Override @@ -47,15 +63,18 @@ public class MetricsModule implements Module { JsonConfigProvider.bind(binder, "druid.monitoring", DruidMonitorSchedulerConfig.class); - binder.bind(JvmMonitor.class).in(LazySingleton.class); - binder.bind(SysMonitor.class).in(LazySingleton.class); // TODO: allow for disabling of this monitor - for (Class monitor : monitors) { binder.bind(monitor).in(LazySingleton.class); } + + // Instantiate eagerly so that we get everything registered and put into the Lifecycle + binder.bind(Key.get(MonitorScheduler.class, Names.named("ForTheEagerness"))) + .to(MonitorScheduler.class) + .asEagerSingleton(); } - @Provides @ManageLifecycle + @Provides + @ManageLifecycle public MonitorScheduler getMonitorScheduler( Supplier config, ServiceEmitter emitter, @@ -64,8 +83,8 @@ public class MetricsModule implements Module { List monitors = Lists.newArrayList(); - for (Key key: injector.getBindings().keySet()) { - if (Monitor.class.isAssignableFrom(key.getClass())) { + for (Key key : injector.getBindings().keySet()) { + if (Monitor.class.isAssignableFrom(key.getTypeLiteral().getRawType())) { final Monitor monitor = (Monitor) injector.getInstance(key); log.info("Adding monitor[%s]", monitor); diff --git a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java new file mode 100644 index 00000000000..9d59c4fbdab --- /dev/null +++ b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java @@ -0,0 +1,35 @@ +package com.metamx.druid.metrics; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.metamx.metrics.JvmMonitor; +import com.metamx.metrics.Monitor; +import com.metamx.metrics.SysMonitor; + +import javax.validation.constraints.NotNull; +import java.util.List; + +/** + */ +public class MonitorsConfig +{ + @JsonProperty("monitors") + @NotNull + private List> monitors = ImmutableList.>builder() + .add(JvmMonitor.class) + .add(SysMonitor.class) + .build(); + + public List> getMonitors() + { + return monitors; + } + + @Override + public String toString() + { + return "MonitorsConfig{" + + "monitors=" + monitors + + '}'; + } +} From 9cd26c70c08826258945d6517077dc4f60628b67 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 29 Jul 2013 14:26:51 -0700 Subject: [PATCH 13/92] 1) Adjust the JettyServerModule and other things such that Jetty will actually serve the static artifacts and the various resources of the Master --- .../com/metamx/druid/http/StatusResource.java | 31 +++++++ .../initialization/JettyServerModule.java | 84 ++++++++++++++++++- pom.xml | 6 +- .../metamx/druid/db/DatabaseRuleManager.java | 9 +- .../druid/db/DatabaseSegmentManager.java | 8 +- .../com/metamx/druid/guice/MasterModule.java | 9 +- .../com/metamx/druid/http/MasterMain.java | 43 +++++++--- .../com/metamx/druid/http/RedirectFilter.java | 8 +- .../com/metamx/druid/master/DruidMaster.java | 15 +++- 9 files changed, 182 insertions(+), 31 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/http/StatusResource.java diff --git a/client/src/main/java/com/metamx/druid/http/StatusResource.java b/client/src/main/java/com/metamx/druid/http/StatusResource.java new file mode 100644 index 00000000000..2c4aed29614 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/StatusResource.java @@ -0,0 +1,31 @@ +package com.metamx.druid.http; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Response; + +/** + */ +@Path("/{a:status|health}") +public class StatusResource +{ + @GET + @Produces("text/plain") + public Response doGet() + { + StringBuffer buf = new StringBuffer(); + + Runtime runtime = Runtime.getRuntime(); + long maxMemory = runtime.maxMemory(); + long totalMemory = runtime.totalMemory(); + long freeMemory = runtime.freeMemory(); + + buf.append(String.format("Max Memory:\t%,18d\t%1$d%n", maxMemory)); + buf.append(String.format("Total Memory:\t%,18d\t%1$d%n", totalMemory)); + buf.append(String.format("Free Memory:\t%,18d\t%1$d%n", freeMemory)); + buf.append(String.format("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory)); + + return Response.ok(buf.toString()).build(); + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java index 170c50396aa..a1c83c33641 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -1,22 +1,41 @@ package com.metamx.druid.initialization; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Injector; -import com.google.inject.Module; +import com.google.inject.Key; import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; +import com.google.inject.name.Named; +import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.guice.ConfigProvider; import com.metamx.druid.guice.LazySingleton; +import com.sun.jersey.api.core.DefaultResourceConfig; +import com.sun.jersey.api.core.ResourceConfig; +import com.sun.jersey.guice.JerseyServletModule; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; +import com.sun.jersey.spi.container.servlet.WebConfig; import org.eclipse.jetty.server.Server; +import javax.servlet.ServletException; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** */ -public class JettyServerModule implements Module +public class JettyServerModule extends JerseyServletModule { private static final Logger log = new Logger(JettyServerModule.class); private final JettyServerInitializer initializer; + private final List> resources = Lists.newArrayList(); public JettyServerModule( JettyServerInitializer initializer @@ -25,10 +44,66 @@ public class JettyServerModule implements Module this.initializer = initializer; } - @Override - public void configure(Binder binder) + public JettyServerModule addResource(Class resource) { + resources.add(resource); + return this; + } + + @Override + protected void configureServlets() + { + Binder binder = binder(); + ConfigProvider.bind(binder, ServerConfig.class); + + // The Guice servlet extension doesn't actually like requiring explicit bindings, so we do its job for it here. + try { + final Class classToBind = Class.forName( + "com.google.inject.servlet.InternalServletModule$BackwardsCompatibleServletContextProvider" + ); + binder.bind(classToBind); + } + catch (ClassNotFoundException e) { + throw Throwables.propagate(e); + } + + binder.bind(GuiceContainer.class).to(DruidGuiceContainer.class); + binder.bind(DruidGuiceContainer.class).in(Scopes.SINGLETON); + serve("/*").with(DruidGuiceContainer.class); + + final ImmutableSet> theResources = ImmutableSet.copyOf(resources); + binder.bind(new TypeLiteral>>(){}) + .annotatedWith(Names.named("resourceClasses")) + .toInstance(theResources); + for (Class resource : theResources) { + binder.bind(resource); + } + + binder.bind(Key.get(Server.class, Names.named("ForTheEagerness"))).to(Server.class).asEagerSingleton(); + } + + public static class DruidGuiceContainer extends GuiceContainer + { + private final Set> resources; + + @Inject + public DruidGuiceContainer( + Injector injector, + @Named("resourceClasses") Set> resources + ) + { + super(injector); + this.resources = resources; + } + + @Override + protected ResourceConfig getDefaultResourceConfig( + Map props, WebConfig webConfig + ) throws ServletException + { + return new DefaultResourceConfig(resources); + } } @Provides @LazySingleton @@ -36,6 +111,7 @@ public class JettyServerModule implements Module { final Server server = Initialization.makeJettyServer(config); initializer.initialize(server, injector); + lifecycle.addHandler( new Lifecycle.Handler() { diff --git a/pom.xml b/pom.xml index 95436f18d3f..c528699a9f3 100644 --- a/pom.xml +++ b/pom.xml @@ -253,17 +253,17 @@ com.sun.jersey jersey-core - 1.9.1 + 1.17.1 com.sun.jersey.contribs jersey-guice - 1.9.1 + 1.17.1 com.sun.jersey jersey-server - 1.9.1 + 1.17.1 net.java.dev.jets3t diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index c8cdcc7c14b..334d629e2ad 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -118,12 +118,13 @@ public class DatabaseRuleManager private static final Logger log = new Logger(DatabaseRuleManager.class); private final ObjectMapper jsonMapper; - private final ScheduledExecutorService exec; private final Supplier config; private final Supplier dbTables; private final IDBI dbi; private final AtomicReference>> rules; + private volatile ScheduledExecutorService exec; + private final Object lock = new Object(); private volatile boolean started = false; @@ -141,8 +142,6 @@ public class DatabaseRuleManager this.dbTables = dbTables; this.dbi = dbi; - this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"); - this.rules = new AtomicReference>>( new ConcurrentHashMap>() ); @@ -156,6 +155,8 @@ public class DatabaseRuleManager return; } + this.exec = Execs.scheduledSingleThreaded("DatabaseRuleManager-Exec--%d"); + createDefaultRule(dbi, getRulesTable(), config.get().getDefaultTier(), jsonMapper); ScheduledExecutors.scheduleWithFixedDelay( exec, @@ -186,6 +187,8 @@ public class DatabaseRuleManager rules.set(new ConcurrentHashMap>()); started = false; + exec.shutdownNow(); + exec = null; } } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index b5beabc9988..815680fb0c8 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -68,12 +68,13 @@ public class DatabaseSegmentManager private final Object lock = new Object(); private final ObjectMapper jsonMapper; - private final ScheduledExecutorService exec; private final Supplier config; private final Supplier dbTables; private final AtomicReference> dataSources; private final IDBI dbi; + private volatile ScheduledExecutorService exec; + private volatile boolean started = false; @Inject @@ -91,8 +92,6 @@ public class DatabaseSegmentManager new ConcurrentHashMap() ); this.dbi = dbi; - - this.exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); } @LifecycleStart @@ -103,6 +102,8 @@ public class DatabaseSegmentManager return; } + this.exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); + final Duration delay = config.get().getPollDuration().toStandardDuration(); ScheduledExecutors.scheduleWithFixedDelay( exec, @@ -133,6 +134,7 @@ public class DatabaseSegmentManager started = false; dataSources.set(new ConcurrentHashMap()); exec.shutdownNow(); + exec = null; } } diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/MasterModule.java index fdb7bb771c7..ad751bb045c 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/MasterModule.java @@ -8,6 +8,7 @@ import com.google.inject.TypeLiteral; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.indexing.IndexingService; @@ -23,7 +24,9 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DatabaseSegmentManagerProvider; import com.metamx.druid.http.MasterRedirectInfo; +import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; +import com.metamx.druid.http.RedirectServlet; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; @@ -49,8 +52,10 @@ public class MasterModule implements Module JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); - binder.bind(DruidMaster.class).asEagerSingleton(); + binder.bind(InventoryView.class).to(ServerInventoryView.class); binder.bind(ServerInventoryView.class); + binder.bind(RedirectServlet.class).in(LazySingleton.class); + binder.bind(RedirectFilter.class).in(LazySingleton.class); binder.bind(DatabaseSegmentManager.class) .toProvider(DatabaseSegmentManagerProvider.class) @@ -67,6 +72,8 @@ public class MasterModule implements Module binder.bind(IndexingServiceClient.class).in(LazySingleton.class); binder.bind(RedirectInfo.class).to(MasterRedirectInfo.class).in(LazySingleton.class); + + binder.bind(DruidMaster.class); } @Provides @LazySingleton @IndexingService 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 8fcbfb61baf..4d8e45c18ef 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,7 +19,10 @@ package com.metamx.druid.http; +import com.google.common.collect.Iterables; +import com.google.inject.ConfigurationException; import com.google.inject.Injector; +import com.google.inject.ProvisionException; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -37,8 +40,13 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerInitializer; import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.log.LogLevelAdjuster; +import com.metamx.druid.master.DruidMaster; import com.metamx.druid.metrics.MetricsModule; +import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; @@ -56,7 +64,7 @@ public class MasterMain LogLevelAdjuster.register(); Injector injector = Initialization.makeInjector( - new LifecycleModule(), + new LifecycleModule().register(DruidMaster.class), EmitterModule.class, HttpClientModule.class, DbConnectorModule.class, @@ -65,7 +73,10 @@ public class MasterMain new MetricsModule(), new DiscoveryModule().register(Self.class), ServerModule.class, - new JettyServerModule(new MasterJettyServerInitializer()), + new JettyServerModule(new MasterJettyServerInitializer()) + .addResource(InfoResource.class) + .addResource(MasterResource.class) + .addResource(StatusResource.class), MasterModule.class ); @@ -87,19 +98,25 @@ public class MasterMain @Override public void initialize(Server server, Injector injector) { - final ServletContextHandler staticContext = new ServletContextHandler(server, "/static", ServletContextHandler.SESSIONS); - staticContext.addServlet(new ServletHolder(injector.getInstance(RedirectServlet.class)), "/*"); + try { + ResourceHandler resourceHandler = new ResourceHandler(); + resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); - staticContext.setResourceBase(ComputeMain.class.getClassLoader().getResource("static").toExternalForm()); + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.setContextPath("/"); - final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); - root.addFilter(GuiceFilter.class, "/info/*", null); - root.addFilter(GuiceFilter.class, "/master/*", null); + HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); + server.setHandler(handlerList); + + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + } + catch (ConfigurationException e) { + throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage()); + } } } } diff --git a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java index 5ed2bff98ee..1535e90ff71 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java +++ b/server/src/main/java/com/metamx/druid/http/RedirectFilter.java @@ -71,7 +71,13 @@ public class RedirectFilter implements Filter chain.doFilter(request, response); } else { URL url = redirectInfo.getRedirectURL(request.getQueryString(), request.getRequestURI()); - log.info("Forwarding request to [%s]", url); + log.debug("Forwarding request to [%s]", url); + + if (url == null) { + // We apparently have no master, so let's do a Service Unavailable + response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE); + return; + } response.setStatus(HttpServletResponse.SC_MOVED_TEMPORARILY); response.setHeader("Location", url.toString()); 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 4561fe502ac..2978ff3f43d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -54,6 +54,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; import org.apache.curator.framework.recipes.leader.LeaderLatchListener; +import org.apache.curator.framework.recipes.leader.Participant; import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -241,7 +242,17 @@ public class DruidMaster { try { final LeaderLatch latch = leaderLatch.get(); - return latch == null ? null : latch.getLeader().getId(); + + if (latch == null) { + return null; + } + + Participant participant = latch.getLeader(); + if (participant.isLeader()) { + return participant.getId(); + } + + return null; } catch (Exception e) { throw Throwables.propagate(e); @@ -562,8 +573,6 @@ public class DruidMaster try { log.info("I am no longer the master..."); - leaderLatch.get().close(); - for (String server : loadManagementPeons.keySet()) { LoadQueuePeon peon = loadManagementPeons.remove(server); peon.stop(); From 4c9be81912886419548df3908c8704676b5aaa36 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 29 Jul 2013 14:35:30 -0700 Subject: [PATCH 14/92] 1) Pull try/catch check in the JettyServerInitializer out of the initializer. --- .../initialization/JettyServerModule.java | 11 ++++++- .../com/metamx/druid/http/MasterMain.java | 30 +++++++------------ 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java index a1c83c33641..abbf3b091b5 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -2,12 +2,15 @@ package com.metamx.druid.initialization; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.inject.Binder; +import com.google.inject.ConfigurationException; import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Provides; +import com.google.inject.ProvisionException; import com.google.inject.Scopes; import com.google.inject.TypeLiteral; import com.google.inject.name.Named; @@ -110,7 +113,13 @@ public class JettyServerModule extends JerseyServletModule public Server getServer(Injector injector, Lifecycle lifecycle, ServerConfig config) { final Server server = Initialization.makeJettyServer(config); - initializer.initialize(server, injector); + try { + initializer.initialize(server, injector); + } + catch (ConfigurationException e) { + throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage()); + } + lifecycle.addHandler( new Lifecycle.Handler() 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 4d8e45c18ef..a60b104e6d6 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -19,10 +19,7 @@ package com.metamx.druid.http; -import com.google.common.collect.Iterables; -import com.google.inject.ConfigurationException; import com.google.inject.Injector; -import com.google.inject.ProvisionException; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -98,25 +95,20 @@ public class MasterMain @Override public void initialize(Server server, Injector injector) { - try { - ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); + ResourceHandler resourceHandler = new ResourceHandler(); + resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.setContextPath("/"); + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.setContextPath("/"); - HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); - server.setHandler(handlerList); + HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); + server.setHandler(handlerList); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); - root.addFilter(GuiceFilter.class, "/*", null); - } - catch (ConfigurationException e) { - throw new ProvisionException(Iterables.getFirst(e.getErrorMessages(), null).getMessage()); - } + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); } } } From 2b587cba856d36d8751ac69bf9f4d8e4e8f24b7c Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 29 Jul 2013 15:14:20 -0700 Subject: [PATCH 15/92] 1) Introduce airlift/airline for CLI stuff and add a Coordinator CLI verb to make it all happiness --- .../druid/client/cache/CacheMonitor.java | 2 + pom.xml | 5 + services/pom.xml | 4 + .../main/java/io/druid/cli/Coordinator.java | 112 ++++++++++++++++++ services/src/main/java/io/druid/cli/Main.java | 26 ++++ 5 files changed, 149 insertions(+) create mode 100644 services/src/main/java/io/druid/cli/Coordinator.java create mode 100644 services/src/main/java/io/druid/cli/Main.java diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java b/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java index e89c1113e8d..ce9ac5e27d2 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java +++ b/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java @@ -19,6 +19,7 @@ package com.metamx.druid.client.cache; +import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.AbstractMonitor; @@ -31,6 +32,7 @@ public class CacheMonitor extends AbstractMonitor private volatile CacheStats prevCacheStats = null; + @Inject public CacheMonitor( Cache cache ) diff --git a/pom.xml b/pom.xml index c528699a9f3..45a7bac2a21 100644 --- a/pom.xml +++ b/pom.xml @@ -119,6 +119,11 @@ compress-lzf 0.8.4 + + io.airlift + airline + 0.5 + org.skife.config config-magic diff --git a/services/pom.xml b/services/pom.xml index e8dff6c3aa3..1f9a12dbccf 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -42,6 +42,10 @@ druid-server ${project.parent.version} + + io.airlift + airline + diff --git a/services/src/main/java/io/druid/cli/Coordinator.java b/services/src/main/java/io/druid/cli/Coordinator.java new file mode 100644 index 00000000000..0d342e9267f --- /dev/null +++ b/services/src/main/java/io/druid/cli/Coordinator.java @@ -0,0 +1,112 @@ +package io.druid.cli; + +import com.google.common.base.Throwables; +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.DbConnectorModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.JacksonConfigManagerModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.MasterModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.http.InfoResource; +import com.metamx.druid.http.MasterMain; +import com.metamx.druid.http.MasterResource; +import com.metamx.druid.http.RedirectFilter; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerInitializer; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.log.LogLevelAdjuster; +import com.metamx.druid.master.DruidMaster; +import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Command; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.server.handler.ResourceHandler; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; + +/** + */ +@Command( + name = "coordinator", + description = "Runs the Coordinator, see https://github.com/metamx/druid/wiki/Master for a description." +) +public class Coordinator implements Runnable +{ + private static final Logger log = new Logger(Coordinator.class); + + @Override + public void run() + { + try { + LogLevelAdjuster.register(); + + Injector injector = Initialization.makeInjector( + new LifecycleModule().register(DruidMaster.class), + EmitterModule.class, + HttpClientModule.class, + DbConnectorModule.class, + JacksonConfigManagerModule.class, + CuratorModule.class, + new MetricsModule(), + new DiscoveryModule().register(Self.class), + ServerModule.class, + new JettyServerModule(new MasterJettyServerInitializer()) + .addResource(InfoResource.class) + .addResource(MasterResource.class) + .addResource(StatusResource.class), + MasterModule.class + ); + + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + try { + lifecycle.start(); + } + catch (Throwable t) { + log.error(t, "Error when starting up. Failing."); + System.exit(1); + } + + lifecycle.join(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private static class MasterJettyServerInitializer implements JettyServerInitializer + { + @Override + public void initialize(Server server, Injector injector) + { + ResourceHandler resourceHandler = new ResourceHandler(); + resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); + + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.setContextPath("/"); + + HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); + server.setHandler(handlerList); + + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + } + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java new file mode 100644 index 00000000000..5d465cce706 --- /dev/null +++ b/services/src/main/java/io/druid/cli/Main.java @@ -0,0 +1,26 @@ +package io.druid.cli; + +import io.airlift.command.Cli; +import io.airlift.command.Help; + +/** + */ +public class Main +{ + @SuppressWarnings("unchecked") + public static void main(String[] args) + { + final Cli.CliBuilder builder = Cli.builder("druid"); + + builder.withDescription("Druid command-line runner.") + .withDefaultCommand(Help.class) + .withCommands(Help.class); + + builder.withGroup("server") + .withDescription("Run one of the Druid server types.") + .withDefaultCommand(Help.class) + .withCommands(Coordinator.class); + + builder.build().parse(args).run(); + } +} From 97fb971f849fc245f9eb38a268354477589fe52f Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 30 Jul 2013 11:58:05 -0700 Subject: [PATCH 16/92] 1) Start the journey of making the historical nodes work with Guice. Doesn't work yet, but at least it builds. --- .../coordination/DruidServerMetadata.java | 3 +- .../query/MetricsEmittingExecutorService.java | 4 ++ .../metamx/druid/guice/ConfigProvider.java | 22 +++++- pom.xml | 5 ++ server/pom.xml | 4 ++ .../druid/coordination/ServerManager.java | 5 +- ...sterModule.java => CoordinatorModule.java} | 2 +- .../metamx/druid/guice/HistoricalModule.java | 68 ++++++++++++++++++ .../druid/guice/annotations/Processing.java | 17 +++++ .../com/metamx/druid/http/ComputeNode.java | 3 +- .../com/metamx/druid/http/MasterMain.java | 4 +- .../druid/loading/BaseSegmentLoader.java | 62 ++++++++++++++++ .../loading/DelegatingSegmentLoader.java | 5 -- .../druid/loading/SegmentLoaderConfig.java | 23 ++++-- .../metamx/druid/metrics/ServerMonitor.java | 32 +++++---- .../{Coordinator.java => CliCoordinator.java} | 71 +++++++------------ .../main/java/io/druid/cli/CliHistorical.java | 68 ++++++++++++++++++ services/src/main/java/io/druid/cli/Main.java | 2 +- .../java/io/druid/cli/ServerRunnable.java | 45 ++++++++++++ 19 files changed, 364 insertions(+), 81 deletions(-) rename server/src/main/java/com/metamx/druid/guice/{MasterModule.java => CoordinatorModule.java} (98%) create mode 100644 server/src/main/java/com/metamx/druid/guice/HistoricalModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/annotations/Processing.java create mode 100644 server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java rename services/src/main/java/io/druid/cli/{Coordinator.java => CliCoordinator.java} (64%) create mode 100644 services/src/main/java/io/druid/cli/CliHistorical.java create mode 100644 services/src/main/java/io/druid/cli/ServerRunnable.java diff --git a/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java b/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java index 25c9c9875e8..7c4e55cdd8e 100644 --- a/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java +++ b/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java @@ -37,7 +37,8 @@ public class DruidServerMetadata @JsonProperty("name") String name, @JsonProperty("host") String host, @JsonProperty("maxSize") long maxSize, - @JsonProperty("type") String type, @JsonProperty("tier") String tier + @JsonProperty("type") String type, + @JsonProperty("tier") String tier ) { this.name = name; diff --git a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java b/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java index 2e453ddbe0b..1aee5cb48a5 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java +++ b/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java @@ -1,5 +1,7 @@ package com.metamx.druid.query; +import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -9,6 +11,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +@ManageLifecycle public class MetricsEmittingExecutorService extends AbstractExecutorService { private final ExecutorService base; @@ -33,6 +36,7 @@ public class MetricsEmittingExecutorService extends AbstractExecutorService } @Override + @LifecycleStop public List shutdownNow() { return base.shutdownNow(); diff --git a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java index d86bcbc04b7..40559a1e798 100644 --- a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java @@ -7,6 +7,8 @@ import com.google.inject.Provider; import com.metamx.common.logger.Logger; import org.skife.config.ConfigurationObjectFactory; +import java.util.Map; + /** */ public class ConfigProvider implements Provider @@ -18,27 +20,41 @@ public class ConfigProvider implements Provider binder.bind(clazz).toProvider(of(clazz)).in(LazySingleton.class); } + public static void bind(Binder binder, Class clazz, Map replacements) + { + binder.bind(clazz).toProvider(of(clazz, replacements)).in(LazySingleton.class); + } + public static Provider of(Class clazz) { - return new ConfigProvider(clazz); + return of(clazz, null); + } + + public static Provider of(Class clazz, Map replacements) + { + return new ConfigProvider(clazz, replacements); } private final Class clazz; + private final Map replacements; private T object = null; public ConfigProvider( - Class clazz + Class clazz, + Map replacements ) { this.clazz = clazz; + this.replacements = replacements; } @Inject public void inject(ConfigurationObjectFactory factory) { try { - object = factory.build(clazz); + // ConfigMagic handles a null replacements + object = factory.buildWithReplacements(clazz, replacements); } catch (IllegalArgumentException e) { log.info("Unable to build instance of class[%s]", clazz); diff --git a/pom.xml b/pom.xml index 45a7bac2a21..9795d116c3b 100644 --- a/pom.xml +++ b/pom.xml @@ -180,6 +180,11 @@ guice-servlet 3.0 + + com.google.inject.extensions + guice-multibindings + 3.0 + com.ibm.icu icu4j diff --git a/server/pom.xml b/server/pom.xml index 48289cca972..ddeabbdd62e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -105,6 +105,10 @@ com.google.inject.extensions guice-servlet + + com.google.inject.extensions + guice-multibindings + com.fasterxml.jackson.core jackson-core diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 7803475d091..eb22c560e99 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -22,6 +22,7 @@ package com.metamx.druid.coordination; import com.google.common.base.Function; import com.google.common.base.Predicates; import com.google.common.collect.Ordering; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.Query; @@ -29,6 +30,7 @@ import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; +import com.metamx.druid.guice.annotations.Processing; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; @@ -75,11 +77,12 @@ public class ServerManager implements QuerySegmentWalker private final CountingMap dataSourceSizes = new CountingMap(); private final CountingMap dataSourceCounts = new CountingMap(); + @Inject public ServerManager( SegmentLoader segmentLoader, QueryRunnerFactoryConglomerate conglomerate, ServiceEmitter emitter, - ExecutorService exec + @Processing ExecutorService exec ) { this.segmentLoader = segmentLoader; diff --git a/server/src/main/java/com/metamx/druid/guice/MasterModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java similarity index 98% rename from server/src/main/java/com/metamx/druid/guice/MasterModule.java rename to server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index ad751bb045c..4007230d743 100644 --- a/server/src/main/java/com/metamx/druid/guice/MasterModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -40,7 +40,7 @@ import java.io.IOException; /** */ -public class MasterModule implements Module +public class CoordinatorModule implements Module { @Override public void configure(Binder binder) diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java new file mode 100644 index 00000000000..466acc37cc6 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -0,0 +1,68 @@ +package com.metamx.druid.guice; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.multibindings.MapBinder; +import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.metamx.druid.client.DruidServerConfig; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.coordination.ServerManager; +import com.metamx.druid.guice.annotations.Processing; +import com.metamx.druid.loading.BaseSegmentLoader; +import com.metamx.druid.loading.DataSegmentPuller; +import com.metamx.druid.loading.HdfsDataSegmentPuller; +import com.metamx.druid.loading.LocalDataSegmentPuller; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.QueryableIndexFactory; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoader; +import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; +import com.metamx.druid.query.MetricsEmittingExecutorService; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + */ +public class HistoricalModule implements Module +{ + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, DruidServerConfig.class); + ConfigProvider.bind(binder, ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing")); + + JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); + + binder.bind(ServerManager.class).in(LazySingleton.class); + + binder.bind(SegmentLoader.class).to(BaseSegmentLoader.class).in(LazySingleton.class); + binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); + + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, + String.class, + DataSegmentPuller.class + ); + + segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); + segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); + } + + @Provides @Processing @ManageLifecycle + public ExecutorService getProcessingExecutorService(ExecutorServiceConfig config, ServiceEmitter emitter) + { + return new MetricsEmittingExecutorService( + Executors.newFixedThreadPool(config.getNumThreads(), Execs.makeThreadFactory(config.getFormatString())), + emitter, + new ServiceMetricEvent.Builder() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java b/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java new file mode 100644 index 00000000000..f3279d2af90 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java @@ -0,0 +1,17 @@ +package com.metamx.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Processing +{ +} diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 5ffc6c234bc..f2e2c6f2b12 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -31,6 +31,7 @@ import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.BaseServerNode; +import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.coordination.ZkCoordinatorConfig; @@ -121,7 +122,7 @@ public class ComputeNode extends BaseServerNode ); lifecycle.addManagedInstance(coordinator); - monitors.add(new ServerMonitor(getDruidServerMetadata(), serverManager)); + monitors.add(new ServerMonitor(getConfigFactory().build(DruidServerConfig.class), serverManager)); startMonitoring(monitors); final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); 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 a60b104e6d6..5cad41a63ef 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -25,11 +25,11 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.CoordinatorModule; import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.EmitterModule; @@ -74,7 +74,7 @@ public class MasterMain .addResource(InfoResource.class) .addResource(MasterResource.class) .addResource(StatusResource.class), - MasterModule.class + CoordinatorModule.class ); final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); diff --git a/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java new file mode 100644 index 00000000000..b3a44a99629 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java @@ -0,0 +1,62 @@ +package com.metamx.druid.loading; + +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.Segment; + +import java.util.Map; + +/** + */ +public class BaseSegmentLoader implements SegmentLoader +{ + private final Map pullers; + private final QueryableIndexFactory factory; + private final Supplier config; + + @Inject + public BaseSegmentLoader( + Map pullers, + QueryableIndexFactory factory, + Supplier config + ) + { + this.pullers = pullers; + this.factory = factory; + this.config = config; + } + + @Override + public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException + { + return getLoader(segment.getLoadSpec()).isSegmentLoaded(segment); + } + + @Override + public Segment getSegment(DataSegment segment) throws SegmentLoadingException + { + return getLoader(segment.getLoadSpec()).getSegment(segment); + } + + @Override + public void cleanup(DataSegment segment) throws SegmentLoadingException + { + getLoader(segment.getLoadSpec()).cleanup(segment); + } + + private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException + { + String type = MapUtils.getString(loadSpec, "type"); + DataSegmentPuller loader = pullers.get(type); + + if (loader == null) { + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, pullers.keySet()); + } + + // TODO: SingleSegmentLoader should die when Guice goes out. The logic should just be in this class. + return new SingleSegmentLoader(loader, factory, config.get()); + } + +} diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index 089f137db1a..135a14ccb90 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -19,9 +19,7 @@ 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 com.metamx.druid.index.Segment; @@ -31,11 +29,8 @@ import java.util.Map; */ public class DelegatingSegmentLoader implements SegmentLoader { - private static final Logger log = new Logger(DelegatingSegmentLoader.class); - private volatile Map loaderTypes; - @Inject public void setLoaderTypes( Map loaderTypes ) diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index 294c91b9a38..1ec5e1b6584 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -19,20 +19,29 @@ package com.metamx.druid.loading; -import org.skife.config.Config; +import com.fasterxml.jackson.annotation.JsonProperty; +import javax.validation.constraints.NotNull; import java.io.File; /** */ -public abstract class SegmentLoaderConfig +public class SegmentLoaderConfig { - @Config({"druid.paths.indexCache", "druid.segmentCache.path"}) - public abstract File getCacheDirectory(); + @JsonProperty("path") + @NotNull + private File cacheDirectory; - @Config("druid.segmentCache.deleteOnRemove") - public boolean deleteOnRemove() + @JsonProperty("deleteOnRemove") + private boolean deleteOnRemove = true; + + public File getCacheDirectory() { - return true; + return cacheDirectory; + } + + public boolean isDeleteOnRemove() + { + return deleteOnRemove; } } diff --git a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java index 9898d85a76e..72e24f40a32 100644 --- a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java +++ b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java @@ -19,7 +19,8 @@ package com.metamx.druid.metrics; -import com.metamx.druid.coordination.DruidServerMetadata; +import com.google.inject.Inject; +import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.coordination.ServerManager; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -29,39 +30,40 @@ import java.util.Map; public class ServerMonitor extends AbstractMonitor { - private final DruidServerMetadata druidServer; + private final DruidServerConfig serverConfig; private final ServerManager serverManager; + @Inject public ServerMonitor( - DruidServerMetadata druidServer, + DruidServerConfig serverConfig, ServerManager serverManager ) { - this.druidServer = druidServer; + this.serverConfig = serverConfig; this.serverManager = serverManager; } @Override public boolean doMonitor(ServiceEmitter emitter) { - emitter.emit(new ServiceMetricEvent.Builder().build("server/segment/max", druidServer.getMaxSize())); + emitter.emit(new ServiceMetricEvent.Builder().build("server/segment/max", serverConfig.getMaxSize())); for (Map.Entry entry : serverManager.getDataSourceSizes().entrySet()) { String dataSource = entry.getKey(); long used = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser1(dataSource) - .build("server/segment/used", used) - ); + final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder().setUser1(dataSource) + .setUser2(serverConfig.getTier()); + + emitter.emit(builder.build("server/segment/used", used)); + emitter.emit(builder.build("server/segment/usedPercent", used / (double) serverConfig.getMaxSize())); } + for (Map.Entry entry : serverManager.getDataSourceCounts().entrySet()) { String dataSource = entry.getKey(); long count = entry.getValue(); - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser1(dataSource) - .build("server/segment/count", count) - ); + final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder().setUser1(dataSource) + .setUser2(serverConfig.getTier()); + + emitter.emit(builder.build("server/segment/count", count)); } return true; diff --git a/services/src/main/java/io/druid/cli/Coordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java similarity index 64% rename from services/src/main/java/io/druid/cli/Coordinator.java rename to services/src/main/java/io/druid/cli/CliCoordinator.java index 0d342e9267f..549259ab711 100644 --- a/services/src/main/java/io/druid/cli/Coordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -1,17 +1,15 @@ package io.druid.cli; -import com.google.common.base.Throwables; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.CoordinatorModule; import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.MasterModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.http.InfoResource; @@ -23,7 +21,6 @@ import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerInitializer; import com.metamx.druid.initialization.JettyServerModule; -import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Command; @@ -44,51 +41,37 @@ import org.eclipse.jetty.servlets.GzipFilter; name = "coordinator", description = "Runs the Coordinator, see https://github.com/metamx/druid/wiki/Master for a description." ) -public class Coordinator implements Runnable +public class CliCoordinator extends ServerRunnable { - private static final Logger log = new Logger(Coordinator.class); + private static final Logger log = new Logger(CliCoordinator.class); - @Override - public void run() + public CliCoordinator() { - try { - LogLevelAdjuster.register(); - - Injector injector = Initialization.makeInjector( - new LifecycleModule().register(DruidMaster.class), - EmitterModule.class, - HttpClientModule.class, - DbConnectorModule.class, - JacksonConfigManagerModule.class, - CuratorModule.class, - new MetricsModule(), - new DiscoveryModule().register(Self.class), - ServerModule.class, - new JettyServerModule(new MasterJettyServerInitializer()) - .addResource(InfoResource.class) - .addResource(MasterResource.class) - .addResource(StatusResource.class), - MasterModule.class - ); - - final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.error(t, "Error when starting up. Failing."); - System.exit(1); - } - - lifecycle.join(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + super(log); } - private static class MasterJettyServerInitializer implements JettyServerInitializer + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule().register(DruidMaster.class), + EmitterModule.class, + HttpClientModule.class, + DbConnectorModule.class, + JacksonConfigManagerModule.class, + CuratorModule.class, + new MetricsModule(), + new DiscoveryModule().register(Self.class), + ServerModule.class, + new JettyServerModule(new CoordinatorJettyServerInitializer()) + .addResource(InfoResource.class) + .addResource(MasterResource.class) + .addResource(StatusResource.class), + CoordinatorModule.class + ); + } + + private static class CoordinatorJettyServerInitializer implements JettyServerInitializer { @Override public void initialize(Server server, Injector injector) diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java new file mode 100644 index 00000000000..c3303ff9add --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -0,0 +1,68 @@ +package io.druid.cli; + +import com.google.inject.Injector; +import com.metamx.common.logger.Logger; +import com.metamx.druid.coordination.ZkCoordinator; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.HistoricalModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.http.QueryServlet; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerInitializer; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.metrics.MetricsModule; +import com.metamx.druid.metrics.ServerMonitor; +import io.airlift.command.Command; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; + +/** + */ +@Command( + name = "historical", + description = "Runs a Historical node, see https://github.com/metamx/druid/wiki/Compute for a description" +) +public class CliHistorical extends ServerRunnable +{ + private static final Logger log = new Logger(CliHistorical.class); + + public CliHistorical() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule().register(ZkCoordinator.class), + EmitterModule.class, + HttpClientModule.class, + CuratorModule.class, + new MetricsModule().register(ServerMonitor.class), + ServerModule.class, + new JettyServerModule(new HistoricalJettyServerInitializer()) + .addResource(StatusResource.class), + HistoricalModule.class + ); + } + + private static class HistoricalJettyServerInitializer implements JettyServerInitializer + { + @Override + public void initialize(Server server, Injector injector) + { + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.setResourceBase("/"); + + root.addServlet(new ServletHolder(new StatusServlet()), "/status"); + root.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); + } + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 5d465cce706..401e4b08203 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -19,7 +19,7 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(Coordinator.class); + .withCommands(CliCoordinator.class, CliHistorical.class); builder.build().parse(args).run(); } diff --git a/services/src/main/java/io/druid/cli/ServerRunnable.java b/services/src/main/java/io/druid/cli/ServerRunnable.java new file mode 100644 index 00000000000..1ebdc23ae27 --- /dev/null +++ b/services/src/main/java/io/druid/cli/ServerRunnable.java @@ -0,0 +1,45 @@ +package io.druid.cli; + +import com.google.common.base.Throwables; +import com.google.inject.Injector; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; +import com.metamx.druid.log.LogLevelAdjuster; + +/** + */ +public abstract class ServerRunnable implements Runnable +{ + private final Logger log; + + public ServerRunnable(Logger log) + { + this.log = log; + } + + protected abstract Injector getInjector(); + + @Override + public void run() + { + try { + LogLevelAdjuster.register(); + + final Lifecycle lifecycle = getInjector().getInstance(Lifecycle.class); + + try { + lifecycle.start(); + } + catch (Throwable t) { + log.error(t, "Error when starting up. Failing."); + System.exit(1); + } + + lifecycle.join(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + } +} From 9de2e5b096349f5d2a6349b3ebded80a72426945 Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 31 Jul 2013 14:55:34 -0700 Subject: [PATCH 17/92] 1) Adjust JsonConfigurator to be able to attach the proper property path if it is renamed via a @JsonProperty annotation 2) JsonConfigurator now requires that fields be annotated with @JsonProperty. It will fail fast if they aren't. --- .../metamx/druid/guice/JsonConfigurator.java | 50 ++++++++++++++++++- 1 file changed, 48 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java index 2d642260562..1a940fb05ca 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -1,8 +1,12 @@ package com.metamx.druid.guice; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotatedField; +import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition; import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -13,8 +17,12 @@ import com.metamx.common.logger.Logger; import javax.annotation.Nullable; import javax.validation.ConstraintViolation; +import javax.validation.ElementKind; +import javax.validation.Path; import javax.validation.Validator; import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -26,7 +34,7 @@ public class JsonConfigurator { private static final Logger log = new Logger(JsonConfigurator.class); - private static final Joiner JOINER = Joiner.on("; "); + private static final Joiner JOINER = Joiner.on("."); private final ObjectMapper jsonMapper; private final Validator validator; @@ -43,6 +51,8 @@ public class JsonConfigurator public T configurate(Properties props, String propertyPrefix, Class clazz) throws ProvisionException { + verifyClazzIsConfigurable(clazz); + // Make it end with a period so we only include properties with sub-object thingies. final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + "."; @@ -75,7 +85,25 @@ public class JsonConfigurator List messages = Lists.newArrayList(); for (ConstraintViolation violation : violations) { - messages.add(String.format("%s - %s", violation.getPropertyPath().toString(), violation.getMessage())); + List pathParts = Lists.newArrayList(); + try { + Class beanClazz = violation.getRootBeanClass(); + final Iterator iter = violation.getPropertyPath().iterator(); + while (iter.hasNext()) { + Path.Node next = iter.next(); + if (next.getKind() == ElementKind.PROPERTY) { + final String fieldName = next.getName(); + final Field theField = beanClazz.getDeclaredField(fieldName); + JsonProperty annotation = theField.getAnnotation(JsonProperty.class); + pathParts.add(annotation == null || annotation.value() == null ? fieldName : annotation.value()); + } + } + } + catch (NoSuchFieldException e) { + throw Throwables.propagate(e); + } + + messages.add(String.format("%s - %s", JOINER.join(pathParts), violation.getMessage())); } throw new ProvisionException( @@ -98,4 +126,22 @@ public class JsonConfigurator return config; } + + private void verifyClazzIsConfigurable(Class clazz) + { + final List beanDefs = jsonMapper.getSerializationConfig() + .introspect(jsonMapper.constructType(clazz)) + .findProperties(); + for (BeanPropertyDefinition beanDef : beanDefs) { + final AnnotatedField field = beanDef.getField(); + if (field == null || !field.hasAnnotation(JsonProperty.class)) { + throw new ProvisionException( + String.format( + "JsonConfigurator requires Jackson-annotated Config objects to have field annotations. %s doesn't", + clazz + ) + ); + } + } + } } From f4fc8b2f6e6f448ba5ed7c0fd93a881390e02ecf Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 31 Jul 2013 18:36:47 -0700 Subject: [PATCH 18/92] 1) A whole boondoggle of changes to Guicify the instantiation of things for the Historical nodes 2) Add GuiceInjectableValues that supports @JacksonInject resolution via Guice 3) Refactor RequestLogger stuff and setup polymorphic deserialization method of creating objects --- .../java/com/metamx/druid/QueryableNode.java | 15 +- .../metamx/druid/guice/QueryableModule.java | 47 +++++ .../com/metamx/druid/http/QueryServlet.java | 9 +- .../http/{ => log}/EmittingRequestLogger.java | 39 +++-- .../log/EmittingRequestLoggerProvider.java | 36 ++++ .../http/{ => log}/FileRequestLogger.java | 3 +- .../http/log/FileRequestLoggerProvider.java | 37 ++++ .../druid/http/log/NoopRequestLogger.java | 14 ++ .../http/log/NoopRequestLoggerProvider.java | 12 ++ .../druid/http/{ => log}/RequestLogger.java | 4 +- .../druid/http/log/RequestLoggerProvider.java | 13 ++ .../druid/initialization/DruidModule.java | 10 ++ .../druid/initialization/DruidNode.java | 6 +- .../druid/initialization/Initialization.java | 42 +++-- .../introspect/GuiceInjectableValues.java | 43 +++++ .../druid/guice/DruidSecondaryModule.java | 19 +- .../druid/guice/JsonConfigProvider.java | 14 +- .../metamx/druid/guice/SupplierProvider.java | 35 ++++ .../metamx/druid/guice/HistoricalModule.java | 165 +++++++++++++++++- .../com/metamx/druid/guice/ServerModule.java | 10 ++ .../druid/initialization/ServerInit.java | 31 ++-- .../druid/loading/HdfsDataSegmentPuller.java | 2 + .../druid/loading/S3CredentialsConfig.java | 24 +++ .../druid/loading/SegmentLoaderConfig.java | 5 + .../cassandra/CassandraDataSegmentPuller.java | 11 +- .../metamx/druid/metrics/MonitorsConfig.java | 2 +- ...DefaultQueryRunnerFactoryConglomerate.java | 2 + .../druid/query/group/GroupByQueryConfig.java | 34 ++++ .../druid/query/group/GroupByQueryEngine.java | 16 +- .../group/GroupByQueryRunnerFactory.java | 9 +- .../GroupByQueryRunnerFactoryConfig.java | 14 -- .../query/group/GroupByQueryRunnerTest.java | 17 +- .../GroupByTimeseriesQueryRunnerTest.java | 16 +- .../main/java/io/druid/cli/CliHistorical.java | 3 + 34 files changed, 645 insertions(+), 114 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/guice/QueryableModule.java rename client/src/main/java/com/metamx/druid/http/{ => log}/EmittingRequestLogger.java (76%) create mode 100644 client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java rename client/src/main/java/com/metamx/druid/http/{ => log}/FileRequestLogger.java (97%) create mode 100644 client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java create mode 100644 client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java create mode 100644 client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java rename client/src/main/java/com/metamx/druid/http/{ => log}/RequestLogger.java (91%) create mode 100644 client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/DruidModule.java create mode 100644 common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java create mode 100644 common/src/main/java/com/metamx/druid/guice/SupplierProvider.java create mode 100644 server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java create mode 100644 server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java delete mode 100644 server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactoryConfig.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index ce70dca4398..28ddbb9c7f9 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -44,7 +44,7 @@ import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.guice.JsonConfigurator; -import com.metamx.druid.http.RequestLogger; +import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkPathsConfig; @@ -372,17 +372,12 @@ public abstract class QueryableNode extends Registering try { final String loggingType = props.getProperty("druid.request.logging.type"); if("emitter".equals(loggingType)) { - setRequestLogger(Initialization.makeEmittingRequestLogger( - getProps(), - getEmitter() - )); + setRequestLogger(Initialization.makeEmittingRequestLogger(getProps(), getEmitter())); } else { - setRequestLogger(Initialization.makeFileRequestLogger( - getJsonMapper(), - getScheduledExecutorFactory(), - getProps() - )); + setRequestLogger( + Initialization.makeFileRequestLogger(getJsonMapper(), getScheduledExecutorFactory(), getProps()) + ); } } catch (IOException e) { diff --git a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java new file mode 100644 index 00000000000..b28f7bd4021 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java @@ -0,0 +1,47 @@ +package com.metamx.druid.guice; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import com.metamx.druid.http.QueryServlet; +import com.metamx.druid.http.log.EmittingRequestLoggerProvider; +import com.metamx.druid.http.log.FileRequestLoggerProvider; +import com.metamx.druid.http.log.RequestLogger; +import com.metamx.druid.http.log.RequestLoggerProvider; +import com.metamx.druid.initialization.DruidModule; +import com.metamx.druid.query.segment.QuerySegmentWalker; + +import java.util.Arrays; +import java.util.List; + +/** + */ +public class QueryableModule implements DruidModule +{ + private final Class walkerClass; + + public QueryableModule( + Class walkerClass + ) + { + this.walkerClass = walkerClass; + } + + @Override + public void configure(Binder binder) + { + binder.bind(QueryServlet.class).in(LazySingleton.class); + binder.bind(QuerySegmentWalker.class).to(walkerClass).in(LazySingleton.class); + binder.bind(RequestLogger.class).toProvider(RequestLoggerProvider.class).in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.request.logging", RequestLoggerProvider.class); + } + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("QueryableModule") + .registerSubtypes(EmittingRequestLoggerProvider.class, FileRequestLoggerProvider.class) + ); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 78cb428998a..5c78bc3aad5 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -25,10 +25,14 @@ import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.inject.Inject; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; import com.metamx.druid.Query; +import com.metamx.druid.guice.annotations.Json; +import com.metamx.druid.guice.annotations.Smile; +import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; @@ -57,9 +61,10 @@ public class QueryServlet extends HttpServlet private final ServiceEmitter emitter; private final RequestLogger requestLogger; + @Inject public QueryServlet( - ObjectMapper jsonMapper, - ObjectMapper smileMapper, + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper, QuerySegmentWalker texasRanger, ServiceEmitter emitter, RequestLogger requestLogger diff --git a/client/src/main/java/com/metamx/druid/http/EmittingRequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java similarity index 76% rename from client/src/main/java/com/metamx/druid/http/EmittingRequestLogger.java rename to client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java index 600eceae6f9..08691a3b637 100644 --- a/client/src/main/java/com/metamx/druid/http/EmittingRequestLogger.java +++ b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java @@ -17,38 +17,35 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package com.metamx.druid.http.log; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.collect.ImmutableMap; import com.metamx.druid.Query; -import com.metamx.emitter.core.Emitter; +import com.metamx.druid.http.RequestLogLine; import com.metamx.emitter.core.Event; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceEventBuilder; import org.joda.time.DateTime; import java.util.Map; public class EmittingRequestLogger implements RequestLogger { - - final String service; - final String host; - final Emitter emitter; + final ServiceEmitter emitter; final String feed; - public EmittingRequestLogger(String service, String host, Emitter emitter, String feed) + public EmittingRequestLogger(ServiceEmitter emitter, String feed) { this.emitter = emitter; - this.host = host; - this.service = service; this.feed = feed; } @Override public void log(final RequestLogLine requestLogLine) throws Exception { - emitter.emit(new RequestLogEvent(service, host, feed, requestLogLine)); + emitter.emit(new RequestLogEventBuilder(feed, requestLogLine)); } public static class RequestLogEvent implements Event @@ -119,4 +116,26 @@ public class EmittingRequestLogger implements RequestLogger return true; } } + + private class RequestLogEventBuilder implements ServiceEventBuilder + { + private final String feed; + private final RequestLogLine requestLogLine; + + public RequestLogEventBuilder( + String feed, + RequestLogLine requestLogLine + ) + { + this.feed = feed; + this.requestLogLine = requestLogLine; + } + + + @Override + public Event build(String service, String host) + { + return new RequestLogEvent(service, host, feed, requestLogLine); + } + } } diff --git a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java new file mode 100644 index 00000000000..9aaca387ab9 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java @@ -0,0 +1,36 @@ +package com.metamx.druid.http.log; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.metamx.emitter.service.ServiceEmitter; + +import javax.validation.constraints.NotNull; + +/** + */ +@JsonTypeName("emitter") +public class EmittingRequestLoggerProvider implements RequestLoggerProvider +{ + @JsonProperty + @NotNull + private String feed = null; + + @JacksonInject + @NotNull + private ServiceEmitter emitter = null; + + @Inject + public void injectMe(Injector injector) + { + System.out.println("YAYAYAYAYAYA!!!"); + } + + @Override + public RequestLogger get() + { + return new EmittingRequestLogger(emitter, feed); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/FileRequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java similarity index 97% rename from client/src/main/java/com/metamx/druid/http/FileRequestLogger.java rename to client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java index 32b245e6c60..0891edbd713 100644 --- a/client/src/main/java/com/metamx/druid/http/FileRequestLogger.java +++ b/client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package com.metamx.druid.http.log; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; @@ -25,6 +25,7 @@ import com.google.common.io.Closeables; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.http.RequestLogLine; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.MutableDateTime; diff --git a/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java new file mode 100644 index 00000000000..9526aa65ffe --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java @@ -0,0 +1,37 @@ +package com.metamx.druid.http.log; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.druid.guice.annotations.Json; + +import javax.validation.constraints.NotNull; +import java.io.File; + +/** + */ +@JsonTypeName("file") +public class FileRequestLoggerProvider implements RequestLoggerProvider +{ + @JsonProperty + @NotNull + private File dir = null; + + @JacksonInject + @NotNull + private ScheduledExecutorFactory factory = null; + + + @JacksonInject + @NotNull + @Json + private ObjectMapper jsonMapper = null; + + @Override + public RequestLogger get() + { + return new FileRequestLogger(jsonMapper, factory.create(1, "RequestLogger-%s"), dir); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java new file mode 100644 index 00000000000..062f8ddc82c --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java @@ -0,0 +1,14 @@ +package com.metamx.druid.http.log; + +import com.metamx.druid.http.RequestLogLine; + +/** + */ +public class NoopRequestLogger implements RequestLogger +{ + @Override + public void log(RequestLogLine requestLogLine) throws Exception + { + // This is a no op! + } +} diff --git a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java new file mode 100644 index 00000000000..0bafb7945a3 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java @@ -0,0 +1,12 @@ +package com.metamx.druid.http.log; + +/** + */ +public class NoopRequestLoggerProvider implements RequestLoggerProvider +{ + @Override + public RequestLogger get() + { + return new NoopRequestLogger(); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/RequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/RequestLogger.java similarity index 91% rename from client/src/main/java/com/metamx/druid/http/RequestLogger.java rename to client/src/main/java/com/metamx/druid/http/log/RequestLogger.java index 7720938c4bc..ff3b0e84401 100644 --- a/client/src/main/java/com/metamx/druid/http/RequestLogger.java +++ b/client/src/main/java/com/metamx/druid/http/log/RequestLogger.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package com.metamx.druid.http.log; + +import com.metamx.druid.http.RequestLogLine; /** */ diff --git a/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java new file mode 100644 index 00000000000..0e2ecc15b44 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java @@ -0,0 +1,13 @@ +package com.metamx.druid.http.log; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + * A Marker interface for things that can provide a RequestLogger. This can be combined with jackson polymorphic serde + * to provide new RequestLogger implementations as plugins. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopRequestLoggerProvider.class) +public interface RequestLoggerProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidModule.java b/client/src/main/java/com/metamx/druid/initialization/DruidModule.java new file mode 100644 index 00000000000..0677d7110cf --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/DruidModule.java @@ -0,0 +1,10 @@ +package com.metamx.druid.initialization; + +import java.util.List; + +/** + */ +public interface DruidModule extends com.google.inject.Module +{ + public List getJacksonModules(); +} diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java index bd8be448e66..35bfb5415b3 100644 --- a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java @@ -31,12 +31,15 @@ import javax.validation.constraints.NotNull; */ public class DruidNode { + @JsonProperty("service") @NotNull private String serviceName = null; + @JsonProperty @NotNull private String host = null; + @JsonProperty @Min(0) @Max(0xffff) private int port = -1; @@ -82,19 +85,16 @@ public class DruidNode this.port = port; } - @JsonProperty("service") public String getServiceName() { return serviceName; } - @JsonProperty public String getHost() { return host; } - @JsonProperty public int getPort() { return port; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 81bb6e2f750..e6460c0f147 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -28,6 +28,7 @@ import com.google.common.io.Closeables; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.Module; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -40,12 +41,14 @@ import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.guice.DruidGuiceExtensions; import com.metamx.druid.guice.DruidSecondaryModule; -import com.metamx.druid.http.EmittingRequestLogger; -import com.metamx.druid.http.FileRequestLogger; -import com.metamx.druid.http.RequestLogger; +import com.metamx.druid.guice.annotations.Json; +import com.metamx.druid.guice.annotations.Smile; +import com.metamx.druid.http.log.EmittingRequestLogger; +import com.metamx.druid.http.log.FileRequestLogger; +import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.jackson.JacksonModule; import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.core.Emitter; +import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.BoundedExponentialBackoffRetry; @@ -355,14 +358,9 @@ public class Initialization ); } - public static RequestLogger makeEmittingRequestLogger(Properties props, Emitter emitter) + public static RequestLogger makeEmittingRequestLogger(Properties props, ServiceEmitter emitter) { - return new EmittingRequestLogger( - PropUtils.getProperty(props, "druid.service"), - PropUtils.getProperty(props, "druid.host"), - emitter, - PropUtils.getProperty(props, "druid.request.logging.feed") - ); + return new EmittingRequestLogger(emitter, PropUtils.getProperty(props, "druid.request.logging.feed")); } public static Injector makeInjector(final Object... modules) @@ -398,15 +396,27 @@ public class Initialization actualModules, new Function() { + ObjectMapper jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class)); + ObjectMapper smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class)); + @Override @SuppressWarnings("unchecked") public Module apply(@Nullable Object input) { + if (input instanceof DruidModule) { + baseInjector.injectMembers(input); + return registerJacksonModules(((DruidModule) input)); + } + if (input instanceof Module) { baseInjector.injectMembers(input); return (Module) input; } + if (input instanceof Class) { + if (DruidModule.class.isAssignableFrom((Class) input)) { + return registerJacksonModules(baseInjector.getInstance((Class) input)); + } if (Module.class.isAssignableFrom((Class) input)) { return baseInjector.getInstance((Class) input); } @@ -414,8 +424,18 @@ public class Initialization throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); } } + throw new ISE("Unknown module type[%s]", input.getClass()); } + + private DruidModule registerJacksonModules(DruidModule module) + { + for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + smileMapper.registerModule(jacksonModule); + } + return module; + } } ) ); diff --git a/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java new file mode 100644 index 00000000000..52175898461 --- /dev/null +++ b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java @@ -0,0 +1,43 @@ +package com.fasterxml.jackson.databind.introspect; + +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.google.inject.BindingAnnotation; +import com.google.inject.Injector; +import com.google.inject.Key; + +import java.lang.annotation.Annotation; + +/** +*/ +public class GuiceInjectableValues extends InjectableValues +{ + private final Injector injector; + + public GuiceInjectableValues(Injector injector) {this.injector = injector;} + + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + final AnnotatedMember member = forProperty.getMember(); + Annotation guiceAnnotation = null; + for (Annotation annotation : member.getAllAnnotations()._annotations.values()) { + if (annotation.annotationType().isAnnotationPresent(BindingAnnotation.class)) { + guiceAnnotation = annotation; + break; + } + } + + final Key key; + if (guiceAnnotation == null) { + key = Key.get(forProperty.getMember().getGenericType()); + } + else { + key = Key.get(forProperty.getMember().getGenericType(), guiceAnnotation); + } + return injector.getInstance(key); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java index 9d60f5efcc8..24c5466d22d 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -1,10 +1,13 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; +import com.google.inject.Provides; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; import org.skife.config.ConfigurationObjectFactory; @@ -48,9 +51,21 @@ public class DruidSecondaryModule implements Module binder.bind(Properties.class).toInstance(properties); binder.bind(ConfigurationObjectFactory.class).toInstance(factory); binder.bind(ObjectMapper.class).to(Key.get(ObjectMapper.class, Json.class)); - binder.bind(ObjectMapper.class).annotatedWith(Json.class).toInstance(jsonMapper); - binder.bind(ObjectMapper.class).annotatedWith(Smile.class).toInstance(smileMapper); binder.bind(Validator.class).toInstance(validator); binder.bind(JsonConfigurator.class).toInstance(jsonConfigurator); } + + @Provides @LazySingleton @Json + public ObjectMapper getJsonMapper(final Injector injector) + { + jsonMapper.setInjectableValues(new GuiceInjectableValues(injector)); + return jsonMapper; + } + + @Provides @LazySingleton @Smile + public ObjectMapper getSmileMapper(Injector injector) + { + smileMapper.setInjectableValues(new GuiceInjectableValues(injector)); + return smileMapper; + } } diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java index d85a99d02b3..3066286ea7f 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java @@ -41,6 +41,7 @@ public class JsonConfigProvider implements Provider> binder, propertyBase, classToProvide, + Key.get(classToProvide), (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide)) ); } @@ -52,6 +53,7 @@ public class JsonConfigProvider implements Provider> binder, propertyBase, classToProvide, + Key.get(classToProvide, annotation), (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation) ); } @@ -68,14 +70,22 @@ public class JsonConfigProvider implements Provider> binder, propertyBase, classToProvide, + Key.get(classToProvide, annotation), (Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation) ); } @SuppressWarnings("unchecked") - public static void bind(Binder binder, String propertyBase, Class clazz, Key> key) + public static void bind( + Binder binder, + String propertyBase, + Class clazz, + Key instanceKey, + Key> supplierKey + ) { - binder.bind(key).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class); + binder.bind(supplierKey).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class); + binder.bind(instanceKey).toProvider(new SupplierProvider(supplierKey)); } public static JsonConfigProvider of(String propertyBase, Class classToProvide) diff --git a/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java b/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java new file mode 100644 index 00000000000..e7dcc875949 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java @@ -0,0 +1,35 @@ +package com.metamx.druid.guice; + +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; + +/** + */ +public class SupplierProvider implements Provider +{ + private final Key> supplierKey; + + private Provider> supplierProvider; + + public SupplierProvider( + Key> supplierKey + ) + { + this.supplierKey = supplierKey; + } + + @Inject + public void configure(Injector injector) + { + this.supplierProvider = injector.getProvider(supplierKey); + } + + @Override + public T get() + { + return supplierProvider.get().get(); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index 466acc37cc6..7c931baa12c 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -1,14 +1,22 @@ package com.metamx.druid.guice; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.ProvisionException; +import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.metamx.common.logger.Logger; +import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.Query; import com.metamx.druid.client.DruidServerConfig; +import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.ServerManager; +import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.guice.annotations.Processing; import com.metamx.druid.loading.BaseSegmentLoader; import com.metamx.druid.loading.DataSegmentPuller; @@ -16,26 +24,53 @@ import com.metamx.druid.loading.HdfsDataSegmentPuller; import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; +import com.metamx.druid.loading.S3CredentialsConfig; import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; +import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.MetricsEmittingExecutorService; +import com.metamx.druid.query.QueryRunnerFactory; +import com.metamx.druid.query.QueryRunnerFactoryConglomerate; +import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.group.GroupByQueryConfig; +import com.metamx.druid.query.group.GroupByQueryEngine; +import com.metamx.druid.query.group.GroupByQueryRunnerFactory; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; +import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.search.SearchQueryRunnerFactory; +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.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import org.apache.hadoop.conf.Configuration; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; +import java.lang.reflect.InvocationTargetException; +import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; /** */ public class HistoricalModule implements Module { + private static final Logger log = new Logger(HistoricalModule.class); + @Override public void configure(Binder binder) { ConfigProvider.bind(binder, DruidServerConfig.class); - ConfigProvider.bind(binder, ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing")); + ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); + binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); @@ -44,16 +79,58 @@ public class HistoricalModule implements Module binder.bind(SegmentLoader.class).to(BaseSegmentLoader.class).in(LazySingleton.class); binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, - String.class, - DataSegmentPuller.class + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); + + bindDeepStorageS3(binder); + bindDeepStorageHdfs(binder); + bindDeepStorageCassandra(binder); + + + final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( + binder, new TypeLiteral>(){}, new TypeLiteral(){} ); - segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); - segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + queryFactoryBinder.addBinding(TimeseriesQuery.class).to(TimeseriesQueryRunnerFactory.class).in(LazySingleton.class); + queryFactoryBinder.addBinding(SearchQuery.class).to(SearchQueryRunnerFactory.class).in(LazySingleton.class); + queryFactoryBinder.addBinding(TimeBoundaryQuery.class).to(TimeBoundaryQueryRunnerFactory.class).in(LazySingleton.class); + queryFactoryBinder.addBinding(SegmentMetadataQuery.class).to(SegmentMetadataQueryRunnerFactory.class).in(LazySingleton.class); + + queryFactoryBinder.addBinding(GroupByQuery.class).to(GroupByQueryRunnerFactory.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); + binder.bind(GroupByQueryEngine.class).in(LazySingleton.class); + + binder.bind(QueryRunnerFactoryConglomerate.class).to(DefaultQueryRunnerFactoryConglomerate.class).in(LazySingleton.class); + } + + private void bindDeepStorageS3(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); + } + + private void bindDeepStorageHdfs(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + binder.bind(Configuration.class).toInstance(new Configuration()); + } + + private void bindDeepStorageCassandra(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); + ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); } @Provides @Processing @ManageLifecycle @@ -65,4 +142,80 @@ public class HistoricalModule implements Module new ServiceMetricEvent.Builder() ); } + + @Provides @LazySingleton + public RestS3Service getRestS3Service(Supplier config) + { + try { + return new RestS3Service(new AWSCredentials(config.get().getAccessKey(), config.get().getSecretKey())); + } + catch (S3ServiceException e) { + throw new ProvisionException("Unable to create a RestS3Service", e); + } + } + + @Provides @LazySingleton @Global + public StupidPool getIntermediateResultsPool(DruidProcessingConfig config) + { + try { + Class vmClass = Class.forName("sun.misc.VM"); + Object maxDirectMemoryObj = vmClass.getMethod("maxDirectMemory").invoke(null); + + if (maxDirectMemoryObj == null || !(maxDirectMemoryObj instanceof Number)) { + log.info("Cannot determine maxDirectMemory from[%s]", maxDirectMemoryObj); + } else { + long maxDirectMemory = ((Number) maxDirectMemoryObj).longValue(); + + final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * (config.getNumThreads() + 1); + if (maxDirectMemory < memoryNeeded) { + throw new ProvisionException( + String.format( + "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize or druid.computation.buffer.size: " + + "maxDirectMemory[%,d], memoryNeeded[%,d], druid.computation.buffer.size[%,d], numThreads[%,d]", + maxDirectMemory, memoryNeeded, config.intermediateComputeSizeBytes(), config.getNumThreads() + ) + ); + } + } + } + catch (ClassNotFoundException e) { + log.info("No VM class, cannot do memory check."); + } + catch (NoSuchMethodException e) { + log.info("VM.maxDirectMemory doesn't exist, cannot do memory check."); + } + catch (InvocationTargetException e) { + log.warn(e, "static method shouldn't throw this"); + } + catch (IllegalAccessException e) { + log.warn(e, "public method, shouldn't throw this"); + } + + return new IntermediateProcessingBufferPool(config.intermediateComputeSizeBytes()); + } + + private static class IntermediateProcessingBufferPool extends StupidPool + { + private static final Logger log = new Logger(IntermediateProcessingBufferPool.class); + + public IntermediateProcessingBufferPool(final int computationBufferSize) + { + super( + new Supplier() + { + final AtomicLong count = new AtomicLong(0); + + @Override + public ByteBuffer get() + { + log.info( + "Allocating new intermediate processing buffer[%,d] of size[%,d]", + count.getAndIncrement(), computationBufferSize + ); + return ByteBuffer.allocateDirect(computationBufferSize); + } + } + ); + } + } } diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java index 57e98a3c7b4..0bcda8d113c 100644 --- a/server/src/main/java/com/metamx/druid/guice/ServerModule.java +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.java @@ -2,6 +2,10 @@ package com.metamx.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.DruidNode; @@ -14,4 +18,10 @@ public class ServerModule implements Module { JsonConfigProvider.bind(binder, "druid", DruidNode.class, Self.class); } + + @Provides @LazySingleton + public ScheduledExecutorFactory getScheduledExecutorFactory(Lifecycle lifecycle) + { + return ScheduledExecutors.createFactory(lifecycle); + } } 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 0ac5e67a4de..32479965651 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -19,20 +19,9 @@ package com.metamx.druid.initialization; -import java.lang.reflect.InvocationTargetException; -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.conf.Configuration; -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 com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -62,10 +51,9 @@ import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; import com.metamx.druid.loading.cassandra.CassandraDataSegmentPusher; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.group.GroupByQueryConfig; import com.metamx.druid.query.group.GroupByQueryEngine; -import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.query.group.GroupByQueryRunnerFactory; -import com.metamx.druid.query.group.GroupByQueryRunnerFactoryConfig; import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; import com.metamx.druid.query.search.SearchQuery; @@ -75,6 +63,17 @@ 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.apache.hadoop.conf.Configuration; +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; /** */ @@ -160,10 +159,10 @@ public class ServerInit GroupByQuery.class, new GroupByQueryRunnerFactory( new GroupByQueryEngine( - configFactory.build(GroupByQueryEngineConfig.class), + Suppliers.ofInstance(new GroupByQueryConfig()), // TODO: Get rid of this computationBufferPool ), - configFactory.build(GroupByQueryRunnerFactoryConfig.class) + Suppliers.ofInstance(new GroupByQueryConfig()) ) ); queryRunners.put(SearchQuery.class, new SearchQueryRunnerFactory()); diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java index fd0c93264ab..0e25a2de792 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java @@ -1,6 +1,7 @@ package com.metamx.druid.loading; import com.google.common.io.Closeables; +import com.google.inject.Inject; import com.metamx.druid.client.DataSegment; import com.metamx.druid.utils.CompressionUtils; import org.apache.hadoop.conf.Configuration; @@ -17,6 +18,7 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller { private final Configuration config; + @Inject public HdfsDataSegmentPuller(final Configuration config) { this.config = config; diff --git a/server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java b/server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java new file mode 100644 index 00000000000..252cf3c92df --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java @@ -0,0 +1,24 @@ +package com.metamx.druid.loading; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class S3CredentialsConfig +{ + @JsonProperty + private String accessKey = ""; + + @JsonProperty + private String secretKey = ""; + + public String getAccessKey() + { + return accessKey; + } + + public String getSecretKey() + { + return secretKey; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index 1ec5e1b6584..2271126ac51 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -40,6 +40,11 @@ public class SegmentLoaderConfig return cacheDirectory; } + public void setCacheDirectory(File cacheDirectory) + { + this.cacheDirectory = cacheDirectory; + } + public boolean isDeleteOnRemove() { return deleteOnRemove; diff --git a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java index 87e6105f161..a11c9e2ec2f 100644 --- a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java @@ -19,12 +19,8 @@ package com.metamx.druid.loading.cassandra; -import java.io.File; -import java.io.OutputStream; - -import org.apache.commons.io.FileUtils; - import com.google.common.io.Files; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; @@ -36,6 +32,10 @@ import com.netflix.astyanax.connectionpool.exceptions.ConnectionException; import com.netflix.astyanax.model.ColumnList; import com.netflix.astyanax.recipes.storage.ChunkedStorage; import com.netflix.astyanax.recipes.storage.ObjectMetadata; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.OutputStream; /** * Cassandra Segment Puller @@ -48,6 +48,7 @@ public class CassandraDataSegmentPuller extends CassandraStorage implements Data private static final int CONCURRENCY = 10; private static final int BATCH_SIZE = 10; + @Inject public CassandraDataSegmentPuller(CassandraDataSegmentConfig config) { super(config); diff --git a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java index 9d59c4fbdab..72f8feea6ca 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java +++ b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java @@ -13,7 +13,7 @@ import java.util.List; */ public class MonitorsConfig { - @JsonProperty("monitors") + @JsonProperty("monitorExclusions") @NotNull private List> monitors = ImmutableList.>builder() .add(JvmMonitor.class) diff --git a/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java b/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java index d8db68446b0..53208308719 100644 --- a/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java +++ b/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java @@ -19,6 +19,7 @@ package com.metamx.druid.query; +import com.google.inject.Inject; import com.metamx.druid.Query; import java.util.Map; @@ -29,6 +30,7 @@ public class DefaultQueryRunnerFactoryConglomerate implements QueryRunnerFactory { private final Map, QueryRunnerFactory> factories; + @Inject public DefaultQueryRunnerFactoryConglomerate( Map, QueryRunnerFactory> factories ) diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java new file mode 100644 index 00000000000..bb3c4693b69 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java @@ -0,0 +1,34 @@ +package com.metamx.druid.query.group; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class GroupByQueryConfig +{ + @JsonProperty + private boolean singleThreaded = false; + + @JsonProperty + private int maxIntermediateRows = 50000; + + public boolean isSingleThreaded() + { + return singleThreaded; + } + + public void setSingleThreaded(boolean singleThreaded) + { + this.singleThreaded = singleThreaded; + } + + public int getMaxIntermediateRows() + { + return maxIntermediateRows; + } + + public void setMaxIntermediateRows(int maxIntermediateRows) + { + this.maxIntermediateRows = maxIntermediateRows; + } +} diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 3c123bd64b3..b61f0b71944 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -20,12 +20,14 @@ package com.metamx.druid.query.group; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Closeables; import com.google.common.primitives.Ints; +import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; @@ -39,6 +41,7 @@ import com.metamx.druid.aggregation.BufferAggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.index.brita.Filters; import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.index.v1.processing.DimensionSelector; @@ -60,12 +63,13 @@ import java.util.TreeMap; */ public class GroupByQueryEngine { - private final GroupByQueryEngineConfig config; + private final Supplier config; private final StupidPool intermediateResultsBufferPool; + @Inject public GroupByQueryEngine ( - GroupByQueryEngineConfig config, - StupidPool intermediateResultsBufferPool + Supplier config, + @Global StupidPool intermediateResultsBufferPool ) { this.config = config; @@ -106,7 +110,7 @@ public class GroupByQueryEngine @Override public CloseableIterator make() { - return new RowIterator(query, cursor, bufferHolder.get()); + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); } @Override @@ -269,6 +273,7 @@ public class GroupByQueryEngine private final GroupByQuery query; private final Cursor cursor; private final ByteBuffer metricsBuffer; + private final GroupByQueryConfig config; private final List dimensionSpecs; private final List dimensions; @@ -281,11 +286,12 @@ public class GroupByQueryEngine private List unprocessedKeys; private Iterator delegate; - public RowIterator(GroupByQuery query, Cursor cursor, ByteBuffer metricsBuffer) + public RowIterator(GroupByQuery query, Cursor cursor, ByteBuffer metricsBuffer, GroupByQueryConfig config) { this.query = query; this.cursor = cursor; this.metricsBuffer = metricsBuffer; + this.config = config; unprocessedKeys = null; delegate = Iterators.emptyIterator(); diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java index 0ecec11b76c..c69e16c7715 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java @@ -20,9 +20,11 @@ package com.metamx.druid.query.group; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import com.google.common.primitives.Longs; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; @@ -49,11 +51,12 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory config; + @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, - GroupByQueryRunnerFactoryConfig config + Supplier config ) { this.engine = engine; @@ -69,7 +72,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory mergeRunners(final ExecutorService queryExecutor, Iterable> queryRunners) { - if (config.isSingleThreaded()) { + if (config.get().isSingleThreaded()) { return new ConcatQueryRunner( Sequences.map( Sequences.simple(queryRunners), diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactoryConfig.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactoryConfig.java deleted file mode 100644 index 52e3227c1ff..00000000000 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactoryConfig.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.query.group; - -import org.skife.config.Config; - -/** - */ -public abstract class GroupByQueryRunnerFactoryConfig -{ - @Config("druid.query.groupBy.singleThreaded") - public boolean isSingleThreaded() - { - return false; - } -} diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index b60b8ce825e..9ba4e51f628 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -22,6 +22,7 @@ package com.metamx.druid.query.group; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -77,16 +78,12 @@ public class GroupByQueryRunnerTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { + GroupByQueryConfig config = new GroupByQueryConfig(); + config.setMaxIntermediateRows(10000); + final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( new GroupByQueryEngine( - new GroupByQueryEngineConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 10000; - } - }, + Suppliers.ofInstance(config), new StupidPool( new Supplier() { @@ -98,9 +95,7 @@ public class GroupByQueryRunnerTest } ) ), - new GroupByQueryRunnerFactoryConfig() - { - } + Suppliers.ofInstance(config) ); return Lists.newArrayList( diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index ca6dab39c98..5ca55f111bc 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -22,6 +22,7 @@ package com.metamx.druid.query.group; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.Query; @@ -51,17 +52,12 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { + GroupByQueryConfig config = new GroupByQueryConfig(); + config.setMaxIntermediateRows(10000); final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( new GroupByQueryEngine( - new GroupByQueryEngineConfig() - { - @Override - public int getMaxIntermediateRows() - { - return 10000; - } - }, + Suppliers.ofInstance(config), new StupidPool( new Supplier() { @@ -73,9 +69,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest } ) ), - new GroupByQueryRunnerFactoryConfig() - { - } + Suppliers.ofInstance(config) ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index c3303ff9add..ab32d53649b 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -2,11 +2,13 @@ package io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; +import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusResource; @@ -49,6 +51,7 @@ public class CliHistorical extends ServerRunnable ServerModule.class, new JettyServerModule(new HistoricalJettyServerInitializer()) .addResource(StatusResource.class), + new QueryableModule(ServerManager.class), HistoricalModule.class ); } From 019bb5d453d5b75b253d718b9a8b1508b466a850 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 1 Aug 2013 15:28:08 -0700 Subject: [PATCH 19/92] 1) Another whole bunch of changes to annotate things and create Modules and bind stuff. But OMFG, the compute node actually appears to be working! 2) The compute node works with Guice 3) The compute node fires up with Guice and appears to work 4) Did I mention that the compute node, now called historical node, fires up with Guice and appears to work? --- .../java/com/metamx/druid/QueryableNode.java | 4 +- .../com/metamx/druid/client/DruidServer.java | 6 ++- .../druid/client/DruidServerConfig.java | 29 +++++++------ .../CuratorDataSegmentAnnouncer.java | 2 + .../com/metamx/druid/http/ZkPathsModule.java | 14 ++++++ .../curator/announcement/AnnouncerTest.java | 2 - .../GuiceAnnotationIntrospector.java | 33 ++++++++++++++ .../introspect/GuiceInjectableValues.java | 21 +-------- .../com/metamx/druid/db/DbTablesConfig.java | 14 +++--- .../druid/guice/DruidSecondaryModule.java | 20 ++++++++- .../metamx/druid/guice/JsonConfigurator.java | 3 +- .../metamx/druid/guice/LifecycleModule.java | 5 ++- .../metamx/druid/guice/LifecycleScope.java | 8 +++- .../druid/guice/ManageLifecycleLast.java | 21 +++++++++ .../druid/indexing/common/TaskToolbox.java | 10 ++--- pom.xml | 6 +-- .../druid/coordination/ZkCoordinator.java | 17 +++++--- .../metamx/druid/guice/CoordinatorModule.java | 11 ----- .../metamx/druid/guice/HistoricalModule.java | 43 ++++++++++++++++--- .../com/metamx/druid/guice/ServerModule.java | 3 ++ .../com/metamx/druid/http/ComputeNode.java | 3 +- .../druid/loading/SegmentLoaderConfig.java | 20 +++++---- .../druid/loading/SingleSegmentLoader.java | 14 +++--- .../druid/coordination/ZkCoordinatorTest.java | 36 ++++++++-------- .../main/java/io/druid/cli/CliHistorical.java | 7 +++ 25 files changed, 235 insertions(+), 117 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/http/ZkPathsModule.java create mode 100644 common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceAnnotationIntrospector.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 28ddbb9c7f9..5e119c33fd4 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -327,8 +327,8 @@ public abstract class QueryableNode extends Registering final DruidServerConfig serverConfig = getConfigFactory().build(DruidServerConfig.class); setDruidServerMetadata( new DruidServerMetadata( - serverConfig.getServerName(), - serverConfig.getHost(), + null, // TODO: serverConfig.getServerName(), + null, // TODO: serverConfig.getHost(), serverConfig.getMaxSize(), nodeType, serverConfig.getTier() diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index ebd078e9383..d93003c9ca3 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.initialization.DruidNode; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -45,13 +46,14 @@ public class DruidServer implements Comparable private volatile long currSize; public DruidServer( + DruidNode node, DruidServerConfig config, String type ) { this( - config.getServerName(), - config.getHost(), + node.getHost(), + node.getHost(), config.getMaxSize(), type, config.getTier() diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 89028021c18..6a57a38b20f 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -19,23 +19,28 @@ package com.metamx.druid.client; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Min; /** */ -public abstract class DruidServerConfig +public class DruidServerConfig { - @Config("druid.host") - public abstract String getServerName(); + @JsonProperty + @Min(0) + private long maxSize = -1; - @Config("druid.host") - public abstract String getHost(); + @JsonProperty + private String tier = "_default_tier"; - @Config("druid.server.maxSize") - public abstract long getMaxSize(); + public long getMaxSize() + { + return maxSize; + } - @Config("druid.server.tier") - @Default("_default_tier") - public abstract String getTier(); + public String getTier() + { + return tier; + } } diff --git a/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java index a7efc6cb9e8..f420d90bbbd 100644 --- a/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/CuratorDataSegmentAnnouncer.java @@ -22,6 +22,7 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; @@ -46,6 +47,7 @@ public class CuratorDataSegmentAnnouncer implements DataSegmentAnnouncer private volatile boolean started = false; + @Inject public CuratorDataSegmentAnnouncer( DruidServerMetadata server, ZkPathsConfig config, diff --git a/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java b/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java new file mode 100644 index 00000000000..f0a75fce898 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java @@ -0,0 +1,14 @@ +package com.metamx.druid.http; + +import com.google.inject.Binder; +import com.google.inject.Module; + +/** + */ +public class ZkPathsModule implements Module +{ + @Override + public void configure(Binder binder) + { + } +} diff --git a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java b/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java index 21c9ae2eee0..4b8b4806bdd 100644 --- a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java +++ b/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java @@ -113,8 +113,6 @@ public class AnnouncerTest extends CuratorTestBase @Test public void testSessionKilled() throws Exception { - final ExecutorService exec = Execs.singleThreaded("test-announcer-sanity-%s"); - curator.start(); Announcer announcer = new Announcer(curator, exec); try { diff --git a/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceAnnotationIntrospector.java b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceAnnotationIntrospector.java new file mode 100644 index 00000000000..76146f5cfb4 --- /dev/null +++ b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceAnnotationIntrospector.java @@ -0,0 +1,33 @@ +package com.fasterxml.jackson.databind.introspect; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.google.inject.BindingAnnotation; +import com.google.inject.Key; + +import java.lang.annotation.Annotation; + +/** + */ +public class GuiceAnnotationIntrospector extends NopAnnotationIntrospector +{ + @Override + public Object findInjectableValueId(AnnotatedMember m) + { + if (m.getAnnotation(JacksonInject.class) == null) { + return null; + } + + Annotation guiceAnnotation = null; + for (Annotation annotation : m.getAllAnnotations()._annotations.values()) { + if (annotation.annotationType().isAnnotationPresent(BindingAnnotation.class)) { + guiceAnnotation = annotation; + break; + } + } + + if (guiceAnnotation == null) { + return Key.get(m.getGenericType()); + } + return Key.get(m.getGenericType(), guiceAnnotation); + } +} diff --git a/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java index 52175898461..dc50cd2603b 100644 --- a/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java +++ b/common/src/main/java/com/fasterxml/jackson/databind/introspect/GuiceInjectableValues.java @@ -3,12 +3,9 @@ package com.fasterxml.jackson.databind.introspect; import com.fasterxml.jackson.databind.BeanProperty; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; -import com.google.inject.BindingAnnotation; import com.google.inject.Injector; import com.google.inject.Key; -import java.lang.annotation.Annotation; - /** */ public class GuiceInjectableValues extends InjectableValues @@ -22,22 +19,6 @@ public class GuiceInjectableValues extends InjectableValues Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance ) { - final AnnotatedMember member = forProperty.getMember(); - Annotation guiceAnnotation = null; - for (Annotation annotation : member.getAllAnnotations()._annotations.values()) { - if (annotation.annotationType().isAnnotationPresent(BindingAnnotation.class)) { - guiceAnnotation = annotation; - break; - } - } - - final Key key; - if (guiceAnnotation == null) { - key = Key.get(forProperty.getMember().getGenericType()); - } - else { - key = Key.get(forProperty.getMember().getGenericType(), guiceAnnotation); - } - return injector.getInstance(key); + return injector.getInstance((Key) valueId); } } diff --git a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java index 9065f0d1bf2..2a4282a9cc9 100644 --- a/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java +++ b/common/src/main/java/com/metamx/druid/db/DbTablesConfig.java @@ -14,19 +14,26 @@ public class DbTablesConfig return new DbTablesConfig(base, null, null, null, null, null, null); } + @JsonProperty @NotNull private final String base; + @JsonProperty("segments") private final String segmentsTable; + @JsonProperty("rules") private final String rulesTable; + @JsonProperty("config") private final String configTable; + @JsonProperty("tasks") private final String tasksTable; + @JsonProperty("taskLog") private final String taskLogTable; + @JsonProperty("taskLock") private final String taskLockTable; @JsonCreator @@ -61,43 +68,36 @@ public class DbTablesConfig return explicitTableName; } - @JsonProperty public String getBase() { return base; } - @JsonProperty("segments") public String getSegmentsTable() { return segmentsTable; } - @JsonProperty("rules") public String getRulesTable() { return rulesTable; } - @JsonProperty("config") public String getConfigTable() { return configTable; } - @JsonProperty("tasks") public String getTasksTable() { return tasksTable; } - @JsonProperty("taskLog") public String getTaskLogTable() { return taskLogTable; } - @JsonProperty("taskLock") public String getTaskLockTable() { return taskLockTable; diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java index 24c5466d22d..2793b9481a8 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java @@ -1,6 +1,8 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; +import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector; import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues; import com.google.inject.Binder; import com.google.inject.Inject; @@ -58,14 +60,28 @@ public class DruidSecondaryModule implements Module @Provides @LazySingleton @Json public ObjectMapper getJsonMapper(final Injector injector) { - jsonMapper.setInjectableValues(new GuiceInjectableValues(injector)); + setupJackson(injector, jsonMapper); return jsonMapper; } @Provides @LazySingleton @Smile public ObjectMapper getSmileMapper(Injector injector) { - smileMapper.setInjectableValues(new GuiceInjectableValues(injector)); + setupJackson(injector, smileMapper); return smileMapper; } + + private void setupJackson(Injector injector, final ObjectMapper mapper) { + final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); + + mapper.setInjectableValues(new GuiceInjectableValues(injector)); + mapper.setAnnotationIntrospectors( + new AnnotationIntrospectorPair( + guiceIntrospector, mapper.getSerializationConfig().getAnnotationIntrospector() + ), + new AnnotationIntrospectorPair( + guiceIntrospector, mapper.getDeserializationConfig().getAnnotationIntrospector() + ) + ); + } } diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java index 1a940fb05ca..500760af670 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -6,6 +6,7 @@ import com.fasterxml.jackson.databind.introspect.AnnotatedField; import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition; import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -95,7 +96,7 @@ public class JsonConfigurator final String fieldName = next.getName(); final Field theField = beanClazz.getDeclaredField(fieldName); JsonProperty annotation = theField.getAnnotation(JsonProperty.class); - pathParts.add(annotation == null || annotation.value() == null ? fieldName : annotation.value()); + pathParts.add(annotation == null || Strings.isNullOrEmpty(annotation.value()) ? fieldName : annotation.value()); } } } diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java index 3d5ee5c1dac..5ec992a96b1 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java @@ -36,7 +36,8 @@ import java.util.concurrent.CopyOnWriteArrayList; */ public class LifecycleModule implements Module { - private final LifecycleScope scope = new LifecycleScope(); + private final LifecycleScope scope = new LifecycleScope(Lifecycle.Stage.NORMAL); + private final LifecycleScope lastScope = new LifecycleScope(Lifecycle.Stage.LAST); private final List> eagerClasses = new CopyOnWriteArrayList>(); public boolean configured = false; @@ -136,6 +137,7 @@ public class LifecycleModule implements Module synchronized (eagerClasses) { configured = true; binder.bindScope(ManageLifecycle.class, scope); + binder.bindScope(ManageLifecycleLast.class, lastScope); } } @@ -153,6 +155,7 @@ public class LifecycleModule implements Module } }; scope.setLifecycle(lifecycle); + lastScope.setLifecycle(lifecycle); return lifecycle; } diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java index f9d8afad575..a4b076ebb67 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java +++ b/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java @@ -15,10 +15,16 @@ import java.util.List; public class LifecycleScope implements Scope { private static final Logger log = new Logger(LifecycleScope.class); + private final Lifecycle.Stage stage; private Lifecycle lifecycle; private List instances = Lists.newLinkedList(); + public LifecycleScope(Lifecycle.Stage stage) + { + this.stage = stage; + } + public void setLifecycle(Lifecycle lifecycle) { this.lifecycle = lifecycle; @@ -48,7 +54,7 @@ public class LifecycleScope implements Scope } else { try { - lifecycle.addMaybeStartManagedInstance(retVal); + lifecycle.addMaybeStartManagedInstance(retVal, stage); } catch (Exception e) { log.warn(e, "Caught exception when trying to create a[%s]", key); diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java new file mode 100644 index 00000000000..ae34f21dc9e --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java @@ -0,0 +1,21 @@ +package com.metamx.druid.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST + * + * This Scope gets defined by {@link com.metamx.druid.guice.LifecycleModule} + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface ManageLifecycleLast +{ +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java index 0ee2a6e7632..f7119af3f48 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java @@ -24,6 +24,10 @@ import com.google.common.collect.Maps; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.ServerView; import com.metamx.druid.coordination.DataSegmentAnnouncer; +import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.MMappedQueryableIndexFactory; @@ -31,10 +35,6 @@ import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; -import com.metamx.druid.indexing.common.config.TaskConfig; -import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -141,7 +141,7 @@ public class TaskToolbox new SegmentLoaderConfig() { @Override - public File getCacheDirectory() + public File getSegmentLocations() { return new File(getTaskWorkDir(), "fetched_segments"); } diff --git a/pom.xml b/pom.xml index 9795d116c3b..e8d1a0375c1 100644 --- a/pom.xml +++ b/pom.xml @@ -173,17 +173,17 @@ com.google.inject guice - 3.0 + 4.0-beta com.google.inject.extensions guice-servlet - 3.0 + 4.0-beta com.google.inject.extensions guice-multibindings - 3.0 + 4.0-beta com.ibm.icu diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 1befc1df888..f719f4985cb 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -22,10 +22,12 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.client.DataSegment; import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; @@ -47,7 +49,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler private final Object lock = new Object(); private final ObjectMapper jsonMapper; - private final ZkCoordinatorConfig config; + private final SegmentLoaderConfig config; private final DruidServerMetadata me; private final DataSegmentAnnouncer announcer; private final CuratorFramework curator; @@ -59,9 +61,10 @@ public class ZkCoordinator implements DataSegmentChangeHandler private volatile PathChildrenCache loadQueueCache; private volatile boolean started; + @Inject public ZkCoordinator( ObjectMapper jsonMapper, - ZkCoordinatorConfig config, + SegmentLoaderConfig config, ZkPathsConfig zkPaths, DruidServerMetadata me, DataSegmentAnnouncer announcer, @@ -98,7 +101,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler ); try { - config.getSegmentInfoCacheDirectory().mkdirs(); + config.getInfoDir().mkdirs(); curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient()); curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient()); @@ -187,7 +190,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler private void loadCache() { - File baseDir = config.getSegmentInfoCacheDirectory(); + File baseDir = config.getInfoDir(); if (!baseDir.exists()) { return; } @@ -201,7 +204,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler } else { log.warn("Unable to find cache file for %s. Deleting lookup entry", segment.getIdentifier()); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } @@ -221,7 +224,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler try { serverManager.loadSegment(segment); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } @@ -253,7 +256,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler try { serverManager.dropSegment(segment); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index 4007230d743..412fa9a61ff 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -5,9 +5,6 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; @@ -27,7 +24,6 @@ import com.metamx.druid.http.MasterRedirectInfo; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.RedirectServlet; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; @@ -46,7 +42,6 @@ public class CoordinatorModule implements Module public void configure(Binder binder) { ConfigProvider.bind(binder, DruidMasterConfig.class); - ConfigProvider.bind(binder, ZkPathsConfig.class); ConfigProvider.bind(binder, ServerInventoryViewConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); @@ -110,10 +105,4 @@ public class CoordinatorModule implements Module { return new LoadQueueTaskMaster(curator, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d")); } - - @Provides @LazySingleton - public ScheduledExecutorFactory getScheduledExecutorFactory(Lifecycle lifecycle) - { - return ScheduledExecutors.createFactory(lifecycle); - } } diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index 7c931baa12c..dc31e07c073 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -15,9 +15,16 @@ import com.metamx.druid.Query; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer; +import com.metamx.druid.coordination.DataSegmentAnnouncer; +import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.coordination.ServerManager; +import com.metamx.druid.coordination.ZkCoordinator; +import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.guice.annotations.Processing; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.loading.BaseSegmentLoader; import com.metamx.druid.loading.DataSegmentPuller; import com.metamx.druid.loading.HdfsDataSegmentPuller; @@ -48,6 +55,7 @@ import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import org.apache.curator.framework.CuratorFramework; import org.apache.hadoop.conf.Configuration; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -68,10 +76,10 @@ public class HistoricalModule implements Module @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, DruidServerConfig.class); ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); + JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class); JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); binder.bind(ServerManager.class).in(LazySingleton.class); @@ -79,7 +87,6 @@ public class HistoricalModule implements Module binder.bind(SegmentLoader.class).to(BaseSegmentLoader.class).in(LazySingleton.class); binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( binder, String.class, DataSegmentPuller.class ); @@ -96,14 +103,24 @@ public class HistoricalModule implements Module queryFactoryBinder.addBinding(TimeseriesQuery.class).to(TimeseriesQueryRunnerFactory.class).in(LazySingleton.class); queryFactoryBinder.addBinding(SearchQuery.class).to(SearchQueryRunnerFactory.class).in(LazySingleton.class); - queryFactoryBinder.addBinding(TimeBoundaryQuery.class).to(TimeBoundaryQueryRunnerFactory.class).in(LazySingleton.class); - queryFactoryBinder.addBinding(SegmentMetadataQuery.class).to(SegmentMetadataQueryRunnerFactory.class).in(LazySingleton.class); + queryFactoryBinder.addBinding(TimeBoundaryQuery.class) + .to(TimeBoundaryQueryRunnerFactory.class) + .in(LazySingleton.class); + queryFactoryBinder.addBinding(SegmentMetadataQuery.class) + .to(SegmentMetadataQueryRunnerFactory.class) + .in(LazySingleton.class); queryFactoryBinder.addBinding(GroupByQuery.class).to(GroupByQueryRunnerFactory.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); binder.bind(GroupByQueryEngine.class).in(LazySingleton.class); - binder.bind(QueryRunnerFactoryConglomerate.class).to(DefaultQueryRunnerFactoryConglomerate.class).in(LazySingleton.class); + binder.bind(QueryRunnerFactoryConglomerate.class) + .to(DefaultQueryRunnerFactoryConglomerate.class) + .in(LazySingleton.class); + + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + + binder.bind(DataSegmentAnnouncer.class).to(CuratorDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); } private void bindDeepStorageS3(Binder binder) @@ -133,6 +150,18 @@ public class HistoricalModule implements Module ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); } + @Provides @LazySingleton + public DruidServerMetadata getMetadata(@Self DruidNode node, DruidServerConfig config) + { + return new DruidServerMetadata(node.getHost(), node.getHost(), config.getMaxSize(), "historical", config.getTier()); + } + + @Provides @ManageLifecycle + public Announcer getAnnouncer(CuratorFramework curator) + { + return new Announcer(curator, Execs.singleThreaded("Announcer-%s")); + } + @Provides @Processing @ManageLifecycle public ExecutorService getProcessingExecutorService(ExecutorServiceConfig config, ServiceEmitter emitter) { @@ -144,10 +173,10 @@ public class HistoricalModule implements Module } @Provides @LazySingleton - public RestS3Service getRestS3Service(Supplier config) + public RestS3Service getRestS3Service(S3CredentialsConfig config) { try { - return new RestS3Service(new AWSCredentials(config.get().getAccessKey(), config.get().getSecretKey())); + return new RestS3Service(new AWSCredentials(config.getAccessKey(), config.getSecretKey())); } catch (S3ServiceException e) { throw new ProvisionException("Unable to create a RestS3Service", e); diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java index 0bcda8d113c..74d0605e5f9 100644 --- a/server/src/main/java/com/metamx/druid/guice/ServerModule.java +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.java @@ -8,6 +8,7 @@ import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.DruidNode; +import com.metamx.druid.initialization.ZkPathsConfig; /** */ @@ -16,6 +17,8 @@ public class ServerModule implements Module @Override public void configure(Binder binder) { + ConfigProvider.bind(binder, ZkPathsConfig.class); + JsonConfigProvider.bind(binder, "druid", DruidNode.class, Self.class); } diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index f2e2c6f2b12..32d21714bfe 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -34,7 +34,6 @@ import com.metamx.druid.BaseServerNode; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; -import com.metamx.druid.coordination.ZkCoordinatorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -113,7 +112,7 @@ public class ComputeNode extends BaseServerNode final ZkCoordinator coordinator = new ZkCoordinator( getJsonMapper(), - getConfigFactory().build(ZkCoordinatorConfig.class), + null, // TODO: getConfigFactory().build(ZkCoordinatorConfig.class), getZkPaths(), getDruidServerMetadata(), getAnnouncer(), diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index 2271126ac51..0f9bea560d5 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -28,25 +28,29 @@ import java.io.File; */ public class SegmentLoaderConfig { - @JsonProperty("path") + @JsonProperty @NotNull - private File cacheDirectory; + private String locations; @JsonProperty("deleteOnRemove") private boolean deleteOnRemove = true; - public File getCacheDirectory() - { - return cacheDirectory; - } + @JsonProperty + @NotNull + private File infoDir = null; - public void setCacheDirectory(File cacheDirectory) + public String getLocations() { - this.cacheDirectory = cacheDirectory; + return locations; } public boolean isDeleteOnRemove() { return deleteOnRemove; } + + public File getInfoDir() + { + return infoDir; + } } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 61e9986f484..ef5e5117a85 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -29,7 +29,9 @@ import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; import org.apache.commons.io.FileUtils; -import java.io.*; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; /** */ @@ -57,13 +59,13 @@ public class SingleSegmentLoader implements SegmentLoader @Override public boolean isSegmentLoaded(final DataSegment segment) { - File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + File localStorageDir = new File(config.getLocations(), DataSegmentPusherUtil.getStorageDir(segment)); if (localStorageDir.exists()) { return true; } final File legacyStorageDir = new File( - config.getCacheDirectory(), + config.getLocations(), DataSegmentPusherUtil.getLegacyStorageDir(segment) ); return legacyStorageDir.exists(); @@ -80,11 +82,11 @@ public class SingleSegmentLoader implements SegmentLoader public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + File localStorageDir = new File(config.getLocations(), DataSegmentPusherUtil.getStorageDir(segment)); final String legacyDir = DataSegmentPusherUtil.getLegacyStorageDir(segment); if (legacyDir != null) { - File legacyStorageDir = new File(config.getCacheDirectory(), legacyDir); + File legacyStorageDir = new File(config.getLocations(), legacyDir); if (legacyStorageDir.exists()) { log.info("Found legacyStorageDir[%s], moving to new storage location[%s]", legacyStorageDir, localStorageDir); @@ -148,7 +150,7 @@ public class SingleSegmentLoader implements SegmentLoader segment.getShardSpec().getPartitionNum() ); - return new File(config.getCacheDirectory(), outputKey); + return new File(config.getLocations(), outputKey); } private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index 1a34c75cdbc..2a5a125eb5c 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -32,6 +32,7 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.CacheTestSegmentLoader; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; import com.metamx.druid.shard.NoneShardSpec; @@ -54,7 +55,7 @@ public class ZkCoordinatorTest extends CuratorTestBase private ZkCoordinator zkCoordinator; private ServerManager serverManager; private DataSegmentAnnouncer announcer; - private File cacheDir; + private File infoDir; private final ObjectMapper jsonMapper = new DefaultObjectMapper(); private static final Logger log = new Logger(ZkCoordinatorTest.class); @@ -64,12 +65,12 @@ public class ZkCoordinatorTest extends CuratorTestBase setupServerAndCurator(); curator.start(); try { - cacheDir = new File(File.createTempFile("blah", "blah2").getParent(), "ZkCoordinatorTest"); - cacheDir.mkdirs(); - for (File file : cacheDir.listFiles()) { + infoDir = new File(File.createTempFile("blah", "blah2").getParent(), "ZkCoordinatorTest"); + infoDir.mkdirs(); + for (File file : infoDir.listFiles()) { file.delete(); } - log.info("Creating tmp test files in [%s]", cacheDir); + log.info("Creating tmp test files in [%s]", infoDir); } catch (IOException e) { throw new RuntimeException(e); @@ -99,12 +100,11 @@ public class ZkCoordinatorTest extends CuratorTestBase zkCoordinator = new ZkCoordinator( jsonMapper, - new ZkCoordinatorConfig() - { + new SegmentLoaderConfig(){ @Override - public File getSegmentInfoCacheDirectory() + public File getInfoDir() { - return cacheDir; + return infoDir; } }, zkPaths, @@ -155,8 +155,8 @@ public class ZkCoordinatorTest extends CuratorTestBase deleteSegmentFromCache(segment); } - Assert.assertEquals(0, cacheDir.listFiles().length); - Assert.assertTrue(cacheDir.delete()); + Assert.assertEquals(0, infoDir.listFiles().length); + Assert.assertTrue(infoDir.delete()); } private DataSegment makeSegment(String dataSource, String version, Interval interval) @@ -165,7 +165,7 @@ public class ZkCoordinatorTest extends CuratorTestBase dataSource, interval, version, - ImmutableMap.of("version", version, "interval", interval, "cacheDir", cacheDir), + ImmutableMap.of("version", version, "interval", interval, "cacheDir", infoDir), Arrays.asList("dim1", "dim2", "dim3"), Arrays.asList("metric1", "metric2"), new NoneShardSpec(), @@ -176,12 +176,12 @@ public class ZkCoordinatorTest extends CuratorTestBase private void writeSegmentToCache(final DataSegment segment) throws IOException { - if (!cacheDir.exists()) { - cacheDir.mkdir(); + if (!infoDir.exists()) { + infoDir.mkdir(); } File segmentInfoCacheFile = new File( - cacheDir, + infoDir, segment.getIdentifier() ); try { @@ -197,7 +197,7 @@ public class ZkCoordinatorTest extends CuratorTestBase private void deleteSegmentFromCache(final DataSegment segment) throws IOException { File segmentInfoCacheFile = new File( - cacheDir, + infoDir, segment.getIdentifier() ); if (segmentInfoCacheFile.exists()) { @@ -209,8 +209,8 @@ public class ZkCoordinatorTest extends CuratorTestBase private void checkCache(List segments) throws IOException { - Assert.assertTrue(cacheDir.exists()); - File[] files = cacheDir.listFiles(); + Assert.assertTrue(infoDir.exists()); + File[] files = infoDir.listFiles(); List sortedFiles = Lists.newArrayList(files); Collections.sort(sortedFiles); diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index ab32d53649b..c49c42022af 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -20,7 +20,10 @@ import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import com.metamx.druid.metrics.ServerMonitor; import io.airlift.command.Command; +import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -66,6 +69,10 @@ public class CliHistorical extends ServerRunnable root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); + + final HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + server.setHandler(handlerList); } } } From 2361e0112ad03ebc92e97b123ae60b15d51e3ab6 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 2 Aug 2013 10:14:46 -0700 Subject: [PATCH 20/92] Make it all compile again... --- .gitignore | 2 - .../java/com/metamx/druid/QueryableNode.java | 12 ++---- .../client/BatchServerInventoryView.java | 29 +++----------- .../druid/client/CachingClusteredClient.java | 3 -- .../druid/client/ServerInventoryView.java | 21 +++++----- .../client/SingleServerInventoryView.java | 30 +++----------- .../druid/client/cache/MemcachedCache.java | 1 - ...aSegmentAnnouncerDataSegmentAnnouncer.java | 3 -- .../SingleDataSegmentAnnouncer.java | 4 -- .../druid/initialization/ZkPathsConfig.java | 1 - .../query/MetricsEmittingExecutorService.java | 11 ----- .../query/PrioritizedExecutorService.java | 1 - .../SegmentMetadataQueryQueryToolChest.java | 2 - .../client/BatchServerInventoryViewTest.java | 16 -------- .../rabbitmq/RabbitMQProducerMain.java | 15 ++++++- .../TwitterSpritzerFirehoseFactory.java | 2 +- .../metamx/druid/index/QueryableIndex.java | 1 - .../v1/SpatialDimensionRowFormatter.java | 2 - .../indexer/HadoopDruidIndexerConfigTest.java | 6 ++- .../druid/indexing/common/TaskToolbox.java | 6 +-- .../indexing/common/TaskToolboxFactory.java | 4 +- .../actions/RemoteTaskActionClient.java | 2 +- .../common/task/RealtimeIndexTask.java | 2 - .../coordinator/ForkingTaskRunner.java | 4 -- .../config/ForkingTaskRunnerConfig.java | 2 - .../http/IndexerCoordinatorNode.java | 4 -- .../worker/executor/ExecutorLifecycle.java | 5 --- .../worker/executor/ExecutorNode.java | 17 +++----- .../indexing/worker/http/WorkerNode.java | 11 ++--- .../indexing/common/task/TaskSerdeTest.java | 2 +- .../JacksonifiedConnectionFactory.java | 1 - .../plumber/RealtimePlumberSchool.java | 1 - .../druid/coordination/ZkCoordinator.java | 10 +++-- .../druid/db/DatabaseSegmentManager.java | 1 - .../metamx/druid/guice/CoordinatorModule.java | 8 ++-- .../metamx/druid/guice/HistoricalModule.java | 6 ++- .../com/metamx/druid/http/ComputeNode.java | 2 - .../com/metamx/druid/http/MasterMain.java | 21 ---------- .../druid/loading/DataSegmentPusherUtil.java | 3 +- .../druid/loading/QueryableIndexFactory.java | 1 - .../druid/loading/S3DataSegmentPuller.java | 1 - .../druid/loading/SingleSegmentLoader.java | 40 +++++++++---------- .../druid/master/LoadQueueTaskMaster.java | 1 - .../com/metamx/druid/master/rules/Rule.java | 1 - .../metamx/druid/master/DruidMasterTest.java | 1 - 45 files changed, 94 insertions(+), 225 deletions(-) diff --git a/.gitignore b/.gitignore index 845a2ddc007..dc61b625f65 100644 --- a/.gitignore +++ b/.gitignore @@ -10,7 +10,5 @@ target .idea .project .settings/ -examples/rand/RealtimeNode.out -examples/twitter/RealtimeNode.out *.log *.DS_Store diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 0b8f4b6e5f0..9d2ef45c0a1 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -46,15 +46,13 @@ import com.metamx.druid.coordination.AbstractDataSegmentAnnouncer; import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer; import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.guice.JsonConfigurator; +import com.metamx.druid.http.log.NoopRequestLogger; import com.metamx.druid.http.log.RequestLogger; -import com.metamx.druid.http.NoopRequestLogger; -import com.metamx.druid.http.RequestLogger; -import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; @@ -82,6 +80,8 @@ import java.lang.reflect.Field; import java.util.Arrays; import java.util.List; import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -376,18 +376,14 @@ public abstract class QueryableNode extends Registering if ("legacy".equalsIgnoreCase(announcerType)) { serverInventoryView = new SingleServerInventoryView( - serverInventoryViewConfig, getZkPaths(), getCuratorFramework(), - exec, getJsonMapper() ); } else if ("batch".equalsIgnoreCase(announcerType)) { serverInventoryView = new BatchServerInventoryView( - serverInventoryViewConfig, getZkPaths(), getCuratorFramework(), - exec, getJsonMapper() ); } else { diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java index 58497e293db..ad155bac3b7 100644 --- a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java @@ -23,55 +23,38 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.MapMaker; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.ISE; -import com.metamx.druid.curator.inventory.InventoryManagerConfig; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; import java.util.Set; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorService; /** */ +@ManageLifecycle public class BatchServerInventoryView extends ServerInventoryView> { private static final EmittingLogger log = new EmittingLogger(BatchServerInventoryView.class); final ConcurrentMap> zNodes = new MapMaker().makeMap(); + @Inject public BatchServerInventoryView( - final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, - final ExecutorService exec, final ObjectMapper jsonMapper ) { super( - config, log, - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return zkPaths.getAnnouncementsPath(); - } - - @Override - public String getInventoryPath() - { - return zkPaths.getLiveSegmentsPath(); - } - }, + zkPaths, curator, - exec, jsonMapper, - new TypeReference>() - { - } + new TypeReference>(){} ); } diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 72519cbeb8c..b0f460f050b 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -32,13 +32,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.LazySequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.common.logger.Logger; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; @@ -49,7 +47,6 @@ import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.Queries; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.QueryToolChestWarehouse; diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java index ef2dfb19257..d17b4ea1f22 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java @@ -25,15 +25,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.MapMaker; -import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.concurrent.Execs; -import com.metamx.common.logger.Logger; import com.metamx.druid.curator.inventory.CuratorInventoryManager; import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy; import com.metamx.druid.curator.inventory.InventoryManagerConfig; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; @@ -49,8 +46,8 @@ import java.util.concurrent.atomic.AtomicBoolean; */ public abstract class ServerInventoryView implements ServerView, InventoryView { - private final ServerInventoryViewConfig config; - private final Logger log; + + private final EmittingLogger log; private final CuratorInventoryManager inventoryManager; private final AtomicBoolean started = new AtomicBoolean(false); @@ -59,18 +56,14 @@ public abstract class ServerInventoryView implements ServerView, private final Map removedSegments = new MapMaker().makeMap(); - @Inject public ServerInventoryView( - final ServerInventoryViewConfig config, - final Logger log, - final InventoryManagerConfig inventoryManagerConfig, + final EmittingLogger log, + final ZkPathsConfig zkPaths, final CuratorFramework curator, - final ExecutorService exec, final ObjectMapper jsonMapper, final TypeReference typeReference ) { - this.config = config; this.log = log; this.inventoryManager = new CuratorInventoryManager( curator, @@ -89,7 +82,7 @@ public abstract class ServerInventoryView implements ServerView, } }, Execs.singleThreaded("ServerInventoryView-%s"), - new CuratorInventoryManagerStrategy() + new CuratorInventoryManagerStrategy() { @Override public DruidServer deserializeContainer(byte[] bytes) @@ -299,6 +292,7 @@ public abstract class ServerInventoryView implements ServerView, final DataSegment inventory ) { +/* TODO log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); if (container.getSegment(inventory.getIdentifier()) != null) { @@ -323,10 +317,12 @@ public abstract class ServerInventoryView implements ServerView, } } ); +*/ } protected void removeSingleInventory(final DruidServer container, String inventoryKey) { +/* TODO log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey); final DataSegment segment = container.getSegment(inventoryKey); @@ -354,6 +350,7 @@ public abstract class ServerInventoryView implements ServerView, ); removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime()); +*/ } protected abstract DruidServer addInnerInventory( diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java index 4b345dc5a29..5845c465b1a 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -21,50 +21,32 @@ package com.metamx.druid.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.curator.inventory.InventoryManagerConfig; +import com.google.inject.Inject; +import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; -import java.util.concurrent.ExecutorService; - /** */ +@ManageLifecycle public class SingleServerInventoryView extends ServerInventoryView { private static final EmittingLogger log = new EmittingLogger(SingleServerInventoryView.class); + @Inject public SingleServerInventoryView( - final ServerInventoryViewConfig config, final ZkPathsConfig zkPaths, final CuratorFramework curator, - final ExecutorService exec, final ObjectMapper jsonMapper ) { super( - config, log, - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return zkPaths.getAnnouncementsPath(); - } - - @Override - public String getInventoryPath() - { - return zkPaths.getServedSegmentsPath(); - } - }, + zkPaths, curator, - exec, jsonMapper, - new TypeReference() - { - } + new TypeReference(){} ); } diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index fb6fa72ce46..befac85326e 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -32,7 +32,6 @@ import net.spy.memcached.FailureMode; import net.spy.memcached.MemcachedClient; import net.spy.memcached.MemcachedClientIF; import net.spy.memcached.internal.BulkFuture; -import net.spy.memcached.transcoders.SerializingTranscoder; import org.apache.commons.codec.digest.DigestUtils; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java index 053a3d85fac..b9f42d5b2bd 100644 --- a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java @@ -19,12 +19,9 @@ package com.metamx.druid.coordination; -import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkPathsConfig; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java index e238e3b91a3..5c91a0c41a2 100644 --- a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java @@ -20,10 +20,7 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.inject.Inject; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.announcement.Announcer; @@ -43,7 +40,6 @@ public class SingleDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer private volatile boolean started = false; @Inject - public CuratorDataSegmentAnnouncer( public SingleDataSegmentAnnouncer( DruidServerMetadata server, ZkPathsConfig config, diff --git a/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java b/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java index 065829fd9ea..6f484c1bc63 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java @@ -21,7 +21,6 @@ package com.metamx.druid.initialization; import org.apache.curator.utils.ZKPaths; import org.skife.config.Config; -import org.skife.config.Default; public abstract class ZkPathsConfig { diff --git a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java b/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java index b981fde0975..65f6f2833e9 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java +++ b/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java @@ -19,22 +19,11 @@ package com.metamx.druid.query; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.AbstractExecutorService; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.RunnableFuture; import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; public class MetricsEmittingExecutorService extends DelegatingExecutorService { diff --git a/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java b/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java index f943a0c112f..b3892477a3d 100644 --- a/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java +++ b/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java @@ -27,7 +27,6 @@ import com.metamx.common.lifecycle.Lifecycle; import java.util.List; import java.util.concurrent.AbstractExecutorService; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; import java.util.concurrent.FutureTask; import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.RunnableFuture; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 1cfceac4db8..9b210c5aa30 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -40,13 +40,11 @@ import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; - import org.joda.time.Interval; import org.joda.time.Minutes; import javax.annotation.Nullable; import java.nio.ByteBuffer; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java index 7690b0ab06b..1a54651423d 100644 --- a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java +++ b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java @@ -45,7 +45,6 @@ import org.junit.Before; import org.junit.Test; import java.util.Set; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; /** @@ -130,20 +129,6 @@ public class BatchServerInventoryViewTest } batchServerInventoryView = new BatchServerInventoryView( - new ServerInventoryViewConfig() - { - @Override - public int getRemovedSegmentLifetime() - { - return 0; - } - - @Override - public String getAnnouncerType() - { - return "batch"; - } - }, new ZkPathsConfig() { @Override @@ -153,7 +138,6 @@ public class BatchServerInventoryViewTest } }, cf, - Executors.newSingleThreadExecutor(), jsonMapper ); diff --git a/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java b/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java index a070bfff929..0e9bd2be4dd 100644 --- a/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java +++ b/examples/src/main/java/druid/examples/rabbitmq/RabbitMQProducerMain.java @@ -3,10 +3,21 @@ package druid.examples.rabbitmq; import com.rabbitmq.client.Channel; import com.rabbitmq.client.Connection; import com.rabbitmq.client.ConnectionFactory; -import org.apache.commons.cli.*; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; import java.text.SimpleDateFormat; -import java.util.*; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Comparator; +import java.util.Date; +import java.util.List; +import java.util.Random; /** * diff --git a/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index e87b5e44de2..36843522d13 100644 --- a/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -11,13 +11,13 @@ import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; import twitter4j.ConnectionLifeCycleListener; import twitter4j.HashtagEntity; +import twitter4j.StallWarning; import twitter4j.Status; import twitter4j.StatusDeletionNotice; import twitter4j.StatusListener; import twitter4j.TwitterStream; import twitter4j.TwitterStreamFactory; import twitter4j.User; -import twitter4j.StallWarning; import java.io.IOException; import java.util.Arrays; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java index 8e429d45879..963c62dc964 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java @@ -23,7 +23,6 @@ import com.metamx.druid.index.column.ColumnSelector; import com.metamx.druid.kv.Indexed; import org.joda.time.Interval; -import java.io.Closeable; import java.io.IOException; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java index 864a0087831..22c0c5c7edf 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java @@ -30,9 +30,7 @@ import com.google.common.collect.Sets; import com.google.common.primitives.Floats; import com.metamx.common.ISE; import com.metamx.druid.input.InputRow; -import org.joda.time.DateTime; -import javax.annotation.Nullable; import java.util.Arrays; import java.util.List; import java.util.Map; diff --git a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index 70b5ddfa903..6a0110e0b2a 100644 --- a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -27,9 +27,11 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; - +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.joda.time.DateTime; import org.joda.time.Interval; -import org.joda.time.format.ISODateTimeFormat; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java index e716cc3c369..5304228e648 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java @@ -35,10 +35,6 @@ import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; -import com.metamx.druid.indexing.common.config.TaskConfig; -import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; @@ -154,7 +150,7 @@ public class TaskToolbox new SegmentLoaderConfig() { @Override - public File getSegmentLocations() + public String getLocations() { return new File(getTaskWorkDir(), "fetched_segments").toString(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java index 91f5c99a333..5acd075bee7 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java @@ -22,11 +22,11 @@ package com.metamx.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.client.ServerView; import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.loading.DataSegmentKiller; -import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java index 2b698f1b234..289fcf005d9 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -1,6 +1,7 @@ package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.ISE; @@ -10,7 +11,6 @@ import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; import org.joda.time.Duration; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index e2cb18ccb05..cf390eb68ef 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -46,13 +46,11 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.FireDepartmentConfig; -import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.RealtimeMetricsMonitor; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; -import com.metamx.druid.realtime.plumber.NoopRejectionPolicyFactory; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.RealtimePlumberSchool; import com.metamx.druid.realtime.plumber.RejectionPolicyFactory; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 4ac78c96c40..f993bcf3a1e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -26,12 +26,10 @@ import com.google.common.base.Optional; import com.google.common.base.Splitter; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; import com.google.common.io.Closer; import com.google.common.io.Files; import com.google.common.io.InputSupplier; @@ -49,7 +47,6 @@ import com.metamx.druid.indexing.worker.executor.ExecutorMain; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; -import java.io.Closeable; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -63,7 +60,6 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicInteger; /** * Runs tasks in separate processes using {@link ExecutorMain}. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index 4ff499dbd92..31efa5331ec 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -1,14 +1,12 @@ package com.metamx.druid.indexing.coordinator.config; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.metamx.druid.indexing.worker.executor.ExecutorMain; import org.skife.config.Config; import org.skife.config.Default; import java.io.File; import java.util.List; -import java.util.Set; public abstract class ForkingTaskRunnerConfig { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index a2183e98d60..5c7d600c542 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -102,10 +102,6 @@ import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java index d4f47a19771..5d5543b9de7 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -1,13 +1,11 @@ package com.metamx.druid.indexing.worker.executor; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.indexing.common.TaskStatus; @@ -15,12 +13,9 @@ import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.emitter.EmittingLogger; -import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 820306a4df9..3e8aaed5512 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -36,6 +36,7 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.BaseServerNode; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.NoopServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; @@ -49,7 +50,6 @@ import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; @@ -62,14 +62,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentKiller; -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.S3DataSegmentKiller; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -353,11 +345,14 @@ public class ExecutorNode extends BaseServerNode public void initializeServiceDiscovery() throws Exception { - final CuratorDiscoveryConfig config = configFactory.build(CuratorDiscoveryConfig.class); + final CuratorConfig config = configFactory.build(CuratorConfig.class); if (serviceDiscovery == null) { final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(config, lifecycle); + CuratorDiscoveryConfig discoveryConfig = getJsonConfigurator() + .configurate(getProps(), "druid.discovery.curator", CuratorDiscoveryConfig.class); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - serviceDiscoveryCuratorFramework, config, lifecycle + serviceDiscoveryCuratorFramework, discoveryConfig, lifecycle ); } if (serviceAnnouncer == null) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index d490d631796..ec39b6e93a3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -34,7 +34,7 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.QueryableNode; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.config.IndexerZkConfig; @@ -103,7 +103,6 @@ public class WorkerNode extends QueryableNode private ServiceEmitter emitter = null; private WorkerConfig workerConfig = null; private ServiceDiscovery serviceDiscovery = null; - private ServiceAnnouncer serviceAnnouncer = null; private ServiceProvider coordinatorServiceProvider = null; private WorkerCuratorCoordinator workerCuratorCoordinator = null; private WorkerTaskMonitor workerTaskMonitor = null; @@ -330,15 +329,13 @@ public class WorkerNode extends QueryableNode public void initializeServiceDiscovery() throws Exception { if (serviceDiscovery == null) { - final CuratorDiscoveryConfig config = getConfigFactory().build(CuratorDiscoveryConfig.class); + final CuratorDiscoveryConfig config = getJsonConfigurator() + .configurate(getProps(), "druid.discovery.curator", CuratorDiscoveryConfig.class); this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - getCuratorFramework(), + Initialization.makeCuratorFramework(getConfigFactory().build(CuratorConfig.class), getLifecycle()), config, getLifecycle() ); - this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - serviceDiscoveryCuratorFramework, config, getLifecycle() - ); } if (coordinatorServiceProvider == null) { this.coordinatorServiceProvider = Initialization.makeServiceProvider( diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java index d4d2e99d4f2..0c54c296973 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java @@ -1,5 +1,6 @@ package com.metamx.druid.indexing.common.task; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.metamx.common.Granularity; @@ -18,7 +19,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import junit.framework.Assert; -import com.fasterxml.jackson.databind.ObjectMapper; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Test; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/JacksonifiedConnectionFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/JacksonifiedConnectionFactory.java index 7da92c6a8e7..b304191faf4 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/JacksonifiedConnectionFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/JacksonifiedConnectionFactory.java @@ -3,7 +3,6 @@ package com.metamx.druid.realtime.firehose; import com.fasterxml.jackson.annotation.JsonProperty; import com.rabbitmq.client.ConnectionFactory; -import java.net.URI; import java.net.URISyntaxException; import java.security.KeyManagementException; import java.security.NoSuchAlgorithmException; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index 08383bc6456..391fd305e21 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index ebc5bc2c1fb..f9737571d66 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -52,7 +52,6 @@ public class ZkCoordinator implements DataSegmentChangeHandler private final ObjectMapper jsonMapper; private final SegmentLoaderConfig config; - private final ZkCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final DruidServerMetadata me; private final DataSegmentAnnouncer announcer; @@ -110,9 +109,12 @@ public class ZkCoordinator implements DataSegmentChangeHandler curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient()); curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); +/* TODO if (config.isLoadFromSegmentCacheEnabled()) { loadCache(); } +*/ + loadCache(); loadQueueCache.getListenable().addListener( new PathChildrenCacheListener() @@ -234,7 +236,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler try { serverManager.loadSegment(segment); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); @@ -269,7 +271,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler for (DataSegment segment : segments) { serverManager.loadSegment(segment); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); @@ -325,7 +327,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler for (DataSegment segment : segments) { serverManager.dropSegment(segment); - File segmentInfoCacheFile = new File(config.getSegmentInfoCacheDirectory(), segment.getIdentifier()); + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index 70733da0f66..06ef0dbccf3 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -20,7 +20,6 @@ package com.metamx.druid.db; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index 412fa9a61ff..985770bf527 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -5,6 +5,7 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; +import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; @@ -13,7 +14,6 @@ import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.client.indexing.IndexingServiceSelector; import com.metamx.druid.client.selector.DiscoverySelector; import com.metamx.druid.client.selector.Server; -import com.metamx.druid.concurrent.Execs; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseRuleManagerProvider; @@ -101,8 +101,10 @@ public class CoordinatorModule implements Module } @Provides @LazySingleton - public LoadQueueTaskMaster getLoadQueueTaskMaster(CuratorFramework curator, ObjectMapper jsonMapper) + public LoadQueueTaskMaster getLoadQueueTaskMaster( + CuratorFramework curator, ObjectMapper jsonMapper, ScheduledExecutorFactory factory, DruidMasterConfig config + ) { - return new LoadQueueTaskMaster(curator, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d")); + return new LoadQueueTaskMaster(curator, jsonMapper, factory.create(1, "Master-PeonExec--%d"), config); } } diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index dc31e07c073..71525bc8d8c 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -15,9 +15,9 @@ import com.metamx.druid.Query; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.coordination.CuratorDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.curator.announcement.Announcer; @@ -120,7 +120,9 @@ public class HistoricalModule implements Module binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); - binder.bind(DataSegmentAnnouncer.class).to(CuratorDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + binder.bind(DataSegmentAnnouncer.class) + .to(MultipleDataSegmentAnnouncerDataSegmentAnnouncer.class) + .in(ManageLifecycleLast.class); } private void bindDeepStorageS3(Binder binder) diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 2b2db2790e6..de91530d34d 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -47,8 +47,6 @@ import com.metamx.metrics.Monitor; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.jets3t.service.S3ServiceException; -import org.mortbay.jetty.servlet.Context; -import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; import java.util.List; 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 bf404147e93..5cad41a63ef 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -21,10 +21,6 @@ package com.metamx.druid.http; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.IAE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; @@ -37,23 +33,6 @@ import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.client.BatchServerInventoryView; -import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.ServerInventoryViewConfig; -import com.metamx.druid.client.SingleServerInventoryView; -import com.metamx.druid.client.indexing.IndexingServiceClient; -import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.config.ConfigManager; -import com.metamx.druid.config.ConfigManagerConfig; -import com.metamx.druid.config.JacksonConfigManager; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.db.DatabaseRuleManagerConfig; -import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.db.DatabaseSegmentManagerConfig; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; -import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerInitializer; import com.metamx.druid.initialization.JettyServerModule; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java index 349194836d9..c71832d9f63 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -19,10 +19,9 @@ package com.metamx.druid.loading; -import org.joda.time.format.ISODateTimeFormat; - import com.google.common.base.Joiner; import com.metamx.druid.client.DataSegment; +import org.joda.time.format.ISODateTimeFormat; /** */ diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index 535649baea4..276bbc2028a 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -22,7 +22,6 @@ package com.metamx.druid.loading; import com.metamx.druid.index.QueryableIndex; import java.io.File; -import java.io.IOException; /** */ diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java index 75d212886da..8729ce9364c 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -40,7 +40,6 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.Map; -import java.util.Random; import java.util.concurrent.Callable; import java.util.zip.GZIPInputStream; diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index d3518f7d53e..db20e8edbb3 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -34,8 +34,10 @@ import com.metamx.druid.index.Segment; import org.apache.commons.io.FileUtils; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Set; /** */ @@ -60,13 +62,14 @@ public class SingleSegmentLoader implements SegmentLoader final ImmutableList.Builder locBuilder = ImmutableList.builder(); + // TODO // This is a really, really stupid way of getting this information. Splitting on commas and bars is error-prone // We should instead switch it up to be a JSON Array of JSON Object or something and cool stuff like that // But, that'll have to wait for some other day. - for (String dirSpec : config.getCacheDirectory().split(",")) { + for (String dirSpec : config.getLocations().split(",")) { String[] dirSplit = dirSpec.split("\\|"); if (dirSplit.length == 1) { - locBuilder.add(new StorageLocation(new File(dirSplit[0]), config.getServerMaxSize())); + locBuilder.add(new StorageLocation(new File(dirSplit[0]), Integer.MAX_VALUE)); } else if (dirSplit.length == 2) { final Long maxSize = Longs.tryParse(dirSplit[1]); @@ -78,7 +81,7 @@ public class SingleSegmentLoader implements SegmentLoader else { throw new ISE( "Unknown segment storage location[%s]=>[%s], config[%s].", - dirSplit.length, dirSpec, config.getCacheDirectory() + dirSplit.length, dirSpec, config.getLocations() ); } } @@ -89,19 +92,21 @@ public class SingleSegmentLoader implements SegmentLoader } @Override - public boolean isSegmentLoaded(final DataSegment segment) - { - File localStorageDir = new File(config.getLocations(), DataSegmentPusherUtil.getStorageDir(segment)); - if (localStorageDir.exists()) { - return true; + public boolean isSegmentLoaded(final DataSegment segment) + { + return findStorageLocationIfLoaded(segment) != null; } - final File legacyStorageDir = new File( - config.getLocations(), - DataSegmentPusherUtil.getLegacyStorageDir(segment) - ); - return legacyStorageDir.exists(); - } + public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) + { + for (StorageLocation location : locations) { + File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + return location; + } + } + return null; + } @Override public Segment getSegment(DataSegment segment) throws SegmentLoadingException @@ -114,12 +119,8 @@ public class SingleSegmentLoader implements SegmentLoader public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - File localStorageDir = new File(config.getLocations(), DataSegmentPusherUtil.getStorageDir(segment)); StorageLocation loc = findStorageLocationIfLoaded(segment); - final String legacyDir = DataSegmentPusherUtil.getLegacyStorageDir(segment); - if (legacyDir != null) { - File legacyStorageDir = new File(config.getLocations(), legacyDir); final File retVal; if (loc == null) { @@ -160,7 +161,6 @@ public class SingleSegmentLoader implements SegmentLoader { StorageLocation loc = findStorageLocationIfLoaded(segment); - return new File(config.getLocations(), outputKey); if (loc == null) { log.info("Asked to cleanup something[%s] that didn't exist. Skipping.", segment); return; diff --git a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java index a5d9d053cf6..b870708d73b 100644 --- a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java +++ b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; -import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; /** diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index 1c77a0ebc8f..82e710cfbfa 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -25,7 +25,6 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; - import org.joda.time.DateTime; /** 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 213e350f76b..a49dc85a582 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -36,7 +36,6 @@ import org.junit.Before; import org.junit.Test; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ScheduledExecutorService; /** */ From f593b23aac6bf76ec0e0127ce8b4a4fcf2a01107 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 2 Aug 2013 10:15:57 -0700 Subject: [PATCH 21/92] Remove unused boolean parameter --- .../metamx/druid/coordination/SingleDataSegmentAnnouncer.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java index 5c91a0c41a2..20e617a8e32 100644 --- a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java @@ -37,8 +37,6 @@ public class SingleDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer private final ObjectMapper jsonMapper; private final String servedSegmentsLocation; - private volatile boolean started = false; - @Inject public SingleDataSegmentAnnouncer( DruidServerMetadata server, From d66af7625cde369badd2ba715a703728d2667ca3 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 2 Aug 2013 17:05:01 -0700 Subject: [PATCH 22/92] 1) Eliminate the need for SingleSegmentLoader 2) Setup the configuration glue for ServerInventoryViews and DataSegmentAnnouncer 3) Make processes run and work again! --- .../java/com/metamx/druid/QueryableNode.java | 11 +- .../client/BatchServerInventoryView.java | 3 +- .../BatchServerInventoryViewProvider.java | 31 +++ .../druid/client/ServerInventoryView.java | 37 +-- .../client/ServerInventoryViewProvider.java | 16 ++ .../client/SingleServerInventoryProvider.java | 31 +++ .../client/SingleServerInventoryView.java | 3 +- .../BatchDataSegmentAnnouncer.java | 28 +- .../BatchDataSegmentAnnouncerProvider.java | 20 ++ .../DataSegmentAnnouncerProvider.java | 16 ++ .../LegacyDataSegmentAnnouncerProvider.java | 32 +++ ...aSegmentAnnouncerDataSegmentAnnouncer.java | 22 +- .../BatchDataSegmentAnnouncerConfig.java | 30 +++ .../ZkDataSegmentAnnouncerConfig.java | 21 -- .../client/BatchServerInventoryViewTest.java | 24 +- .../BatchDataSegmentAnnouncerTest.java | 25 +- .../metamx/druid/guice/JsonConfigurator.java | 23 +- .../metamx/druid/guice/CoordinatorModule.java | 5 +- .../metamx/druid/guice/HistoricalModule.java | 71 ++--- .../druid/loading/BaseSegmentLoader.java | 62 ----- .../druid/loading/OmniSegmentLoader.java | 250 ++++++++++++++++++ .../druid/loading/SegmentLoaderConfig.java | 2 +- .../druid/loading/SingleSegmentLoader.java | 26 +- .../com/metamx/druid/master/DruidMaster.java | 12 +- 24 files changed, 548 insertions(+), 253 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java create mode 100644 client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java create mode 100644 client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java create mode 100644 client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java create mode 100644 client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java create mode 100644 client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java create mode 100644 client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java delete mode 100644 client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java delete mode 100644 server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java create mode 100644 server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 9d2ef45c0a1..23b50bc9597 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -42,7 +42,6 @@ import com.metamx.druid.client.ServerInventoryViewConfig; import com.metamx.druid.client.ServerView; import com.metamx.druid.client.SingleServerInventoryView; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.coordination.AbstractDataSegmentAnnouncer; import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; @@ -53,9 +52,9 @@ import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.guice.JsonConfigurator; import com.metamx.druid.http.log.NoopRequestLogger; import com.metamx.druid.http.log.RequestLogger; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; @@ -456,23 +455,25 @@ public abstract class QueryableNode extends Registering final Announcer announcer = new Announcer(getCuratorFramework(), Execs.singleThreaded("Announcer-%s")); lifecycle.addManagedInstance(announcer); - final ZkDataSegmentAnnouncerConfig config = getConfigFactory().build(ZkDataSegmentAnnouncerConfig.class); - final String announcerType = config.getAnnouncerType(); + final BatchDataSegmentAnnouncerConfig config = getConfigFactory().build(BatchDataSegmentAnnouncerConfig.class); + final String announcerType = "legacy"; final DataSegmentAnnouncer dataSegmentAnnouncer; if ("batch".equalsIgnoreCase(announcerType)) { dataSegmentAnnouncer = new BatchDataSegmentAnnouncer( getDruidServerMetadata(), config, + getZkPaths(), announcer, getJsonMapper() ); } else if ("legacy".equalsIgnoreCase(announcerType)) { dataSegmentAnnouncer = new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( - Arrays.asList( + Arrays.asList( new BatchDataSegmentAnnouncer( getDruidServerMetadata(), config, + getZkPaths(), announcer, getJsonMapper() ), diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java index ad155bac3b7..f5410536926 100644 --- a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java @@ -51,7 +51,8 @@ public class BatchServerInventoryView extends ServerInventoryView>(){} diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java new file mode 100644 index 00000000000..a0a320b1879 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java @@ -0,0 +1,31 @@ +package com.metamx.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class BatchServerInventoryViewProvider implements ServerInventoryViewProvider +{ + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public ServerInventoryView get() + { + return new BatchServerInventoryView(zkPaths, curator, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java index d17b4ea1f22..32cade52c1d 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java @@ -31,12 +31,10 @@ import com.metamx.druid.concurrent.Execs; import com.metamx.druid.curator.inventory.CuratorInventoryManager; import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy; import com.metamx.druid.curator.inventory.InventoryManagerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; import java.io.IOException; -import java.util.Iterator; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; @@ -54,11 +52,10 @@ public abstract class ServerInventoryView implements ServerView, private final ConcurrentMap serverCallbacks = new MapMaker().makeMap(); private final ConcurrentMap segmentCallbacks = new MapMaker().makeMap(); - private final Map removedSegments = new MapMaker().makeMap(); - public ServerInventoryView( final EmittingLogger log, - final ZkPathsConfig zkPaths, + final String announcementsPath, + final String inventoryPath, final CuratorFramework curator, final ObjectMapper jsonMapper, final TypeReference typeReference @@ -72,13 +69,13 @@ public abstract class ServerInventoryView implements ServerView, @Override public String getContainerPath() { - return zkPaths.getAnnouncementsPath(); + return announcementsPath; } @Override public String getInventoryPath() { - return zkPaths.getServedSegmentsPath(); + return inventoryPath; } }, Execs.singleThreaded("ServerInventoryView-%s"), @@ -174,26 +171,6 @@ public abstract class ServerInventoryView implements ServerView, ); } - public int lookupSegmentLifetime(DataSegment segment) - { - Integer lifetime = removedSegments.get(segment.getIdentifier()); - return (lifetime == null) ? 0 : lifetime; - } - - public void decrementRemovedSegmentsLifetime() - { - for (Iterator> mapIter = removedSegments.entrySet().iterator(); mapIter.hasNext(); ) { - Map.Entry segment = mapIter.next(); - int lifetime = segment.getValue() - 1; - - if (lifetime < 0) { - mapIter.remove(); - } else { - segment.setValue(lifetime); - } - } - } - @LifecycleStart public void start() throws Exception { @@ -292,7 +269,6 @@ public abstract class ServerInventoryView implements ServerView, final DataSegment inventory ) { -/* TODO log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); if (container.getSegment(inventory.getIdentifier()) != null) { @@ -317,12 +293,10 @@ public abstract class ServerInventoryView implements ServerView, } } ); -*/ } protected void removeSingleInventory(final DruidServer container, String inventoryKey) { -/* TODO log.info("Server[%s] removed segment[%s]", container.getName(), inventoryKey); final DataSegment segment = container.getSegment(inventoryKey); @@ -348,9 +322,6 @@ public abstract class ServerInventoryView implements ServerView, } } ); - - removedSegments.put(inventoryKey, config.getRemovedSegmentLifetime()); -*/ } protected abstract DruidServer addInnerInventory( diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java new file mode 100644 index 00000000000..6eea6e5a5e6 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java @@ -0,0 +1,16 @@ +package com.metamx.druid.client; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleServerInventoryProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "legacy", value = SingleServerInventoryProvider.class), + @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class) +}) +public interface ServerInventoryViewProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java new file mode 100644 index 00000000000..93fae69a77b --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java @@ -0,0 +1,31 @@ +package com.metamx.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class SingleServerInventoryProvider implements ServerInventoryViewProvider +{ + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public ServerInventoryView get() + { + return new SingleServerInventoryView(zkPaths, curator, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java index 5845c465b1a..1c04800887f 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -43,7 +43,8 @@ public class SingleServerInventoryView extends ServerInventoryView { super( log, - zkPaths, + zkPaths.getAnnouncementsPath(), + zkPaths.getServedSegmentsPath(), curator, jsonMapper, new TypeReference(){} diff --git a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java index 5911ec2a642..ea9cc26412d 100644 --- a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java @@ -24,11 +24,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; @@ -43,7 +45,7 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer { private static final Logger log = new Logger(BatchDataSegmentAnnouncer.class); - private final ZkDataSegmentAnnouncerConfig config; + private final BatchDataSegmentAnnouncerConfig config; private final Announcer announcer; private final ObjectMapper jsonMapper; private final String liveSegmentLocation; @@ -51,27 +53,29 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer private final Set availableZNodes = Sets.newHashSet(); private final Map segmentLookup = Maps.newHashMap(); + @Inject public BatchDataSegmentAnnouncer( DruidServerMetadata server, - ZkDataSegmentAnnouncerConfig config, + BatchDataSegmentAnnouncerConfig config, + ZkPathsConfig zkPaths, Announcer announcer, ObjectMapper jsonMapper ) { - super(server, config, announcer, jsonMapper); - + super(server, zkPaths, announcer, jsonMapper); this.config = config; this.announcer = announcer; this.jsonMapper = jsonMapper; - this.liveSegmentLocation = ZKPaths.makePath(config.getLiveSegmentsPath(), server.getName()); + + this.liveSegmentLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), server.getName()); } @Override public void announceSegment(DataSegment segment) throws IOException { int newBytesLen = jsonMapper.writeValueAsBytes(segment).length; - if (newBytesLen > config.getMaxNumBytes()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxNumBytes()); + if (newBytesLen > config.getMaxBytesPerNode()) { + throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); } // create new batch @@ -88,7 +92,7 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer boolean done = false; while (iter.hasNext() && !done) { SegmentZNode availableZNode = iter.next(); - if (availableZNode.getBytes().length + newBytesLen < config.getMaxNumBytes()) { + if (availableZNode.getBytes().length + newBytesLen < config.getMaxBytesPerNode()) { availableZNode.addSegment(segment); log.info("Announcing segment[%s] at path[%s]", segment.getIdentifier(), availableZNode.getPath()); @@ -132,11 +136,11 @@ public class BatchDataSegmentAnnouncer extends AbstractDataSegmentAnnouncer for (DataSegment segment : segments) { int newBytesLen = jsonMapper.writeValueAsBytes(segment).length; - if (newBytesLen > config.getMaxNumBytes()) { - throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxNumBytes()); + if (newBytesLen > config.getMaxBytesPerNode()) { + throw new ISE("byte size %,d exceeds %,d", newBytesLen, config.getMaxBytesPerNode()); } - if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxNumBytes()) { + if (count >= config.getSegmentsPerNode() || byteSize + newBytesLen > config.getMaxBytesPerNode()) { segmentZNode.addSegments(batch); announcer.announce(segmentZNode.getPath(), segmentZNode.getBytes()); diff --git a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..7b849575835 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java @@ -0,0 +1,20 @@ +package com.metamx.druid.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; + +import javax.validation.constraints.NotNull; + +/** + */ +public class BatchDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider +{ + @JacksonInject + @NotNull + private BatchDataSegmentAnnouncer batchAnnouncer = null; + + @Override + public DataSegmentAnnouncer get() + { + return batchAnnouncer; + } +} diff --git a/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..6f43fb15c0b --- /dev/null +++ b/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java @@ -0,0 +1,16 @@ +package com.metamx.druid.coordination; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSegmentAnnouncerProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "legacy", value = LegacyDataSegmentAnnouncerProvider.class), + @JsonSubTypes.Type(name = "batch", value = BatchDataSegmentAnnouncerProvider.class) +}) +public interface DataSegmentAnnouncerProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..cd99119b2a4 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java @@ -0,0 +1,32 @@ +package com.metamx.druid.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.metamx.common.lifecycle.Lifecycle; + +import javax.validation.constraints.NotNull; +import java.util.Arrays; + +/** + */ +public class LegacyDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider +{ + @JacksonInject + @NotNull + private SingleDataSegmentAnnouncer singleAnnouncer = null; + + @JacksonInject + @NotNull + private BatchDataSegmentAnnouncer batchAnnouncer = null; + + @JacksonInject + @NotNull + private Lifecycle lifecycle = null; + + @Override + public DataSegmentAnnouncer get() + { + return new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( + Arrays.asList(singleAnnouncer, batchAnnouncer) + ); + } +} diff --git a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java b/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java index b9f42d5b2bd..8bdc14e7d95 100644 --- a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java +++ b/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java @@ -19,8 +19,6 @@ package com.metamx.druid.coordination; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.client.DataSegment; import java.io.IOException; @@ -30,31 +28,15 @@ import java.io.IOException; */ public class MultipleDataSegmentAnnouncerDataSegmentAnnouncer implements DataSegmentAnnouncer { - private final Iterable dataSegmentAnnouncers; + private final Iterable dataSegmentAnnouncers; public MultipleDataSegmentAnnouncerDataSegmentAnnouncer( - Iterable dataSegmentAnnouncers + Iterable dataSegmentAnnouncers ) { this.dataSegmentAnnouncers = dataSegmentAnnouncers; } - @LifecycleStart - public void start() - { - for (AbstractDataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) { - dataSegmentAnnouncer.start(); - } - } - - @LifecycleStop - public void stop() - { - for (AbstractDataSegmentAnnouncer dataSegmentAnnouncer : dataSegmentAnnouncers) { - dataSegmentAnnouncer.stop(); - } - } - @Override public void announceSegment(DataSegment segment) throws IOException { diff --git a/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java b/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java new file mode 100644 index 00000000000..3b80641f943 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java @@ -0,0 +1,30 @@ +package com.metamx.druid.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; + +/** + */ +public class BatchDataSegmentAnnouncerConfig +{ + @JsonProperty + @Min(1) + private int segmentsPerNode = 50; + + @JsonProperty + @Max(1024 * 1024) + @Min(1024) + private long maxBytesPerNode = 512 * 1024; + + public int getSegmentsPerNode() + { + return segmentsPerNode; + } + + public long getMaxBytesPerNode() + { + return maxBytesPerNode; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java b/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java deleted file mode 100644 index e14c65027eb..00000000000 --- a/client/src/main/java/com/metamx/druid/initialization/ZkDataSegmentAnnouncerConfig.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.metamx.druid.initialization; - -import org.skife.config.Config; -import org.skife.config.Default; - -/** - */ -public abstract class ZkDataSegmentAnnouncerConfig extends ZkPathsConfig -{ - @Config("druid.zk.segmentsPerNode") - @Default("50") - public abstract int getSegmentsPerNode(); - - @Config("druid.zk.maxNumBytesPerNode") - @Default("512000") - public abstract long getMaxNumBytes(); - - @Config("druid.announcer.type") - @Default("legacy") - public abstract String getAnnouncerType(); -} diff --git a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java index 1a54651423d..bb7bc307808 100644 --- a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java +++ b/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java @@ -30,7 +30,7 @@ import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import junit.framework.Assert; @@ -92,30 +92,20 @@ public class BatchServerInventoryViewTest "type", "tier" ), - new ZkDataSegmentAnnouncerConfig() + new BatchDataSegmentAnnouncerConfig() { - @Override - public String getZkBasePath() - { - return testBasePath; - } - @Override public int getSegmentsPerNode() { return 50; } - + }, + new ZkPathsConfig() + { @Override - public long getMaxNumBytes() + public String getZkBasePath() { - return 100000; - } - - @Override - public String getAnnouncerType() - { - return "batch"; + return testBasePath; } }, announcer, diff --git a/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java b/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java index 608fe850a74..f8d7abed828 100644 --- a/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java +++ b/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java @@ -28,7 +28,8 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; +import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; @@ -92,30 +93,20 @@ public class BatchDataSegmentAnnouncerTest "type", "tier" ), - new ZkDataSegmentAnnouncerConfig() + new BatchDataSegmentAnnouncerConfig() { - @Override - public String getZkBasePath() - { - return testBasePath; - } - @Override public int getSegmentsPerNode() { return 50; } - + }, + new ZkPathsConfig() + { @Override - public long getMaxNumBytes() + public String getZkBasePath() { - return 100000; - } - - @Override - public String getAnnouncerType() - { - return "batch"; + return testBasePath; } }, announcer, diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java index 500760af670..a06636dad35 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java @@ -1,11 +1,11 @@ package com.metamx.druid.guice; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.introspect.AnnotatedField; import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition; import com.google.common.base.Function; -import com.google.common.base.Joiner; import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; @@ -35,8 +35,6 @@ public class JsonConfigurator { private static final Logger log = new Logger(JsonConfigurator.class); - private static final Joiner JOINER = Joiner.on("."); - private final ObjectMapper jsonMapper; private final Validator validator; @@ -86,7 +84,7 @@ public class JsonConfigurator List messages = Lists.newArrayList(); for (ConstraintViolation violation : violations) { - List pathParts = Lists.newArrayList(); + String path = ""; try { Class beanClazz = violation.getRootBeanClass(); final Iterator iter = violation.getPropertyPath().iterator(); @@ -95,8 +93,21 @@ public class JsonConfigurator if (next.getKind() == ElementKind.PROPERTY) { final String fieldName = next.getName(); final Field theField = beanClazz.getDeclaredField(fieldName); + + if (theField.getAnnotation(JacksonInject.class) != null) { + path = String.format(" -- Injected field[%s] not bound!?", fieldName); + break; + } + JsonProperty annotation = theField.getAnnotation(JsonProperty.class); - pathParts.add(annotation == null || Strings.isNullOrEmpty(annotation.value()) ? fieldName : annotation.value()); + final boolean noAnnotationValue = annotation == null || Strings.isNullOrEmpty(annotation.value()); + final String pathPart = noAnnotationValue ? fieldName : annotation.value(); + if (path.isEmpty()) { + path += pathPart; + } + else { + path += "." + pathPart; + } } } } @@ -104,7 +115,7 @@ public class JsonConfigurator throw Throwables.propagate(e); } - messages.add(String.format("%s - %s", JOINER.join(pathParts), violation.getMessage())); + messages.add(String.format("%s - %s", path, violation.getMessage())); } throw new ProvisionException( diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index 985770bf527..5e242fa2c57 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -9,6 +9,7 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; +import com.metamx.druid.client.ServerInventoryViewProvider; import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.client.indexing.IndexingServiceSelector; @@ -48,10 +49,12 @@ public class CoordinatorModule implements Module JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); binder.bind(InventoryView.class).to(ServerInventoryView.class); - binder.bind(ServerInventoryView.class); binder.bind(RedirectServlet.class).in(LazySingleton.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); + binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); + binder.bind(DatabaseSegmentManager.class) .toProvider(DatabaseSegmentManagerProvider.class) .in(ManageLifecycle.class); diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index 71525bc8d8c..4885ccff755 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -15,21 +15,24 @@ import com.metamx.druid.Query; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; import com.metamx.druid.coordination.DataSegmentAnnouncer; +import com.metamx.druid.coordination.DataSegmentAnnouncerProvider; import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer; import com.metamx.druid.coordination.ServerManager; +import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.guice.annotations.Processing; import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.loading.BaseSegmentLoader; import com.metamx.druid.loading.DataSegmentPuller; import com.metamx.druid.loading.HdfsDataSegmentPuller; import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.OmniSegmentLoader; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.S3CredentialsConfig; import com.metamx.druid.loading.S3DataSegmentPuller; @@ -84,7 +87,7 @@ public class HistoricalModule implements Module binder.bind(ServerManager.class).in(LazySingleton.class); - binder.bind(SegmentLoader.class).to(BaseSegmentLoader.class).in(LazySingleton.class); + binder.bind(SegmentLoader.class).to(OmniSegmentLoader.class).in(LazySingleton.class); binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); final MapBinder segmentPullerBinder = MapBinder.newMapBinder( @@ -120,36 +123,11 @@ public class HistoricalModule implements Module binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); - binder.bind(DataSegmentAnnouncer.class) - .to(MultipleDataSegmentAnnouncerDataSegmentAnnouncer.class) - .in(ManageLifecycleLast.class); - } - - private void bindDeepStorageS3(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); - } - - private void bindDeepStorageHdfs(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); - binder.bind(Configuration.class).toInstance(new Configuration()); - } - - private void bindDeepStorageCassandra(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); - ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); + JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); + JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); + binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); + binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + binder.bind(SingleDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); } @Provides @LazySingleton @@ -225,6 +203,33 @@ public class HistoricalModule implements Module return new IntermediateProcessingBufferPool(config.intermediateComputeSizeBytes()); } + private static void bindDeepStorageS3(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); + } + + private static void bindDeepStorageHdfs(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + binder.bind(Configuration.class).toInstance(new Configuration()); + } + + private static void bindDeepStorageCassandra(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); + ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); + } + private static class IntermediateProcessingBufferPool extends StupidPool { private static final Logger log = new Logger(IntermediateProcessingBufferPool.class); diff --git a/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java deleted file mode 100644 index b3a44a99629..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/BaseSegmentLoader.java +++ /dev/null @@ -1,62 +0,0 @@ -package com.metamx.druid.loading; - -import com.google.common.base.Supplier; -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.Segment; - -import java.util.Map; - -/** - */ -public class BaseSegmentLoader implements SegmentLoader -{ - private final Map pullers; - private final QueryableIndexFactory factory; - private final Supplier config; - - @Inject - public BaseSegmentLoader( - Map pullers, - QueryableIndexFactory factory, - Supplier config - ) - { - this.pullers = pullers; - this.factory = factory; - this.config = config; - } - - @Override - public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException - { - return getLoader(segment.getLoadSpec()).isSegmentLoaded(segment); - } - - @Override - public Segment getSegment(DataSegment segment) throws SegmentLoadingException - { - return getLoader(segment.getLoadSpec()).getSegment(segment); - } - - @Override - public void cleanup(DataSegment segment) throws SegmentLoadingException - { - getLoader(segment.getLoadSpec()).cleanup(segment); - } - - private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException - { - String type = MapUtils.getString(loadSpec, "type"); - DataSegmentPuller loader = pullers.get(type); - - if (loader == null) { - throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, pullers.keySet()); - } - - // TODO: SingleSegmentLoader should die when Guice goes out. The logic should just be in this class. - return new SingleSegmentLoader(loader, factory, config.get()); - } - -} diff --git a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java new file mode 100644 index 00000000000..38b811ca179 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java @@ -0,0 +1,250 @@ +package com.metamx.druid.loading; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.google.common.primitives.Longs; +import com.google.inject.Inject; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.QueryableIndexSegment; +import com.metamx.druid.index.Segment; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + */ +public class OmniSegmentLoader implements SegmentLoader +{ + private static final Logger log = new Logger(OmniSegmentLoader.class); + + private final Map pullers; + private final QueryableIndexFactory factory; + private final SegmentLoaderConfig config; + + private final List locations; + + @Inject + public OmniSegmentLoader( + Map pullers, + QueryableIndexFactory factory, + SegmentLoaderConfig config + ) + { + this.pullers = pullers; + this.factory = factory; + this.config = config; + + final ImmutableList.Builder locBuilder = ImmutableList.builder(); + + // TODO + // This is a really, really stupid way of getting this information. Splitting on commas and bars is error-prone + // We should instead switch it up to be a JSON Array of JSON Object or something and cool stuff like that + // But, that'll have to wait for some other day. + for (String dirSpec : config.getLocations().split(",")) { + String[] dirSplit = dirSpec.split("\\|"); + if (dirSplit.length == 1) { + locBuilder.add(new StorageLocation(new File(dirSplit[0]), Integer.MAX_VALUE)); + } + else if (dirSplit.length == 2) { + final Long maxSize = Longs.tryParse(dirSplit[1]); + if (maxSize == null) { + throw new IAE("Size of a local segment storage location must be an integral number, got[%s]", dirSplit[1]); + } + locBuilder.add(new StorageLocation(new File(dirSplit[0]), maxSize)); + } + else { + throw new ISE( + "Unknown segment storage location[%s]=>[%s], config[%s].", + dirSplit.length, dirSpec, config.getLocations() + ); + } + } + locations = locBuilder.build(); + + Preconditions.checkArgument(locations.size() > 0, "Must have at least one segment cache directory."); + log.info("Using storage locations[%s]", locations); + + } + + @Override + public boolean isSegmentLoaded(final DataSegment segment) + { + return findStorageLocationIfLoaded(segment) != null; + } + + public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) + { + for (StorageLocation location : locations) { + File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + return location; + } + } + return null; + } + + @Override + public Segment getSegment(DataSegment segment) throws SegmentLoadingException + { + File segmentFiles = loadSegmentFiles(segment); + final QueryableIndex index = factory.factorize(segmentFiles); + + return new QueryableIndexSegment(segment.getIdentifier(), index); + } + + public File loadSegmentFiles(DataSegment segment) throws SegmentLoadingException + { + StorageLocation loc = findStorageLocationIfLoaded(segment); + + final File retVal; + + if (loc == null) { + Iterator locIter = locations.iterator(); + loc = locIter.next(); + while (locIter.hasNext()) { + loc = loc.mostEmpty(locIter.next()); + } + + if (!loc.canHandle(segment.getSize())) { + throw new ISE( + "Segment[%s:%,d] too large for storage[%s:%,d].", + segment.getIdentifier(), segment.getSize(), loc.getPath(), loc.available() + ); + } + + File storageDir = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + if (!storageDir.mkdirs()) { + log.debug("Unable to make parent file[%s]", storageDir); + } + + getPuller(segment.getLoadSpec()).getSegmentFiles(segment, storageDir); + loc.addSegment(segment); + + retVal = storageDir; + } + else { + retVal = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + } + + loc.addSegment(segment); + + return retVal; + } + + @Override + public void cleanup(DataSegment segment) throws SegmentLoadingException + { + if (!config.isDeleteOnRemove()) { + return; + } + + StorageLocation loc = findStorageLocationIfLoaded(segment); + + if (loc == null) { + log.info("Asked to cleanup something[%s] that didn't exist. Skipping.", segment); + return; + } + + try { + File cacheFile = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + log.info("Deleting directory[%s]", cacheFile); + FileUtils.deleteDirectory(cacheFile); + loc.removeSegment(segment); + } + catch (IOException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + } + + private DataSegmentPuller getPuller(Map loadSpec) throws SegmentLoadingException + { + String type = MapUtils.getString(loadSpec, "type"); + DataSegmentPuller loader = pullers.get(type); + + if (loader == null) { + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, pullers.keySet()); + } + + return loader; + } + + + private static class StorageLocation + { + private final File path; + private final long maxSize; + private final Set segments; + + private volatile long currSize = 0; + + StorageLocation( + File path, + long maxSize + ) + { + this.path = path; + this.maxSize = maxSize; + + this.segments = Sets.newHashSet(); + } + + private File getPath() + { + return path; + } + + private Long getMaxSize() + { + return maxSize; + } + + private synchronized void addSegment(DataSegment segment) + { + if (! segments.add(segment)) { + currSize += segment.getSize(); + } + } + + private synchronized void removeSegment(DataSegment segment) + { + if (segments.remove(segment)) { + currSize -= segment.getSize(); + } + } + + private boolean canHandle(long size) + { + return available() > size; + } + + private synchronized long available() + { + return maxSize - currSize; + } + + private StorageLocation mostEmpty(StorageLocation other) + { + return available() > other.available() ? this : other; + } + + @Override + public String toString() + { + return "StorageLocation{" + + "path=" + path + + ", maxSize=" + maxSize + + '}'; + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index 0f9bea560d5..9b2b0342150 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -30,7 +30,7 @@ public class SegmentLoaderConfig { @JsonProperty @NotNull - private String locations; + private String locations = null; @JsonProperty("deleteOnRemove") private boolean deleteOnRemove = true; diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index db20e8edbb3..a9fe23d1af6 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -40,7 +40,9 @@ import java.util.List; import java.util.Set; /** + * TODO: Kill this along with the Guicification of the IndexingService stuff */ +@Deprecated public class SingleSegmentLoader implements SegmentLoader { private static final Logger log = new Logger(SingleSegmentLoader.class); @@ -92,21 +94,21 @@ public class SingleSegmentLoader implements SegmentLoader } @Override - public boolean isSegmentLoaded(final DataSegment segment) - { - return findStorageLocationIfLoaded(segment) != null; - } + public boolean isSegmentLoaded(final DataSegment segment) + { + return findStorageLocationIfLoaded(segment) != null; + } - public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) - { - for (StorageLocation location : locations) { - File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - if (localStorageDir.exists()) { - return location; - } + public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) + { + for (StorageLocation location : locations) { + File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + return location; } - return null; } + return null; + } @Override public Segment getSegment(DataSegment segment) throws SegmentLoadingException 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 699b5b21a90..d955dca5e91 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -214,16 +214,6 @@ public class DruidMaster return loadStatus; } - public int lookupSegmentLifetime(DataSegment segment) - { - return serverInventoryView.lookupSegmentLifetime(segment); - } - - public void decrementRemovedSegmentsLifetime() - { - serverInventoryView.decrementRemovedSegmentsLifetime(); - } - public void removeSegment(DataSegment segment) { log.info("Removing Segment[%s]", segment); @@ -763,7 +753,7 @@ public class DruidMaster peon.stop(); } - decrementRemovedSegmentsLifetime(); + // TODO: decrementRemovedSegmentsLifetime(); return params.buildFromExisting() .withDruidCluster(cluster) From d64bc2f7621adc500106286141b7f386a6f56899 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 5 Aug 2013 13:33:43 -0700 Subject: [PATCH 23/92] 1) Eliminate Old mains() that aren't needed anymore. --- examples/bin/ec2/run.sh | 4 +- .../com/metamx/druid/http/ComputeMain.java | 52 ----- .../com/metamx/druid/http/ComputeNode.java | 210 ------------------ .../com/metamx/druid/http/MasterMain.java | 114 ---------- .../com/metamx/druid/http/ServerMain.java | 44 ---- .../druid/initialization/ServerInit.java | 41 ---- .../java/io/druid/cli/CliCoordinator.java | 3 +- 7 files changed, 3 insertions(+), 465 deletions(-) delete mode 100644 server/src/main/java/com/metamx/druid/http/ComputeMain.java delete mode 100644 server/src/main/java/com/metamx/druid/http/ComputeNode.java delete mode 100644 server/src/main/java/com/metamx/druid/http/MasterMain.java delete mode 100644 server/src/main/java/com/metamx/druid/http/ServerMain.java diff --git a/examples/bin/ec2/run.sh b/examples/bin/ec2/run.sh index 933de0a2b42..630881c4742 100644 --- a/examples/bin/ec2/run.sh +++ b/examples/bin/ec2/run.sh @@ -10,10 +10,10 @@ mkdir logs 2>&1 > /dev/null nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime com.metamx.druid.realtime.RealtimeMain 2>&1 > logs/realtime.log & # And a master node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master com.metamx.druid.http.MasterMain 2>&1 > logs/master.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master com.metamx.druid.cli.Main server coordinator 2>&1 > logs/master.log & # And a compute node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute com.metamx.druid.http.ComputeMain 2>&1 > logs/compute.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute com.metamx.druid.cli.Main server historical 2>&1 > logs/compute.log & # And a broker node nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/broker com.metamx.druid.http.BrokerMain 2>&1 > logs/broker.log & diff --git a/server/src/main/java/com/metamx/druid/http/ComputeMain.java b/server/src/main/java/com/metamx/druid/http/ComputeMain.java deleted file mode 100644 index 828d1568214..00000000000 --- a/server/src/main/java/com/metamx/druid/http/ComputeMain.java +++ /dev/null @@ -1,52 +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.http; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; - -/** - */ -public class ComputeMain -{ - private static final Logger log = new Logger(ComputeMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance( - ComputeNode.builder().build() - ); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - - lifecycle.join(); - } -} diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java deleted file mode 100644 index de91530d34d..00000000000 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ /dev/null @@ -1,210 +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.http; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ExecutorServiceConfig; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.BaseServerNode; -import com.metamx.druid.client.DruidServerConfig; -import com.metamx.druid.coordination.ServerManager; -import com.metamx.druid.coordination.ZkCoordinator; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerInit; -import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.metrics.ServerMonitor; -import com.metamx.druid.query.MetricsEmittingExecutorService; -import com.metamx.druid.query.PrioritizedExecutorService; -import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import com.metamx.metrics.Monitor; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.jets3t.service.S3ServiceException; -import org.skife.config.ConfigurationObjectFactory; - -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ExecutorService; - -/** - */ -public class ComputeNode extends BaseServerNode -{ - private static final Logger log = new Logger(ComputeNode.class); - - public static Builder builder() - { - return new Builder(); - } - - private SegmentLoader segmentLoader; - - public ComputeNode( - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super("historical", log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public ComputeNode setSegmentLoader(SegmentLoader segmentLoader) - { - Preconditions.checkState(this.segmentLoader == null, "Cannot set segmentLoader once it has already been set."); - this.segmentLoader = segmentLoader; - return this; - } - - public SegmentLoader getSegmentLoader() - { - initializeSegmentLoader(); - return segmentLoader; - } - - protected void doInit() throws Exception - { - final Lifecycle lifecycle = getLifecycle(); - final ServiceEmitter emitter = getEmitter(); - final List monitors = getMonitors(); - final QueryRunnerFactoryConglomerate conglomerate = getConglomerate(); - - final PrioritizedExecutorService innerExecutorService = PrioritizedExecutorService.create( - getLifecycle(), - getConfigFactory().buildWithReplacements( - ExecutorServiceConfig.class, ImmutableMap.of("base_path", "druid.processing") - ) - ); - - final ExecutorService executorService = new MetricsEmittingExecutorService( - innerExecutorService, - emitter, - new ServiceMetricEvent.Builder() - ); - - final ServerManager serverManager = new ServerManager(getSegmentLoader(), conglomerate, emitter, executorService); - - final ZkCoordinator coordinator = new ZkCoordinator( - getJsonMapper(), - null, // TODO: getConfigFactory().build(ZkCoordinatorConfig.class), - getZkPaths(), - getDruidServerMetadata(), - getAnnouncer(), - getCuratorFramework(), - serverManager - ); - lifecycle.addManagedInstance(coordinator); - - monitors.add(new ServerMonitor(getConfigFactory().build(DruidServerConfig.class), serverManager)); - startMonitoring(monitors); - - final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet( - new ServletHolder( - new QueryServlet(getJsonMapper(), getSmileMapper(), serverManager, emitter, getRequestLogger()) - ), - "/druid/v2/*" - ); - } - - private void initializeSegmentLoader() - { - if (segmentLoader == null) { - try { - setSegmentLoader( - ServerInit.makeDefaultQueryableLoader(getConfigFactory(), getProps()) - ); - } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMappers(ObjectMapper jsonMapper, ObjectMapper smileMapper) - { - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public ComputeNode build() - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } else if (jsonMapper == null || smileMapper == null) { - throw new ISE( - "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", - jsonMapper, - smileMapper - ); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new ComputeNode(props, lifecycle, jsonMapper, smileMapper, configFactory); - } - } - -} diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java deleted file mode 100644 index 5cad41a63ef..00000000000 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ /dev/null @@ -1,114 +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.http; - -import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.curator.discovery.DiscoveryModule; -import com.metamx.druid.guice.CoordinatorModule; -import com.metamx.druid.guice.DbConnectorModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.JacksonConfigManagerModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerInitializer; -import com.metamx.druid.initialization.JettyServerModule; -import com.metamx.druid.log.LogLevelAdjuster; -import com.metamx.druid.master.DruidMaster; -import com.metamx.druid.metrics.MetricsModule; -import org.eclipse.jetty.server.Handler; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; -import org.eclipse.jetty.server.handler.HandlerList; -import org.eclipse.jetty.server.handler.ResourceHandler; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; - -/** - */ -public class MasterMain -{ - private static final Logger log = new Logger(MasterMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - Injector injector = Initialization.makeInjector( - new LifecycleModule().register(DruidMaster.class), - EmitterModule.class, - HttpClientModule.class, - DbConnectorModule.class, - JacksonConfigManagerModule.class, - CuratorModule.class, - new MetricsModule(), - new DiscoveryModule().register(Self.class), - ServerModule.class, - new JettyServerModule(new MasterJettyServerInitializer()) - .addResource(InfoResource.class) - .addResource(MasterResource.class) - .addResource(StatusResource.class), - CoordinatorModule.class - ); - - final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.error(t, "Error when starting up. Failing."); - System.exit(1); - } - - lifecycle.join(); - } - - private static class MasterJettyServerInitializer implements JettyServerInitializer - { - @Override - public void initialize(Server server, Injector injector) - { - ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); - - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.setContextPath("/"); - - HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); - server.setHandler(handlerList); - - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); - root.addFilter(GuiceFilter.class, "/*", null); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/http/ServerMain.java b/server/src/main/java/com/metamx/druid/http/ServerMain.java deleted file mode 100644 index 11dcb879584..00000000000 --- a/server/src/main/java/com/metamx/druid/http/ServerMain.java +++ /dev/null @@ -1,44 +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.http; - -/** - */ -@Deprecated -public class ServerMain -{ - public static void main(String[] args) throws Exception - { - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ServerMain!!!! PLZ Stop. Use ComputeMain instead."); - System.out.println("K thx bye."); - ComputeMain.main(args); - } -} 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 32479965651..d49b0062b72 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; 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; @@ -31,23 +30,13 @@ import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.DelegatingSegmentLoader; -import com.metamx.druid.loading.HdfsDataSegmentPuller; import com.metamx.druid.loading.HdfsDataSegmentPusher; import com.metamx.druid.loading.HdfsDataSegmentPusherConfig; -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.SegmentLoader; -import com.metamx.druid.loading.SegmentLoaderConfig; -import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; -import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; import com.metamx.druid.loading.cassandra.CassandraDataSegmentPusher; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -81,36 +70,6 @@ public class ServerInit { private static Logger log = new Logger(ServerInit.class); - public static SegmentLoader makeDefaultQueryableLoader( - final ConfigurationObjectFactory configFactory, - final Properties props - ) throws S3ServiceException - { - SegmentLoaderConfig config = configFactory.build(SegmentLoaderConfig.class); - DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final QueryableIndexFactory factory = new MMappedQueryableIndexFactory(); - - final RestS3Service s3Client = new RestS3Service( - new AWSCredentials( - props.getProperty("com.metamx.aws.accessKey", ""), - props.getProperty("com.metamx.aws.secretKey", "") - ) - ); - final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); - final SingleSegmentLoader s3segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config); - - delegateLoader.setLoaderTypes( - ImmutableMap.builder() - .put("local", new SingleSegmentLoader(new LocalDataSegmentPuller(), factory, config)) - .put("hdfs", new SingleSegmentLoader(new HdfsDataSegmentPuller(new Configuration()), factory, config)) - .put("s3", s3segmentLoader) - .put("s3_zip", s3segmentLoader) - .put("c*",new SingleSegmentLoader(new CassandraDataSegmentPuller(configFactory.build(CassandraDataSegmentConfig.class)), factory, config)) - .build() - ); - return delegateLoader; - } - public static StupidPool makeComputeScratchPool(DruidProcessingConfig config) { try { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 549259ab711..9aa3dc18930 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -13,7 +13,6 @@ import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.http.InfoResource; -import com.metamx.druid.http.MasterMain; import com.metamx.druid.http.MasterResource; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.StatusResource; @@ -77,7 +76,7 @@ public class CliCoordinator extends ServerRunnable public void initialize(Server server, Injector injector) { ResourceHandler resourceHandler = new ResourceHandler(); - resourceHandler.setResourceBase(MasterMain.class.getClassLoader().getResource("static").toExternalForm()); + resourceHandler.setResourceBase(DruidMaster.class.getClassLoader().getResource("static").toExternalForm()); final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.setContextPath("/"); From 293c0e8418a503d1ad173abffa172f1755781366 Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 5 Aug 2013 13:58:45 -0700 Subject: [PATCH 24/92] 1) Clean up the servlet registration stuff, resource registrations on CliHistorical weren't working, they do now. --- .../main/java/io/druid/cli/CliHistorical.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index c49c42022af..a6c34c3d4f8 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -1,6 +1,7 @@ package io.druid.cli; import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; @@ -12,7 +13,6 @@ import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusResource; -import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerInitializer; @@ -24,8 +24,10 @@ import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; /** */ @@ -64,14 +66,17 @@ public class CliHistorical extends ServerRunnable @Override public void initialize(Server server, Injector injector) { - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.setResourceBase("/"); + final ServletContextHandler queries = new ServletContextHandler(ServletContextHandler.SESSIONS); + queries.setResourceBase("/"); + queries.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + handlerList.setHandlers(new Handler[]{queries, root, new DefaultHandler()}); server.setHandler(handlerList); } } From 9f71d42314644a4842956a459eccaa3ca24be407 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 6 Aug 2013 12:01:52 -0700 Subject: [PATCH 25/92] 1) Broker works with Guice 2) Extract ServerViewModule to handle various ServerView thingies 3) Extract QueryToolChestModule and QueryRunnerFactoryModule to reuse code for configuration of Query stuff 4) Extract QueryJettyServerInitializer to reuse between Historical and Broker nodes 5) Remove ClientMain, BrokerMain and BrokerNode --- client/pom.xml | 8 + .../metamx/druid/client/BrokerServerView.java | 10 +- .../druid/client/CachingClusteredClient.java | 2 + .../druid/client/cache/CacheConfig.java | 11 - .../druid/client/cache/CacheProvider.java | 14 + ...cheConfig.java => LocalCacheProvider.java} | 32 +- .../metamx/druid/client/cache/MapCache.java | 11 - .../druid/client/cache/MemcachedCache.java | 2 +- .../client/cache/MemcachedCacheConfig.java | 26 -- .../client/cache/MemcachedCacheConfiger.java | 49 +++ .../client/cache/MemcachedCacheProvider.java | 10 + .../com/metamx/druid/guice/BrokerModule.java | 28 ++ .../metamx/druid/guice/ServerViewModule.java | 22 ++ .../druid/guice/annotations/Client.java | 17 + .../com/metamx/druid/http/BrokerMain.java | 53 --- .../com/metamx/druid/http/BrokerNode.java | 355 ------------------ .../com/metamx/druid/http/ClientMain.java | 41 -- .../druid/http/ClientQuerySegmentWalker.java | 2 + .../query/MapQueryToolChestWarehouse.java | 28 ++ .../druid/query/group/GroupByQueryConfig.java | 13 +- .../group/GroupByQueryQueryToolChest.java | 23 +- examples/bin/ec2/run.sh | 6 +- pom.xml | 2 +- server/pom.xml | 8 - .../metamx/druid/guice/CoordinatorModule.java | 7 - .../metamx/druid/guice/HistoricalModule.java | 33 -- .../metamx/druid/guice/HttpClientModule.java | 118 +++++- .../druid/guice/QueryRunnerFactoryModule.java | 52 +++ .../druid/initialization/ServerInit.java | 11 +- .../group/GroupByQueryRunnerFactory.java | 9 +- .../query/group/GroupByQueryRunnerTest.java | 22 +- .../GroupByTimeseriesQueryRunnerTest.java | 6 +- .../src/main/java/io/druid/cli/CliBroker.java | 56 +++ .../java/io/druid/cli/CliCoordinator.java | 4 +- .../main/java/io/druid/cli/CliHistorical.java | 37 +- services/src/main/java/io/druid/cli/Main.java | 2 +- .../cli/QueryJettyServerInitializer.java | 36 ++ 37 files changed, 508 insertions(+), 658 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/client/cache/CacheConfig.java create mode 100644 client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java rename client/src/main/java/com/metamx/druid/client/cache/{MapCacheConfig.java => LocalCacheProvider.java} (65%) delete mode 100644 client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java create mode 100644 client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java create mode 100644 client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java create mode 100644 client/src/main/java/com/metamx/druid/guice/BrokerModule.java create mode 100644 client/src/main/java/com/metamx/druid/guice/ServerViewModule.java create mode 100644 client/src/main/java/com/metamx/druid/guice/annotations/Client.java delete mode 100644 client/src/main/java/com/metamx/druid/http/BrokerMain.java delete mode 100644 client/src/main/java/com/metamx/druid/http/BrokerNode.java delete mode 100644 client/src/main/java/com/metamx/druid/http/ClientMain.java create mode 100644 client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java rename {server => client}/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java (83%) create mode 100644 server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java create mode 100644 services/src/main/java/io/druid/cli/CliBroker.java create mode 100644 services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java diff --git a/client/pom.xml b/client/pom.xml index daf8fbd96b2..f86fd9c031a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -108,6 +108,14 @@ com.google.inject guice + + com.google.inject.extensions + guice-servlet + + + com.google.inject.extensions + guice-multibindings + com.ibm.icu icu4j diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java index a61b6b0c00f..0f55560449b 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java @@ -22,10 +22,13 @@ package com.metamx.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.selector.QueryableDruidServer; import com.metamx.druid.client.selector.ServerSelector; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.annotations.Client; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; @@ -54,12 +57,12 @@ public class BrokerServerView implements TimelineServerView private final HttpClient httpClient; private final ServerView baseView; + @Inject public BrokerServerView( QueryToolChestWarehouse warehose, ObjectMapper smileMapper, - HttpClient httpClient, - ServerView baseView, - ExecutorService exec + @Client HttpClient httpClient, + ServerView baseView ) { this.warehose = warehose; @@ -71,6 +74,7 @@ public class BrokerServerView implements TimelineServerView this.selectors = Maps.newHashMap(); this.timelines = Maps.newHashMap(); + ExecutorService exec = Execs.singleThreaded("BrokerServerView-%s"); baseView.registerSegmentCallback( exec, new ServerView.SegmentCallback() diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index b0f460f050b..3bbd8146ebe 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -32,6 +32,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.inject.Inject; import com.metamx.common.Pair; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.LazySequence; @@ -79,6 +80,7 @@ public class CachingClusteredClient implements QueryRunner private final Cache cache; private final ObjectMapper objectMapper; + @Inject public CachingClusteredClient( QueryToolChestWarehouse warehouse, TimelineServerView serverView, diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/CacheConfig.java deleted file mode 100644 index e970363e0e5..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheConfig.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.metamx.druid.client.cache; - -import org.skife.config.Config; -import org.skife.config.Default; - -public abstract class CacheConfig -{ - @Config("druid.bard.cache.type") - @Default("local") - public abstract String getType(); -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java b/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java new file mode 100644 index 00000000000..cd0e09700b9 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java @@ -0,0 +1,14 @@ +package com.metamx.druid.client.cache; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LocalCacheProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "local", value = LocalCacheProvider.class), + @JsonSubTypes.Type(name = "memcached", value = MemcachedCacheProvider.class) +}) +public interface CacheProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java similarity index 65% rename from client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java rename to client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java index 2bc468bf899..1457ec928e6 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCacheConfig.java +++ b/client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java @@ -19,26 +19,30 @@ package com.metamx.druid.client.cache; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Min; /** */ -public abstract class MapCacheConfig +public class LocalCacheProvider implements CacheProvider { - @Config("${prefix}.sizeInBytes") - @Default("0") - public abstract long getSizeInBytes(); + @JsonProperty + @Min(0) + private long sizeInBytes = 0; - @Config("${prefix}.initialSize") - public int getInitialSize() - { - return 500000; - } + @JsonProperty + @Min(0) + private int initialSize = 500000; - @Config("${prefix}.logEvictionCount") - public int getLogEvictionCount() + @JsonProperty + @Min(0) + private int logEvictionCount = 0; + + + @Override + public Cache get() { - return 0; + return new MapCache(new ByteCountingLRUMap(initialSize, logEvictionCount, sizeInBytes)); } } diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java index bf549ec31b9..0480776cfde 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MapCache.java @@ -44,17 +44,6 @@ public class MapCache implements Cache private final AtomicLong hitCount = new AtomicLong(0); private final AtomicLong missCount = new AtomicLong(0); - public static com.metamx.druid.client.cache.Cache create(final MapCacheConfig config) - { - return new MapCache( - new ByteCountingLRUMap( - config.getInitialSize(), - config.getLogEvictionCount(), - config.getSizeInBytes() - ) - ); - } - MapCache( ByteCountingLRUMap byteCountingLRUMap ) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index befac85326e..6f77373556c 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -49,7 +49,7 @@ public class MemcachedCache implements Cache { private static final Logger log = new Logger(MemcachedCache.class); - public static MemcachedCache create(final MemcachedCacheConfig config) + public static MemcachedCache create(final MemcachedCacheConfiger config) { try { LZ4Transcoder transcoder = new LZ4Transcoder(config.getMaxObjectSize()); diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java deleted file mode 100644 index 2bd7b84e4cc..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java +++ /dev/null @@ -1,26 +0,0 @@ -package com.metamx.druid.client.cache; - -import org.skife.config.Config; -import org.skife.config.Default; - -public abstract class MemcachedCacheConfig -{ - @Config("${prefix}.expiration") - @Default("2592000") - public abstract int getExpiration(); - - @Config("${prefix}.timeout") - @Default("500") - public abstract int getTimeout(); - - @Config("${prefix}.hosts") - public abstract String getHosts(); - - @Config("${prefix}.maxObjectSize") - @Default("52428800") - public abstract int getMaxObjectSize(); - - @Config("${prefix}.memcachedPrefix") - @Default("druid") - public abstract String getMemcachedPrefix(); -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java new file mode 100644 index 00000000000..491335c81f5 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java @@ -0,0 +1,49 @@ +package com.metamx.druid.client.cache; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.NotNull; + +public class MemcachedCacheConfiger +{ + @JsonProperty + private int expiration = 2592000; // What is this number? + + @JsonProperty + private int timeout = 500; + + @JsonProperty + @NotNull + private String hosts; + + @JsonProperty + private int maxObjectSize = 50 * 1024 * 1024; + + @JsonProperty + private String memcachedPrefix = "druid"; + + public int getExpiration() + { + return expiration; + } + + public int getTimeout() + { + return timeout; + } + + public String getHosts() + { + return hosts; + } + + public int getMaxObjectSize() + { + return maxObjectSize; + } + + public String getMemcachedPrefix() + { + return memcachedPrefix; + } +} diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java new file mode 100644 index 00000000000..c0bedbe5309 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java @@ -0,0 +1,10 @@ +package com.metamx.druid.client.cache; + +public class MemcachedCacheProvider extends MemcachedCacheConfiger implements CacheProvider +{ + @Override + public Cache get() + { + return MemcachedCache.create(this); + } +} diff --git a/client/src/main/java/com/metamx/druid/guice/BrokerModule.java b/client/src/main/java/com/metamx/druid/guice/BrokerModule.java new file mode 100644 index 00000000000..9cf33e4f71c --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/BrokerModule.java @@ -0,0 +1,28 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.druid.client.BrokerServerView; +import com.metamx.druid.client.CachingClusteredClient; +import com.metamx.druid.client.TimelineServerView; +import com.metamx.druid.client.cache.Cache; +import com.metamx.druid.client.cache.CacheProvider; +import com.metamx.druid.query.MapQueryToolChestWarehouse; +import com.metamx.druid.query.QueryToolChestWarehouse; + +/** + */ +public class BrokerModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); + + binder.bind(CachingClusteredClient.class).in(LazySingleton.class); + binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); + + binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class); + } +} diff --git a/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java b/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java new file mode 100644 index 00000000000..3c9dd142deb --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java @@ -0,0 +1,22 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.druid.client.InventoryView; +import com.metamx.druid.client.ServerInventoryView; +import com.metamx.druid.client.ServerInventoryViewProvider; +import com.metamx.druid.client.ServerView; + +/** + */ +public class ServerViewModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); + binder.bind(InventoryView.class).to(ServerInventoryView.class); + binder.bind(ServerView.class).to(ServerInventoryView.class); + binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); + } +} diff --git a/client/src/main/java/com/metamx/druid/guice/annotations/Client.java b/client/src/main/java/com/metamx/druid/guice/annotations/Client.java new file mode 100644 index 00000000000..6a8eabc28dc --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/annotations/Client.java @@ -0,0 +1,17 @@ +package com.metamx.druid.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Client +{ +} diff --git a/client/src/main/java/com/metamx/druid/http/BrokerMain.java b/client/src/main/java/com/metamx/druid/http/BrokerMain.java deleted file mode 100644 index a1856552a1f..00000000000 --- a/client/src/main/java/com/metamx/druid/http/BrokerMain.java +++ /dev/null @@ -1,53 +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.http; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; - -/** - */ - -public class BrokerMain -{ - private static final Logger log = new Logger(BrokerMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance( - BrokerNode.builder().build() - ); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - - lifecycle.join(); - } -} diff --git a/client/src/main/java/com/metamx/druid/http/BrokerNode.java b/client/src/main/java/com/metamx/druid/http/BrokerNode.java deleted file mode 100644 index a68ff4def5a..00000000000 --- a/client/src/main/java/com/metamx/druid/http/BrokerNode.java +++ /dev/null @@ -1,355 +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.http; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.ISE; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryableNode; -import com.metamx.druid.client.BrokerServerView; -import com.metamx.druid.client.CachingClusteredClient; -import com.metamx.druid.client.cache.Cache; -import com.metamx.druid.client.cache.CacheConfig; -import com.metamx.druid.client.cache.CacheMonitor; -import com.metamx.druid.client.cache.MapCache; -import com.metamx.druid.client.cache.MapCacheConfig; -import com.metamx.druid.client.cache.MemcachedCache; -import com.metamx.druid.client.cache.MemcachedCacheConfig; -import com.metamx.druid.curator.CuratorConfig; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.ReflectionQueryToolChestWarehouse; -import com.metamx.druid.utils.PropUtils; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.Monitor; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; -import org.skife.config.ConfigurationObjectFactory; - -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - */ - -public class BrokerNode extends QueryableNode -{ - private static final Logger log = new Logger(BrokerNode.class); - - public static final String CACHE_TYPE_LOCAL = "local"; - public static final String CACHE_TYPE_MEMCACHED = "memcached"; - public static final String CACHE_PROPERTY_PREFIX = "druid.bard.cache"; - - private final List extraModules = Lists.newArrayList(); - private final List pathsForGuiceFilter = Lists.newArrayList(); - - private QueryToolChestWarehouse warehouse = null; - private HttpClient brokerHttpClient = null; - private Cache cache = null; - - private boolean useDiscovery = true; - - public static Builder builder() - { - return new Builder(); - } - - public BrokerNode( - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super("broker", log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public QueryToolChestWarehouse getWarehouse() - { - initializeWarehouse(); - return warehouse; - } - - public BrokerNode setWarehouse(QueryToolChestWarehouse warehouse) - { - checkFieldNotSetAndSet("warehouse", warehouse); - return this; - } - - public HttpClient getBrokerHttpClient() - { - initializeBrokerHttpClient(); - return brokerHttpClient; - } - - public BrokerNode setBrokerHttpClient(HttpClient brokerHttpClient) - { - checkFieldNotSetAndSet("brokerHttpClient", brokerHttpClient); - return this; - } - - public Cache getCache() - { - initializeCacheBroker(); - return cache; - } - - public BrokerNode setCache(Cache cache) - { - checkFieldNotSetAndSet("cache", cache); - return this; - } - - public BrokerNode useDiscovery(boolean useDiscovery) - { - this.useDiscovery = useDiscovery; - return this; - } - - /** - * This method allows you to specify more Guice modules to use primarily for injected extra Jersey resources. - * I'd like to remove the Guice dependency for this, but I don't know how to set up Jersey without Guice... - * - * This is deprecated because at some point in the future, we will eliminate the Guice dependency and anything - * that uses this will break. Use at your own risk. - * - * @param module the module to register with Guice - * - * @return this - */ - @Deprecated - public BrokerNode addModule(Module module) - { - extraModules.add(module); - return this; - } - - /** - * This method is used to specify extra paths that the GuiceFilter should pay attention to. - * - * This is deprecated for the same reason that addModule is deprecated. - * - * @param path the path that the GuiceFilter should pay attention to. - * - * @return this - */ - @Deprecated - public BrokerNode addPathForGuiceFilter(String path) - { - pathsForGuiceFilter.add(path); - return this; - } - - @Override - protected void doInit() throws Exception - { - initializeWarehouse(); - initializeBrokerHttpClient(); - initializeCacheBroker(); - initializeDiscovery(); - - final Lifecycle lifecycle = getLifecycle(); - - final List monitors = getMonitors(); - monitors.add(new CacheMonitor(cache)); - startMonitoring(monitors); - - final ExecutorService viewExec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("BrokerServerView-%s").build() - ); - final BrokerServerView view = new BrokerServerView( - warehouse, getSmileMapper(), brokerHttpClient, getServerView(), viewExec - ); - - final CachingClusteredClient baseClient = new CachingClusteredClient(warehouse, view, cache, getSmileMapper()); - lifecycle.addManagedInstance(baseClient); - - final ClientQuerySegmentWalker texasRanger = new ClientQuerySegmentWalker(warehouse, getEmitter(), baseClient); - - List theModules = Lists.newArrayList(); - theModules.add(new ClientServletModule(texasRanger, getInventoryView(), getJsonMapper())); - theModules.addAll(extraModules); - - final Injector injector = Guice.createInjector(theModules); - final ServletContextHandler root = new ServletContextHandler(); - root.setContextPath("/"); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet( - new ServletHolder( - new QueryServlet(getJsonMapper(), getSmileMapper(), texasRanger, getEmitter(), getRequestLogger()) - ), - "/druid/v2/*" - ); - root.addFilter(GzipFilter.class, "/*", null); - - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/v2/datasources/*", null); - - for (String path : pathsForGuiceFilter) { - root.addFilter(GuiceFilter.class, path, null); - } - } - - private void initializeDiscovery() throws Exception - { - if (useDiscovery) { - final Lifecycle lifecycle = getLifecycle(); - final CuratorDiscoveryConfig curatorDiscoveryConfig = getConfigFactory().build(CuratorDiscoveryConfig.class); - final DruidNode nodeConfig = getConfigFactory().build(DruidNode.class); - final CuratorFramework curatorFramework = Initialization.makeCuratorFramework( - getConfigFactory().build(CuratorConfig.class), lifecycle - ); - final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, curatorDiscoveryConfig, lifecycle - ); - final ServiceAnnouncer serviceAnnouncer = Initialization.makeServiceAnnouncer(serviceDiscovery); - Initialization.announceDefaultService(nodeConfig, serviceAnnouncer, lifecycle); - } - } - - private void initializeCacheBroker() - { - if (cache == null) { - String cacheType = getConfigFactory() - .build(CacheConfig.class) - .getType(); - - if (cacheType.equals(CACHE_TYPE_LOCAL)) { - setCache( - MapCache.create( - getConfigFactory().buildWithReplacements( - MapCacheConfig.class, - ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX) - ) - ) - ); - } else if (cacheType.equals(CACHE_TYPE_MEMCACHED)) { - setCache( - MemcachedCache.create( - getConfigFactory().buildWithReplacements( - MemcachedCacheConfig.class, - ImmutableMap.of("prefix", CACHE_PROPERTY_PREFIX) - ) - ) - ); - } else { - throw new ISE("Unknown cache type [%s]", cacheType); - } - } - } - - private void initializeBrokerHttpClient() - { - if (brokerHttpClient == null) { - setBrokerHttpClient( - HttpClientInit.createClient( - HttpClientConfig - .builder() - .withNumConnections(PropUtils.getPropertyAsInt(getProps(), "druid.client.http.connections")) - .build(), - getLifecycle() - ) - ); - } - } - - private void initializeWarehouse() - { - if (warehouse == null) { - setWarehouse(new ReflectionQueryToolChestWarehouse()); - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMappers(ObjectMapper jsonMapper, ObjectMapper smileMapper) - { - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public BrokerNode build() - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } - else if (jsonMapper == null || smileMapper == null) { - throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new BrokerNode(props, lifecycle, jsonMapper, smileMapper, configFactory); - } - } -} diff --git a/client/src/main/java/com/metamx/druid/http/ClientMain.java b/client/src/main/java/com/metamx/druid/http/ClientMain.java deleted file mode 100644 index 1beda5c405d..00000000000 --- a/client/src/main/java/com/metamx/druid/http/ClientMain.java +++ /dev/null @@ -1,41 +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.http; - -/** - */ -@Deprecated -public class ClientMain -{ - public static void main(String[] args) throws Exception - { - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("!@(*&$#!(*@(@*&$! You are running with ClientMain!!!! PLZ Stop. Use BrokerMain instead."); - System.out.println("K thx bye."); - BrokerMain.main(args); - } -} diff --git a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java index d053d6507a2..e7c187e5545 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java @@ -20,6 +20,7 @@ package com.metamx.druid.http; import com.google.common.base.Function; +import com.google.inject.Inject; import com.metamx.druid.Query; import com.metamx.druid.client.CachingClusteredClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; @@ -43,6 +44,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final ServiceEmitter emitter; private final CachingClusteredClient baseClient; + @Inject public ClientQuerySegmentWalker( QueryToolChestWarehouse warehouse, ServiceEmitter emitter, diff --git a/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java b/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java new file mode 100644 index 00000000000..9b0700f6dc8 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java @@ -0,0 +1,28 @@ +package com.metamx.druid.query; + +import com.google.inject.Inject; +import com.metamx.druid.Query; + +import java.util.Map; + +/** + */ +public class MapQueryToolChestWarehouse implements QueryToolChestWarehouse +{ + private final Map, QueryToolChest> toolchests; + + @Inject + public MapQueryToolChestWarehouse( + Map, QueryToolChest> toolchests + ) + { + this.toolchests = toolchests; + } + + @Override + @SuppressWarnings("unchecked") + public > QueryToolChest getToolChest(QueryType query) + { + return toolchests.get(query.getClass()); + } +} diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java similarity index 83% rename from server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java rename to client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java index bb3c4693b69..ea7be8f9a2c 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java @@ -12,16 +12,14 @@ public class GroupByQueryConfig @JsonProperty private int maxIntermediateRows = 50000; + @JsonProperty + private int maxResults = 500000; + public boolean isSingleThreaded() { return singleThreaded; } - public void setSingleThreaded(boolean singleThreaded) - { - this.singleThreaded = singleThreaded; - } - public int getMaxIntermediateRows() { return maxIntermediateRows; @@ -31,4 +29,9 @@ public class GroupByQueryConfig { this.maxIntermediateRows = maxIntermediateRows; } + + public int getMaxResults() + { + return maxResults; + } } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index db39bc4b513..bf3c7c0eaee 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -22,9 +22,11 @@ package com.metamx.druid.query.group; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ConcatSequence; @@ -34,7 +36,6 @@ import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.initialization.Initialization; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; @@ -42,7 +43,6 @@ import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; import org.joda.time.Minutes; @@ -50,7 +50,6 @@ import org.joda.time.Minutes; import javax.annotation.Nullable; import java.util.List; import java.util.Map; -import java.util.Properties; /** */ @@ -62,13 +61,14 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); - private static final int maxRows; + private final Supplier configSupplier; - static { - // I dislike this static loading of properies, but it's the only mechanism available right now. - Properties props = Initialization.loadProperties(); - - maxRows = PropUtils.getPropertyAsInt(props, "com.metamx.query.groupBy.maxResults", 500000); + @Inject + public GroupByQueryQueryToolChest( + Supplier configSupplier + ) + { + this.configSupplier = configSupplier; } @Override @@ -90,6 +90,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { + final GroupByQueryConfig config = configSupplier.get(); final QueryGranularity gran = query.getGranularity(); final long timeStart = query.getIntervals().get(0).getStartMillis(); @@ -133,8 +134,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest maxRows) { - throw new ISE("Computation exceeds maxRows limit[%s]", maxRows); + if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions)) > config.getMaxResults()) { + throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); } return accumulated; diff --git a/examples/bin/ec2/run.sh b/examples/bin/ec2/run.sh index 630881c4742..9f695c1e001 100644 --- a/examples/bin/ec2/run.sh +++ b/examples/bin/ec2/run.sh @@ -10,13 +10,13 @@ mkdir logs 2>&1 > /dev/null nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime com.metamx.druid.realtime.RealtimeMain 2>&1 > logs/realtime.log & # And a master node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master com.metamx.druid.cli.Main server coordinator 2>&1 > logs/master.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master io.druid.cli.Main server coordinator 2>&1 > logs/master.log & # And a compute node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute com.metamx.druid.cli.Main server historical 2>&1 > logs/compute.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/compute io.druid.cli.Main server historical 2>&1 > logs/compute.log & # And a broker node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/broker com.metamx.druid.http.BrokerMain 2>&1 > logs/broker.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/broker io.druid.cli.Main server broker 2>&1 > logs/broker.log & echo "Hit CTRL-C to continue..." exit 0 diff --git a/pom.xml b/pom.xml index 2ca8434b40f..cb7376bdad1 100644 --- a/pom.xml +++ b/pom.xml @@ -80,7 +80,7 @@ com.metamx server-metrics - 0.0.3 + 0.0.4 diff --git a/server/pom.xml b/server/pom.xml index 6431dc2ac5a..e165b329d23 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -100,14 +100,6 @@ com.google.inject guice - - com.google.inject.extensions - guice-servlet - - - com.google.inject.extensions - guice-multibindings - com.fasterxml.jackson.core jackson-core diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index 5e242fa2c57..86d857bcb37 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -6,10 +6,7 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.ServerInventoryView; import com.metamx.druid.client.ServerInventoryViewConfig; -import com.metamx.druid.client.ServerInventoryViewProvider; import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.client.indexing.IndexingServiceSelector; @@ -48,13 +45,9 @@ public class CoordinatorModule implements Module JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); - binder.bind(InventoryView.class).to(ServerInventoryView.class); binder.bind(RedirectServlet.class).in(LazySingleton.class); binder.bind(RedirectFilter.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); - binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); - binder.bind(DatabaseSegmentManager.class) .toProvider(DatabaseSegmentManagerProvider.class) .in(ManageLifecycle.class); diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index 4885ccff755..c96b15150e8 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -6,12 +6,10 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.ProvisionException; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.metamx.common.concurrent.ExecutorServiceConfig; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; -import com.metamx.druid.Query; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; @@ -42,20 +40,7 @@ import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.MetricsEmittingExecutorService; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.group.GroupByQueryConfig; -import com.metamx.druid.query.group.GroupByQueryEngine; -import com.metamx.druid.query.group.GroupByQueryRunnerFactory; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQueryRunnerFactory; -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.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import org.apache.curator.framework.CuratorFramework; @@ -99,24 +84,6 @@ public class HistoricalModule implements Module bindDeepStorageHdfs(binder); bindDeepStorageCassandra(binder); - - final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( - binder, new TypeLiteral>(){}, new TypeLiteral(){} - ); - - queryFactoryBinder.addBinding(TimeseriesQuery.class).to(TimeseriesQueryRunnerFactory.class).in(LazySingleton.class); - queryFactoryBinder.addBinding(SearchQuery.class).to(SearchQueryRunnerFactory.class).in(LazySingleton.class); - queryFactoryBinder.addBinding(TimeBoundaryQuery.class) - .to(TimeBoundaryQueryRunnerFactory.class) - .in(LazySingleton.class); - queryFactoryBinder.addBinding(SegmentMetadataQuery.class) - .to(SegmentMetadataQueryRunnerFactory.class) - .in(LazySingleton.class); - - queryFactoryBinder.addBinding(GroupByQuery.class).to(GroupByQueryRunnerFactory.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); - binder.bind(GroupByQueryEngine.class).in(LazySingleton.class); - binder.bind(QueryRunnerFactoryConglomerate.class) .to(DefaultQueryRunnerFactoryConglomerate.class) .in(LazySingleton.class); diff --git a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java index 757c75b3a71..28e4dce5f15 100644 --- a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java @@ -3,8 +3,13 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; import com.google.inject.Binder; +import com.google.inject.Binding; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provides; +import com.google.inject.Provider; +import com.google.inject.TypeLiteral; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.druid.guice.annotations.Global; import com.metamx.http.client.HttpClient; @@ -13,18 +18,63 @@ import com.metamx.http.client.HttpClientInit; import org.joda.time.Duration; import org.joda.time.Period; -import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import javax.validation.constraints.Min; +import java.lang.annotation.Annotation; /** */ public class HttpClientModule implements Module { + public static HttpClientModule global() + { + return new HttpClientModule("druid.global.http", Global.class); + } + + private final String propertyPrefix; + private Annotation annotation = null; + private Class annotationClazz = null; + + public HttpClientModule(String propertyPrefix) + { + this.propertyPrefix = propertyPrefix; + } + + public HttpClientModule(String propertyPrefix, Class annotation) + { + this.propertyPrefix = propertyPrefix; + this.annotationClazz = annotation; + } + + public HttpClientModule(String propertyPrefix, Annotation annotation) + { + this.propertyPrefix = propertyPrefix; + this.annotation = annotation; + } + @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.global.http", DruidHttpClientConfig.class); + if (annotation != null) { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotation); + binder.bind(HttpClient.class) + .annotatedWith(annotation) + .toProvider(new HttpClientProvider(annotation)) + .in(LazySingleton.class); + } + else if (annotationClazz != null) { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotationClazz); + binder.bind(HttpClient.class) + .annotatedWith(annotationClazz) + .toProvider(new HttpClientProvider(annotationClazz)) + .in(LazySingleton.class); + } + else { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class); + binder.bind(HttpClient.class) + .toProvider(new HttpClientProvider()) + .in(LazySingleton.class); + } } public static class DruidHttpClientConfig @@ -47,26 +97,56 @@ public class HttpClientModule implements Module } } - @Provides @LazySingleton @Global - public HttpClient makeHttpClient( - Supplier configSupplier, - Lifecycle lifecycle, - @Nullable SSLContext sslContext - ) + public static class HttpClientProvider implements Provider { - final DruidHttpClientConfig config = configSupplier.get(); + private final Key> configKey; + private final Key sslContextKey; - final HttpClientConfig.Builder builder = HttpClientConfig - .builder() - .withNumConnections(config.getNumConnections()) - .withReadTimeout(config.getReadTimeout()); + private Provider> configProvider; + private Provider lifecycleProvider; + private Binding sslContextBinding; - if (sslContext != null) { - builder.withSslContext(sslContext); + public HttpClientProvider() + { + configKey = Key.get(new TypeLiteral>(){}); + sslContextKey = Key.get(SSLContext.class); } - return HttpClientInit.createClient(builder.build(), lifecycle); + public HttpClientProvider(Annotation annotation) + { + configKey = Key.get(new TypeLiteral>(){}, annotation); + sslContextKey = Key.get(SSLContext.class, annotation); + } + + public HttpClientProvider(Class annotation) + { + configKey = Key.get(new TypeLiteral>(){}, annotation); + sslContextKey = Key.get(SSLContext.class, annotation); + } + + @Inject + public void configure(Injector injector) + { + configProvider = injector.getProvider(configKey); + sslContextBinding = injector.getExistingBinding(sslContextKey); + lifecycleProvider = injector.getProvider(Lifecycle.class); + } + + @Override + public HttpClient get() + { + final DruidHttpClientConfig config = configProvider.get().get(); + + final HttpClientConfig.Builder builder = HttpClientConfig + .builder() + .withNumConnections(config.getNumConnections()) + .withReadTimeout(config.getReadTimeout()); + + if (sslContextBinding != null) { + builder.withSslContext(sslContextBinding.getProvider().get()); + } + + return HttpClientInit.createClient(builder.build(), lifecycleProvider.get()); + } } - - } diff --git a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java new file mode 100644 index 00000000000..81078b56595 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java @@ -0,0 +1,52 @@ +package com.metamx.druid.guice; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.Query; +import com.metamx.druid.query.QueryRunnerFactory; +import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.group.GroupByQueryEngine; +import com.metamx.druid.query.group.GroupByQueryRunnerFactory; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; +import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.search.SearchQueryRunnerFactory; +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 java.util.Map; + +/** + */ +public class QueryRunnerFactoryModule extends QueryToolChestModule +{ + final Map, Class> mappings = + ImmutableMap., Class>builder() + .put(TimeseriesQuery.class, TimeseriesQueryRunnerFactory.class) + .put(SearchQuery.class, SearchQueryRunnerFactory.class) + .put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class) + .put(SegmentMetadataQuery.class, SegmentMetadataQueryRunnerFactory.class) + .put(GroupByQuery.class, GroupByQueryRunnerFactory.class) + .build(); + + @Override + public void configure(Binder binder) + { + super.configure(binder); + + final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( + binder, new TypeLiteral>(){}, new TypeLiteral(){} + ); + + for (Map.Entry, Class> entry : mappings.entrySet()) { + queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue()); + binder.bind(entry.getValue()).in(LazySingleton.class); + } + + binder.bind(GroupByQueryEngine.class).in(LazySingleton.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 d49b0062b72..96f5d3a5bda 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -42,6 +42,7 @@ import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQueryConfig; import com.metamx.druid.query.group.GroupByQueryEngine; +import com.metamx.druid.query.group.GroupByQueryQueryToolChest; import com.metamx.druid.query.group.GroupByQueryRunnerFactory; import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; @@ -107,6 +108,7 @@ public class ServerInit return new ComputeScratchPool(config.intermediateComputeSizeBytes()); } + // TODO: Get rid of this method public static Map, QueryRunnerFactory> initDefaultQueryTypes( ConfigurationObjectFactory configFactory, StupidPool computationBufferPool @@ -114,14 +116,13 @@ public class ServerInit { Map, QueryRunnerFactory> queryRunners = Maps.newLinkedHashMap(); queryRunners.put(TimeseriesQuery.class, new TimeseriesQueryRunnerFactory()); + final Supplier configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); queryRunners.put( GroupByQuery.class, new GroupByQueryRunnerFactory( - new GroupByQueryEngine( - Suppliers.ofInstance(new GroupByQueryConfig()), // TODO: Get rid of this - computationBufferPool - ), - Suppliers.ofInstance(new GroupByQueryConfig()) + new GroupByQueryEngine(configSupplier, computationBufferPool), + configSupplier, + new GroupByQueryQueryToolChest(configSupplier) ) ); queryRunners.put(SearchQuery.class, new SearchQueryRunnerFactory()); diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java index c69e16c7715..9cc502ff1e6 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java @@ -48,19 +48,20 @@ import java.util.concurrent.Future; */ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { - private static final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(); - private final GroupByQueryEngine engine; private final Supplier config; + private final GroupByQueryQueryToolChest toolChest; @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, - Supplier config + Supplier config, + GroupByQueryQueryToolChest toolChest ) { this.engine = engine; this.config = config; + this.toolChest = toolChest; } @Override @@ -122,7 +123,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory getToolchest() { return toolChest; } diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index 9ba4e51f628..98a7eff3c8b 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -81,9 +81,10 @@ public class GroupByQueryRunnerTest GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); + final Supplier configSupplier = Suppliers.ofInstance(config); final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( new GroupByQueryEngine( - Suppliers.ofInstance(config), + configSupplier, new StupidPool( new Supplier() { @@ -95,7 +96,8 @@ public class GroupByQueryRunnerTest } ) ), - Suppliers.ofInstance(config) + configSupplier, + new GroupByQueryQueryToolChest(configSupplier) ); return Lists.newArrayList( @@ -245,7 +247,7 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); final GroupByQuery allGranQuery = builder.copy().setGranularity(QueryGranularity.ALL).build(); - QueryRunner mergedRunner = new GroupByQueryQueryToolChest().mergeResults( + QueryRunner mergedRunner = factory.getToolchest().mergeResults( new QueryRunner() { @Override @@ -332,7 +334,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "travel", "rows", 2L, "idx", 243L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, limit), mergeRunner.run(fullQuery), String.format("limit: %d", limit) @@ -437,7 +439,7 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); - QueryRunner mergedRunner = new GroupByQueryQueryToolChest().mergeResults( + QueryRunner mergedRunner = factory.getToolchest().mergeResults( new QueryRunner() { @Override @@ -490,7 +492,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 6L, "idx", 4420L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects( @@ -530,7 +532,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "automotive", "rows", 2L, "idx", 269L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" @@ -569,7 +571,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "alias", "technology", "rows", 2L, "idx", 178.24917602539062D) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); TestHelper.assertExpectedObjects( Iterables.limit(expectedResults, 5), mergeRunner.run(builder.limit(5).build()), "limited" @@ -608,7 +610,7 @@ public class GroupByQueryRunnerTest final GroupByQuery fullQuery = builder.build(); - QueryRunner mergedRunner = new GroupByQueryQueryToolChest().mergeResults( + QueryRunner mergedRunner = factory.getToolchest().mergeResults( new QueryRunner() { @Override @@ -651,7 +653,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = factory.getToolchest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index 5ca55f111bc..4c4a801487c 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -55,9 +55,10 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); + final Supplier configSupplier = Suppliers.ofInstance(config); final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( new GroupByQueryEngine( - Suppliers.ofInstance(config), + configSupplier, new StupidPool( new Supplier() { @@ -69,7 +70,8 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest } ) ), - Suppliers.ofInstance(config) + configSupplier, + new GroupByQueryQueryToolChest(configSupplier) ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java new file mode 100644 index 00000000000..d5e00e4c7ff --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -0,0 +1,56 @@ +package io.druid.cli; + +import com.google.inject.Injector; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.BrokerModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.QueryToolChestModule; +import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.ServerViewModule; +import com.metamx.druid.guice.annotations.Client; +import com.metamx.druid.http.ClientQuerySegmentWalker; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Command; + +/** + */ +@Command( + name = "broker", + description = "Runs a broker node, see https://github.com/metamx/druid/wiki/Broker for a description" +) +public class CliBroker extends ServerRunnable +{ + private static final Logger log = new Logger(CliBroker.class); + + public CliBroker() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule(), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule(), + ServerModule.class, + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class), + new QueryableModule(ClientQuerySegmentWalker.class), + new QueryToolChestModule(), + new ServerViewModule(), + new HttpClientModule("druid.broker.http", Client.class), + new BrokerModule() + ); + } +} diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 9aa3dc18930..7c60be03346 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -11,6 +11,7 @@ import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.ServerViewModule; import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.http.InfoResource; import com.metamx.druid.http.MasterResource; @@ -55,7 +56,7 @@ public class CliCoordinator extends ServerRunnable return Initialization.makeInjector( new LifecycleModule().register(DruidMaster.class), EmitterModule.class, - HttpClientModule.class, + HttpClientModule.global(), DbConnectorModule.class, JacksonConfigManagerModule.class, CuratorModule.class, @@ -66,6 +67,7 @@ public class CliCoordinator extends ServerRunnable .addResource(InfoResource.class) .addResource(MasterResource.class) .addResource(StatusResource.class), + new ServerViewModule(), CoordinatorModule.class ); } diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index a6c34c3d4f8..01bb7484198 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -1,7 +1,6 @@ package io.druid.cli; import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; @@ -9,25 +8,16 @@ import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerInitializer; import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import com.metamx.druid.metrics.ServerMonitor; import io.airlift.command.Command; -import org.eclipse.jetty.server.Handler; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.handler.DefaultHandler; -import org.eclipse.jetty.server.handler.HandlerList; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; /** */ @@ -50,34 +40,15 @@ public class CliHistorical extends ServerRunnable return Initialization.makeInjector( new LifecycleModule().register(ZkCoordinator.class), EmitterModule.class, - HttpClientModule.class, + HttpClientModule.global(), CuratorModule.class, new MetricsModule().register(ServerMonitor.class), ServerModule.class, - new JettyServerModule(new HistoricalJettyServerInitializer()) + new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ServerManager.class), + new QueryRunnerFactoryModule(), HistoricalModule.class ); } - - private static class HistoricalJettyServerInitializer implements JettyServerInitializer - { - @Override - public void initialize(Server server, Injector injector) - { - final ServletContextHandler queries = new ServletContextHandler(ServletContextHandler.SESSIONS); - queries.setResourceBase("/"); - queries.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); - - final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); - root.addFilter(GuiceFilter.class, "/*", null); - - final HandlerList handlerList = new HandlerList(); - handlerList.setHandlers(new Handler[]{queries, root, new DefaultHandler()}); - server.setHandler(handlerList); - } - } } diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 401e4b08203..594deb05af5 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -19,7 +19,7 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(CliCoordinator.class, CliHistorical.class); + .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class); builder.build().parse(args).run(); } diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java new file mode 100644 index 00000000000..78ea25219e1 --- /dev/null +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -0,0 +1,36 @@ +package io.druid.cli; + +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; +import com.metamx.druid.http.QueryServlet; +import com.metamx.druid.initialization.JettyServerInitializer; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; + +/** +*/ +public class QueryJettyServerInitializer implements JettyServerInitializer +{ + @Override + public void initialize(Server server, Injector injector) + { + final ServletContextHandler queries = new ServletContextHandler(ServletContextHandler.SESSIONS); + queries.setResourceBase("/"); + queries.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*"); + + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + + final HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{queries, root, new DefaultHandler()}); + server.setHandler(handlerList); + } +} From c3ca9f5a3b7c63d3d914495c1d344dc0daaed359 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 6 Aug 2013 12:02:19 -0700 Subject: [PATCH 26/92] 1) Rename MemcachedCacheConfig back to what it should be --- .../main/java/com/metamx/druid/client/cache/MemcachedCache.java | 2 +- .../{MemcachedCacheConfiger.java => MemcachedCacheConfig.java} | 2 +- .../com/metamx/druid/client/cache/MemcachedCacheProvider.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename client/src/main/java/com/metamx/druid/client/cache/{MemcachedCacheConfiger.java => MemcachedCacheConfig.java} (95%) diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java index 6f77373556c..befac85326e 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java @@ -49,7 +49,7 @@ public class MemcachedCache implements Cache { private static final Logger log = new Logger(MemcachedCache.class); - public static MemcachedCache create(final MemcachedCacheConfiger config) + public static MemcachedCache create(final MemcachedCacheConfig config) { try { LZ4Transcoder transcoder = new LZ4Transcoder(config.getMaxObjectSize()); diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java similarity index 95% rename from client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java rename to client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java index 491335c81f5..672fa9a2bba 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfiger.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import javax.validation.constraints.NotNull; -public class MemcachedCacheConfiger +public class MemcachedCacheConfig { @JsonProperty private int expiration = 2592000; // What is this number? diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java index c0bedbe5309..a199b6323bd 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java +++ b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java @@ -1,6 +1,6 @@ package com.metamx.druid.client.cache; -public class MemcachedCacheProvider extends MemcachedCacheConfiger implements CacheProvider +public class MemcachedCacheProvider extends MemcachedCacheConfig implements CacheProvider { @Override public Cache get() From 69f7831d826327ac1ac84f9dbe3c2f441904ee94 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 6 Aug 2013 12:04:04 -0700 Subject: [PATCH 27/92] 1) Rename QueryToolChestModule because there was a case sensitivity issue, grrr --- .../guice/QueryToolChestModuleStupidCase.java | 51 +++++++++++++++++++ ...> QueryRunnerFactoryModuleStupidCase.java} | 2 +- .../src/main/java/io/druid/cli/CliBroker.java | 4 +- .../main/java/io/druid/cli/CliHistorical.java | 4 +- 4 files changed, 56 insertions(+), 5 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java rename server/src/main/java/com/metamx/druid/guice/{QueryRunnerFactoryModule.java => QueryRunnerFactoryModuleStupidCase.java} (96%) diff --git a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java new file mode 100644 index 00000000000..54a9f46b94c --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java @@ -0,0 +1,51 @@ +package com.metamx.druid.guice; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.Query; +import com.metamx.druid.query.QueryToolChest; +import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.group.GroupByQueryConfig; +import com.metamx.druid.query.group.GroupByQueryQueryToolChest; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQueryQueryToolChest; +import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.search.SearchQueryQueryToolChest; +import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; +import com.metamx.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; +import com.metamx.druid.query.timeseries.TimeseriesQuery; +import com.metamx.druid.query.timeseries.TimeseriesQueryQueryToolChest; + +import java.util.Map; + +/** + */ +public class QueryToolChestModuleStupidCase implements Module +{ + final Map, Class> mappings = + ImmutableMap., Class>builder() + .put(TimeseriesQuery.class, TimeseriesQueryQueryToolChest.class) + .put(SearchQuery.class, SearchQueryQueryToolChest.class) + .put(TimeBoundaryQuery.class, TimeBoundaryQueryQueryToolChest.class) + .put(SegmentMetadataQuery.class, SegmentMetadataQueryQueryToolChest.class) + .put(GroupByQuery.class, GroupByQueryQueryToolChest.class) + .build(); + + @Override + public void configure(Binder binder) + { + MapBinder, QueryToolChest> toolChests = MapBinder.newMapBinder( + binder, new TypeLiteral>(){}, new TypeLiteral(){} + ); + + for (Map.Entry, Class> entry : mappings.entrySet()) { + toolChests.addBinding(entry.getKey()).to(entry.getValue()); + binder.bind(entry.getValue()).in(LazySingleton.class); + } + + JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java similarity index 96% rename from server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java rename to server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java index 81078b56595..6e6c015ab38 100644 --- a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java @@ -22,7 +22,7 @@ import java.util.Map; /** */ -public class QueryRunnerFactoryModule extends QueryToolChestModule +public class QueryRunnerFactoryModuleStupidCase extends QueryToolChestModuleStupidCase { final Map, Class> mappings = ImmutableMap., Class>builder() diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index d5e00e4c7ff..53aa0d8554b 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -6,7 +6,7 @@ import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.BrokerModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryToolChestModule; +import com.metamx.druid.guice.QueryToolChestModuleStupidCase; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.ServerViewModule; @@ -47,7 +47,7 @@ public class CliBroker extends ServerRunnable new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ClientQuerySegmentWalker.class), - new QueryToolChestModule(), + new QueryToolChestModuleStupidCase(), new ServerViewModule(), new HttpClientModule("druid.broker.http", Client.class), new BrokerModule() diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 01bb7484198..654f9b7d58e 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -8,7 +8,7 @@ import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryRunnerFactoryModule; +import com.metamx.druid.guice.QueryRunnerFactoryModuleStupidCase; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.http.StatusResource; @@ -47,7 +47,7 @@ public class CliHistorical extends ServerRunnable new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ServerManager.class), - new QueryRunnerFactoryModule(), + new QueryRunnerFactoryModuleStupidCase(), HistoricalModule.class ); } From 51f648ec114df19828de663965770d84038151b6 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 6 Aug 2013 12:05:56 -0700 Subject: [PATCH 28/92] 1) Rename QueryToolChestModule back --- ...olChestModuleStupidCase.java => QueryToolChestModule.java} | 2 +- ...oryModuleStupidCase.java => QueryRunnerFactoryModule.java} | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 4 ++-- services/src/main/java/io/druid/cli/CliHistorical.java | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) rename client/src/main/java/com/metamx/druid/guice/{QueryToolChestModuleStupidCase.java => QueryToolChestModule.java} (97%) rename server/src/main/java/com/metamx/druid/guice/{QueryRunnerFactoryModuleStupidCase.java => QueryRunnerFactoryModule.java} (96%) diff --git a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java similarity index 97% rename from client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java rename to client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java index 54a9f46b94c..3a5e39833c5 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModuleStupidCase.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java @@ -23,7 +23,7 @@ import java.util.Map; /** */ -public class QueryToolChestModuleStupidCase implements Module +public class QueryToolChestModule implements Module { final Map, Class> mappings = ImmutableMap., Class>builder() diff --git a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java similarity index 96% rename from server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java rename to server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java index 6e6c015ab38..81078b56595 100644 --- a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModuleStupidCase.java +++ b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java @@ -22,7 +22,7 @@ import java.util.Map; /** */ -public class QueryRunnerFactoryModuleStupidCase extends QueryToolChestModuleStupidCase +public class QueryRunnerFactoryModule extends QueryToolChestModule { final Map, Class> mappings = ImmutableMap., Class>builder() diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 53aa0d8554b..d5e00e4c7ff 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -6,7 +6,7 @@ import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.BrokerModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryToolChestModuleStupidCase; +import com.metamx.druid.guice.QueryToolChestModule; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.ServerViewModule; @@ -47,7 +47,7 @@ public class CliBroker extends ServerRunnable new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ClientQuerySegmentWalker.class), - new QueryToolChestModuleStupidCase(), + new QueryToolChestModule(), new ServerViewModule(), new HttpClientModule("druid.broker.http", Client.class), new BrokerModule() diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 654f9b7d58e..01bb7484198 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -8,7 +8,7 @@ import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryRunnerFactoryModuleStupidCase; +import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.ServerModule; import com.metamx.druid.http.StatusResource; @@ -47,7 +47,7 @@ public class CliHistorical extends ServerRunnable new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ServerManager.class), - new QueryRunnerFactoryModuleStupidCase(), + new QueryRunnerFactoryModule(), HistoricalModule.class ); } From 4a64ce37ed3207c858a1dee41e2a6d366218d8be Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 6 Aug 2013 13:34:24 -0700 Subject: [PATCH 29/92] Finish the merging, wtf IntelliJ? --- .../common/index/ChatHandlerProvider.java | 64 +------------------ .../EventReceivingChatHandlerProvider.java | 36 +++++------ .../task/IndexDeterminePartitionsTask.java | 4 +- .../common/task/VersionConverterTask.java | 2 +- .../coordinator/TaskMasterLifecycle.java | 8 +-- .../coordinator/ThreadPoolTaskRunner.java | 1 - .../indexing/worker/config/WorkerConfig.java | 1 - .../worker/executor/ExecutorNode.java | 6 +- .../coordinator/RemoteTaskRunnerTest.java | 1 - .../coordinator/TaskLifecycleTest.java | 8 +-- .../scaling/EC2AutoScalingStrategyTest.java | 2 +- .../SimpleResourceManagementStrategyTest.java | 3 - 12 files changed, 30 insertions(+), 106 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java index c89b079a30c..1662abc7d9b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java @@ -20,11 +20,6 @@ package com.metamx.druid.indexing.common.index; import com.google.common.base.Optional; -import com.google.common.collect.Maps; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; /** */ @@ -34,62 +29,5 @@ public interface ChatHandlerProvider public void unregister(final String key); - public ChatHandlerProvider( - ChatHandlerProviderConfig config, - ServiceAnnouncer serviceAnnouncer - ) - { - this.config = config; - this.serviceAnnouncer = serviceAnnouncer; - this.handlers = Maps.newConcurrentMap(); - } - - public void register(final String key, ChatHandler handler) - { - final String service = serviceName(key); - log.info("Registering Eventhandler: %s", key); - - if (handlers.putIfAbsent(key, handler) != null) { - throw new ISE("handler already registered for key: %s", key); - } - - try { - serviceAnnouncer.announce(service); - } - catch (Exception e) { - log.warn(e, "Failed to register service: %s", service); - handlers.remove(key, handler); - } - } - - public void unregister(final String key) - { - final String service = serviceName(key); - - log.info("Unregistering chat handler: %s", key); - - final ChatHandler handler = handlers.get(key); - if (handler == null) { - log.warn("handler not currently registered, ignoring: %s", key); - } - - try { - serviceAnnouncer.unannounce(service); - } - catch (Exception e) { - log.warn(e, "Failed to unregister service: %s", service); - } - - handlers.remove(key, handler); - } - - public Optional get(final String key) - { - return Optional.fromNullable(handlers.get(key)); - } - - private String serviceName(String key) - { - return String.format(config.getServiceFormat(), key); - } + public Optional get(final String key); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java index f18a79c241c..c4ae8cec851 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java @@ -25,6 +25,7 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.initialization.DruidNode; import java.util.concurrent.ConcurrentMap; @@ -52,44 +53,43 @@ public class EventReceivingChatHandlerProvider implements ChatHandlerProvider } @Override - public void register(final String key, ChatHandler handler) + public void register(final String service, ChatHandler handler) { - final String service = serviceName(key); - log.info("Registering Eventhandler: %s", key); + final DruidNode node = makeDruidNode(service); + log.info("Registering Eventhandler[%s]", service); - if (handlers.putIfAbsent(key, handler) != null) { - throw new ISE("handler already registered for key: %s", key); + if (handlers.putIfAbsent(service, handler) != null) { + throw new ISE("handler already registered for service[%s]", service); } try { - serviceAnnouncer.announce(service); + serviceAnnouncer.announce(node); } catch (Exception e) { - log.warn(e, "Failed to register service: %s", service); - handlers.remove(key, handler); + log.warn(e, "Failed to register service[%s]", service); + handlers.remove(service, handler); } } @Override - public void unregister(final String key) + public void unregister(final String service) { - final String service = serviceName(key); - log.info("Unregistering chat handler: %s", key); + log.info("Unregistering chat handler[%s]", service); - final ChatHandler handler = handlers.get(key); + final ChatHandler handler = handlers.get(service); if (handler == null) { - log.warn("handler not currently registered, ignoring: %s", key); + log.warn("handler[%s] not currently registered, ignoring.", service); } try { - serviceAnnouncer.unannounce(service); + serviceAnnouncer.unannounce(makeDruidNode(service)); } catch (Exception e) { - log.warn(e, "Failed to unregister service: %s", service); + log.warn(e, "Failed to unregister service[%s]", service); } - handlers.remove(key, handler); + handlers.remove(service, handler); } @Override @@ -98,8 +98,8 @@ public class EventReceivingChatHandlerProvider implements ChatHandlerProvider return Optional.fromNullable(handlers.get(key)); } - private String serviceName(String key) + private DruidNode makeDruidNode(String key) { - return String.format(config.getServiceFormat(), key); + return new DruidNode(key, config.getHost(), config.getPort()); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java index 8388c9653e3..7f43a31a746 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java @@ -31,13 +31,13 @@ import com.google.common.collect.Sets; import com.google.common.collect.TreeMultiset; import com.google.common.primitives.Ints; import com.metamx.common.logger.Logger; -import com.metamx.druid.input.InputRow; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.SpawnTasksAction; +import com.metamx.druid.input.InputRow; +import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; -import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java index 4722222bf52..d98868f989d 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java @@ -30,13 +30,13 @@ 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.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.indexing.common.actions.SegmentListUsedAction; import com.metamx.druid.indexing.common.actions.SpawnTasksAction; import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.loading.SegmentLoadingException; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index 4ef331e2de5..d1823d36cda 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -26,8 +26,6 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.task.Task; @@ -35,6 +33,8 @@ import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.initialization.DruidNode; +import com.metamx.druid.initialization.Initialization; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -66,7 +66,7 @@ public class TaskMasterLifecycle final TaskQueue taskQueue, final TaskActionClientFactory taskActionClientFactory, final IndexerCoordinatorConfig indexerCoordinatorConfig, - final DruidNode nodeConfig, + final DruidNode node, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, @@ -119,7 +119,7 @@ public class TaskMasterLifecycle } ); leaderLifecycle.addManagedInstance(taskQueue); - Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); + Initialization.announceDefaultService(node, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); if ("remote".equalsIgnoreCase(indexerCoordinatorConfig.getRunnerImpl())) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index ac2d680ebc0..202ca0f55f5 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -40,7 +40,6 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; -import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.File; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java index deca87c56f5..2474704a418 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java @@ -20,7 +20,6 @@ package com.metamx.druid.indexing.worker.config; import org.skife.config.Config; -import org.skife.config.Default; /** */ diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 4b89b40e949..351d617fdcf 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -388,14 +388,12 @@ public class ExecutorNode extends BaseServerNode { if (chatHandlerProvider == null) { final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); - final ServiceAnnouncer myServiceAnnouncer; if (config.isPublishDiscovery()) { - myServiceAnnouncer = serviceAnnouncer; + this.chatHandlerProvider = new EventReceivingChatHandlerProvider(config, serviceAnnouncer); } else { log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); - myServiceAnnouncer = new NoopServiceAnnouncer(); + this.chatHandlerProvider = new NoopChatHandlerProvider(); } - this.chatHandlerProvider = new ChatHandlerProvider(config, myServiceAnnouncer); } } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 6c6da7ccb89..0c39406b821 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -45,7 +45,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import javax.annotation.Nullable; import java.io.File; import java.util.Arrays; import java.util.Set; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index 6e8171f2072..77e7c2fcb49 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -36,13 +36,6 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; -import com.metamx.druid.indexing.common.task.TaskResource; -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.DataSegmentKiller; -import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; @@ -57,6 +50,7 @@ import com.metamx.druid.indexing.common.task.AbstractTask; import com.metamx.druid.indexing.common.task.IndexTask; import com.metamx.druid.indexing.common.task.KillTask; import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java index 8f512390f14..27442ed1cdc 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -28,11 +28,11 @@ import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Lists; -import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.indexing.coordinator.setup.EC2NodeData; import com.metamx.druid.indexing.coordinator.setup.GalaxyUserData; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index f73d289526d..fcbc1d4113b 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -22,7 +22,6 @@ package com.metamx.druid.indexing.coordinator.scaling; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.TestTask; @@ -45,10 +44,8 @@ import org.junit.Before; import org.junit.Test; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.atomic.AtomicReference; /** From 85ee8bb26781cf4245ad04fea73ef0f8f8c90bdf Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 13 Aug 2013 17:08:45 -0700 Subject: [PATCH 30/92] port realtime to guice --- .../druid/client/DruidServerConfig.java | 8 + .../LegacyDataSegmentAnnouncerProvider.java | 6 +- .../metamx/druid/guice/AnnouncerModule.java | 35 ++ examples/bin/ec2/run.sh | 2 +- examples/bin/run_example_server.sh | 2 +- .../examples/RealtimeStandaloneMain.java | 161 ---------- .../guice/RealtimeStandaloneModule.java | 164 ++++++++++ .../guice/DbSegmentPublisherProvider.java | 52 +++ .../guice/NoopSegmentPublisherProvider.java | 34 ++ .../druid/guice/RealtimeManagerConfig.java | 37 +++ .../druid/guice/RealtimeManagerProvider.java | 71 +++++ .../metamx/druid/guice/RealtimeModule.java | 45 +++ .../druid/guice/SegmentPublisherProvider.java | 35 ++ .../realtime/DbSegmentPublisherConfig.java | 21 +- .../metamx/druid/realtime/FireDepartment.java | 1 + ...imeMain.java => NoopSegmentPublisher.java} | 33 +- .../druid/realtime/RealtimeManager.java | 2 + .../metamx/druid/realtime/RealtimeNode.java | 298 ------------------ .../druid/realtime/firehose/Firehose.java | 2 +- .../plumber/RealtimePlumberSchool.java | 54 ++-- .../CassandraDataSegmentPusherProvider.java | 47 +++ .../druid/guice/DataSegmentPullerModule.java | 83 +++++ .../druid/guice/DataSegmentPusherModule.java | 27 ++ .../guice/DataSegmentPusherProvider.java | 37 +++ .../druid/guice/DruidProcessingModule.java | 136 ++++++++ .../guice/HdfsDataSegmentPusherProvider.java | 52 +++ .../metamx/druid/guice/HistoricalModule.java | 222 ++----------- .../guice/LocalDataSegmentPusherProvider.java | 47 +++ .../guice/S3DataSegmentPusherProvider.java | 52 +++ .../java/com/metamx/druid/guice/S3Module.java | 52 +++ .../com/metamx/druid/guice/ServerModule.java | 19 ++ .../metamx/druid/guice/StorageNodeModule.java | 65 ++++ .../druid/loading/HdfsDataSegmentPusher.java | 21 ++ .../loading/HdfsDataSegmentPusherConfig.java | 11 +- .../druid/loading/LocalDataSegmentPusher.java | 2 + .../loading/LocalDataSegmentPusherConfig.java | 12 +- .../druid/loading/S3DataSegmentPusher.java | 2 + .../loading/S3DataSegmentPusherConfig.java | 32 +- .../cassandra/CassandraDataSegmentConfig.java | 23 +- .../cassandra/CassandraDataSegmentPusher.java | 29 +- .../metamx/druid/metrics/MonitorsConfig.java | 21 +- .../src/main/java/io/druid/cli/CliBroker.java | 22 +- .../main/java/io/druid/cli/CliHistorical.java | 31 +- .../main/java/io/druid/cli/CliRealtime.java | 86 +++++ .../io/druid/cli/CliRealtimeStandalone.java | 67 ++++ services/src/main/java/io/druid/cli/Main.java | 21 +- 46 files changed, 1520 insertions(+), 762 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java delete mode 100644 examples/src/main/java/druid/examples/RealtimeStandaloneMain.java create mode 100644 examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/NoopSegmentPublisherProvider.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerConfig.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java create mode 100644 realtime/src/main/java/com/metamx/druid/guice/SegmentPublisherProvider.java rename realtime/src/main/java/com/metamx/druid/realtime/{RealtimeMain.java => NoopSegmentPublisher.java} (57%) delete mode 100644 realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java create mode 100644 server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java create mode 100644 server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java create mode 100644 server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java create mode 100644 server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java create mode 100644 server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java create mode 100644 server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java create mode 100644 server/src/main/java/com/metamx/druid/guice/S3Module.java create mode 100644 server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java create mode 100644 services/src/main/java/io/druid/cli/CliRealtime.java create mode 100644 services/src/main/java/io/druid/cli/CliRealtimeStandalone.java diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 6a57a38b20f..1dcf38c4dc0 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -34,11 +34,19 @@ public class DruidServerConfig @JsonProperty private String tier = "_default_tier"; + @JsonProperty + private String type = "historical"; + public long getMaxSize() { return maxSize; } + public String getType() + { + return type; + } + public String getTier() { return tier; diff --git a/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java index cd99119b2a4..86b1f5fce88 100644 --- a/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java +++ b/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java @@ -1,7 +1,9 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.druid.guice.annotations.Json; import javax.validation.constraints.NotNull; import java.util.Arrays; @@ -18,10 +20,6 @@ public class LegacyDataSegmentAnnouncerProvider implements DataSegmentAnnouncerP @NotNull private BatchDataSegmentAnnouncer batchAnnouncer = null; - @JacksonInject - @NotNull - private Lifecycle lifecycle = null; - @Override public DataSegmentAnnouncer get() { diff --git a/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java b/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java new file mode 100644 index 00000000000..f3a6cd46a05 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java @@ -0,0 +1,35 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; +import com.metamx.druid.coordination.DataSegmentAnnouncer; +import com.metamx.druid.coordination.DataSegmentAnnouncerProvider; +import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; +import com.metamx.druid.curator.announcement.Announcer; +import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; +import org.apache.curator.framework.CuratorFramework; + +/** + */ +public class AnnouncerModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); + JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); + binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); + binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + binder.bind(SingleDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + } + + @Provides + @ManageLifecycle + public Announcer getAnnouncer(CuratorFramework curator) + { + return new Announcer(curator, Execs.singleThreaded("Announcer-%s")); + } +} diff --git a/examples/bin/ec2/run.sh b/examples/bin/ec2/run.sh index 9f695c1e001..df7438ccf5b 100644 --- a/examples/bin/ec2/run.sh +++ b/examples/bin/ec2/run.sh @@ -7,7 +7,7 @@ cd druid-services-* 2>&1 > /dev/null mkdir logs 2>&1 > /dev/null # Now start a realtime node -nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime com.metamx.druid.realtime.RealtimeMain 2>&1 > logs/realtime.log & +nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=config/realtime/realtime.spec -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/realtime io.druid.cli.Main server realtime 2>&1 > logs/realtime.log & # And a master node nohup java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/druid-services-0.5.5-SNAPSHOT-selfcontained.jar:config/master io.druid.cli.Main server coordinator 2>&1 > logs/master.log & diff --git a/examples/bin/run_example_server.sh b/examples/bin/run_example_server.sh index e2cc076ce9c..e970ffcc7e0 100755 --- a/examples/bin/run_example_server.sh +++ b/examples/bin/run_example_server.sh @@ -63,4 +63,4 @@ DRUID_CP=${DRUID_CP}:${SCRIPT_DIR}/config echo "Running command:" -(set -x; java ${JAVA_ARGS} -classpath ${DRUID_CP} druid.examples.RealtimeStandaloneMain) +(set -x; java ${JAVA_ARGS} -classpath ${DRUID_CP} io.druid.cli.Main druid server realtimeStandalone) diff --git a/examples/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/src/main/java/druid/examples/RealtimeStandaloneMain.java deleted file mode 100644 index ebdd59d46c1..00000000000 --- a/examples/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ /dev/null @@ -1,161 +0,0 @@ -package druid.examples; - -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.collect.ImmutableList; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.ServerView; -import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.log.LogLevelAdjuster; -import com.metamx.druid.realtime.RealtimeNode; -import com.metamx.druid.realtime.SegmentPublisher; -import druid.examples.flights.FlightsFirehoseFactory; -import druid.examples.rand.RandomFirehoseFactory; -import druid.examples.twitter.TwitterSpritzerFirehoseFactory; -import druid.examples.web.WebFirehoseFactory; - -import java.io.File; -import java.io.IOException; -import java.util.concurrent.Executor; - -/** - * Standalone Demo Realtime process. - */ -public class RealtimeStandaloneMain -{ - private static final Logger log = new Logger(RealtimeStandaloneMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - final Lifecycle lifecycle = new Lifecycle(); - - RealtimeNode rn = RealtimeNode.builder().build(); - lifecycle.addManagedInstance(rn); - - // register the Firehoses - rn.registerJacksonSubtype( - new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"), - new NamedType(FlightsFirehoseFactory.class, "flights"), - new NamedType(RandomFirehoseFactory.class, "rand"), - new NamedType(WebFirehoseFactory.class, "webstream") - - ); - - // Create dummy objects for the various interfaces that interact with the DB, ZK and deep storage - rn.setSegmentPublisher(new NoopSegmentPublisher()); - rn.setAnnouncer(new NoopDataSegmentAnnouncer()); - rn.setDataSegmentPusher(new NoopDataSegmentPusher()); - rn.setServerView(new NoopServerView()); - rn.setInventoryView(new NoopInventoryView()); - - Runtime.getRuntime().addShutdownHook( - new Thread( - new Runnable() - { - @Override - public void run() - { - log.info("Running shutdown hook"); - lifecycle.stop(); - } - } - ) - ); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - t.printStackTrace(); - System.exit(2); - } - - lifecycle.join(); - } - - private static class NoopServerView implements ServerView - { - @Override - public void registerServerCallback( - Executor exec, ServerCallback callback - ) - { - - } - - @Override - public void registerSegmentCallback( - Executor exec, SegmentCallback callback - ) - { - - } - } - - private static class NoopInventoryView implements InventoryView - { - @Override - public DruidServer getInventoryValue(String string) - { - return null; - } - - @Override - public Iterable getInventory() - { - return ImmutableList.of(); - } - } - - private static class NoopDataSegmentPusher implements DataSegmentPusher - { - @Override - public DataSegment push(File file, DataSegment segment) throws IOException - { - return segment; - } - } - - private static class NoopSegmentPublisher implements SegmentPublisher - { - @Override - public void publishSegment(DataSegment segment) throws IOException - { - // do nothing - } - } - - private static class NoopDataSegmentAnnouncer implements DataSegmentAnnouncer - { - @Override - public void announceSegment(DataSegment segment) throws IOException - { - // do nothing - } - - @Override - public void unannounceSegment(DataSegment segment) throws IOException - { - // do nothing - } - - @Override - public void announceSegments(Iterable segments) throws IOException - { - // do nothing - } - - @Override - public void unannounceSegments(Iterable segments) throws IOException - { - // do nothing - } - } -} \ No newline at end of file diff --git a/examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java b/examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java new file mode 100644 index 00000000000..4013b6a236a --- /dev/null +++ b/examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java @@ -0,0 +1,164 @@ +package druid.examples.guice; + +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.ProvisionException; +import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.metamx.common.logger.Logger; +import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.client.DruidServer; +import com.metamx.druid.client.DruidServerConfig; +import com.metamx.druid.client.InventoryView; +import com.metamx.druid.client.ServerView; +import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.coordination.DataSegmentAnnouncer; +import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.JsonConfigProvider; +import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.ManageLifecycle; +import com.metamx.druid.guice.NoopSegmentPublisherProvider; +import com.metamx.druid.guice.RealtimeManagerConfig; +import com.metamx.druid.guice.RealtimeManagerProvider; +import com.metamx.druid.guice.annotations.Global; +import com.metamx.druid.guice.annotations.Processing; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.initialization.DruidModule; +import com.metamx.druid.initialization.DruidNode; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.QueryableIndexFactory; +import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; +import com.metamx.druid.query.MetricsEmittingExecutorService; +import com.metamx.druid.query.QueryRunnerFactoryConglomerate; +import com.metamx.druid.realtime.RealtimeManager; +import com.metamx.druid.realtime.SegmentPublisher; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; +import druid.examples.flights.FlightsFirehoseFactory; +import druid.examples.rand.RandomFirehoseFactory; +import druid.examples.twitter.TwitterSpritzerFirehoseFactory; +import druid.examples.web.WebFirehoseFactory; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; + +/** + */ +public class RealtimeStandaloneModule implements DruidModule +{ + private static final Logger log = new Logger(RealtimeStandaloneModule.class); + + @Override + public void configure(Binder binder) + { + binder.bind(SegmentPublisher.class).toProvider(NoopSegmentPublisherProvider.class); + binder.bind(DataSegmentPusher.class).to(NoopDataSegmentPusher.class); + binder.bind(DataSegmentAnnouncer.class).to(NoopDataSegmentAnnouncer.class); + binder.bind(InventoryView.class).to(NoopInventoryView.class); + binder.bind(ServerView.class).to(NoopServerView.class); + + JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class); + binder.bind(RealtimeManager.class).toProvider(RealtimeManagerProvider.class).in(ManageLifecycle.class); + } + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("RealtimestandAloneModule") + .registerSubtypes( + new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"), + new NamedType(FlightsFirehoseFactory.class, "flights"), + new NamedType(RandomFirehoseFactory.class, "rand"), + new NamedType(WebFirehoseFactory.class, "webstream") + ) + ); + } + + private static class NoopServerView implements ServerView + { + @Override + public void registerServerCallback( + Executor exec, ServerCallback callback + ) + { + // do nothing + } + + @Override + public void registerSegmentCallback( + Executor exec, SegmentCallback callback + ) + { + // do nothing + } + } + + private static class NoopInventoryView implements InventoryView + { + @Override + public DruidServer getInventoryValue(String string) + { + return null; + } + + @Override + public Iterable getInventory() + { + return ImmutableList.of(); + } + } + + private static class NoopDataSegmentPusher implements DataSegmentPusher + { + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + return segment; + } + } + + private static class NoopDataSegmentAnnouncer implements DataSegmentAnnouncer + { + @Override + public void announceSegment(DataSegment segment) throws IOException + { + // do nothing + } + + @Override + public void unannounceSegment(DataSegment segment) throws IOException + { + // do nothing + } + + @Override + public void announceSegments(Iterable segments) throws IOException + { + // do nothing + } + + @Override + public void unannounceSegments(Iterable segments) throws IOException + { + // do nothing + } + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java b/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java new file mode 100644 index 00000000000..3a8aa5f8731 --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java @@ -0,0 +1,52 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.realtime.DbSegmentPublisher; +import com.metamx.druid.realtime.DbSegmentPublisherConfig; +import com.metamx.druid.realtime.SegmentPublisher; +import org.skife.jdbi.v2.IDBI; + +import javax.validation.constraints.NotNull; + +/** + */ +public class DbSegmentPublisherProvider implements SegmentPublisherProvider +{ + @JacksonInject + @NotNull + private IDBI idbi = null; + + @JacksonInject + @NotNull + private DbSegmentPublisherConfig config = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public SegmentPublisher get() + { + return new DbSegmentPublisher(jsonMapper, config, idbi); + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/NoopSegmentPublisherProvider.java b/realtime/src/main/java/com/metamx/druid/guice/NoopSegmentPublisherProvider.java new file mode 100644 index 00000000000..361d54cb40b --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/NoopSegmentPublisherProvider.java @@ -0,0 +1,34 @@ +/* + * 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.guice; + +import com.metamx.druid.realtime.NoopSegmentPublisher; +import com.metamx.druid.realtime.SegmentPublisher; + +/** + */ +public class NoopSegmentPublisherProvider implements SegmentPublisherProvider +{ + @Override + public SegmentPublisher get() + { + return new NoopSegmentPublisher(); + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerConfig.java b/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerConfig.java new file mode 100644 index 00000000000..44d66bcf99b --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerConfig.java @@ -0,0 +1,37 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.File; + +/** + */ +public class RealtimeManagerConfig +{ + @JsonProperty + private File specFile; + + public File getSpecFile() + { + return specFile; + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java b/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java new file mode 100644 index 00000000000..e03e7875f94 --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java @@ -0,0 +1,71 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.metamx.common.logger.Logger; +import com.metamx.druid.query.QueryRunnerFactoryConglomerate; +import com.metamx.druid.realtime.FireDepartment; +import com.metamx.druid.realtime.RealtimeManager; + +import java.util.List; + +/** + */ +public class RealtimeManagerProvider implements Provider +{ + private static final Logger log = new Logger(RealtimeManagerProvider.class); + + private final QueryRunnerFactoryConglomerate conglomerate; + private final List fireDepartments = Lists.newArrayList(); + + @Inject + public RealtimeManagerProvider( + ObjectMapper jsonMapper, + RealtimeManagerConfig config, + QueryRunnerFactoryConglomerate conglomerate + ) + { + this.conglomerate = conglomerate; + try { + this.fireDepartments.addAll( + (List) jsonMapper.readValue( + config.getSpecFile(), new TypeReference>() + { + } + ) + ); + } + catch (Exception e) { + log.error(e, "Unable to read fireDepartments from config"); + } + } + + + @Override + public RealtimeManager get() + { + return new RealtimeManager(fireDepartments, conglomerate); + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java new file mode 100644 index 00000000000..633c85f81ba --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java @@ -0,0 +1,45 @@ +/* + * 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.common.logger.Logger; +import com.metamx.druid.realtime.DbSegmentPublisherConfig; +import com.metamx.druid.realtime.RealtimeManager; +import com.metamx.druid.realtime.SegmentPublisher; + +/** + */ +public class RealtimeModule implements Module +{ + private static final Logger log = new Logger(RealtimeModule.class); + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.publish", SegmentPublisherProvider.class); + JsonConfigProvider.bind(binder, "druid.db.tables", DbSegmentPublisherConfig.class); + binder.bind(SegmentPublisher.class).toProvider(SegmentPublisherProvider.class); + + JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class); + binder.bind(RealtimeManager.class).toProvider(RealtimeManagerProvider.class).in(ManageLifecycle.class); + } +} diff --git a/realtime/src/main/java/com/metamx/druid/guice/SegmentPublisherProvider.java b/realtime/src/main/java/com/metamx/druid/guice/SegmentPublisherProvider.java new file mode 100644 index 00000000000..832acac32ff --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/guice/SegmentPublisherProvider.java @@ -0,0 +1,35 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; +import com.metamx.druid.realtime.SegmentPublisher; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopSegmentPublisherProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "db", value = DbSegmentPublisherProvider.class) +}) +public interface SegmentPublisherProvider extends Provider +{ +} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisherConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisherConfig.java index 5dcaccac49b..dcab06ef0fd 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisherConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisherConfig.java @@ -1,9 +1,28 @@ +/* + * 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.realtime; import org.skife.config.Config; public abstract class DbSegmentPublisherConfig { - @Config("druid.database.segmentTable") + @Config("druid.db.tables.segmentTable") public abstract String getSegmentTable(); } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java index 591486335ea..cc2cf2c6553 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartment.java @@ -21,6 +21,7 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.inject.Inject; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.realtime.plumber.Plumber; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMain.java b/realtime/src/main/java/com/metamx/druid/realtime/NoopSegmentPublisher.java similarity index 57% rename from realtime/src/main/java/com/metamx/druid/realtime/RealtimeMain.java rename to realtime/src/main/java/com/metamx/druid/realtime/NoopSegmentPublisher.java index c81ab548a2e..6d51397e997 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMain.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/NoopSegmentPublisher.java @@ -19,34 +19,17 @@ package com.metamx.druid.realtime; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; +import com.metamx.druid.client.DataSegment; + +import java.io.IOException; /** */ -public class RealtimeMain +public class NoopSegmentPublisher implements SegmentPublisher { - private static final Logger log = new Logger(RealtimeMain.class); - - public static void main(String[] args) throws Exception + @Override + public void publishSegment(DataSegment segment) throws IOException { - LogLevelAdjuster.register(); - - Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance( - RealtimeNode.builder().build() - ); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - - lifecycle.join(); + // do nothing } -} \ No newline at end of file +} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java index 0874a900f42..4fafd14b98a 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.Closeables; +import com.google.inject.Inject; import com.metamx.common.exception.FormattedException; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -60,6 +61,7 @@ public class RealtimeManager implements QuerySegmentWalker private final Map chiefs; + @Inject public RealtimeManager( List fireDepartments, QueryRunnerFactoryConglomerate conglomerate diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java deleted file mode 100644 index 4357f39c2b5..00000000000 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ /dev/null @@ -1,298 +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.realtime; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.BeanProperty; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Preconditions; -import com.google.common.base.Suppliers; -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import com.metamx.common.ISE; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import com.metamx.druid.BaseServerNode; -import com.metamx.druid.db.DbConnector; -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.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.metrics.Monitor; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.skife.config.ConfigurationObjectFactory; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - */ -public class RealtimeNode extends BaseServerNode -{ - private static final Logger log = new Logger(RealtimeNode.class); - - public static Builder builder() - { - return new Builder(); - } - - private final Map injectablesMap = Maps.newLinkedHashMap(); - - private SegmentPublisher segmentPublisher = null; - private DataSegmentPusher dataSegmentPusher = null; - private List fireDepartments = null; - - private boolean initialized = false; - - public RealtimeNode( - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super("realtime", log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public RealtimeNode setSegmentPublisher(SegmentPublisher segmentPublisher) - { - checkFieldNotSetAndSet("segmentPublisher", segmentPublisher); - return this; - } - - public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) - { - checkFieldNotSetAndSet("dataSegmentPusher", dataSegmentPusher); - return this; - } - - public RealtimeNode setFireDepartments(List fireDepartments) - { - checkFieldNotSetAndSet("fireDepartments", fireDepartments); - return this; - } - - public RealtimeNode registerJacksonInjectable(String name, Object object) - { - Preconditions.checkState(injectablesMap.containsKey(name), "Already registered jackson object[%s]", name); - injectablesMap.put(name, object); - return this; - } - - public SegmentPublisher getSegmentPublisher() - { - initializeSegmentPublisher(); - return segmentPublisher; - } - - public DataSegmentPusher getDataSegmentPusher() - { - initializeSegmentPusher(); - return dataSegmentPusher; - } - - public List getFireDepartments() - { - initializeFireDepartments(); - return fireDepartments; - } - - protected void doInit() throws Exception - { - initializeJacksonInjectables(); - - final Lifecycle lifecycle = getLifecycle(); - final ServiceEmitter emitter = getEmitter(); - final QueryRunnerFactoryConglomerate conglomerate = getConglomerate(); - final List monitors = getMonitors(); - final List departments = getFireDepartments(); - - monitors.add(new RealtimeMetricsMonitor(departments)); - - final RealtimeManager realtimeManager = new RealtimeManager(departments, conglomerate); - lifecycle.addManagedInstance(realtimeManager); - - startMonitoring(monitors); - - final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet( - new ServletHolder( - new QueryServlet(getJsonMapper(), getSmileMapper(), realtimeManager, emitter, getRequestLogger()) - ), - "/druid/v2/*" - ); - - initialized = true; - } - - @LifecycleStart - public synchronized void start() throws Exception - { - if (! initialized) { - init(); - } - - getLifecycle().start(); - } - - @LifecycleStop - public synchronized void stop() - { - getLifecycle().stop(); - } - - protected void initializeJacksonInjectables() - { - final Map injectables = Maps.newHashMap(); - - for (Map.Entry entry : injectablesMap.entrySet()) { - injectables.put(entry.getKey(), entry.getValue()); - } - - injectables.put("queryRunnerFactoryConglomerate", getConglomerate()); - injectables.put("segmentPusher", getDataSegmentPusher()); - injectables.put("segmentAnnouncer", getAnnouncer()); - injectables.put("segmentPublisher", getSegmentPublisher()); - injectables.put("serverView", getServerView()); - injectables.put("serviceEmitter", getEmitter()); - - getJsonMapper().setInjectableValues( - new InjectableValues() - { - @Override - public Object findInjectableValue( - Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance - ) - { - return injectables.get(valueId); - } - } - ); - } - - private void initializeFireDepartments() - { - if (fireDepartments == null) { - try { - setFireDepartments( - getJsonMapper().>readValue( - new File(PropUtils.getProperty(getProps(), "druid.realtime.specFile")), - new TypeReference>(){} - ) - ); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - } - - private void initializeSegmentPusher() - { - if (dataSegmentPusher == null) { - dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper()); - } - } - - protected void initializeSegmentPublisher() - { - if (segmentPublisher == null) { - final DbSegmentPublisherConfig dbSegmentPublisherConfig = getConfigFactory().build(DbSegmentPublisherConfig.class); - segmentPublisher = new DbSegmentPublisher( - getJsonMapper(), - dbSegmentPublisherConfig, - new DbConnector(Suppliers.ofInstance(getConfigFactory().build(DbConnectorConfig.class)), null).getDBI() // TODO - ); - getLifecycle().addManagedInstance(segmentPublisher); - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMappers(ObjectMapper jsonMapper, ObjectMapper smileMapper) - { - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public RealtimeNode build() - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } - else if (jsonMapper == null || smileMapper == null) { - throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new RealtimeNode(props, lifecycle, jsonMapper, smileMapper, configFactory); - } - } -} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/Firehose.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/Firehose.java index d5a8a2bbdf0..434300031f8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/Firehose.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/Firehose.java @@ -26,7 +26,7 @@ import java.io.Closeable; /** * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement - * one of these and register it with the RealtimeMain. + * one of these and register it with the Main. * * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index 391fd305e21..aa26c735da7 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -68,6 +68,7 @@ import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import com.sun.istack.internal.NotNull; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -98,14 +99,31 @@ public class RealtimePlumberSchool implements PlumberSchool private final IndexGranularity segmentGranularity; private final Object handoffCondition = new Object(); + @JacksonInject + @NotNull private ServiceEmitter emitter; private volatile VersioningPolicy versioningPolicy = null; private volatile RejectionPolicyFactory rejectionPolicyFactory = null; + + @JacksonInject + @NotNull private volatile QueryRunnerFactoryConglomerate conglomerate = null; + + @JacksonInject + @NotNull private volatile DataSegmentPusher dataSegmentPusher = null; + + @JacksonInject + @NotNull private volatile DataSegmentAnnouncer segmentAnnouncer = null; + + @JacksonInject + @NotNull private volatile SegmentPublisher segmentPublisher = null; + + @JacksonInject + @NotNull private volatile ServerView serverView = null; @JsonCreator @@ -138,42 +156,6 @@ public class RealtimePlumberSchool implements PlumberSchool this.rejectionPolicyFactory = factory; } - @JacksonInject("queryRunnerFactoryConglomerate") - public void setConglomerate(QueryRunnerFactoryConglomerate conglomerate) - { - this.conglomerate = conglomerate; - } - - @JacksonInject("segmentPusher") - public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) - { - this.dataSegmentPusher = dataSegmentPusher; - } - - @JacksonInject("segmentAnnouncer") - public void setSegmentAnnouncer(DataSegmentAnnouncer segmentAnnouncer) - { - this.segmentAnnouncer = segmentAnnouncer; - } - - @JacksonInject("segmentPublisher") - public void setSegmentPublisher(SegmentPublisher segmentPublisher) - { - this.segmentPublisher = segmentPublisher; - } - - @JacksonInject("serverView") - public void setServerView(ServerView serverView) - { - this.serverView = serverView; - } - - @JacksonInject("serviceEmitter") - public void setServiceEmitter(ServiceEmitter emitter) - { - this.emitter = emitter; - } - @Override public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics) { diff --git a/server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java new file mode 100644 index 00000000000..6d28aff0f87 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.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.guice; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentPusher; + +import javax.validation.constraints.NotNull; + +/** + */ +public class CassandraDataSegmentPusherProvider implements DataSegmentPusherProvider +{ + @JacksonInject + @NotNull + private CassandraDataSegmentConfig config = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public DataSegmentPusher get() + { + return new CassandraDataSegmentPusher(config, jsonMapper); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java b/server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java new file mode 100644 index 00000000000..25a21ff3af2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java @@ -0,0 +1,83 @@ +/* + * 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.loading.DataSegmentPuller; +import com.metamx.druid.loading.HdfsDataSegmentPuller; +import com.metamx.druid.loading.LocalDataSegmentPuller; +import com.metamx.druid.loading.OmniSegmentLoader; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoader; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; +import org.apache.hadoop.conf.Configuration; + +/** + */ +public class DataSegmentPullerModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(SegmentLoader.class).to(OmniSegmentLoader.class).in(LazySingleton.class); + + bindDeepStorageLocal(binder); + bindDeepStorageS3(binder); + bindDeepStorageHdfs(binder); + bindDeepStorageCassandra(binder); + } + + private static void bindDeepStorageLocal(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); + } + + private static void bindDeepStorageS3(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + } + + private static void bindDeepStorageHdfs(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + binder.bind(Configuration.class).toInstance(new Configuration()); + } + + private static void bindDeepStorageCassandra(Binder binder) + { + final MapBinder segmentPullerBinder = MapBinder.newMapBinder( + binder, String.class, DataSegmentPuller.class + ); + segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); + ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java new file mode 100644 index 00000000000..7368eaf7ca1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java @@ -0,0 +1,27 @@ +package com.metamx.druid.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.HdfsDataSegmentPusherConfig; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; +import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; +import org.apache.hadoop.conf.Configuration; + +/** + */ +public class DataSegmentPusherModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.pusher", DataSegmentPusherProvider.class); + JsonConfigProvider.bind(binder, "druid.pusher.local", LocalDataSegmentPusherConfig.class); + JsonConfigProvider.bind(binder, "druid.pusher.s3", S3DataSegmentPusherConfig.class); + binder.bind(Configuration.class).toInstance(new Configuration()); + JsonConfigProvider.bind(binder, "druid.pusher.hdfs", HdfsDataSegmentPusherConfig.class); + JsonConfigProvider.bind(binder, "druid.pusher.cassandra", CassandraDataSegmentConfig.class); + binder.bind(DataSegmentPusher.class).toProvider(DataSegmentPusherProvider.class); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java new file mode 100644 index 00000000000..f69c413663e --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java @@ -0,0 +1,37 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; +import com.metamx.druid.loading.DataSegmentPusher; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LocalDataSegmentPusherProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "s3_zip", value = S3DataSegmentPusherProvider.class), + @JsonSubTypes.Type(name = "hdfs", value = HdfsDataSegmentPusherProvider.class), + @JsonSubTypes.Type(name = "c*", value = CassandraDataSegmentPusherProvider.class) +}) +public interface DataSegmentPusherProvider extends Provider +{ +} diff --git a/server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java b/server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java new file mode 100644 index 00000000000..27a8d069829 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java @@ -0,0 +1,136 @@ +/* + * 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.guice; + +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.ProvisionException; +import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.metamx.common.logger.Logger; +import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.collect.StupidPool; +import com.metamx.druid.concurrent.Execs; +import com.metamx.druid.guice.annotations.Global; +import com.metamx.druid.guice.annotations.Processing; +import com.metamx.druid.query.MetricsEmittingExecutorService; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; + +import java.lang.reflect.InvocationTargetException; +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicLong; + +/** + */ +public class DruidProcessingModule implements Module +{ + private static final Logger log = new Logger(DruidProcessingModule.class); + + @Override + public void configure(Binder binder) + { + ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); + binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); + } + + @Provides + @Processing + @ManageLifecycle + public ExecutorService getProcessingExecutorService(ExecutorServiceConfig config, ServiceEmitter emitter) + { + return new MetricsEmittingExecutorService( + Executors.newFixedThreadPool(config.getNumThreads(), Execs.makeThreadFactory(config.getFormatString())), + emitter, + new ServiceMetricEvent.Builder() + ); + } + + @Provides + @LazySingleton + @Global + public StupidPool getIntermediateResultsPool(DruidProcessingConfig config) + { + try { + Class vmClass = Class.forName("sun.misc.VM"); + Object maxDirectMemoryObj = vmClass.getMethod("maxDirectMemory").invoke(null); + + if (maxDirectMemoryObj == null || !(maxDirectMemoryObj instanceof Number)) { + log.info("Cannot determine maxDirectMemory from[%s]", maxDirectMemoryObj); + } else { + long maxDirectMemory = ((Number) maxDirectMemoryObj).longValue(); + + final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * (config.getNumThreads() + 1); + if (maxDirectMemory < memoryNeeded) { + throw new ProvisionException( + String.format( + "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize or druid.computation.buffer.size: " + + "maxDirectMemory[%,d], memoryNeeded[%,d], druid.computation.buffer.size[%,d], numThreads[%,d]", + maxDirectMemory, memoryNeeded, config.intermediateComputeSizeBytes(), config.getNumThreads() + ) + ); + } + } + } + catch (ClassNotFoundException e) { + log.info("No VM class, cannot do memory check."); + } + catch (NoSuchMethodException e) { + log.info("VM.maxDirectMemory doesn't exist, cannot do memory check."); + } + catch (InvocationTargetException e) { + log.warn(e, "static method shouldn't throw this"); + } + catch (IllegalAccessException e) { + log.warn(e, "public method, shouldn't throw this"); + } + + return new IntermediateProcessingBufferPool(config.intermediateComputeSizeBytes()); + } + + private static class IntermediateProcessingBufferPool extends StupidPool + { + private static final Logger log = new Logger(IntermediateProcessingBufferPool.class); + + public IntermediateProcessingBufferPool(final int computationBufferSize) + { + super( + new Supplier() + { + final AtomicLong count = new AtomicLong(0); + + @Override + public ByteBuffer get() + { + log.info( + "Allocating new intermediate processing buffer[%,d] of size[%,d]", + count.getAndIncrement(), computationBufferSize + ); + return ByteBuffer.allocateDirect(computationBufferSize); + } + } + ); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java new file mode 100644 index 00000000000..837b41ce1e1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java @@ -0,0 +1,52 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.HdfsDataSegmentPusher; +import com.metamx.druid.loading.HdfsDataSegmentPusherConfig; +import org.apache.hadoop.conf.Configuration; + +import javax.validation.constraints.NotNull; + +/** + */ +public class HdfsDataSegmentPusherProvider implements DataSegmentPusherProvider +{ + @JacksonInject + @NotNull + private HdfsDataSegmentPusherConfig hdfsDataSegmentPusherConfig = null; + + @JacksonInject + @NotNull + private Configuration config = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public DataSegmentPusher get() + { + return new HdfsDataSegmentPusher(hdfsDataSegmentPusherConfig, config, jsonMapper); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java index c96b15150e8..56526a5f286 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java @@ -1,59 +1,29 @@ +/* + * 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.guice; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Module; -import com.google.inject.Provides; -import com.google.inject.ProvisionException; -import com.google.inject.multibindings.MapBinder; -import com.metamx.common.concurrent.ExecutorServiceConfig; import com.metamx.common.logger.Logger; -import com.metamx.druid.DruidProcessingConfig; -import com.metamx.druid.client.DruidServerConfig; -import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; -import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.coordination.DataSegmentAnnouncerProvider; -import com.metamx.druid.coordination.DruidServerMetadata; import com.metamx.druid.coordination.ServerManager; -import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; import com.metamx.druid.coordination.ZkCoordinator; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.guice.annotations.Global; -import com.metamx.druid.guice.annotations.Processing; -import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.loading.DataSegmentPuller; -import com.metamx.druid.loading.HdfsDataSegmentPuller; -import com.metamx.druid.loading.LocalDataSegmentPuller; -import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.OmniSegmentLoader; -import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.S3CredentialsConfig; -import com.metamx.druid.loading.S3DataSegmentPuller; -import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.SegmentLoaderConfig; -import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; -import com.metamx.druid.loading.cassandra.CassandraDataSegmentPuller; -import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; -import com.metamx.druid.query.MetricsEmittingExecutorService; -import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import org.apache.curator.framework.CuratorFramework; -import org.apache.hadoop.conf.Configuration; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; - -import java.lang.reflect.InvocationTargetException; -import java.nio.ByteBuffer; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicLong; /** */ @@ -64,161 +34,7 @@ public class HistoricalModule implements Module @Override public void configure(Binder binder) { - ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); - binder.bind(ExecutorServiceConfig.class).to(DruidProcessingConfig.class); - - JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class); - JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); - binder.bind(ServerManager.class).in(LazySingleton.class); - - binder.bind(SegmentLoader.class).to(OmniSegmentLoader.class).in(LazySingleton.class); - binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); - - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); - - bindDeepStorageS3(binder); - bindDeepStorageHdfs(binder); - bindDeepStorageCassandra(binder); - - binder.bind(QueryRunnerFactoryConglomerate.class) - .to(DefaultQueryRunnerFactoryConglomerate.class) - .in(LazySingleton.class); - binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); - - JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); - JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); - binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); - binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); - binder.bind(SingleDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); - } - - @Provides @LazySingleton - public DruidServerMetadata getMetadata(@Self DruidNode node, DruidServerConfig config) - { - return new DruidServerMetadata(node.getHost(), node.getHost(), config.getMaxSize(), "historical", config.getTier()); - } - - @Provides @ManageLifecycle - public Announcer getAnnouncer(CuratorFramework curator) - { - return new Announcer(curator, Execs.singleThreaded("Announcer-%s")); - } - - @Provides @Processing @ManageLifecycle - public ExecutorService getProcessingExecutorService(ExecutorServiceConfig config, ServiceEmitter emitter) - { - return new MetricsEmittingExecutorService( - Executors.newFixedThreadPool(config.getNumThreads(), Execs.makeThreadFactory(config.getFormatString())), - emitter, - new ServiceMetricEvent.Builder() - ); - } - - @Provides @LazySingleton - public RestS3Service getRestS3Service(S3CredentialsConfig config) - { - try { - return new RestS3Service(new AWSCredentials(config.getAccessKey(), config.getSecretKey())); - } - catch (S3ServiceException e) { - throw new ProvisionException("Unable to create a RestS3Service", e); - } - } - - @Provides @LazySingleton @Global - public StupidPool getIntermediateResultsPool(DruidProcessingConfig config) - { - try { - Class vmClass = Class.forName("sun.misc.VM"); - Object maxDirectMemoryObj = vmClass.getMethod("maxDirectMemory").invoke(null); - - if (maxDirectMemoryObj == null || !(maxDirectMemoryObj instanceof Number)) { - log.info("Cannot determine maxDirectMemory from[%s]", maxDirectMemoryObj); - } else { - long maxDirectMemory = ((Number) maxDirectMemoryObj).longValue(); - - final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * (config.getNumThreads() + 1); - if (maxDirectMemory < memoryNeeded) { - throw new ProvisionException( - String.format( - "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize or druid.computation.buffer.size: " - + "maxDirectMemory[%,d], memoryNeeded[%,d], druid.computation.buffer.size[%,d], numThreads[%,d]", - maxDirectMemory, memoryNeeded, config.intermediateComputeSizeBytes(), config.getNumThreads() - ) - ); - } - } - } - catch (ClassNotFoundException e) { - log.info("No VM class, cannot do memory check."); - } - catch (NoSuchMethodException e) { - log.info("VM.maxDirectMemory doesn't exist, cannot do memory check."); - } - catch (InvocationTargetException e) { - log.warn(e, "static method shouldn't throw this"); - } - catch (IllegalAccessException e) { - log.warn(e, "public method, shouldn't throw this"); - } - - return new IntermediateProcessingBufferPool(config.intermediateComputeSizeBytes()); - } - - private static void bindDeepStorageS3(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); - } - - private static void bindDeepStorageHdfs(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); - binder.bind(Configuration.class).toInstance(new Configuration()); - } - - private static void bindDeepStorageCassandra(Binder binder) - { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); - ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); - } - - private static class IntermediateProcessingBufferPool extends StupidPool - { - private static final Logger log = new Logger(IntermediateProcessingBufferPool.class); - - public IntermediateProcessingBufferPool(final int computationBufferSize) - { - super( - new Supplier() - { - final AtomicLong count = new AtomicLong(0); - - @Override - public ByteBuffer get() - { - log.info( - "Allocating new intermediate processing buffer[%,d] of size[%,d]", - count.getAndIncrement(), computationBufferSize - ); - return ByteBuffer.allocateDirect(computationBufferSize); - } - } - ); - } } } diff --git a/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java new file mode 100644 index 00000000000..5e9578f066c --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.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.guice; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; + +import javax.validation.constraints.NotNull; + +/** + */ +public class LocalDataSegmentPusherProvider implements DataSegmentPusherProvider +{ + @JacksonInject + @NotNull + private LocalDataSegmentPusherConfig config = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public DataSegmentPusher get() + { + return new LocalDataSegmentPusher(config, jsonMapper); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java new file mode 100644 index 00000000000..b86fecfdf7a --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java @@ -0,0 +1,52 @@ +/* + * 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.guice; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import javax.validation.constraints.NotNull; + +/** + */ +public class S3DataSegmentPusherProvider implements DataSegmentPusherProvider +{ + @JacksonInject + @NotNull + private RestS3Service restS3Service = null; + + @JacksonInject + @NotNull + private S3DataSegmentPusherConfig config = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public DataSegmentPusher get() + { + return new S3DataSegmentPusher(restS3Service, config, jsonMapper); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/S3Module.java b/server/src/main/java/com/metamx/druid/guice/S3Module.java new file mode 100644 index 00000000000..e60827285a3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/S3Module.java @@ -0,0 +1,52 @@ +/* + * 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.ProvisionException; +import com.metamx.druid.loading.S3CredentialsConfig; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; + +/** + */ +public class S3Module implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); + } + + @Provides + @LazySingleton + public RestS3Service getRestS3Service(S3CredentialsConfig config) + { + try { + return new RestS3Service(new AWSCredentials(config.getAccessKey(), config.getSecretKey())); + } + catch (S3ServiceException e) { + throw new ProvisionException("Unable to create a RestS3Service", e); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/com/metamx/druid/guice/ServerModule.java index 74d0605e5f9..400f4ad040b 100644 --- a/server/src/main/java/com/metamx/druid/guice/ServerModule.java +++ b/server/src/main/java/com/metamx/druid/guice/ServerModule.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.guice; import com.google.inject.Binder; diff --git a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java new file mode 100644 index 00000000000..c2a5200915e --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java @@ -0,0 +1,65 @@ +/* + * 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.metamx.druid.client.DruidServerConfig; +import com.metamx.druid.coordination.DruidServerMetadata; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.initialization.DruidNode; +import com.metamx.druid.loading.MMappedQueryableIndexFactory; +import com.metamx.druid.loading.QueryableIndexFactory; +import com.metamx.druid.loading.SegmentLoaderConfig; +import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; +import com.metamx.druid.query.QueryRunnerFactoryConglomerate; + +/** + */ +public class StorageNodeModule extends ServerModule +{ + @Override + public void configure(Binder binder) + { + super.configure(binder); + + JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class); + JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); + + binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); + + binder.bind(QueryRunnerFactoryConglomerate.class) + .to(DefaultQueryRunnerFactoryConglomerate.class) + .in(LazySingleton.class); + } + + @Provides + @LazySingleton + public DruidServerMetadata getMetadata(@Self DruidNode node, DruidServerConfig config) + { + return new DruidServerMetadata( + node.getHost(), + node.getHost(), + config.getMaxSize(), + config.getType(), + config.getTier() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java index e232bbb8f08..e68a93f5282 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.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.loading; import com.fasterxml.jackson.databind.ObjectMapper; @@ -5,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.common.io.OutputSupplier; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; @@ -28,6 +48,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher private final Configuration hadoopConfig; private final ObjectMapper jsonMapper; + @Inject public HdfsDataSegmentPusher( HdfsDataSegmentPusherConfig config, Configuration hadoopConfig, diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusherConfig.java index b27d03672bc..61061f7ecbf 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusherConfig.java @@ -19,12 +19,17 @@ package com.metamx.druid.loading; -import org.skife.config.Config; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ public abstract class HdfsDataSegmentPusherConfig { - @Config("druid.pusher.hdfs.storageDirectory") - public abstract String getStorageDirectory(); + @JsonProperty + public String storageDirectory = ""; + + public String getStorageDirectory() + { + return storageDirectory; + } } diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java index 1493b162572..ed4ab5472c9 100644 --- a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; @@ -40,6 +41,7 @@ public class LocalDataSegmentPusher implements DataSegmentPusher private final LocalDataSegmentPusherConfig config; private final ObjectMapper jsonMapper; + @Inject public LocalDataSegmentPusher( LocalDataSegmentPusherConfig config, ObjectMapper jsonMapper diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java index d33a9a5130b..7addab21ce5 100644 --- a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java @@ -19,14 +19,20 @@ package com.metamx.druid.loading; +import com.fasterxml.jackson.annotation.JsonProperty; import org.skife.config.Config; import java.io.File; /** */ -public abstract class LocalDataSegmentPusherConfig +public class LocalDataSegmentPusherConfig { - @Config("druid.pusher.local.storageDirectory") - public abstract File getStorageDirectory(); + @JsonProperty + public File storageDirectory; + + public File getStorageDirectory() + { + return storageDirectory; + } } 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 d9ac69e5c5b..f70055456af 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -24,6 +24,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; +import com.google.inject.Inject; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.utils.CompressionUtils; @@ -46,6 +47,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; + @Inject public S3DataSegmentPusher( RestS3Service s3Client, S3DataSegmentPusherConfig config, diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index 3fbbe2d311f..70fa23c4278 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -19,21 +19,35 @@ package com.metamx.druid.loading; +import com.fasterxml.jackson.annotation.JsonProperty; import org.skife.config.Config; import org.skife.config.Default; /** */ -public abstract class S3DataSegmentPusherConfig +public class S3DataSegmentPusherConfig { - @Config("druid.pusher.s3.bucket") - public abstract String getBucket(); + @JsonProperty + public String bucket = ""; - @Config("druid.pusher.s3.baseKey") - @Default("") - public abstract String getBaseKey(); + @JsonProperty + public String baseKey = ""; - @Config("druid.pusher.s3.disableAcl") - @Default("false") - public abstract boolean getDisableAcl(); + @JsonProperty + public boolean disableAcl = false; + + public String getBucket() + { + return bucket; + } + + public String getBaseKey() + { + return baseKey; + } + + public boolean getDisableAcl() + { + return disableAcl; + } } diff --git a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentConfig.java b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentConfig.java index 6e7976f824d..e392cc4bb0d 100644 --- a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentConfig.java @@ -19,8 +19,7 @@ package com.metamx.druid.loading.cassandra; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; /** * Cassandra Config @@ -29,11 +28,19 @@ import org.skife.config.Default; */ public abstract class CassandraDataSegmentConfig { - @Config("druid.pusher.cassandra.host") - @Default("") - public abstract String getHost(); + @JsonProperty + public String host = ""; - @Config("druid.pusher.cassandra.keyspace") - @Default("") - public abstract String getKeyspace(); + @JsonProperty + public String keyspace = ""; + + public String getKeyspace() + { + return keyspace; + } + + public String getHost() + { + return host; + } } diff --git a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java index 57bc72b9124..63483075db9 100644 --- a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java @@ -1,12 +1,28 @@ -package com.metamx.druid.loading.cassandra; +/* + * 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. + */ -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; +package com.metamx.druid.loading.cassandra; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; @@ -16,6 +32,10 @@ import com.metamx.druid.utils.CompressionUtils; import com.netflix.astyanax.MutationBatch; import com.netflix.astyanax.recipes.storage.ChunkedStorage; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; + /** * Cassandra Segment Pusher * @@ -28,6 +48,7 @@ public class CassandraDataSegmentPusher extends CassandraStorage implements Data private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final ObjectMapper jsonMapper; + @Inject public CassandraDataSegmentPusher( CassandraDataSegmentConfig config, ObjectMapper jsonMapper) diff --git a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java index 72f8feea6ca..4061eb7b4ba 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java +++ b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.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.metrics; import com.fasterxml.jackson.annotation.JsonProperty; @@ -13,7 +32,7 @@ import java.util.List; */ public class MonitorsConfig { - @JsonProperty("monitorExclusions") + @JsonProperty("monitors") @NotNull private List> monitors = ImmutableList.>builder() .add(JvmMonitor.class) diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index d5e00e4c7ff..14339320320 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -1,7 +1,27 @@ +/* + * 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 io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.cache.CacheMonitor; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.guice.BrokerModule; import com.metamx.druid.guice.HttpClientModule; @@ -42,7 +62,7 @@ public class CliBroker extends ServerRunnable EmitterModule.class, HttpClientModule.global(), CuratorModule.class, - new MetricsModule(), + new MetricsModule().register(CacheMonitor.class), ServerModule.class, new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 01bb7484198..a79c03a7aa3 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.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 io.druid.cli; import com.google.inject.Injector; @@ -5,12 +24,18 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.AnnouncerModule; +import com.metamx.druid.guice.DataSegmentPullerModule; +import com.metamx.druid.guice.DataSegmentPusherModule; +import com.metamx.druid.guice.DruidProcessingModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.S3Module; import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; @@ -42,8 +67,12 @@ public class CliHistorical extends ServerRunnable EmitterModule.class, HttpClientModule.global(), CuratorModule.class, + AnnouncerModule.class, + DruidProcessingModule.class, + S3Module.class, + DataSegmentPullerModule.class, new MetricsModule().register(ServerMonitor.class), - ServerModule.class, + StorageNodeModule.class, new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ServerManager.class), diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java new file mode 100644 index 00000000000..be3f8de0191 --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -0,0 +1,86 @@ +/* + * 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 io.druid.cli; + +import com.google.inject.Injector; +import com.metamx.common.logger.Logger; +import com.metamx.druid.DruidProcessingConfig; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.AnnouncerModule; +import com.metamx.druid.guice.DataSegmentPusherModule; +import com.metamx.druid.guice.DbConnectorModule; +import com.metamx.druid.guice.DruidProcessingModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.QueryRunnerFactoryModule; +import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.RealtimeModule; +import com.metamx.druid.guice.S3Module; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.ServerViewModule; +import com.metamx.druid.guice.StorageNodeModule; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.metrics.MetricsModule; +import com.metamx.druid.realtime.RealtimeManager; +import io.airlift.command.Command; + +/** + */ +@Command( + name = "realtime", + description = "Runs a realtime node, see https://github.com/metamx/druid/wiki/Realtime for a description" +) +public class CliRealtime extends ServerRunnable +{ + private static final Logger log = new Logger(CliBroker.class); + + public CliRealtime() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule(), + EmitterModule.class, + DbConnectorModule.class, + HttpClientModule.global(), + CuratorModule.class, + AnnouncerModule.class, + DruidProcessingModule.class, + S3Module.class, + DataSegmentPusherModule.class, + new MetricsModule(), + StorageNodeModule.class, + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class), + new ServerViewModule(), + new QueryableModule(RealtimeManager.class), + new QueryRunnerFactoryModule(), + RealtimeModule.class + ); + } +} diff --git a/services/src/main/java/io/druid/cli/CliRealtimeStandalone.java b/services/src/main/java/io/druid/cli/CliRealtimeStandalone.java new file mode 100644 index 00000000000..39b39a25ce1 --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliRealtimeStandalone.java @@ -0,0 +1,67 @@ +/* + * 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 io.druid.cli; + +import com.google.inject.Injector; +import com.metamx.common.logger.Logger; +import com.metamx.druid.guice.DruidProcessingModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.QueryRunnerFactoryModule; +import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.StorageNodeModule; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.realtime.RealtimeManager; +import druid.examples.guice.RealtimeStandaloneModule; +import io.airlift.command.Command; + +/** + */ +@Command( + name = "realtimeStandalone", + description = "Runs a standalone realtime node for examples, see https://github.com/metamx/druid/wiki/Realtime for a description" +) +public class CliRealtimeStandalone extends ServerRunnable +{ + private static final Logger log = new Logger(CliBroker.class); + + public CliRealtimeStandalone() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule(), + EmitterModule.class, + DruidProcessingModule.class, + StorageNodeModule.class, + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class), + new QueryableModule(RealtimeManager.class), + new QueryRunnerFactoryModule(), + RealtimeStandaloneModule.class + ); + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 594deb05af5..a261071394a 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.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 io.druid.cli; import io.airlift.command.Cli; @@ -19,7 +38,7 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class); + .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliRealtimeStandalone.class); builder.build().parse(args).run(); } From 7ebe053ac1abce6e3b218beaee801ebbc6da2ecb Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 14 Aug 2013 09:37:22 -0700 Subject: [PATCH 31/92] Fix bug with segment loader counting thingie --- .../main/java/com/metamx/druid/loading/OmniSegmentLoader.java | 2 +- .../main/java/com/metamx/druid/loading/SingleSegmentLoader.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java index 38b811ca179..ffe419246ed 100644 --- a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java @@ -211,7 +211,7 @@ public class OmniSegmentLoader implements SegmentLoader private synchronized void addSegment(DataSegment segment) { - if (! segments.add(segment)) { + if (segments.add(segment)) { currSize += segment.getSize(); } } diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index a9fe23d1af6..e7aa60d4bff 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -210,7 +210,7 @@ public class SingleSegmentLoader implements SegmentLoader private synchronized void addSegment(DataSegment segment) { - if (! segments.add(segment)) { + if (segments.add(segment)) { currSize += segment.getSize(); } } From b95d9498ca59300da02ae7b5c15977d023aa6864 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 21 Aug 2013 13:37:56 -0700 Subject: [PATCH 32/92] add version information to status --- .../com/metamx/druid/http/StatusResource.java | 98 ++++++++++++++++--- 1 file changed, 85 insertions(+), 13 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/http/StatusResource.java b/client/src/main/java/com/metamx/druid/http/StatusResource.java index 2c4aed29614..ebe2f0e855a 100644 --- a/client/src/main/java/com/metamx/druid/http/StatusResource.java +++ b/client/src/main/java/com/metamx/druid/http/StatusResource.java @@ -1,9 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2013 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.http; +import com.fasterxml.jackson.annotation.JsonProperty; + import javax.ws.rs.GET; import javax.ws.rs.Path; import javax.ws.rs.Produces; -import javax.ws.rs.core.Response; /** */ @@ -11,21 +31,73 @@ import javax.ws.rs.core.Response; public class StatusResource { @GET - @Produces("text/plain") - public Response doGet() + @Produces("application/json") + public Status doGet() { - StringBuffer buf = new StringBuffer(); + return new Status( + StatusResource.class.getPackage().getImplementationVersion(), + new Memory(Runtime.getRuntime()) + ); + } - Runtime runtime = Runtime.getRuntime(); - long maxMemory = runtime.maxMemory(); - long totalMemory = runtime.totalMemory(); - long freeMemory = runtime.freeMemory(); + public static class Status { + final String version; + final Memory memory; - buf.append(String.format("Max Memory:\t%,18d\t%1$d%n", maxMemory)); - buf.append(String.format("Total Memory:\t%,18d\t%1$d%n", totalMemory)); - buf.append(String.format("Free Memory:\t%,18d\t%1$d%n", freeMemory)); - buf.append(String.format("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory)); + public Status(String version, Memory memory) + { + this.version = version; + this.memory = memory; + } - return Response.ok(buf.toString()).build(); + @JsonProperty + public String getVersion() + { + return version; + } + + @JsonProperty + public Memory getMemory() + { + return memory; + } + } + + public static class Memory { + final long maxMemory; + final long totalMemory; + final long freeMemory; + final long usedMemory; + + public Memory(Runtime runtime) { + maxMemory = runtime.maxMemory(); + totalMemory = runtime.totalMemory(); + freeMemory = runtime.freeMemory(); + usedMemory = totalMemory - freeMemory; + } + + @JsonProperty + public long getMaxMemory() + { + return maxMemory; + } + + @JsonProperty + public long getTotalMemory() + { + return totalMemory; + } + + @JsonProperty + public long getFreeMemory() + { + return freeMemory; + } + + @JsonProperty + public long getUsedMemory() + { + return usedMemory; + } } } From d5b3f7103359587ad60b5f71b31454f6e6dc71ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 21 Aug 2013 13:45:39 -0700 Subject: [PATCH 33/92] replace StatusServlet with StatusResource --- .../com/metamx/druid/http/StatusServlet.java | 57 ------------------- .../http/IndexerCoordinatorNode.java | 3 +- .../http/IndexerCoordinatorServletModule.java | 2 + .../worker/executor/ExecutorNode.java | 3 +- .../executor/ExecutorServletModule.java | 2 + .../indexing/worker/http/WorkerNode.java | 3 +- .../worker/http/WorkerServletModule.java | 2 + .../metamx/druid/realtime/RealtimeNode.java | 12 +++- .../druid/realtime/RealtimeServletModule.java | 37 ++++++++++++ 9 files changed, 56 insertions(+), 65 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/http/StatusServlet.java create mode 100644 realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java diff --git a/client/src/main/java/com/metamx/druid/http/StatusServlet.java b/client/src/main/java/com/metamx/druid/http/StatusServlet.java deleted file mode 100644 index 95a40c543b8..00000000000 --- a/client/src/main/java/com/metamx/druid/http/StatusServlet.java +++ /dev/null @@ -1,57 +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.http; - -import javax.servlet.ServletException; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; - -/** - */ -public class StatusServlet extends HttpServlet -{ - @Override - protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException - { - ByteArrayOutputStream retVal = new ByteArrayOutputStream(); - PrintWriter out = new PrintWriter(new OutputStreamWriter(retVal)); - - Runtime runtime = Runtime.getRuntime(); - long maxMemory = runtime.maxMemory(); - long totalMemory = runtime.totalMemory(); - long freeMemory = runtime.freeMemory(); - - out.printf("Max Memory:\t%,18d\t%1$d%n", maxMemory); - out.printf("Total Memory:\t%,18d\t%1$d%n", totalMemory); - out.printf("Free Memory:\t%,18d\t%1$d%n", freeMemory); - out.printf("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory); - - out.flush(); - - resp.setStatus(HttpServletResponse.SC_OK); - resp.setContentType("text/plain"); - resp.getOutputStream().write(retVal.toByteArray()); - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 61191da91e5..bf278590f48 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -56,7 +56,6 @@ import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; -import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionToolbox; @@ -321,7 +320,6 @@ public class IndexerCoordinatorNode extends QueryableNode ); final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); + root.addFilter(GuiceFilter.class, "/status", null); root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); root.addServlet( new ServletHolder( diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java index 1350094420c..d10d136d9c2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java @@ -3,6 +3,7 @@ package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; +import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -26,6 +27,7 @@ public class ExecutorServletModule extends JerseyServletModule @Override protected void configureServlets() { + bind(StatusResource.class); bind(ChatHandlerResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(ChatHandlerProvider.class).toInstance(receivers); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 90890ca7a37..767823301be 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -36,7 +36,6 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.QueryableNode; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.http.GuiceServletConfig; -import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; @@ -205,9 +204,9 @@ public class WorkerNode extends QueryableNode ); final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); + root.addFilter(GuiceFilter.class, "/status", null); root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java index b135af7dcb6..1fec950af48 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java @@ -22,6 +22,7 @@ package com.metamx.druid.indexing.worker.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; +import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; @@ -51,6 +52,7 @@ public class WorkerServletModule extends JerseyServletModule @Override protected void configureServlets() { + bind(StatusResource.class); bind(WorkerResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(ServiceEmitter.class).toInstance(emitter); 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 4357f39c2b5..efc599db89d 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -29,6 +29,9 @@ import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.Maps; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; import com.metamx.common.ISE; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; @@ -38,8 +41,8 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.BaseServerNode; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.http.GuiceServletConfig; 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; @@ -148,8 +151,13 @@ public class RealtimeNode extends BaseServerNode startMonitoring(monitors); + final Injector injector = Guice.createInjector( + new RealtimeServletModule(getJsonMapper()) + ); + final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); + root.addEventListener(new GuiceServletConfig(injector)); + root.addFilter(GuiceFilter.class, "/status", null); root.addServlet( new ServletHolder( new QueryServlet(getJsonMapper(), getSmileMapper(), realtimeManager, emitter, getRequestLogger()) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java new file mode 100644 index 00000000000..4419e69b0fe --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java @@ -0,0 +1,37 @@ +package com.metamx.druid.realtime; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; +import com.google.inject.Provides; +import com.metamx.druid.http.StatusResource; +import com.sun.jersey.guice.JerseyServletModule; +import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; + +import javax.inject.Singleton; + +public class RealtimeServletModule extends JerseyServletModule +{ + private final ObjectMapper jsonMapper; + + public RealtimeServletModule(ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + protected void configureServlets() + { + bind(StatusResource.class); + + serve("/*").with(GuiceContainer.class); + } + + @Provides + @Singleton + public JacksonJsonProvider getJacksonJsonProvider() + { + final JacksonJsonProvider provider = new JacksonJsonProvider(); + provider.setMapper(jsonMapper); + return provider; + } +} From 3adcaab1ef4bb221b9dbae86a3d8548b9d2f3a0b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 21 Aug 2013 13:58:02 -0700 Subject: [PATCH 34/92] Eric said I did too much work This reverts commit d5b3f7103359587ad60b5f71b31454f6e6dc71ba. --- .../com/metamx/druid/http/StatusServlet.java | 57 +++++++++++++++++++ .../http/IndexerCoordinatorNode.java | 3 +- .../http/IndexerCoordinatorServletModule.java | 2 - .../worker/executor/ExecutorNode.java | 3 +- .../executor/ExecutorServletModule.java | 2 - .../indexing/worker/http/WorkerNode.java | 3 +- .../worker/http/WorkerServletModule.java | 2 - .../metamx/druid/realtime/RealtimeNode.java | 12 +--- .../druid/realtime/RealtimeServletModule.java | 37 ------------ 9 files changed, 65 insertions(+), 56 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/http/StatusServlet.java delete mode 100644 realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java diff --git a/client/src/main/java/com/metamx/druid/http/StatusServlet.java b/client/src/main/java/com/metamx/druid/http/StatusServlet.java new file mode 100644 index 00000000000..95a40c543b8 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/http/StatusServlet.java @@ -0,0 +1,57 @@ +/* + * 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.http; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; + +/** + */ +public class StatusServlet extends HttpServlet +{ + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException + { + ByteArrayOutputStream retVal = new ByteArrayOutputStream(); + PrintWriter out = new PrintWriter(new OutputStreamWriter(retVal)); + + Runtime runtime = Runtime.getRuntime(); + long maxMemory = runtime.maxMemory(); + long totalMemory = runtime.totalMemory(); + long freeMemory = runtime.freeMemory(); + + out.printf("Max Memory:\t%,18d\t%1$d%n", maxMemory); + out.printf("Total Memory:\t%,18d\t%1$d%n", totalMemory); + out.printf("Free Memory:\t%,18d\t%1$d%n", freeMemory); + out.printf("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory); + + out.flush(); + + resp.setStatus(HttpServletResponse.SC_OK); + resp.setContentType("text/plain"); + resp.getOutputStream().write(retVal.toByteArray()); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index bf278590f48..61191da91e5 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -56,6 +56,7 @@ import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; +import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionToolbox; @@ -320,6 +321,7 @@ public class IndexerCoordinatorNode extends QueryableNode ); final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/status", null); root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); root.addServlet( new ServletHolder( diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java index d10d136d9c2..1350094420c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java @@ -3,7 +3,6 @@ package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; -import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -27,7 +26,6 @@ public class ExecutorServletModule extends JerseyServletModule @Override protected void configureServlets() { - bind(StatusResource.class); bind(ChatHandlerResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(ChatHandlerProvider.class).toInstance(receivers); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 767823301be..90890ca7a37 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -36,6 +36,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.QueryableNode; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.http.GuiceServletConfig; +import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; @@ -204,9 +205,9 @@ public class WorkerNode extends QueryableNode ); final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/status", null); root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java index 1fec950af48..b135af7dcb6 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java @@ -22,7 +22,6 @@ package com.metamx.druid.indexing.worker.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; -import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; @@ -52,7 +51,6 @@ public class WorkerServletModule extends JerseyServletModule @Override protected void configureServlets() { - bind(StatusResource.class); bind(WorkerResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(ServiceEmitter.class).toInstance(emitter); 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 efc599db89d..4357f39c2b5 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -29,9 +29,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; import com.metamx.common.ISE; import com.metamx.common.config.Config; import com.metamx.common.lifecycle.Lifecycle; @@ -41,8 +38,8 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.BaseServerNode; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; -import com.metamx.druid.http.GuiceServletConfig; 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; @@ -151,13 +148,8 @@ public class RealtimeNode extends BaseServerNode startMonitoring(monitors); - final Injector injector = Guice.createInjector( - new RealtimeServletModule(getJsonMapper()) - ); - final ServletContextHandler root = new ServletContextHandler(getServer(), "/", ServletContextHandler.SESSIONS); - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/status", null); + root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet( new ServletHolder( new QueryServlet(getJsonMapper(), getSmileMapper(), realtimeManager, emitter, getRequestLogger()) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java deleted file mode 100644 index 4419e69b0fe..00000000000 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeServletModule.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.metamx.druid.realtime; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import com.google.inject.Provides; -import com.metamx.druid.http.StatusResource; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -public class RealtimeServletModule extends JerseyServletModule -{ - private final ObjectMapper jsonMapper; - - public RealtimeServletModule(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - } - - @Override - protected void configureServlets() - { - bind(StatusResource.class); - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} From 6a8c160740c04487e13f17f30fab73daf05b7fbb Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 22 Aug 2013 10:46:05 -0700 Subject: [PATCH 35/92] update code according to code review --- .../druid/client/DruidServerConfig.java | 8 ---- ...Module.java => RealtimeExampleModule.java} | 46 +++++-------------- .../guice/DbSegmentPublisherProvider.java | 3 +- ...ider.java => FireDepartmentsProvider.java} | 21 +++------ .../metamx/druid/guice/RealtimeModule.java | 12 ++++- .../druid/realtime/DbSegmentPublisher.java | 11 +++-- .../druid/guice/DataSegmentPusherModule.java | 5 +- .../guice/LocalDataSegmentPusherProvider.java | 8 +--- .../metamx/druid/guice/StorageNodeModule.java | 9 +++- .../metamx/druid/metrics/MonitorsConfig.java | 9 +--- .../main/java/io/druid/cli/CliHistorical.java | 2 +- .../main/java/io/druid/cli/CliRealtime.java | 2 +- ...tandalone.java => CliRealtimeExample.java} | 12 ++--- services/src/main/java/io/druid/cli/Main.java | 2 +- 14 files changed, 61 insertions(+), 89 deletions(-) rename examples/src/main/java/druid/examples/guice/{RealtimeStandaloneModule.java => RealtimeExampleModule.java} (66%) rename realtime/src/main/java/com/metamx/druid/guice/{RealtimeManagerProvider.java => FireDepartmentsProvider.java} (69%) rename services/src/main/java/io/druid/cli/{CliRealtimeStandalone.java => CliRealtimeExample.java} (89%) diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 1dcf38c4dc0..6a57a38b20f 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -34,19 +34,11 @@ public class DruidServerConfig @JsonProperty private String tier = "_default_tier"; - @JsonProperty - private String type = "historical"; - public long getMaxSize() { return maxSize; } - public String getType() - { - return type; - } - public String getTier() { return tier; diff --git a/examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java similarity index 66% rename from examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java rename to examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java index 4013b6a236a..e8925381d58 100644 --- a/examples/src/main/java/druid/examples/guice/RealtimeStandaloneModule.java +++ b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java @@ -2,47 +2,24 @@ package druid.examples.guice; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Provides; -import com.google.inject.ProvisionException; -import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.google.inject.TypeLiteral; import com.metamx.common.logger.Logger; -import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.client.InventoryView; import com.metamx.druid.client.ServerView; -import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.concurrent.Execs; import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.FireDepartmentsProvider; import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.guice.NoopSegmentPublisherProvider; import com.metamx.druid.guice.RealtimeManagerConfig; -import com.metamx.druid.guice.RealtimeManagerProvider; -import com.metamx.druid.guice.annotations.Global; -import com.metamx.druid.guice.annotations.Processing; -import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.DruidModule; -import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.MMappedQueryableIndexFactory; -import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.SegmentLoaderConfig; -import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; -import com.metamx.druid.query.MetricsEmittingExecutorService; -import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.realtime.RealtimeManager; +import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.SegmentPublisher; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import druid.examples.flights.FlightsFirehoseFactory; import druid.examples.rand.RandomFirehoseFactory; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; @@ -50,20 +27,15 @@ import druid.examples.web.WebFirehoseFactory; import java.io.File; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicLong; /** */ -public class RealtimeStandaloneModule implements DruidModule +public class RealtimeExampleModule implements DruidModule { - private static final Logger log = new Logger(RealtimeStandaloneModule.class); + private static final Logger log = new Logger(RealtimeExampleModule.class); @Override public void configure(Binder binder) @@ -75,14 +47,18 @@ public class RealtimeStandaloneModule implements DruidModule binder.bind(ServerView.class).to(NoopServerView.class); JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class); - binder.bind(RealtimeManager.class).toProvider(RealtimeManagerProvider.class).in(ManageLifecycle.class); + binder.bind( + new TypeLiteral>() + { + } + ).toProvider(FireDepartmentsProvider.class).in(ManageLifecycle.class); } @Override public List getJacksonModules() { return Arrays.asList( - new SimpleModule("RealtimestandAloneModule") + new SimpleModule("RealtimeExampleModule") .registerSubtypes( new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"), new NamedType(FlightsFirehoseFactory.class, "flights"), diff --git a/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java b/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java index 3a8aa5f8731..5d6d105fc06 100644 --- a/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java +++ b/realtime/src/main/java/com/metamx/druid/guice/DbSegmentPublisherProvider.java @@ -21,6 +21,7 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.realtime.DbSegmentPublisher; import com.metamx.druid.realtime.DbSegmentPublisherConfig; import com.metamx.druid.realtime.SegmentPublisher; @@ -38,7 +39,7 @@ public class DbSegmentPublisherProvider implements SegmentPublisherProvider @JacksonInject @NotNull - private DbSegmentPublisherConfig config = null; + private DbTablesConfig config = null; @JacksonInject @NotNull diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java b/realtime/src/main/java/com/metamx/druid/guice/FireDepartmentsProvider.java similarity index 69% rename from realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java rename to realtime/src/main/java/com/metamx/druid/guice/FireDepartmentsProvider.java index e03e7875f94..5f2fd146bc3 100644 --- a/realtime/src/main/java/com/metamx/druid/guice/RealtimeManagerProvider.java +++ b/realtime/src/main/java/com/metamx/druid/guice/FireDepartmentsProvider.java @@ -21,33 +21,26 @@ package com.metamx.druid.guice; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.inject.Inject; import com.google.inject.Provider; -import com.metamx.common.logger.Logger; -import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.realtime.FireDepartment; -import com.metamx.druid.realtime.RealtimeManager; import java.util.List; /** */ -public class RealtimeManagerProvider implements Provider +public class FireDepartmentsProvider implements Provider> { - private static final Logger log = new Logger(RealtimeManagerProvider.class); - - private final QueryRunnerFactoryConglomerate conglomerate; private final List fireDepartments = Lists.newArrayList(); @Inject - public RealtimeManagerProvider( + public FireDepartmentsProvider( ObjectMapper jsonMapper, - RealtimeManagerConfig config, - QueryRunnerFactoryConglomerate conglomerate + RealtimeManagerConfig config ) { - this.conglomerate = conglomerate; try { this.fireDepartments.addAll( (List) jsonMapper.readValue( @@ -58,14 +51,14 @@ public class RealtimeManagerProvider implements Provider ); } catch (Exception e) { - log.error(e, "Unable to read fireDepartments from config"); + throw Throwables.propagate(e); } } @Override - public RealtimeManager get() + public List get() { - return new RealtimeManager(fireDepartments, conglomerate); + return fireDepartments; } } diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java index 633c85f81ba..864580f5405 100644 --- a/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java +++ b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java @@ -21,11 +21,15 @@ package com.metamx.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; +import com.google.inject.TypeLiteral; import com.metamx.common.logger.Logger; import com.metamx.druid.realtime.DbSegmentPublisherConfig; +import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.RealtimeManager; import com.metamx.druid.realtime.SegmentPublisher; +import java.util.List; + /** */ public class RealtimeModule implements Module @@ -36,10 +40,14 @@ public class RealtimeModule implements Module public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.publish", SegmentPublisherProvider.class); - JsonConfigProvider.bind(binder, "druid.db.tables", DbSegmentPublisherConfig.class); binder.bind(SegmentPublisher.class).toProvider(SegmentPublisherProvider.class); JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class); - binder.bind(RealtimeManager.class).toProvider(RealtimeManagerProvider.class).in(ManageLifecycle.class); + binder.bind( + new TypeLiteral>() + { + } + ).toProvider(FireDepartmentsProvider.class).in(ManageLifecycle.class); + binder.bind(RealtimeManager.class).in(ManageLifecycle.class); } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java index 9fb501eddd6..20e27c7dc06 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/DbSegmentPublisher.java @@ -3,6 +3,7 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.db.DbTablesConfig; import org.joda.time.DateTime; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; @@ -17,12 +18,12 @@ public class DbSegmentPublisher implements SegmentPublisher private static final Logger log = new Logger(DbSegmentPublisher.class); private final ObjectMapper jsonMapper; - private final DbSegmentPublisherConfig config; + private final DbTablesConfig config; private final IDBI dbi; public DbSegmentPublisher( ObjectMapper jsonMapper, - DbSegmentPublisherConfig config, + DbTablesConfig config, IDBI dbi ) { @@ -41,7 +42,7 @@ public class DbSegmentPublisher implements SegmentPublisher public List> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT id FROM %s WHERE id=:id", config.getSegmentTable()) + String.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable()) ) .bind("id", segment.getIdentifier()) .list(); @@ -65,13 +66,13 @@ public class DbSegmentPublisher implements SegmentPublisher statement = String.format( "INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - config.getSegmentTable() + config.getSegmentsTable() ); } else { statement = String.format( "INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) " + "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)", - config.getSegmentTable() + config.getSegmentsTable() ); } diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java index 7368eaf7ca1..bb95cb45273 100644 --- a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java +++ b/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java @@ -17,11 +17,14 @@ public class DataSegmentPusherModule implements Module public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.pusher", DataSegmentPusherProvider.class); - JsonConfigProvider.bind(binder, "druid.pusher.local", LocalDataSegmentPusherConfig.class); + JsonConfigProvider.bind(binder, "druid.pusher.s3", S3DataSegmentPusherConfig.class); binder.bind(Configuration.class).toInstance(new Configuration()); + JsonConfigProvider.bind(binder, "druid.pusher.hdfs", HdfsDataSegmentPusherConfig.class); + JsonConfigProvider.bind(binder, "druid.pusher.cassandra", CassandraDataSegmentConfig.class); + binder.bind(DataSegmentPusher.class).toProvider(DataSegmentPusherProvider.class); } } diff --git a/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java b/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java index 5e9578f066c..7a51b7264e1 100644 --- a/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java +++ b/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java @@ -29,12 +29,8 @@ import javax.validation.constraints.NotNull; /** */ -public class LocalDataSegmentPusherProvider implements DataSegmentPusherProvider +public class LocalDataSegmentPusherProvider extends LocalDataSegmentPusherConfig implements DataSegmentPusherProvider { - @JacksonInject - @NotNull - private LocalDataSegmentPusherConfig config = null; - @JacksonInject @NotNull private ObjectMapper jsonMapper = null; @@ -42,6 +38,6 @@ public class LocalDataSegmentPusherProvider implements DataSegmentPusherProvider @Override public DataSegmentPusher get() { - return new LocalDataSegmentPusher(config, jsonMapper); + return new LocalDataSegmentPusher(this, jsonMapper); } } diff --git a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java index c2a5200915e..72954cec734 100644 --- a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java @@ -35,6 +35,13 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; */ public class StorageNodeModule extends ServerModule { + private final String nodeType; + + public StorageNodeModule(String nodeType) + { + this.nodeType = nodeType; + } + @Override public void configure(Binder binder) { @@ -58,7 +65,7 @@ public class StorageNodeModule extends ServerModule node.getHost(), node.getHost(), config.getMaxSize(), - config.getType(), + nodeType, config.getTier() ); } diff --git a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java index 4061eb7b4ba..86c351b3a1d 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java +++ b/server/src/main/java/com/metamx/druid/metrics/MonitorsConfig.java @@ -20,10 +20,8 @@ package com.metamx.druid.metrics; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; -import com.metamx.metrics.JvmMonitor; +import com.google.common.collect.Lists; import com.metamx.metrics.Monitor; -import com.metamx.metrics.SysMonitor; import javax.validation.constraints.NotNull; import java.util.List; @@ -34,10 +32,7 @@ public class MonitorsConfig { @JsonProperty("monitors") @NotNull - private List> monitors = ImmutableList.>builder() - .add(JvmMonitor.class) - .add(SysMonitor.class) - .build(); + private List> monitors = Lists.newArrayList(); public List> getMonitors() { diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index a79c03a7aa3..04aca3c58ba 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -72,7 +72,7 @@ public class CliHistorical extends ServerRunnable S3Module.class, DataSegmentPullerModule.class, new MetricsModule().register(ServerMonitor.class), - StorageNodeModule.class, + new StorageNodeModule("historical"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(ServerManager.class), diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index be3f8de0191..2defc0a2ae2 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -74,7 +74,7 @@ public class CliRealtime extends ServerRunnable S3Module.class, DataSegmentPusherModule.class, new MetricsModule(), - StorageNodeModule.class, + new StorageNodeModule("realtime"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new ServerViewModule(), diff --git a/services/src/main/java/io/druid/cli/CliRealtimeStandalone.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java similarity index 89% rename from services/src/main/java/io/druid/cli/CliRealtimeStandalone.java rename to services/src/main/java/io/druid/cli/CliRealtimeExample.java index 39b39a25ce1..1b11ea07e14 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeStandalone.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -31,20 +31,20 @@ import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.realtime.RealtimeManager; -import druid.examples.guice.RealtimeStandaloneModule; +import druid.examples.guice.RealtimeExampleModule; import io.airlift.command.Command; /** */ @Command( - name = "realtimeStandalone", + name = "example realtime", description = "Runs a standalone realtime node for examples, see https://github.com/metamx/druid/wiki/Realtime for a description" ) -public class CliRealtimeStandalone extends ServerRunnable +public class CliRealtimeExample extends ServerRunnable { private static final Logger log = new Logger(CliBroker.class); - public CliRealtimeStandalone() + public CliRealtimeExample() { super(log); } @@ -56,12 +56,12 @@ public class CliRealtimeStandalone extends ServerRunnable new LifecycleModule(), EmitterModule.class, DruidProcessingModule.class, - StorageNodeModule.class, + new StorageNodeModule("realtime"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), new QueryableModule(RealtimeManager.class), new QueryRunnerFactoryModule(), - RealtimeStandaloneModule.class + RealtimeExampleModule.class ); } } diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index a261071394a..d2f290e26c8 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -38,7 +38,7 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliRealtimeStandalone.class); + .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliRealtimeExample.class); builder.build().parse(args).run(); } From 261ef7ce56a1c4fcc66f375617f442270b5eef49 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 22 Aug 2013 10:56:50 -0700 Subject: [PATCH 36/92] add some fixes --- .../java/druid/examples/guice/RealtimeExampleModule.java | 4 +++- .../src/main/java/com/metamx/druid/guice/RealtimeModule.java | 2 +- services/pom.xml | 5 +++++ 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java index e8925381d58..45e06db05b1 100644 --- a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java +++ b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java @@ -19,6 +19,7 @@ import com.metamx.druid.guice.RealtimeManagerConfig; import com.metamx.druid.initialization.DruidModule; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.realtime.FireDepartment; +import com.metamx.druid.realtime.RealtimeManager; import com.metamx.druid.realtime.SegmentPublisher; import druid.examples.flights.FlightsFirehoseFactory; import druid.examples.rand.RandomFirehoseFactory; @@ -51,7 +52,8 @@ public class RealtimeExampleModule implements DruidModule new TypeLiteral>() { } - ).toProvider(FireDepartmentsProvider.class).in(ManageLifecycle.class); + ).toProvider(FireDepartmentsProvider.class); + binder.bind(RealtimeManager.class).in(ManageLifecycle.class); } @Override diff --git a/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java index 864580f5405..c0a463d2850 100644 --- a/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java +++ b/realtime/src/main/java/com/metamx/druid/guice/RealtimeModule.java @@ -47,7 +47,7 @@ public class RealtimeModule implements Module new TypeLiteral>() { } - ).toProvider(FireDepartmentsProvider.class).in(ManageLifecycle.class); + ).toProvider(FireDepartmentsProvider.class); binder.bind(RealtimeManager.class).in(ManageLifecycle.class); } } diff --git a/services/pom.xml b/services/pom.xml index 89dd568b967..aee7193de5d 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -42,6 +42,11 @@ druid-server ${project.parent.version} + + com.metamx.druid + druid-examples + ${project.parent.version} + io.airlift airline From b897c2cb2254371d1754bb7a0d02e5e8b71c3908 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 23 Aug 2013 14:11:34 -0500 Subject: [PATCH 37/92] 1) IndexCoordinator appears to work as the CliOverlord now, yay! --- client/pom.xml | 4 - .../druid/initialization/DruidNode.java | 25 +- .../druid/initialization/Initialization.java | 2 + .../druid/query/filter/SelectorDimFilter.java | 3 +- common/pom.xml | 4 + .../com/metamx/druid/guava/DSuppliers.java | 41 +++ .../guice/JacksonConfigManagerModule.java | 2 +- .../druid/guice/JacksonConfigProvider.java | 94 ++++++ .../com/metamx/druid/guice/ListProvider.java | 74 +++++ .../java/com/metamx/druid/guice/PolyBind.java | 157 ++++++++++ .../com/metamx/druid/guice/PolyBindTest.java | 128 ++++++++ indexing-hadoop/pom.xml | 19 -- .../metamx/druid/guice/OverlordModule.java | 168 ++++++++++ .../actions/LocalTaskActionClientFactory.java | 2 + .../common/actions/SegmentInsertAction.java | 2 +- .../actions/SegmentListUnusedAction.java | 2 +- .../common/actions/SegmentListUsedAction.java | 2 +- .../common/actions/SegmentNukeAction.java | 2 +- .../common/actions/TaskActionToolbox.java | 14 +- .../common/task/RealtimeIndexTask.java | 2 +- .../indexing/common/tasklogs/S3TaskLogs.java | 21 +- .../common/tasklogs/S3TaskLogsConfig.java | 47 +++ ...der.java => SwitchingTaskLogStreamer.java} | 10 +- ...kLogProvider.java => TaskLogStreamer.java} | 2 +- .../indexing/common/tasklogs/TaskLogs.java | 2 +- .../tasklogs/TaskRunnerTaskLogStreamer.java | 54 ++++ .../indexing/coordinator/DbTaskStorage.java | 30 +- .../coordinator/ForkingTaskRunner.java | 25 +- .../coordinator/ForkingTaskRunnerFactory.java | 61 ++++ ...dinator.java => IndexerDBCoordinator.java} | 8 +- .../coordinator/RemoteTaskRunner.java | 30 +- .../coordinator/RemoteTaskRunnerFactory.java | 77 +++++ .../indexing/coordinator/TaskLockbox.java | 6 +- ...skMasterLifecycle.java => TaskMaster.java} | 38 ++- .../druid/indexing/coordinator/TaskQueue.java | 2 + .../coordinator/TaskStorageQueryAdapter.java | 2 + .../config/ForkingTaskRunnerConfig.java | 88 ++++-- .../config/IndexerCoordinatorConfig.java | 4 - .../config/RemoteTaskRunnerConfig.java | 50 ++- .../http/IndexerCoordinatorNode.java | 291 +++++++----------- .../http/IndexerCoordinatorResource.java | 36 +-- .../http/IndexerCoordinatorServletModule.java | 20 +- .../http/OldIndexerCoordinatorResource.java | 10 +- .../http/OverlordRedirectInfo.java | 57 ++++ .../coordinator/scaling/AutoScalingData.java | 8 +- .../scaling/AutoScalingStrategy.java | 6 +- .../scaling/EC2AutoScalingStrategy.java | 31 +- .../scaling/NoopAutoScalingStrategy.java | 6 +- .../scaling/ResourceManagementScheduler.java | 23 +- .../ResourceManagementSchedulerConfig.java | 45 ++- ...esourceManagementSchedulerFactoryImpl.java | 56 ++++ .../SimpleResourceManagementConfig.java | 112 +++++++ .../SimpleResourceManagementStrategy.java | 28 +- .../SimpleResourceManagmentConfig.java | 50 --- .../indexing/worker/http/WorkerNode.java | 7 +- .../coordinator/RemoteTaskRunnerTest.java | 22 +- .../coordinator/TaskLifecycleTest.java | 10 +- .../scaling/EC2AutoScalingStrategyTest.java | 19 +- .../SimpleResourceManagementStrategyTest.java | 43 +-- .../scaling/TestAutoScalingStrategy.java | 6 +- pom.xml | 19 ++ .../plumber/RealtimePlumberSchool.java | 65 ++-- server/pom.xml | 4 + .../java/com/metamx/druid/guice/S3Module.java | 34 +- ...sConfig.java => AWSCredentialsConfig.java} | 2 +- services/pom.xml | 5 + .../main/java/io/druid/cli/CliOverlord.java | 118 +++++++ services/src/main/java/io/druid/cli/Main.java | 19 +- 68 files changed, 1867 insertions(+), 589 deletions(-) create mode 100644 common/src/main/java/com/metamx/druid/guava/DSuppliers.java create mode 100644 common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/ListProvider.java create mode 100644 common/src/main/java/com/metamx/druid/guice/PolyBind.java create mode 100644 common/src/test/java/com/metamx/druid/guice/PolyBindTest.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogsConfig.java rename indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/{SwitchingTaskLogProvider.java => SwitchingTaskLogStreamer.java} (73%) rename indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/{TaskLogProvider.java => TaskLogStreamer.java} (95%) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java rename indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/{MergerDBCoordinator.java => IndexerDBCoordinator.java} (98%) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerFactory.java rename indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/{TaskMasterLifecycle.java => TaskMaster.java} (88%) create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OverlordRedirectInfo.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java rename server/src/main/java/com/metamx/druid/loading/{S3CredentialsConfig.java => AWSCredentialsConfig.java} (90%) create mode 100644 services/src/main/java/io/druid/cli/CliOverlord.java diff --git a/client/pom.xml b/client/pom.xml index efc6ccd59ca..aa00fc5df59 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -112,10 +112,6 @@ com.google.inject.extensions guice-servlet - - com.google.inject.extensions - guice-multibindings - com.ibm.icu icu4j diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java index 35bfb5415b3..43f78ac0682 100644 --- a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java +++ b/client/src/main/java/com/metamx/druid/initialization/DruidNode.java @@ -31,6 +31,8 @@ import javax.validation.constraints.NotNull; */ public class DruidNode { + private String hostNoPort; + @JsonProperty("service") @NotNull private String serviceName = null; @@ -54,35 +56,37 @@ public class DruidNode if (port == null) { if (host == null) { - setHostAndPort(null, -1); + setHostAndPort(null, -1, null); } else if (host.contains(":")) { + final String[] hostParts = host.split(":"); try { - setHostAndPort(host, Integer.parseInt(host.split(":")[1])); + setHostAndPort(host, Integer.parseInt(hostParts[1]), hostParts[0]); } - catch (Exception e) { - setHostAndPort(host, -1); + catch (NumberFormatException e) { + setHostAndPort(host, -1, hostParts[0]); } } else { final int openPort = SocketUtil.findOpenPort(8080); - setHostAndPort(String.format("%s:%d", host, openPort), openPort); + setHostAndPort(String.format("%s:%d", host, openPort), openPort, host); } } else { if (host == null || host.contains(":")) { - setHostAndPort(host, port); + setHostAndPort(host, port, host == null ? null : host.split(":")[0]); } else { - setHostAndPort(String.format("%s:%d", host, port), port); + setHostAndPort(String.format("%s:%d", host, port), port, host); } } } - private void setHostAndPort(String host, int port) + private void setHostAndPort(String host, int port, String hostNoPort) { this.host = host; this.port = port; + this.hostNoPort = hostNoPort; } public String getServiceName() @@ -100,6 +104,11 @@ public class DruidNode return port; } + public String getHostNoPort() + { + return hostNoPort; + } + @Override public String toString() { diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index c4660cc2bfe..353938bf69a 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -220,12 +220,14 @@ public class Initialization @Override public void start() throws Exception { + log.info("Starting Curator"); framework.start(); } @Override public void stop() { + log.info("Stopping Curator"); framework.close(); } } diff --git a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java index 039d9893411..945c753c652 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java @@ -39,7 +39,6 @@ public class SelectorDimFilter implements DimFilter ) { Preconditions.checkArgument(dimension != null, "dimension must not be null"); - Preconditions.checkArgument(value != null, "value must not be null"); this.dimension = dimension; this.value = value; } @@ -48,7 +47,7 @@ public class SelectorDimFilter implements DimFilter public byte[] getCacheKey() { byte[] dimensionBytes = dimension.getBytes(); - byte[] valueBytes = value.getBytes(); + byte[] valueBytes = value == null ? new byte[]{} : value.getBytes(); return ByteBuffer.allocate(1 + dimensionBytes.length + valueBytes.length) .put(DimFilterCacheHelper.SELECTOR_CACHE_ID) diff --git a/common/pom.xml b/common/pom.xml index ca373dee814..af7e334f6b7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -103,6 +103,10 @@ com.google.inject guice + + com.google.inject.extensions + guice-multibindings + org.jdbi jdbi diff --git a/common/src/main/java/com/metamx/druid/guava/DSuppliers.java b/common/src/main/java/com/metamx/druid/guava/DSuppliers.java new file mode 100644 index 00000000000..ce0768afd6e --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guava/DSuppliers.java @@ -0,0 +1,41 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guava; + +import com.google.common.base.Supplier; + +import java.util.concurrent.atomic.AtomicReference; + +/** + */ +public class DSuppliers +{ + public static Supplier of(final AtomicReference ref) + { + return new Supplier() + { + @Override + public T get() + { + return ref.get(); + } + }; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java b/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java index 05838c04383..8bb33dafabf 100644 --- a/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java +++ b/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java @@ -19,7 +19,7 @@ public class JacksonConfigManagerModule implements Module public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.manager.config", ConfigManagerConfig.class); - binder.bind(JacksonConfigManager.class); + binder.bind(JacksonConfigManager.class).in(LazySingleton.class); } @Provides @ManageLifecycle diff --git a/common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java b/common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java new file mode 100644 index 00000000000..0d4bb53f01d --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java @@ -0,0 +1,94 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Supplier; +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.util.Types; +import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.guava.DSuppliers; + +/** + */ +public class JacksonConfigProvider implements Provider> +{ + public static void bind(Binder binder, String key, Class clazz, T defaultVal) + { + binder.bind(Key.get(Types.newParameterizedType(Supplier.class, clazz))) + .toProvider((Provider) of(key, clazz, defaultVal)) + .in(LazySingleton.class); + } + + public static JacksonConfigProvider of(String key, Class clazz) + { + return of(key, clazz, null); + } + + public static JacksonConfigProvider of(String key, Class clazz, T defaultVal) + { + return new JacksonConfigProvider(key, clazz, null, defaultVal); + } + + public static JacksonConfigProvider of(String key, TypeReference clazz) + { + return of(key, clazz, null); + } + + public static JacksonConfigProvider of(String key, TypeReference typeRef, T defaultVal) + { + return new JacksonConfigProvider(key, null, typeRef, defaultVal); + } + + private final String key; + private final Class clazz; + private final TypeReference typeRef; + private final T defaultVal; + private JacksonConfigManager configManager; + + JacksonConfigProvider(String key, Class clazz, TypeReference typeRef, T defaultVal) + { + this.key = key; + this.clazz = clazz; + this.typeRef = typeRef; + this.defaultVal = defaultVal; + } + + @Inject + public void configure(JacksonConfigManager configManager) + { + this.configManager = configManager; + } + + @Override + public Supplier get() + { + if (clazz == null) { + return DSuppliers.of(configManager.watch(key, typeRef, defaultVal)); + } + else { + return DSuppliers.of(configManager.watch(key, clazz, defaultVal)); + } + } + +} diff --git a/common/src/main/java/com/metamx/druid/guice/ListProvider.java b/common/src/main/java/com/metamx/druid/guice/ListProvider.java new file mode 100644 index 00000000000..86d7a03c24a --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/ListProvider.java @@ -0,0 +1,74 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; + +import java.lang.annotation.Annotation; +import java.util.List; + +/** + */ +public class ListProvider implements Provider> +{ + private final List> itemsToLoad = Lists.newArrayList(); + private Injector injector; + + public ListProvider add(Class clazz) + { + return add(Key.get(clazz)); + } + + public ListProvider add(Class clazz, Class annotation) + { + return add(Key.get(clazz, annotation)); + } + + public ListProvider add(Class clazz, Annotation annotation) + { + return add(Key.get(clazz, annotation)); + } + + public ListProvider add(Key key) + { + itemsToLoad.add(key); + return this; + } + + @Inject + private void configure(Injector injector) + { + this.injector = injector; + } + + @Override + public List get() + { + List retVal = Lists.newArrayListWithExpectedSize(itemsToLoad.size()); + for (Key key : itemsToLoad) { + retVal.add(injector.getInstance(key)); + } + return retVal; + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/PolyBind.java b/common/src/main/java/com/metamx/druid/guice/PolyBind.java new file mode 100644 index 00000000000..195f2923306 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/guice/PolyBind.java @@ -0,0 +1,157 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.ProvisionException; +import com.google.inject.TypeLiteral; +import com.google.inject.binder.ScopedBindingBuilder; +import com.google.inject.multibindings.MapBinder; +import com.google.inject.util.Types; + +import javax.annotation.Nullable; +import java.lang.reflect.ParameterizedType; +import java.util.Map; +import java.util.Properties; + +/** + * Provides the ability to create "polymorphic" bindings. Where the polymorphism is actually just making a decision + * based on a value in a Properties. + * + * The workflow is that you first create a choice by calling createChoice(). Then you create options using the binder + * returned by the optionBinder() method. Multiple different modules can call optionBinder and all options will be + * reflected at injection time as long as equivalent interface Key objects are passed into the various methods. + */ +public class PolyBind +{ + /** + * Sets up a "choice" for the injector to resolve at injection time. + * + * @param binder the binder for the injector that is being configured + * @param property the property that will be checked to determine the implementation choice + * @param interfaceKey the interface that will be injected using this choice + * @param defaultKey the default instance to be injected if the property doesn't match a choice. Can be null + * @param interface type + * @return A ScopedBindingBuilder so that scopes can be added to the binding, if required. + */ + public static ScopedBindingBuilder createChoice( + Binder binder, + String property, + Key interfaceKey, + @Nullable Key defaultKey + ) + { + return binder.bind(interfaceKey).toProvider(new ConfiggedProvider(interfaceKey, property, defaultKey)); + } + + /** + * Binds an option for a specific choice. The choice must already be registered on the injector for this to work. + * + * @param binder the binder for the injector that is being configured + * @param interfaceKey the interface that will have an option added to it. This must equal the + * Key provided to createChoice + * @param interface type + * @return A MapBinder that can be used to create the actual option bindings. + */ + public static MapBinder optionBinder(Binder binder, Key interfaceKey) + { + final TypeLiteral interfaceType = interfaceKey.getTypeLiteral(); + + if (interfaceKey.getAnnotation() != null) { + return MapBinder.newMapBinder( + binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation() + ); + } + else if (interfaceKey.getAnnotationType() != null) { + return MapBinder.newMapBinder( + binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotationType() + ); + } + else { + return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType); + } + } + + static class ConfiggedProvider implements Provider + { + private final Key key; + private final String property; + private final Key defaultKey; + + private Injector injector; + private Properties props; + + ConfiggedProvider( + Key key, + String property, + Key defaultKey + ) + { + this.key = key; + this.property = property; + this.defaultKey = defaultKey; + } + + @Inject + void configure(Injector injector, Properties props) + { + this.injector = injector; + this.props = props; + } + + @Override + @SuppressWarnings("unchecked") + public T get() + { + final ParameterizedType mapType = Types.mapOf( + String.class, Types.newParameterizedType(Provider.class, key.getTypeLiteral().getType()) + ); + + final Map> implsMap; + if (key.getAnnotation() != null) { + implsMap = (Map>) injector.getInstance(Key.get(mapType, key.getAnnotation())); + } + else if (key.getAnnotationType() != null) { + implsMap = (Map>) injector.getInstance(Key.get(mapType, key.getAnnotation())); + } + else { + implsMap = (Map>) injector.getInstance(Key.get(mapType)); + } + + final String implName = props.getProperty(property); + final Provider provider = implsMap.get(implName); + + if (provider == null) { + if (defaultKey == null) { + throw new ProvisionException( + String.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet()) + ); + } + return injector.getInstance(defaultKey); + } + + return provider.get(); + } + } +} diff --git a/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java b/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java new file mode 100644 index 00000000000..be688e7b560 --- /dev/null +++ b/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java @@ -0,0 +1,128 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.common.collect.Iterables; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.multibindings.MapBinder; +import com.google.inject.name.Names; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Properties; + +/** + */ +public class PolyBindTest +{ + private Properties props; + private Injector injector; + + public void setUp(Module... modules) throws Exception + { + props = new Properties(); + injector = Guice.createInjector( + Iterables.concat( + Arrays.asList( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(Properties.class).toInstance(props); + PolyBind.createChoice(binder, "billy", Key.get(Gogo.class), Key.get(GoA.class)); + } + } + ), + Arrays.asList(modules) + ) + ); + } + + @Test + public void testSanity() throws Exception + { + setUp( + new Module() + { + @Override + public void configure(Binder binder) + { + final MapBinder gogoBinder = PolyBind.optionBinder(binder, Key.get(Gogo.class)); + gogoBinder.addBinding("a").to(GoA.class); + gogoBinder.addBinding("b").to(GoB.class); + + PolyBind.createChoice( + binder, "billy", Key.get(Gogo.class, Names.named("reverse")), Key.get(GoB.class) + ); + final MapBinder annotatedGogoBinder = PolyBind.optionBinder( + binder, Key.get(Gogo.class, Names.named("reverse")) + ); + annotatedGogoBinder.addBinding("a").to(GoB.class); + annotatedGogoBinder.addBinding("b").to(GoA.class); + } + } + ); + + + Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); + Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); + props.setProperty("billy.type", "b"); + Assert.assertEquals("B", injector.getInstance(Gogo.class).go()); + Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); + props.setProperty("billy.type", "a"); + Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); + Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); + props.setProperty("billy.type", "b"); + Assert.assertEquals("B", injector.getInstance(Gogo.class).go()); + Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); + props.setProperty("billy.type", "c"); + Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); + Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); + } + + public static interface Gogo + { + public String go(); + } + + public static class GoA implements Gogo + { + @Override + public String go() + { + return "A"; + } + } + + public static class GoB implements Gogo + { + @Override + public String go() + { + return "B"; + } + } +} diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 9607f56d542..7495d0b9013 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -48,25 +48,6 @@ java-util - - com.amazonaws - aws-java-sdk - 1.3.27 - - - javax.mail - mail - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - commons-io commons-io diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java new file mode 100644 index 00000000000..0b1c908d40c --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java @@ -0,0 +1,168 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.http.RedirectFilter; +import com.metamx.druid.http.RedirectInfo; +import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionToolbox; +import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogsConfig; +import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; +import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; +import com.metamx.druid.indexing.coordinator.DbTaskStorage; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunnerFactory; +import com.metamx.druid.indexing.coordinator.HeapMemoryTaskStorage; +import com.metamx.druid.indexing.coordinator.IndexerDBCoordinator; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerFactory; +import com.metamx.druid.indexing.coordinator.TaskLockbox; +import com.metamx.druid.indexing.coordinator.TaskMaster; +import com.metamx.druid.indexing.coordinator.TaskQueue; +import com.metamx.druid.indexing.coordinator.TaskRunnerFactory; +import com.metamx.druid.indexing.coordinator.TaskStorage; +import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.http.OverlordRedirectInfo; +import com.metamx.druid.indexing.coordinator.scaling.AutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.NoopAutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerConfig; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactoryImpl; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementStrategy; +import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementConfig; +import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; + +import java.util.List; + +/** + */ +public class OverlordModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(TaskMaster.class).in(ManageLifecycle.class); + + binder.bind(TaskLogStreamer.class).to(SwitchingTaskLogStreamer.class).in(LazySingleton.class); + binder.bind(new TypeLiteral>(){}) + .toProvider( + new ListProvider() + .add(TaskRunnerTaskLogStreamer.class) + .add(TaskLogs.class) + ) + .in(LazySingleton.class); + + binder.bind(TaskActionClientFactory.class).to(LocalTaskActionClientFactory.class).in(LazySingleton.class); + binder.bind(TaskActionToolbox.class).in(LazySingleton.class); + binder.bind(TaskQueue.class).in(LazySingleton.class); // Lifecycle managed by TaskMaster instead + binder.bind(IndexerDBCoordinator.class).in(LazySingleton.class); + binder.bind(TaskLockbox.class).in(LazySingleton.class); + binder.bind(TaskStorageQueryAdapter.class).in(LazySingleton.class); + binder.bind(ResourceManagementSchedulerFactory.class) + .to(ResourceManagementSchedulerFactoryImpl.class) + .in(LazySingleton.class); + + configureTaskStorage(binder); + configureRunners(binder); + configureAutoscale(binder); + + binder.bind(RedirectFilter.class).in(LazySingleton.class); + binder.bind(RedirectInfo.class).to(OverlordRedirectInfo.class).in(LazySingleton.class); + } + + private void configureTaskStorage(Binder binder) + { + PolyBind.createChoice( + binder, "druid.indexer.storage.type", Key.get(TaskStorage.class), Key.get(HeapMemoryTaskStorage.class) + ); + final MapBinder storageBinder = PolyBind.optionBinder(binder, Key.get(TaskStorage.class)); + + storageBinder.addBinding("local").to(HeapMemoryTaskStorage.class); + binder.bind(HeapMemoryTaskStorage.class).in(LazySingleton.class); + + storageBinder.addBinding("db").to(DbTaskStorage.class); + binder.bind(DbTaskStorage.class).in(LazySingleton.class); + + PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(NoopTaskLogs.class)); + final MapBinder taskLogBinder = PolyBind.optionBinder(binder, Key.get(TaskLogs.class)); + + JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); + taskLogBinder.addBinding("s3").to(S3TaskLogs.class); + binder.bind(S3TaskLogs.class).in(LazySingleton.class); + + taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class); + binder.bind(NoopTaskLogs.class).in(LazySingleton.class); + + } + + private void configureRunners(Binder binder) + { + PolyBind.createChoice( + binder, "druid.indexer.runner.type", Key.get(TaskRunnerFactory.class), Key.get(ForkingTaskRunnerFactory.class) + ); + final MapBinder biddy = PolyBind.optionBinder(binder, Key.get(TaskRunnerFactory.class)); + + JsonConfigProvider.bind(binder, "druid.indexer.runner", ForkingTaskRunnerConfig.class); + biddy.addBinding("local").to(ForkingTaskRunnerFactory.class); + binder.bind(ForkingTaskRunnerFactory.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.indexer.runner", RemoteTaskRunnerConfig.class); + biddy.addBinding("remote").to(RemoteTaskRunnerFactory.class).in(LazySingleton.class); + binder.bind(RemoteTaskRunnerFactory.class).in(LazySingleton.class); + } + + private void configureAutoscale(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.indexer.autoscale", ResourceManagementSchedulerConfig.class); + binder.bind(ResourceManagementStrategy.class).to(SimpleResourceManagementStrategy.class).in(LazySingleton.class); + + JacksonConfigProvider.bind(binder, WorkerSetupData.CONFIG_KEY, WorkerSetupData.class, null); + + PolyBind.createChoice( + binder, + "druid.indexer.autoscale.strategy", + Key.get(AutoScalingStrategy.class), + Key.get(NoopAutoScalingStrategy.class) + ); + + final MapBinder autoScalingBinder = PolyBind.optionBinder( + binder, Key.get(AutoScalingStrategy.class) + ); + autoScalingBinder.addBinding("ec2").to(EC2AutoScalingStrategy.class); + binder.bind(EC2AutoScalingStrategy.class).in(LazySingleton.class); + + autoScalingBinder.addBinding("noop").to(NoopAutoScalingStrategy.class); + binder.bind(NoopAutoScalingStrategy.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.indexer.autoscale", SimpleResourceManagementConfig.class); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java index 0f7cdeaa748..e3f44f506d2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexing.common.actions; +import com.google.inject.Inject; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskStorage; @@ -29,6 +30,7 @@ public class LocalTaskActionClientFactory implements TaskActionClientFactory private final TaskStorage storage; private final TaskActionToolbox toolbox; + @Inject public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox) { this.storage = storage; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java index 4a4dd43f67e..76512a0a109 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java @@ -65,7 +65,7 @@ public class SegmentInsertAction implements TaskAction> throw new ISE("Segments not covered by locks for task[%s]: %s", task.getId(), segments); } - final Set retVal = toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments); + final Set retVal = toolbox.getIndexerDBCoordinator().announceHistoricalSegments(segments); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java index efff8dc0a65..adfbfc628e6 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java @@ -49,7 +49,7 @@ public class SegmentListUnusedAction implements TaskAction> @Override public List perform(Task task, TaskActionToolbox toolbox) throws IOException { - return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); + return toolbox.getIndexerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); } @Override diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java index 2d86b393f0c..100c6d15e8f 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java @@ -49,7 +49,7 @@ public class SegmentListUsedAction implements TaskAction> @Override public List perform(Task task, TaskActionToolbox toolbox) throws IOException { - return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); + return toolbox.getIndexerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); } @Override diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java index d2fdc5be392..db142096696 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java @@ -44,7 +44,7 @@ public class SegmentNukeAction implements TaskAction throw new ISE("Segments not covered by locks for task: %s", task.getId()); } - toolbox.getMergerDBCoordinator().deleteSegments(segments); + toolbox.getIndexerDBCoordinator().deleteSegments(segments); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java index 81f8131c77b..c18dbacd747 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java @@ -2,10 +2,11 @@ package com.metamx.druid.indexing.common.actions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; +import com.google.inject.Inject; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.MergerDBCoordinator; +import com.metamx.druid.indexing.coordinator.IndexerDBCoordinator; import com.metamx.druid.indexing.coordinator.TaskLockbox; import com.metamx.druid.indexing.coordinator.TaskQueue; import com.metamx.emitter.service.ServiceEmitter; @@ -17,19 +18,20 @@ public class TaskActionToolbox { private final TaskQueue taskQueue; private final TaskLockbox taskLockbox; - private final MergerDBCoordinator mergerDBCoordinator; + private final IndexerDBCoordinator indexerDBCoordinator; private final ServiceEmitter emitter; + @Inject public TaskActionToolbox( TaskQueue taskQueue, TaskLockbox taskLockbox, - MergerDBCoordinator mergerDBCoordinator, + IndexerDBCoordinator indexerDBCoordinator, ServiceEmitter emitter ) { this.taskQueue = taskQueue; this.taskLockbox = taskLockbox; - this.mergerDBCoordinator = mergerDBCoordinator; + this.indexerDBCoordinator = indexerDBCoordinator; this.emitter = emitter; } @@ -43,9 +45,9 @@ public class TaskActionToolbox return taskLockbox; } - public MergerDBCoordinator getMergerDBCoordinator() + public IndexerDBCoordinator getIndexerDBCoordinator() { - return mergerDBCoordinator; + return indexerDBCoordinator; } public ServiceEmitter getEmitter() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index dc3ad87a9cb..808d5e3fad1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -278,7 +278,7 @@ public class RealtimeIndexTask extends AbstractTask realtimePlumberSchool.setSegmentAnnouncer(lockingSegmentAnnouncer); realtimePlumberSchool.setSegmentPublisher(segmentPublisher); realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView()); - realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter()); + realtimePlumberSchool.setEmitter(toolbox.getEmitter()); if (this.rejectionPolicyFactory != null) { realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java index a09f2ef7e00..8e684ffa06e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java @@ -1,9 +1,9 @@ package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.io.InputSupplier; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import org.jets3t.service.ServiceException; import org.jets3t.service.StorageService; @@ -21,15 +21,14 @@ public class S3TaskLogs implements TaskLogs { private static final Logger log = new Logger(S3TaskLogs.class); - private final String bucket; - private final String prefix; private final StorageService service; + private final S3TaskLogsConfig config; - public S3TaskLogs(String bucket, String prefix, RestS3Service service) + @Inject + public S3TaskLogs(S3TaskLogsConfig config, RestS3Service service) { - this.bucket = Preconditions.checkNotNull(bucket, "bucket"); - this.prefix = Preconditions.checkNotNull(prefix, "prefix"); - this.service = Preconditions.checkNotNull(service, "service"); + this.config = config; + this.service = service; } @Override @@ -38,7 +37,7 @@ public class S3TaskLogs implements TaskLogs final String taskKey = getTaskLogKey(taskid); try { - final StorageObject objectDetails = service.getObjectDetails(bucket, taskKey, null, null, null, null); + final StorageObject objectDetails = service.getObjectDetails(config.getS3Bucket(), taskKey, null, null, null, null); return Optional.>of( new InputSupplier() @@ -59,7 +58,7 @@ public class S3TaskLogs implements TaskLogs } return service.getObject( - bucket, + config.getS3Bucket(), taskKey, null, null, @@ -95,7 +94,7 @@ public class S3TaskLogs implements TaskLogs final StorageObject object = new StorageObject(logFile); object.setKey(taskKey); - service.putObject(bucket, object); + service.putObject(config.getS3Bucket(), object); } catch (Exception e) { Throwables.propagateIfInstanceOf(e, IOException.class); @@ -105,6 +104,6 @@ public class S3TaskLogs implements TaskLogs private String getTaskLogKey(String taskid) { - return String.format("%s/%s/log", prefix, taskid); + return String.format("%s/%s/log", config.getS3Prefix(), taskid); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogsConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogsConfig.java new file mode 100644 index 00000000000..5642d1e3b05 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogsConfig.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.common.tasklogs; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.NotNull; + +/** + */ +public class S3TaskLogsConfig +{ + @JsonProperty + @NotNull + private String s3Bucket = null; + + @JsonProperty + @NotNull + private String s3Prefix = null; + + public String getS3Bucket() + { + return s3Bucket; + } + + public String getS3Prefix() + { + return s3Prefix; + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java similarity index 73% rename from indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java index 4acad86386b..a573d8fc17e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java @@ -3,6 +3,7 @@ package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.io.InputSupplier; +import com.google.inject.Inject; import java.io.IOException; import java.io.InputStream; @@ -11,11 +12,12 @@ import java.util.List; /** * Provides task logs based on a series of underlying task log providers. */ -public class SwitchingTaskLogProvider implements TaskLogProvider +public class SwitchingTaskLogStreamer implements TaskLogStreamer { - private final List providers; + private final List providers; - public SwitchingTaskLogProvider(List providers) + @Inject + public SwitchingTaskLogStreamer(List providers) { this.providers = ImmutableList.copyOf(providers); } @@ -23,7 +25,7 @@ public class SwitchingTaskLogProvider implements TaskLogProvider @Override public Optional> streamTaskLog(String taskid, long offset) throws IOException { - for (TaskLogProvider provider : providers) { + for (TaskLogStreamer provider : providers) { final Optional> stream = provider.streamTaskLog(taskid, offset); if (stream.isPresent()) { return stream; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogStreamer.java similarity index 95% rename from indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogStreamer.java index 46b1bfc2b9c..89bc46aa22f 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogStreamer.java @@ -9,7 +9,7 @@ import java.io.InputStream; /** * Something that knows how to stream logs for tasks. */ -public interface TaskLogProvider +public interface TaskLogStreamer { /** * Stream log for a task. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java index 0c1994d6073..b42aa29fa4b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java @@ -1,5 +1,5 @@ package com.metamx.druid.indexing.common.tasklogs; -public interface TaskLogs extends TaskLogProvider, TaskLogPusher +public interface TaskLogs extends TaskLogStreamer, TaskLogPusher { } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java new file mode 100644 index 00000000000..b6fb47aca3d --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.common.tasklogs; + +import com.google.common.base.Optional; +import com.google.common.io.InputSupplier; +import com.google.inject.Inject; +import com.metamx.druid.indexing.coordinator.TaskMaster; +import com.metamx.druid.indexing.coordinator.TaskRunner; + +import java.io.IOException; +import java.io.InputStream; + +/** +*/ +public class TaskRunnerTaskLogStreamer implements TaskLogStreamer +{ + private final TaskMaster taskMaster; + + @Inject + public TaskRunnerTaskLogStreamer( + final TaskMaster taskMaster + ) { + this.taskMaster = taskMaster; + } + + @Override + public Optional> streamTaskLog(String taskid, long offset) throws IOException + { + final TaskRunner runner = taskMaster.getTaskRunner().orNull(); + if (runner instanceof TaskLogStreamer) { + return ((TaskLogStreamer) runner).streamTaskLog(taskid, offset); + } else { + return Optional.absent(); + } + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java index a9c4ad7c60c..c92257e1413 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java @@ -28,11 +28,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.inject.Inject; +import com.metamx.druid.db.DbTablesConfig; import com.metamx.druid.indexing.common.TaskLock; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.actions.TaskAction; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.skife.jdbi.v2.Handle; @@ -46,15 +47,16 @@ import java.util.Map; public class DbTaskStorage implements TaskStorage { private final ObjectMapper jsonMapper; - private final IndexerDbConnectorConfig dbConnectorConfig; + private final DbTablesConfig dbTables; private final IDBI dbi; private static final EmittingLogger log = new EmittingLogger(DbTaskStorage.class); - public DbTaskStorage(ObjectMapper jsonMapper, IndexerDbConnectorConfig dbConnectorConfig, IDBI dbi) + @Inject + public DbTaskStorage(ObjectMapper jsonMapper, DbTablesConfig dbTables, IDBI dbi) { this.jsonMapper = jsonMapper; - this.dbConnectorConfig = dbConnectorConfig; + this.dbTables = dbTables; this.dbi = dbi; } @@ -82,7 +84,7 @@ public class DbTaskStorage implements TaskStorage handle.createStatement( String.format( "INSERT INTO %s (id, created_date, datasource, payload, active, status_payload) VALUES (:id, :created_date, :datasource, :payload, :active, :status_payload)", - dbConnectorConfig.getTaskTable() + dbTables.getTasksTable() ) ) .bind("id", task.getId()) @@ -123,7 +125,7 @@ public class DbTaskStorage implements TaskStorage return handle.createStatement( String.format( "UPDATE %s SET active = :active, status_payload = :status_payload WHERE id = :id AND active = 1", - dbConnectorConfig.getTaskTable() + dbTables.getTasksTable() ) ) .bind("id", status.getId()) @@ -152,7 +154,7 @@ public class DbTaskStorage implements TaskStorage handle.createQuery( String.format( "SELECT payload FROM %s WHERE id = :id", - dbConnectorConfig.getTaskTable() + dbTables.getTasksTable() ) ) .bind("id", taskid) @@ -182,7 +184,7 @@ public class DbTaskStorage implements TaskStorage handle.createQuery( String.format( "SELECT status_payload FROM %s WHERE id = :id", - dbConnectorConfig.getTaskTable() + dbTables.getTasksTable() ) ) .bind("id", taskid) @@ -212,7 +214,7 @@ public class DbTaskStorage implements TaskStorage handle.createQuery( String.format( "SELECT id, payload, status_payload FROM %s WHERE active = 1", - dbConnectorConfig.getTaskTable() + dbTables.getTasksTable() ) ) .list(); @@ -261,7 +263,7 @@ public class DbTaskStorage implements TaskStorage return handle.createStatement( String.format( "INSERT INTO %s (task_id, lock_payload) VALUES (:task_id, :lock_payload)", - dbConnectorConfig.getTaskLockTable() + dbTables.getTaskLockTable() ) ) .bind("task_id", taskid) @@ -296,7 +298,7 @@ public class DbTaskStorage implements TaskStorage return handle.createStatement( String.format( "DELETE FROM %s WHERE id = :id", - dbConnectorConfig.getTaskLockTable() + dbTables.getTaskLockTable() ) ) .bind("id", id) @@ -341,7 +343,7 @@ public class DbTaskStorage implements TaskStorage return handle.createStatement( String.format( "INSERT INTO %s (task_id, log_payload) VALUES (:task_id, :log_payload)", - dbConnectorConfig.getTaskLogTable() + dbTables.getTaskLogTable() ) ) .bind("task_id", task.getId()) @@ -365,7 +367,7 @@ public class DbTaskStorage implements TaskStorage handle.createQuery( String.format( "SELECT log_payload FROM %s WHERE task_id = :task_id", - dbConnectorConfig.getTaskLogTable() + dbTables.getTaskLogTable() ) ) .bind("task_id", taskid) @@ -402,7 +404,7 @@ public class DbTaskStorage implements TaskStorage handle.createQuery( String.format( "SELECT id, lock_payload FROM %s WHERE task_id = :task_id", - dbConnectorConfig.getTaskLockTable() + dbTables.getTaskLockTable() ) ) .bind("task_id", taskid) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 7934a088a3b..3c223a3f8de 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -38,12 +38,14 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; import com.metamx.druid.indexing.common.tasklogs.TaskLogPusher; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; import com.metamx.druid.indexing.worker.executor.ExecutorMain; +import com.metamx.druid.initialization.DruidNode; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; @@ -59,12 +61,12 @@ import java.util.Map; import java.util.Properties; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * Runs tasks in separate processes using {@link ExecutorMain}. */ -public class ForkingTaskRunner implements TaskRunner, TaskLogProvider +public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer { private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class); private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property."; @@ -72,6 +74,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider private final ForkingTaskRunnerConfig config; private final Properties props; private final TaskLogPusher taskLogPusher; + private final DruidNode node; private final ListeningExecutorService exec; private final ObjectMapper jsonMapper; @@ -81,15 +84,17 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider ForkingTaskRunnerConfig config, Properties props, TaskLogPusher taskLogPusher, - ExecutorService exec, - ObjectMapper jsonMapper + ObjectMapper jsonMapper, + @Self DruidNode node ) { this.config = config; this.props = props; this.taskLogPusher = taskLogPusher; - this.exec = MoreExecutors.listeningDecorator(exec); this.jsonMapper = jsonMapper; + this.node = node; + + this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(config.maxForks())); } @Override @@ -113,7 +118,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider public TaskStatus call() { final String attemptUUID = UUID.randomUUID().toString(); - final File taskDir = new File(config.getBaseTaskDir(), task.getId()); + final File taskDir = new File(config.getTaskDir(), task.getId()); final File attemptDir = new File(taskDir, attemptUUID); final ProcessHolder processHolder; @@ -147,17 +152,17 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider final List command = Lists.newArrayList(); final int childPort = findUnusedPort(); - final String childHost = String.format(config.getHostPattern(), childPort); + final String childHost = String.format("%s:%d", node.getHostNoPort(), childPort); command.add(config.getJavaCommand()); command.add("-cp"); - command.add(config.getJavaClasspath()); + command.add(config.getClasspath()); Iterables.addAll( command, Splitter.on(CharMatcher.WHITESPACE) .omitEmptyStrings() - .split(config.getJavaOptions()) + .split(config.getJavaOpts()) ); for (String propName : props.stringPropertyNames()) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java new file mode 100644 index 00000000000..9dfc4cdb093 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java @@ -0,0 +1,61 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.coordinator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.initialization.DruidNode; + +import java.util.Properties; + +/** +*/ +public class ForkingTaskRunnerFactory implements TaskRunnerFactory +{ + private final ForkingTaskRunnerConfig config; + private final Properties props; + private final ObjectMapper jsonMapper; + private final TaskLogs persistentTaskLogs; + private final DruidNode node; + + @Inject + public ForkingTaskRunnerFactory( + final ForkingTaskRunnerConfig config, + final Properties props, + final ObjectMapper jsonMapper, + final TaskLogs persistentTaskLogs, + @Self DruidNode node + ) { + this.config = config; + this.props = props; + this.jsonMapper = jsonMapper; + this.persistentTaskLogs = persistentTaskLogs; + this.node = node; + } + + @Override + public TaskRunner build() + { + return new ForkingTaskRunner(config, props, persistentTaskLogs, jsonMapper, node); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/IndexerDBCoordinator.java similarity index 98% rename from indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/IndexerDBCoordinator.java index 3f893d79cdd..cfaa56bdb69 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/IndexerDBCoordinator.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; @@ -52,16 +53,17 @@ import java.util.Set; /** */ -public class MergerDBCoordinator +public class IndexerDBCoordinator { - private static final Logger log = new Logger(MergerDBCoordinator.class); + private static final Logger log = new Logger(IndexerDBCoordinator.class); private final ObjectMapper jsonMapper; private final DbConnectorConfig dbConnectorConfig; private final DbTablesConfig dbTables; private final IDBI dbi; - public MergerDBCoordinator( + @Inject + public IndexerDBCoordinator( ObjectMapper jsonMapper, DbConnectorConfig dbConnectorConfig, DbTablesConfig dbTables, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 88b0a6d43ca..c12077eb8a2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -25,6 +25,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; +import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -38,10 +39,11 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.cache.PathChildrenCacheFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; @@ -73,7 +75,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; /** * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes. @@ -90,7 +91,7 @@ import java.util.concurrent.atomic.AtomicReference; *

* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages. */ -public class RemoteTaskRunner implements TaskRunner, TaskLogProvider +public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer { private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class); private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(Charsets.UTF_8); @@ -98,10 +99,11 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; + private final ZkPathsConfig zkPaths; private final CuratorFramework cf; private final PathChildrenCacheFactory pathChildrenCacheFactory; private final PathChildrenCache workerPathCache; - private final AtomicReference workerSetupData; + private final Supplier workerSetupData; private final HttpClient httpClient; // all workers that exist in ZK @@ -120,17 +122,19 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider public RemoteTaskRunner( ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, + ZkPathsConfig zkPaths, CuratorFramework cf, PathChildrenCacheFactory pathChildrenCacheFactory, - AtomicReference workerSetupData, + Supplier workerSetupData, HttpClient httpClient ) { this.jsonMapper = jsonMapper; this.config = config; + this.zkPaths = zkPaths; this.cf = cf; this.pathChildrenCacheFactory = pathChildrenCacheFactory; - this.workerPathCache = pathChildrenCacheFactory.make(cf, config.getIndexerAnnouncementPath()); + this.workerPathCache = pathChildrenCacheFactory.make(cf, zkPaths.getIndexerAnnouncementPath()); this.workerSetupData = workerSetupData; this.httpClient = httpClient; } @@ -440,7 +444,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private void cleanup(final String workerId, final String taskId) { runningTasks.remove(taskId); - final String statusPath = JOINER.join(config.getIndexerStatusPath(), workerId, taskId); + final String statusPath = JOINER.join(zkPaths.getIndexerStatusPath(), workerId, taskId); try { cf.delete().guaranteed().forPath(statusPath); } @@ -490,11 +494,11 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId()); byte[] rawBytes = jsonMapper.writeValueAsBytes(task); - if (rawBytes.length > config.getMaxNumBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); + if (rawBytes.length > config.getMaxZnodeBytes()) { + throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes()); } - String taskPath = JOINER.join(config.getIndexerTaskPath(), theWorker.getHost(), task.getId()); + String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), theWorker.getHost(), task.getId()); if (cf.checkExists().forPath(taskPath) == null) { cf.create() @@ -541,7 +545,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider log.info("Worker[%s] reportin' for duty!", worker.getHost()); try { - final String workerStatusPath = JOINER.join(config.getIndexerStatusPath(), worker.getHost()); + final String workerStatusPath = JOINER.join(zkPaths.getIndexerStatusPath(), worker.getHost()); final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath); final ZkWorker zkWorker = new ZkWorker( worker, @@ -649,10 +653,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (zkWorker != null) { try { for (String assignedTask : cf.getChildren() - .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost()))) { + .forPath(JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost()))) { RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(assignedTask); if (taskRunnerWorkItem != null) { - String taskPath = JOINER.join(config.getIndexerTaskPath(), worker.getHost(), assignedTask); + String taskPath = JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost(), assignedTask); if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerFactory.java new file mode 100644 index 00000000000..fda47dbfe42 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerFactory.java @@ -0,0 +1,77 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.coordinator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; +import com.metamx.druid.guice.annotations.Global; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.initialization.ZkPathsConfig; +import com.metamx.http.client.HttpClient; +import org.apache.curator.framework.CuratorFramework; + +/** +*/ +public class RemoteTaskRunnerFactory implements TaskRunnerFactory +{ + private final CuratorFramework curator; + private final RemoteTaskRunnerConfig remoteTaskRunnerConfig; + private final ZkPathsConfig zkPaths; + private final ObjectMapper jsonMapper; + private final Supplier setupDataWatch; + private final HttpClient httpClient; + + @Inject + public RemoteTaskRunnerFactory( + final CuratorFramework curator, + final RemoteTaskRunnerConfig remoteTaskRunnerConfig, + final ZkPathsConfig zkPaths, + final ObjectMapper jsonMapper, + final Supplier setupDataWatch, + @Global final HttpClient httpClient + ) { + this.curator = curator; + this.remoteTaskRunnerConfig = remoteTaskRunnerConfig; + this.zkPaths = zkPaths; + this.jsonMapper = jsonMapper; + this.setupDataWatch = setupDataWatch; + this.httpClient = httpClient; + } + + @Override + public TaskRunner build() + { + return new RemoteTaskRunner( + jsonMapper, + remoteTaskRunnerConfig, + zkPaths, + curator, + new SimplePathChildrenCacheFactory + .Builder() + .withCompressed(remoteTaskRunnerConfig.isCompressZnodes()) + .build(), + setupDataWatch, + httpClient + ); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java index b47e66aa573..5787238ab79 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java @@ -28,6 +28,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; @@ -63,7 +64,10 @@ public class TaskLockbox private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class); - public TaskLockbox(TaskStorage taskStorage) + @Inject + public TaskLockbox( + TaskStorage taskStorage + ) { this.taskStorage = taskStorage; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java similarity index 88% rename from indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java index d1823d36cda..f8ef078f771 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java @@ -21,20 +21,21 @@ package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Throwables; -import com.metamx.common.ISE; +import com.google.inject.Inject; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -42,13 +43,14 @@ import org.apache.curator.framework.recipes.leader.LeaderSelector; import org.apache.curator.framework.recipes.leader.LeaderSelectorListener; import org.apache.curator.framework.state.ConnectionState; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; /** * Encapsulates the indexer leadership lifecycle. */ -public class TaskMasterLifecycle +public class TaskMaster { private final LeaderSelector leaderSelector; private final ReentrantLock giant = new ReentrantLock(); @@ -56,17 +58,20 @@ public class TaskMasterLifecycle private final TaskQueue taskQueue; private final TaskActionClientFactory taskActionClientFactory; + private final AtomicReference leaderLifecycleRef = new AtomicReference(null); + private volatile boolean leading = false; private volatile TaskRunner taskRunner; private volatile ResourceManagementScheduler resourceManagementScheduler; - private static final EmittingLogger log = new EmittingLogger(TaskMasterLifecycle.class); + private static final EmittingLogger log = new EmittingLogger(TaskMaster.class); - public TaskMasterLifecycle( + @Inject + public TaskMaster( final TaskQueue taskQueue, final TaskActionClientFactory taskActionClientFactory, - final IndexerCoordinatorConfig indexerCoordinatorConfig, - final DruidNode node, + @Self final DruidNode node, + final ZkPathsConfig zkPaths, final TaskRunnerFactory runnerFactory, final ResourceManagementSchedulerFactory managementSchedulerFactory, final CuratorFramework curator, @@ -78,7 +83,7 @@ public class TaskMasterLifecycle this.taskActionClientFactory = taskActionClientFactory; this.leaderSelector = new LeaderSelector( - curator, indexerCoordinatorConfig.getIndexerLeaderLatchPath(), new LeaderSelectorListener() + curator, zkPaths.getIndexerLeaderLatchPath(), new LeaderSelectorListener() { @Override public void takeLeadership(CuratorFramework client) throws Exception @@ -101,6 +106,11 @@ public class TaskMasterLifecycle // Sensible order to start stuff: final Lifecycle leaderLifecycle = new Lifecycle(); + if (leaderLifecycleRef.getAndSet(leaderLifecycle) != null) { + log.makeAlert("TaskMaster set a new Lifecycle without the old one being cleared! Race condition") + .emit(); + } + leaderLifecycle.addManagedInstance(taskRunner); leaderLifecycle.addHandler( new Lifecycle.Handler() @@ -122,10 +132,7 @@ public class TaskMasterLifecycle Initialization.announceDefaultService(node, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); - if ("remote".equalsIgnoreCase(indexerCoordinatorConfig.getRunnerImpl())) { - if (!(taskRunner instanceof RemoteTaskRunner)) { - throw new ISE("WTF?! We configured a remote runner and got %s", taskRunner.getClass()); - } + if (taskRunner instanceof RemoteTaskRunner) { resourceManagementScheduler = managementSchedulerFactory.build((RemoteTaskRunner) taskRunner); leaderLifecycle.addManagedInstance(resourceManagementScheduler); } @@ -144,7 +151,6 @@ public class TaskMasterLifecycle finally { log.info("Bowing out!"); stopLeading(); - leaderLifecycle.stop(); } } catch (Exception e) { @@ -167,7 +173,7 @@ public class TaskMasterLifecycle } ); - leaderSelector.setId(indexerCoordinatorConfig.getServerName()); + leaderSelector.setId(node.getHost()); leaderSelector.autoRequeue(); } @@ -216,6 +222,10 @@ public class TaskMasterLifecycle if (leading) { leading = false; mayBeStopped.signalAll(); + final Lifecycle leaderLifecycle = leaderLifecycleRef.getAndSet(null); + if (leaderLifecycle != null) { + leaderLifecycle.stop(); + } } } finally { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java index 252728b32dc..6c3f169c580 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.indexing.common.TaskLock; @@ -68,6 +69,7 @@ public class TaskQueue private static final EmittingLogger log = new EmittingLogger(TaskQueue.class); + @Inject public TaskQueue(TaskStorage taskStorage, TaskLockbox taskLockbox) { this.taskStorage = Preconditions.checkNotNull(taskStorage, "taskStorage"); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java index 09000104521..463bf540b21 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java @@ -25,6 +25,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.common.TaskStatus; @@ -44,6 +45,7 @@ public class TaskStorageQueryAdapter { private final TaskStorage storage; + @Inject public TaskStorageQueryAdapter(TaskStorage storage) { this.storage = storage; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index 9ee3a5ab1b7..b0a69e9a14a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -1,47 +1,85 @@ package com.metamx.druid.indexing.coordinator.config; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.metamx.druid.indexing.worker.executor.ExecutorMain; -import org.skife.config.Config; -import org.skife.config.Default; -import java.io.File; +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; import java.util.List; -public abstract class ForkingTaskRunnerConfig +public class ForkingTaskRunnerConfig { - @Config("druid.indexer.taskDir") - @Default("/tmp/persistent") - public abstract File getBaseTaskDir(); + @JsonProperty + @Min(1) + private int maxForks = 1; - @Config("druid.indexer.fork.java") - @Default("java") - public abstract String getJavaCommand(); + @JsonProperty + @NotNull + private String taskDir = "/tmp/persistent"; - @Config("druid.indexer.fork.opts") - @Default("") - public abstract String getJavaOptions(); + @JsonProperty + @NotNull + private String javaCommand = "java"; - @Config("druid.indexer.fork.classpath") - public String getJavaClasspath() { - return System.getProperty("java.class.path"); + @JsonProperty + @NotNull + private String javaOpts = ""; + + @JsonProperty + @NotNull + private String classpath = System.getProperty("java.class.path"); + + @JsonProperty + @NotNull + private String mainClass = ExecutorMain.class.getName(); + + @JsonProperty + @Min(1024) @Max(65535) + private int startPort = 8080; + + @JsonProperty + @NotNull + List allowedPrefixes = Lists.newArrayList("com.metamx", "druid", "io.druid"); + + public int maxForks() + { + return maxForks; + } + + public String getTaskDir() + { + return taskDir; + } + + public String getJavaCommand() + { + return javaCommand; + } + + public String getJavaOpts() + { + return javaOpts; + } + + public String getClasspath() + { + return classpath; } - @Config("druid.indexer.fork.main") public String getMainClass() { - return ExecutorMain.class.getName(); + return mainClass; } - @Config("druid.indexer.fork.hostpattern") - public abstract String getHostPattern(); + public int getStartPort() + { + return startPort; + } - @Config("druid.indexer.fork.startport") - public abstract int getStartPort(); - - @Config("druid.indexer.properties.prefixes") public List getAllowedPrefixes() { - return Lists.newArrayList("com.metamx", "druid"); + return allowedPrefixes; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java index f8d2cf8c3dc..70c72c74972 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java @@ -36,10 +36,6 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig @Config("druid.host") public abstract String getServerName(); - @Config("druid.indexer.threads") - @Default("1") - public abstract int getNumLocalThreads(); - @Config("druid.indexer.runner") @Default("local") public abstract String getRunnerImpl(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 2acadd3ae83..fbf3bb1cc8e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -19,25 +19,47 @@ package com.metamx.druid.indexing.coordinator.config; -import com.metamx.druid.indexing.common.config.IndexerZkConfig; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; -import org.skife.config.DefaultNull; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; /** */ -public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig +public class RemoteTaskRunnerConfig { - @Config("druid.indexer.taskAssignmentTimeoutDuration") - @Default("PT5M") - public abstract Duration getTaskAssignmentTimeoutDuration(); + @JsonProperty + @NotNull + private Duration taskAssignmentTimeoutDuration = new Duration("PT5M"); - @Config("druid.curator.compress") - @Default("false") - public abstract boolean enableCompression(); + @JsonProperty + private boolean compressZnodes = false; - @Config("druid.indexer.worker.version") - @DefaultNull - public abstract String getWorkerVersion(); + @JsonProperty + private String workerVersion = null; + + @JsonProperty + @Min(10 * 1024) + private long maxZnodeBytes = 512 * 1024; + + public Duration getTaskAssignmentTimeoutDuration() + { + return taskAssignmentTimeoutDuration; + } + + public boolean isCompressZnodes() + { + return compressZnodes; + } + + public String getWorkerVersion() + { + return workerVersion; + } + + public long getMaxZnodeBytes() + { + return maxZnodeBytes; + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 61191da91e5..1a6c8bf36da 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -24,12 +24,9 @@ import com.amazonaws.services.ec2.AmazonEC2Client; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Optional; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -47,15 +44,14 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.config.ConfigManager; import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; -import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbTablesConfig; +import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; -import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; @@ -66,22 +62,22 @@ import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; -import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogProvider; -import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import com.metamx.druid.indexing.coordinator.DbTaskStorage; -import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunnerFactory; import com.metamx.druid.indexing.coordinator.HeapMemoryTaskStorage; -import com.metamx.druid.indexing.coordinator.MergerDBCoordinator; +import com.metamx.druid.indexing.coordinator.IndexerDBCoordinator; import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerFactory; import com.metamx.druid.indexing.coordinator.TaskLockbox; -import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.TaskQueue; -import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunnerFactory; import com.metamx.druid.indexing.coordinator.TaskStorage; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; @@ -93,8 +89,8 @@ import com.metamx.druid.indexing.coordinator.scaling.NoopResourceManagementSched import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerConfig; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementConfig; import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; -import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagmentConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.initialization.CuratorDiscoveryConfig; import com.metamx.druid.initialization.DruidNode; @@ -113,7 +109,6 @@ import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; -import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.DefaultServlet; @@ -127,12 +122,8 @@ import org.jets3t.service.security.AWSCredentials; import org.joda.time.Duration; import org.skife.config.ConfigurationObjectFactory; -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; import java.util.List; 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; @@ -155,7 +146,7 @@ public class IndexerCoordinatorNode extends QueryableNode dbTables = null; private IndexerCoordinatorConfig config = null; - private MergerDBCoordinator mergerDBCoordinator = null; + private IndexerDBCoordinator indexerDBCoordinator = null; private ServiceDiscovery serviceDiscovery = null; private ServiceAnnouncer serviceAnnouncer = null; private TaskStorage taskStorage = null; @@ -166,9 +157,9 @@ public class IndexerCoordinatorNode extends QueryableNode providers = Lists.newArrayList(); + if (taskLogStreamer == null) { + final List providers = Lists.newArrayList(); - // Use our TaskRunner if it is also a TaskLogProvider - providers.add( - new TaskLogProvider() - { - @Override - public Optional> streamTaskLog(String taskid, long offset) throws IOException - { - final TaskRunner runner = taskMasterLifecycle.getTaskRunner().orNull(); - if (runner instanceof TaskLogProvider) { - return ((TaskLogProvider) runner).streamTaskLog(taskid, offset); - } else { - return Optional.absent(); - } - } - } - ); + // Use our TaskRunner if it is also a TaskLogStreamer + providers.add(new TaskRunnerTaskLogStreamer(IndexerCoordinatorNode.this.taskMaster)); // Use our persistent log storage providers.add(persistentTaskLogs); - taskLogProvider = new SwitchingTaskLogProvider(providers); + taskLogStreamer = new SwitchingTaskLogStreamer(providers); } } @@ -569,8 +515,8 @@ public class IndexerCoordinatorNode extends QueryableNode workerSetupData = configManager.watch( - WorkerSetupData.CONFIG_KEY, WorkerSetupData.class - ); - - AutoScalingStrategy strategy; - if (config.getAutoScalingImpl().equalsIgnoreCase("ec2")) { - strategy = new EC2AutoScalingStrategy( - getJsonMapper(), - new AmazonEC2Client( - new BasicAWSCredentials( - PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"), - PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey") - ) - ), - getConfigFactory().build(EC2AutoScalingStrategyConfig.class), - workerSetupData - ); - } else if (config.getAutoScalingImpl().equalsIgnoreCase("noop")) { - strategy = new NoopAutoScalingStrategy(); - } else { - throw new ISE("Invalid strategy implementation: %s", config.getAutoScalingImpl()); - } - - return new ResourceManagementScheduler( - runner, - new SimpleResourceManagementStrategy( - strategy, - getConfigFactory().build(SimpleResourceManagmentConfig.class), - workerSetupData - ), - getConfigFactory().build(ResourceManagementSchedulerConfig.class), - scalingScheduledExec - ); - } - }; + resourceManagementSchedulerFactory = new WithOpResourceManagementSchedulerFactory(configManager); } } } @@ -800,4 +675,68 @@ public class IndexerCoordinatorNode extends QueryableNode workerSetupData = configManager.watch( + WorkerSetupData.CONFIG_KEY, WorkerSetupData.class + ); + + AutoScalingStrategy strategy; + if (config.getAutoScalingImpl().equalsIgnoreCase("ec2")) { + strategy = new EC2AutoScalingStrategy( + getJsonMapper(), + new AmazonEC2Client( + new BasicAWSCredentials( + PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"), + PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey") + ) + ), + null, // TODO: eliminate + DSuppliers.of(workerSetupData) + ); + } else if (config.getAutoScalingImpl().equalsIgnoreCase("noop")) { + strategy = new NoopAutoScalingStrategy(); + } else { + throw new ISE("Invalid strategy implementation: %s", config.getAutoScalingImpl()); + } + + return new ResourceManagementScheduler( + runner, + new SimpleResourceManagementStrategy( + strategy, + getConfigFactory().build(SimpleResourceManagementConfig.class), + DSuppliers.of(workerSetupData) + ), + getConfigFactory().build(ResourceManagementSchedulerConfig.class), + scalingScheduledExec + ); + } + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java index 15a7e88b82d..63be1366c8c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorResource.java @@ -35,8 +35,8 @@ import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.indexing.common.actions.TaskActionHolder; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; -import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; +import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.TaskQueue; import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; @@ -87,9 +87,9 @@ public class IndexerCoordinatorResource } }; - private final TaskMasterLifecycle taskMasterLifecycle; + private final TaskMaster taskMaster; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final TaskLogProvider taskLogProvider; + private final TaskLogStreamer taskLogStreamer; private final JacksonConfigManager configManager; private final ObjectMapper jsonMapper; @@ -97,16 +97,16 @@ public class IndexerCoordinatorResource @Inject public IndexerCoordinatorResource( - TaskMasterLifecycle taskMasterLifecycle, + TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, - TaskLogProvider taskLogProvider, + TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, ObjectMapper jsonMapper ) throws Exception { - this.taskMasterLifecycle = taskMasterLifecycle; + this.taskMaster = taskMaster; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.taskLogProvider = taskLogProvider; + this.taskLogStreamer = taskLogStreamer; this.configManager = configManager; this.jsonMapper = jsonMapper; } @@ -137,7 +137,7 @@ public class IndexerCoordinatorResource public Response taskPost(final Task task) { return asLeaderWith( - taskMasterLifecycle.getTaskQueue(), + taskMaster.getTaskQueue(), new Function() { @Override @@ -173,7 +173,7 @@ public class IndexerCoordinatorResource public Response doShutdown(@PathParam("taskid") final String taskid) { return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -241,7 +241,7 @@ public class IndexerCoordinatorResource public Response doAction(final TaskActionHolder holder) { return asLeaderWith( - taskMasterLifecycle.getTaskActionClient(holder.getTask()), + taskMaster.getTaskActionClient(holder.getTask()), new Function() { @Override @@ -278,7 +278,7 @@ public class IndexerCoordinatorResource { if (full != null) { return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -291,7 +291,7 @@ public class IndexerCoordinatorResource } return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -317,7 +317,7 @@ public class IndexerCoordinatorResource { if (full != null) { return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -330,7 +330,7 @@ public class IndexerCoordinatorResource } return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -353,7 +353,7 @@ public class IndexerCoordinatorResource public Response getWorkers() { return asLeaderWith( - taskMasterLifecycle.getTaskRunner(), + taskMaster.getTaskRunner(), new Function() { @Override @@ -371,7 +371,7 @@ public class IndexerCoordinatorResource public Response getScalingState() { return asLeaderWith( - taskMasterLifecycle.getResourceManagementScheduler(), + taskMaster.getResourceManagementScheduler(), new Function() { @Override @@ -392,7 +392,7 @@ public class IndexerCoordinatorResource ) { try { - final Optional> stream = taskLogProvider.streamTaskLog(taskid, offset); + final Optional> stream = taskLogStreamer.streamTaskLog(taskid, offset); if (stream.isPresent()) { return Response.ok(stream.get().getInput()).build(); } else { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java index 64383a667a1..7eb8ce7e436 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java @@ -23,8 +23,8 @@ 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.indexing.common.tasklogs.TaskLogProvider; -import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; +import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.emitter.service.ServiceEmitter; @@ -40,27 +40,27 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule private final ObjectMapper jsonMapper; private final IndexerCoordinatorConfig indexerCoordinatorConfig; private final ServiceEmitter emitter; - private final TaskMasterLifecycle taskMasterLifecycle; + private final TaskMaster taskMaster; private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final TaskLogProvider taskLogProvider; + private final TaskLogStreamer taskLogStreamer; private final JacksonConfigManager configManager; public IndexerCoordinatorServletModule( ObjectMapper jsonMapper, IndexerCoordinatorConfig indexerCoordinatorConfig, ServiceEmitter emitter, - TaskMasterLifecycle taskMasterLifecycle, + TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, - TaskLogProvider taskLogProvider, + TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager ) { this.jsonMapper = jsonMapper; this.indexerCoordinatorConfig = indexerCoordinatorConfig; this.emitter = emitter; - this.taskMasterLifecycle = taskMasterLifecycle; + this.taskMaster = taskMaster; this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.taskLogProvider = taskLogProvider; + this.taskLogStreamer = taskLogStreamer; this.configManager = configManager; } @@ -72,9 +72,9 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule bind(ObjectMapper.class).toInstance(jsonMapper); bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); - bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); + bind(TaskMaster.class).toInstance(taskMaster); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); - bind(TaskLogProvider.class).toInstance(taskLogProvider); + bind(TaskLogStreamer.class).toInstance(taskLogStreamer); bind(JacksonConfigManager.class).toInstance(configManager); serve("/*").with(GuiceContainer.class); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java index 84897f424bd..fe91431c163 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java @@ -3,8 +3,8 @@ package com.metamx.druid.indexing.coordinator.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.druid.config.JacksonConfigManager; -import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; -import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; +import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; import javax.ws.rs.Path; @@ -17,13 +17,13 @@ public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource { @Inject public OldIndexerCoordinatorResource( - TaskMasterLifecycle taskMasterLifecycle, + TaskMaster taskMaster, TaskStorageQueryAdapter taskStorageQueryAdapter, - TaskLogProvider taskLogProvider, + TaskLogStreamer taskLogStreamer, JacksonConfigManager configManager, ObjectMapper jsonMapper ) throws Exception { - super(taskMasterLifecycle, taskStorageQueryAdapter, taskLogProvider, configManager, jsonMapper); + super(taskMaster, taskStorageQueryAdapter, taskLogStreamer, configManager, jsonMapper); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OverlordRedirectInfo.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OverlordRedirectInfo.java new file mode 100644 index 00000000000..1d767c713fd --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OverlordRedirectInfo.java @@ -0,0 +1,57 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.coordinator.http; + +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.metamx.druid.http.RedirectInfo; +import com.metamx.druid.indexing.coordinator.TaskMaster; + +import java.net.URL; + +/** +*/ +public class OverlordRedirectInfo implements RedirectInfo +{ + private final TaskMaster taskMaster; + + @Inject + public OverlordRedirectInfo(TaskMaster taskMaster) + { + this.taskMaster = taskMaster; + } + + @Override + public boolean doLocal() + { + return taskMaster.isLeading(); + } + + @Override + public URL getRedirectURL(String queryString, String requestURI) + { + try { + return new URL(String.format("http://%s%s", taskMaster.getLeader(), requestURI)); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java index 8a7bc1840d5..46b13fa9a74 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java @@ -25,12 +25,12 @@ import java.util.List; /** */ -public class AutoScalingData +public class AutoScalingData { private final List nodeIds; - private final List nodes; + private final List nodes; - public AutoScalingData(List nodeIds, List nodes) + public AutoScalingData(List nodeIds, List nodes) { this.nodeIds = nodeIds; this.nodes = nodes; @@ -42,7 +42,7 @@ public class AutoScalingData return nodeIds; } - public List getNodes() + public List getNodes() { return nodes; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java index 68a3170c54a..aab2175a50f 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java @@ -24,11 +24,11 @@ import java.util.List; /** * The AutoScalingStrategy has the actual methods to provision and terminate worker nodes. */ -public interface AutoScalingStrategy +public interface AutoScalingStrategy { - public AutoScalingData provision(); + public AutoScalingData provision(); - public AutoScalingData terminate(List ips); + public AutoScalingData terminate(List ips); /** * Provides a lookup of ip addresses to node ids diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java index 45b5573674a..9141905733c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java @@ -19,7 +19,7 @@ package com.metamx.druid.indexing.coordinator.scaling; -import com.amazonaws.services.ec2.AmazonEC2Client; +import com.amazonaws.services.ec2.AmazonEC2; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; import com.amazonaws.services.ec2.model.DescribeInstancesResult; import com.amazonaws.services.ec2.model.Filter; @@ -30,8 +30,9 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; -import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; +import com.google.inject.Inject; import com.metamx.druid.indexing.coordinator.setup.EC2NodeData; import com.metamx.druid.indexing.coordinator.setup.GalaxyUserData; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; @@ -40,24 +41,24 @@ import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.util.List; -import java.util.concurrent.atomic.AtomicReference; /** */ -public class EC2AutoScalingStrategy implements AutoScalingStrategy +public class EC2AutoScalingStrategy implements AutoScalingStrategy { private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); private final ObjectMapper jsonMapper; - private final AmazonEC2Client amazonEC2Client; - private final EC2AutoScalingStrategyConfig config; - private final AtomicReference workerSetupDataRef; + private final AmazonEC2 amazonEC2Client; + private final SimpleResourceManagementConfig config; + private final Supplier workerSetupDataRef; + @Inject public EC2AutoScalingStrategy( ObjectMapper jsonMapper, - AmazonEC2Client amazonEC2Client, - EC2AutoScalingStrategyConfig config, - AtomicReference workerSetupDataRef + AmazonEC2 amazonEC2Client, + SimpleResourceManagementConfig config, + Supplier workerSetupDataRef ) { this.jsonMapper = jsonMapper; @@ -67,7 +68,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } @Override - public AutoScalingData provision() + public AutoScalingData provision() { try { WorkerSetupData setupData = workerSetupDataRef.get(); @@ -110,7 +111,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy log.info("Created instances: %s", instanceIds); - return new AutoScalingData( + return new AutoScalingData( Lists.transform( result.getReservation().getInstances(), new Function() @@ -133,10 +134,10 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } @Override - public AutoScalingData terminate(List ips) + public AutoScalingData terminate(List ips) { if (ips.isEmpty()) { - return new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()); + return new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()); } DescribeInstancesResult result = amazonEC2Client.describeInstances( @@ -169,7 +170,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy ) ); - return new AutoScalingData( + return new AutoScalingData( Lists.transform( ips, new Function() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java index 84881deb49a..3d036c8f09a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java @@ -26,19 +26,19 @@ import java.util.List; /** * This class just logs when scaling should occur. */ -public class NoopAutoScalingStrategy implements AutoScalingStrategy +public class NoopAutoScalingStrategy implements AutoScalingStrategy { private static final EmittingLogger log = new EmittingLogger(NoopAutoScalingStrategy.class); @Override - public AutoScalingData provision() + public AutoScalingData provision() { log.info("If I were a real strategy I'd create something now"); return null; } @Override - public AutoScalingData terminate(List ips) + public AutoScalingData terminate(List ips) { log.info("If I were a real strategy I'd terminate %s now", ips); return null; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java index 0cfdc94c76a..0e359635202 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java @@ -76,41 +76,32 @@ public class ResourceManagementScheduler ScheduledExecutors.scheduleAtFixedRate( exec, - config.getProvisionResourcesDuration(), + config.getProvisionPeriod().toStandardDuration(), new Runnable() { @Override public void run() { - resourceManagementStrategy.doProvision( - taskRunner.getPendingTasks(), - taskRunner.getWorkers() - ); + resourceManagementStrategy.doProvision(taskRunner.getPendingTasks(), taskRunner.getWorkers()); } } ); // Schedule termination of worker nodes periodically - Period period = new Period(config.getTerminateResourcesDuration()); - PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null); + Period period = config.getTerminatePeriod(); + PeriodGranularity granularity = new PeriodGranularity(period, config.getOriginTime(), null); final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis())); ScheduledExecutors.scheduleAtFixedRate( exec, - new Duration( - System.currentTimeMillis(), - startTime - ), - config.getTerminateResourcesDuration(), + new Duration(System.currentTimeMillis(), startTime), + config.getTerminatePeriod().toStandardDuration(), new Runnable() { @Override public void run() { - resourceManagementStrategy.doTerminate( - taskRunner.getPendingTasks(), - taskRunner.getWorkers() - ); + resourceManagementStrategy.doTerminate(taskRunner.getPendingTasks(), taskRunner.getWorkers()); } } ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java index 4850ecaaf02..eb689b40f81 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java @@ -19,24 +19,43 @@ package com.metamx.druid.indexing.coordinator.scaling; +import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; +import org.joda.time.Period; /** */ -public abstract class ResourceManagementSchedulerConfig +public class ResourceManagementSchedulerConfig { - @Config("druid.indexer.provisionResources.duration") - @Default("PT1M") - public abstract Duration getProvisionResourcesDuration(); + @JsonProperty + private boolean doAutoscale = false; - @Config("druid.indexer.terminateResources.duration") - @Default("PT1H") - public abstract Duration getTerminateResourcesDuration(); + @JsonProperty + private Period provisionPeriod = new Period("PT1M"); - @Config("druid.indexer.terminateResources.originDateTime") - @Default("2012-01-01T00:55:00.000Z") - public abstract DateTime getTerminateResourcesOriginDateTime(); + @JsonProperty + private Period terminatePeriod = new Period("PT1H"); + + @JsonProperty + private DateTime originTime = new DateTime("2012-01-01T00:55:00.000Z"); + + public boolean isDoAutoscale() + { + return doAutoscale; + } + + public Period getProvisionPeriod() + { + return provisionPeriod; + } + + public Period getTerminatePeriod() + { + return terminatePeriod; + } + + public DateTime getOriginTime() + { + return originTime; + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java new file mode 100644 index 00000000000..c828051d150 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactoryImpl.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.coordinator.scaling; + +import com.google.inject.Inject; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; + +/** + */ +public class ResourceManagementSchedulerFactoryImpl implements ResourceManagementSchedulerFactory +{ + private final ResourceManagementSchedulerConfig config; + private final ResourceManagementStrategy strategy; + private final ScheduledExecutorFactory executorFactory; + + @Inject + public ResourceManagementSchedulerFactoryImpl( + ResourceManagementStrategy strategy, + ResourceManagementSchedulerConfig config, + ScheduledExecutorFactory executorFactory + ) + { + this.config = config; + this.strategy = strategy; + this.executorFactory = executorFactory; + } + + @Override + public ResourceManagementScheduler build(RemoteTaskRunner runner) + { + if (config.isDoAutoscale()) { + return new ResourceManagementScheduler(runner, strategy, config, executorFactory.create(1, "ScalingExec--%d")); + } + else { + return new NoopResourceManagementScheduler(); + } + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java new file mode 100644 index 00000000000..deb529e7a09 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementConfig.java @@ -0,0 +1,112 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.coordinator.scaling; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; + +/** + */ +public class SimpleResourceManagementConfig +{ + @JsonProperty + private Period workerIdleTimeout = new Period("PT10m"); + + @JsonProperty + private Period maxScalingDuration = new Period("PT15M"); + + @JsonProperty + private int numEventsToTrack = 50; + + @JsonProperty + private Period pendingTaskTimeout = new Period("PT30s"); + + @JsonProperty + private String workerVersion = null; + + @JsonProperty + private int workerPort = 8080; + + public Period getWorkerIdleTimeout() + { + return workerIdleTimeout; + } + + public SimpleResourceManagementConfig setWorkerIdleTimeout(Period workerIdleTimeout) + { + this.workerIdleTimeout = workerIdleTimeout; + return this; + } + + public Period getMaxScalingDuration() + { + return maxScalingDuration; + } + + public SimpleResourceManagementConfig setMaxScalingDuration(Period maxScalingDuration) + { + this.maxScalingDuration = maxScalingDuration; + return this; + } + + public int getNumEventsToTrack() + { + return numEventsToTrack; + } + + public SimpleResourceManagementConfig setNumEventsToTrack(int numEventsToTrack) + { + this.numEventsToTrack = numEventsToTrack; + return this; + } + + public Period getPendingTaskTimeout() + { + return pendingTaskTimeout; + } + + public SimpleResourceManagementConfig setPendingTaskTimeout(Period pendingTaskTimeout) + { + this.pendingTaskTimeout = pendingTaskTimeout; + return this; + } + + public String getWorkerVersion() + { + return workerVersion; + } + + public SimpleResourceManagementConfig setWorkerVersion(String workerVersion) + { + this.workerVersion = workerVersion; + return this; + } + + public int getWorkerPort() + { + return workerPort; + } + + public SimpleResourceManagementConfig setWorkerPort(int workerPort) + { + this.workerPort = workerPort; + return this; + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 0f9fd927245..a3800345fb1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -21,9 +21,11 @@ package com.metamx.druid.indexing.coordinator.scaling; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; @@ -37,7 +39,6 @@ import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.atomic.AtomicReference; /** */ @@ -46,8 +47,8 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private static final EmittingLogger log = new EmittingLogger(SimpleResourceManagementStrategy.class); private final AutoScalingStrategy autoScalingStrategy; - private final SimpleResourceManagmentConfig config; - private final AtomicReference workerSetupdDataRef; + private final SimpleResourceManagementConfig config; + private final Supplier workerSetupdDataRef; private final ScalingStats scalingStats; private final ConcurrentSkipListSet currentlyProvisioning = new ConcurrentSkipListSet(); @@ -56,10 +57,11 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat private volatile DateTime lastProvisionTime = new DateTime(); private volatile DateTime lastTerminateTime = new DateTime(); + @Inject public SimpleResourceManagementStrategy( AutoScalingStrategy autoScalingStrategy, - SimpleResourceManagmentConfig config, - AtomicReference workerSetupdDataRef + SimpleResourceManagementConfig config, + Supplier workerSetupdDataRef ) { this.autoScalingStrategy = autoScalingStrategy; @@ -96,7 +98,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat List workerNodeIds = autoScalingStrategy.ipToIdLookup( Lists.newArrayList( - Iterables.transform( + Iterables.transform( zkWorkers, new Function() { @@ -134,7 +136,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat durSinceLastProvision ); - if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) { + if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { log.makeAlert("Worker node provisioning taking too long!") .addData("millisSinceLastProvision", durSinceLastProvision.getMillis()) .addData("provisioningCount", currentlyProvisioning.size()) @@ -198,7 +200,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat { return input.getRunningTasks().isEmpty() && System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis() - >= config.getMaxWorkerIdleTimeMillisBeforeDeletion(); + >= config.getWorkerIdleTimeout().getMillis(); } } ) @@ -240,7 +242,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat currentlyTerminating ); - if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) { + if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration().toStandardDuration())) { log.makeAlert("Worker node termination taking too long!") .addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis()) .addData("terminatingCount", currentlyTerminating.size()) @@ -263,11 +265,9 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat { long now = System.currentTimeMillis(); for (TaskRunnerWorkItem pendingTask : pendingTasks) { - if (new Duration(pendingTask.getQueueInsertionTime().getMillis(), now).isEqual(config.getMaxPendingTaskDuration()) - || - new Duration( - pendingTask.getQueueInsertionTime().getMillis(), now - ).isLongerThan(config.getMaxPendingTaskDuration())) { + final Duration durationSinceInsertion = new Duration(pendingTask.getQueueInsertionTime().getMillis(), now); + final Duration timeoutDuration = config.getPendingTaskTimeout().toStandardDuration(); + if (durationSinceInsertion.isEqual(timeoutDuration) || durationSinceInsertion.isLongerThan(timeoutDuration)) { return true; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java deleted file mode 100644 index 184e1aba7ed..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java +++ /dev/null @@ -1,50 +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.indexing.coordinator.scaling; - -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; -import org.skife.config.DefaultNull; - -/** - */ -public abstract class SimpleResourceManagmentConfig -{ - @Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion") - @Default("600000") - public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion(); - - @Config("druid.indexer.maxScalingDuration") - @Default("PT15M") - public abstract Duration getMaxScalingDuration(); - - @Config("druid.indexer.numEventsToTrack") - @Default("50") - public abstract int getNumEventsToTrack(); - - @Config("druid.indexer.maxPendingTaskDuration") - @Default("PT30S") - public abstract Duration getMaxPendingTaskDuration(); - - @Config("druid.indexer.worker.version") - @DefaultNull - public abstract String getWorkerVersion(); -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 90890ca7a37..7391fdf4e93 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -366,8 +366,7 @@ public class WorkerNode extends QueryableNode if (taskLogConfig.getLogStorageBucket() != null) { initializeS3Service(); persistentTaskLogs = new S3TaskLogs( - taskLogConfig.getLogStorageBucket(), - taskLogConfig.getLogStoragePrefix(), + null, // TODO: eliminate s3Service ); } else { @@ -383,8 +382,8 @@ public class WorkerNode extends QueryableNode getConfigFactory().build(ForkingTaskRunnerConfig.class), getProps(), persistentTaskLogs, - Executors.newFixedThreadPool(workerConfig.getCapacity()), - getJsonMapper() + getJsonMapper(), + null // todo: eliminate ); } } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 0c39406b821..cc897a00b9c 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -15,6 +15,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; +import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolboxFactory; @@ -27,6 +28,7 @@ import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; import com.metamx.druid.indexing.worker.WorkerTaskMonitor; +import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; @@ -354,9 +356,17 @@ public class RemoteTaskRunnerTest remoteTaskRunner = new RemoteTaskRunner( jsonMapper, new TestRemoteTaskRunnerConfig(), + new ZkPathsConfig() + { + @Override + public String getZkBasePath() + { + return basePath; + } + }, cf, new SimplePathChildrenCacheFactory.Builder().build(), - new AtomicReference(new WorkerSetupData("0", 0, 1, null, null)), + DSuppliers.of(new AtomicReference(new WorkerSetupData("0", 0, 1, null, null))), null ); @@ -381,17 +391,11 @@ public class RemoteTaskRunnerTest private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { @Override - public boolean enableCompression() + public boolean isCompressZnodes() { return false; } - @Override - public String getZkBasePath() - { - return basePath; - } - @Override public Duration getTaskAssignmentTimeoutDuration() { @@ -399,7 +403,7 @@ public class RemoteTaskRunnerTest } @Override - public long getMaxNumBytes() + public long getMaxZnodeBytes() { return 1000; } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index 77e7c2fcb49..a67ec0326f6 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -87,7 +87,7 @@ public class TaskLifecycleTest private TaskLockbox tl = null; private TaskQueue tq = null; private TaskRunner tr = null; - private MockMergerDBCoordinator mdc = null; + private MockIndexerDBCoordinator mdc = null; private TaskActionClientFactory tac = null; private TaskToolboxFactory tb = null; private TaskConsumer tc = null; @@ -410,12 +410,12 @@ public class TaskLifecycleTest return status; } - private static class MockMergerDBCoordinator extends MergerDBCoordinator + private static class MockIndexerDBCoordinator extends IndexerDBCoordinator { final private Set published = Sets.newHashSet(); final private Set nuked = Sets.newHashSet(); - private MockMergerDBCoordinator() + private MockIndexerDBCoordinator() { super(null, null, null, null); } @@ -462,9 +462,9 @@ public class TaskLifecycleTest } } - private static MockMergerDBCoordinator newMockMDC() + private static MockIndexerDBCoordinator newMockMDC() { - return new MockMergerDBCoordinator(); + return new MockIndexerDBCoordinator(); } private static ServiceEmitter newMockEmitter() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java index 27442ed1cdc..39df4467800 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -28,7 +28,7 @@ import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Lists; -import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.indexing.coordinator.setup.EC2NodeData; import com.metamx.druid.indexing.coordinator.setup.GalaxyUserData; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; @@ -77,21 +77,8 @@ public class EC2AutoScalingStrategyTest strategy = new EC2AutoScalingStrategy( new DefaultObjectMapper(), amazonEC2Client, - new EC2AutoScalingStrategyConfig() - { - @Override - public String getWorkerPort() - { - return "8080"; - } - - @Override - public String getWorkerVersion() - { - return ""; - } - }, - workerSetupData + new SimpleResourceManagementConfig().setWorkerPort(8080).setWorkerVersion(""), + DSuppliers.of(workerSetupData) ); } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index fcbc1d4113b..5f183e83ce3 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; @@ -38,8 +39,8 @@ import com.metamx.emitter.service.ServiceEventBuilder; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; -import org.joda.time.Duration; import org.joda.time.Interval; +import org.joda.time.Period; import org.junit.Before; import org.junit.Test; @@ -88,39 +89,13 @@ public class SimpleResourceManagementStrategyTest ); simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( autoScalingStrategy, - new SimpleResourceManagmentConfig() - { - @Override - public int getMaxWorkerIdleTimeMillisBeforeDeletion() - { - return 0; - } - - @Override - public Duration getMaxScalingDuration() - { - return new Duration(1000); - } - - @Override - public int getNumEventsToTrack() - { - return 1; - } - - @Override - public Duration getMaxPendingTaskDuration() - { - return new Duration(0); - } - - @Override - public String getWorkerVersion() - { - return ""; - } - }, - workerSetupData + new SimpleResourceManagementConfig() + .setWorkerIdleTimeout(new Period(0)) + .setMaxScalingDuration(new Period(1000)) + .setNumEventsToTrack(1) + .setPendingTaskTimeout(new Period(0)) + .setWorkerVersion(""), + DSuppliers.of(workerSetupData) ); } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java index e86180c79ee..b11a859e5fe 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java @@ -23,16 +23,16 @@ import java.util.List; /** */ -public class TestAutoScalingStrategy implements AutoScalingStrategy +public class TestAutoScalingStrategy implements AutoScalingStrategy { @Override - public AutoScalingData provision() + public AutoScalingData provision() { return null; } @Override - public AutoScalingData terminate(List ips) + public AutoScalingData terminate(List ips) { return null; } diff --git a/pom.xml b/pom.xml index e7a23cdc662..61dd45f9417 100644 --- a/pom.xml +++ b/pom.xml @@ -113,6 +113,25 @@ commons-lang 2.6 + + com.amazonaws + aws-java-sdk + 1.3.27 + + + javax.mail + mail + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + com.ning compress-lzf diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index aa26c735da7..e425a70bec8 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -68,7 +68,6 @@ import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.sun.istack.internal.NotNull; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -99,32 +98,16 @@ public class RealtimePlumberSchool implements PlumberSchool private final IndexGranularity segmentGranularity; private final Object handoffCondition = new Object(); - @JacksonInject - @NotNull - private ServiceEmitter emitter; + private volatile ServiceEmitter emitter; + private volatile QueryRunnerFactoryConglomerate conglomerate = null; + private volatile DataSegmentPusher dataSegmentPusher = null; + private volatile DataSegmentAnnouncer segmentAnnouncer = null; + private volatile SegmentPublisher segmentPublisher = null; + private volatile ServerView serverView = null; private volatile VersioningPolicy versioningPolicy = null; private volatile RejectionPolicyFactory rejectionPolicyFactory = null; - @JacksonInject - @NotNull - private volatile QueryRunnerFactoryConglomerate conglomerate = null; - - @JacksonInject - @NotNull - private volatile DataSegmentPusher dataSegmentPusher = null; - - @JacksonInject - @NotNull - private volatile DataSegmentAnnouncer segmentAnnouncer = null; - - @JacksonInject - @NotNull - private volatile SegmentPublisher segmentPublisher = null; - - @JacksonInject - @NotNull - private volatile ServerView serverView = null; @JsonCreator public RealtimePlumberSchool( @@ -156,6 +139,42 @@ public class RealtimePlumberSchool implements PlumberSchool this.rejectionPolicyFactory = factory; } + @JacksonInject + public void setEmitter(ServiceEmitter emitter) + { + this.emitter = emitter; + } + + @JacksonInject + public void setConglomerate(QueryRunnerFactoryConglomerate conglomerate) + { + this.conglomerate = conglomerate; + } + + @JacksonInject + public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) + { + this.dataSegmentPusher = dataSegmentPusher; + } + + @JacksonInject + public void setSegmentAnnouncer(DataSegmentAnnouncer segmentAnnouncer) + { + this.segmentAnnouncer = segmentAnnouncer; + } + + @JacksonInject + public void setSegmentPublisher(SegmentPublisher segmentPublisher) + { + this.segmentPublisher = segmentPublisher; + } + + @JacksonInject + public void setServerView(ServerView serverView) + { + this.serverView = serverView; + } + @Override public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics) { diff --git a/server/pom.xml b/server/pom.xml index 56dcde1f43c..2799613f39f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -72,6 +72,10 @@ commons-io commons-io + + com.amazonaws + aws-java-sdk + com.ning compress-lzf diff --git a/server/src/main/java/com/metamx/druid/guice/S3Module.java b/server/src/main/java/com/metamx/druid/guice/S3Module.java index e60827285a3..7cf04b762e9 100644 --- a/server/src/main/java/com/metamx/druid/guice/S3Module.java +++ b/server/src/main/java/com/metamx/druid/guice/S3Module.java @@ -19,14 +19,17 @@ package com.metamx.druid.guice; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.services.ec2.AmazonEC2; +import com.amazonaws.services.ec2.AmazonEC2Client; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.ProvisionException; -import com.metamx.druid.loading.S3CredentialsConfig; +import com.metamx.druid.loading.AWSCredentialsConfig; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; /** */ @@ -35,18 +38,39 @@ public class S3Module implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.s3", S3CredentialsConfig.class); + JsonConfigProvider.bind(binder, "druid.s3", AWSCredentialsConfig.class); } @Provides @LazySingleton - public RestS3Service getRestS3Service(S3CredentialsConfig config) + public AWSCredentials getAWSCredentials(AWSCredentialsConfig config) + { + return new BasicAWSCredentials(config.getAccessKey(), config.getSecretKey()); + } + + @Provides + @LazySingleton + public org.jets3t.service.security.AWSCredentials getJets3tAWSCredentials(AWSCredentialsConfig config) + { + return new org.jets3t.service.security.AWSCredentials(config.getAccessKey(), config.getSecretKey()); + } + + @Provides + @LazySingleton + public RestS3Service getRestS3Service(org.jets3t.service.security.AWSCredentials credentials) { try { - return new RestS3Service(new AWSCredentials(config.getAccessKey(), config.getSecretKey())); + return new RestS3Service(credentials); } catch (S3ServiceException e) { throw new ProvisionException("Unable to create a RestS3Service", e); } } + + @Provides + @LazySingleton + public AmazonEC2 getEc2Client(AWSCredentials credentials) + { + return new AmazonEC2Client(credentials); + } } diff --git a/server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java b/server/src/main/java/com/metamx/druid/loading/AWSCredentialsConfig.java similarity index 90% rename from server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java rename to server/src/main/java/com/metamx/druid/loading/AWSCredentialsConfig.java index 252cf3c92df..dd8decfab54 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3CredentialsConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/AWSCredentialsConfig.java @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public class S3CredentialsConfig +public class AWSCredentialsConfig { @JsonProperty private String accessKey = ""; diff --git a/services/pom.xml b/services/pom.xml index aee7193de5d..fb69d5c83cc 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -47,6 +47,11 @@ druid-examples ${project.parent.version} + + com.metamx.druid + druid-indexing-service + ${project.parent.version} + io.airlift airline diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java new file mode 100644 index 00000000000..c7e43955bae --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -0,0 +1,118 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli; + +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.DbConnectorModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.JacksonConfigManagerModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.OverlordModule; +import com.metamx.druid.guice.S3Module; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.http.RedirectFilter; +import com.metamx.druid.indexing.coordinator.TaskMaster; +import com.metamx.druid.indexing.coordinator.http.IndexerCoordinatorResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerInitializer; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Command; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.server.handler.ResourceHandler; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.FilterHolder; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; +import org.eclipse.jetty.util.resource.ResourceCollection; + +/** + */ +@Command( + name = "overlord", + description = "Runs an Overlord node, see https://github.com/metamx/druid/wiki/Indexing-Service for a description" +) +public class CliOverlord extends ServerRunnable +{ + private static Logger log = new Logger(CliOverlord.class); + + public CliOverlord() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule(), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule(), + ServerModule.class, + new S3Module(), + new DbConnectorModule(), + new JacksonConfigManagerModule(), + new JettyServerModule(new OverlordJettyServerInitializer()) + .addResource(IndexerCoordinatorResource.class), + new DiscoveryModule(), + new OverlordModule() + ); + } + + private static class OverlordJettyServerInitializer implements JettyServerInitializer + { + @Override + public void initialize(Server server, Injector injector) + { + ResourceHandler resourceHandler = new ResourceHandler(); + resourceHandler.setBaseResource( + new ResourceCollection( + new String[]{ + TaskMaster.class.getClassLoader().getResource("static").toExternalForm(), + TaskMaster.class.getClassLoader().getResource("indexer_static").toExternalForm() + } + ) + ); + + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.setContextPath("/"); + + HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{resourceHandler, root, new DefaultHandler()}); + server.setHandler(handlerList); + + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + } + } +} diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index d2f290e26c8..0ad07dd5a8b 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -21,6 +21,7 @@ package io.druid.cli; import io.airlift.command.Cli; import io.airlift.command.Help; +import io.airlift.command.ParseException; /** */ @@ -38,8 +39,22 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliRealtimeExample.class); + .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliOverlord.class); - builder.build().parse(args).run(); + builder.withGroup("example") + .withDescription("Run an example") + .withDefaultCommand(Help.class) + .withCommands(CliRealtimeExample.class); + + final Cli cli = builder.build(); + try { + cli.parse(args).run(); + } + catch (ParseException e) { + System.out.println("ERROR!!!!"); + System.out.println(e.getMessage()); + System.out.println("==="); + cli.parse(new String[]{"help"}).run(); + } } } From 613ebd54b59edc30d6fb44efa78f02c3482bf9dc Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 23 Aug 2013 14:32:14 -0500 Subject: [PATCH 38/92] 1) Delete unused things --- .../druid/initialization/Initialization.java | 8 - .../http/IndexerCoordinatorMain.java | 50 -- .../http/IndexerCoordinatorNode.java | 742 ------------------ 3 files changed, 800 deletions(-) delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 353938bf69a..c1ef6d28a23 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -37,7 +37,6 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; -import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.guice.DruidGuiceExtensions; import com.metamx.druid.guice.DruidSecondaryModule; @@ -275,13 +274,6 @@ public class Initialization return serviceDiscovery; } - public static ServiceAnnouncer makeServiceAnnouncer( - ServiceDiscovery serviceDiscovery - ) - { - return new CuratorServiceAnnouncer(serviceDiscovery); - } - public static void announceDefaultService( final DruidNode nodeConfig, final ServiceAnnouncer serviceAnnouncer, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java deleted file mode 100644 index 3714986a8a0..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java +++ /dev/null @@ -1,50 +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.indexing.coordinator.http; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; - -/** - */ -public class IndexerCoordinatorMain -{ - private static final Logger log = new Logger(IndexerCoordinatorMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance(IndexerCoordinatorNode.builder().build()); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - - lifecycle.join(); - } -} \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java deleted file mode 100644 index 1a6c8bf36da..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ /dev/null @@ -1,742 +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.indexing.coordinator.http; - -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.services.ec2.AmazonEC2Client; -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.IAE; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryableNode; -import com.metamx.druid.config.ConfigManager; -import com.metamx.druid.config.ConfigManagerConfig; -import com.metamx.druid.config.JacksonConfigManager; -import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; -import com.metamx.druid.db.DbTablesConfig; -import com.metamx.druid.guava.DSuppliers; -import com.metamx.druid.http.GuiceServletConfig; -import com.metamx.druid.http.RedirectFilter; -import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; -import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; -import com.metamx.druid.indexing.common.actions.TaskActionToolbox; -import com.metamx.druid.indexing.common.config.IndexerZkConfig; -import com.metamx.druid.indexing.common.config.TaskLogConfig; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; -import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; -import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; -import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; -import com.metamx.druid.indexing.common.tasklogs.TaskLogs; -import com.metamx.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; -import com.metamx.druid.indexing.coordinator.DbTaskStorage; -import com.metamx.druid.indexing.coordinator.ForkingTaskRunnerFactory; -import com.metamx.druid.indexing.coordinator.HeapMemoryTaskStorage; -import com.metamx.druid.indexing.coordinator.IndexerDBCoordinator; -import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; -import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerFactory; -import com.metamx.druid.indexing.coordinator.TaskLockbox; -import com.metamx.druid.indexing.coordinator.TaskMaster; -import com.metamx.druid.indexing.coordinator.TaskQueue; -import com.metamx.druid.indexing.coordinator.TaskRunnerFactory; -import com.metamx.druid.indexing.coordinator.TaskStorage; -import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; -import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.indexing.coordinator.scaling.AutoScalingStrategy; -import com.metamx.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy; -import com.metamx.druid.indexing.coordinator.scaling.NoopAutoScalingStrategy; -import com.metamx.druid.indexing.coordinator.scaling.NoopResourceManagementScheduler; -import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; -import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerConfig; -import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; -import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementConfig; -import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; -import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.JvmMonitor; -import com.metamx.metrics.Monitor; -import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.FilterHolder; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.util.resource.ResourceCollection; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; -import org.joda.time.Duration; -import org.skife.config.ConfigurationObjectFactory; - -import java.util.List; -import java.util.Properties; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicReference; - -/** - */ -public class IndexerCoordinatorNode extends QueryableNode -{ - private static final Logger log = new Logger(IndexerCoordinatorNode.class); - - public static Builder builder() - { - return new Builder(); - } - - private RestS3Service s3Service = null; - private List monitors = null; - private ServiceEmitter emitter = null; - private DbConnectorConfig dbConnectorConfig = null; - private DbConnector dbi = null; - private Supplier dbTables = null; - private IndexerCoordinatorConfig config = null; - private IndexerDBCoordinator indexerDBCoordinator = null; - private ServiceDiscovery serviceDiscovery = null; - private ServiceAnnouncer serviceAnnouncer = null; - private TaskStorage taskStorage = null; - private TaskQueue taskQueue = null; - private TaskLockbox taskLockbox = null; - private IndexerZkConfig indexerZkConfig; - private TaskRunnerFactory taskRunnerFactory = null; - private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; - private HttpClient httpClient = null; - private TaskActionClientFactory taskActionClientFactory = null; - private TaskMaster taskMaster = null; - private TaskLogs persistentTaskLogs = null; - private TaskLogStreamer taskLogStreamer = null; - private Server server = null; - - private boolean initialized = false; - - public IndexerCoordinatorNode( - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super("index-coordinator", log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public IndexerCoordinatorNode setEmitter(ServiceEmitter emitter) - { - this.emitter = emitter; - return this; - } - - public IndexerCoordinatorNode setIndexerDBCoordinator(IndexerDBCoordinator indexerDBCoordinator) - { - this.indexerDBCoordinator = indexerDBCoordinator; - return this; - } - - public IndexerCoordinatorNode setS3Service(RestS3Service s3Service) - { - this.s3Service = s3Service; - return this; - } - - public IndexerCoordinatorNode setTaskQueue(TaskQueue taskQueue) - { - this.taskQueue = taskQueue; - return this; - } - - public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox) - { - this.taskLockbox = taskLockbox; - return this; - } - - public IndexerCoordinatorNode setMergeDbCoordinator(IndexerDBCoordinator mergeDbCoordinator) - { - this.indexerDBCoordinator = mergeDbCoordinator; - return this; - } - - public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) - { - this.taskRunnerFactory = taskRunnerFactory; - return this; - } - - public IndexerCoordinatorNode setResourceManagementSchedulerFactory(ResourceManagementSchedulerFactory resourceManagementSchedulerFactory) - { - this.resourceManagementSchedulerFactory = resourceManagementSchedulerFactory; - return this; - } - - public IndexerCoordinatorNode setHttpClient(HttpClient httpClient) - { - this.httpClient = httpClient; - return this; - } - - public Supplier getDbTables() - { - if (dbTables == null) { - dbTables = Suppliers.ofInstance( - getJsonConfigurator().configurate(getProps(), "druid.database.tables", DbTablesConfig.class) - ); - } - return dbTables; - } - - public void doInit() throws Exception - { - final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(getLifecycle()); - initializeDB(); - - final ConfigManagerConfig managerConfig = getConfigFactory().build(ConfigManagerConfig.class); - dbi.createConfigTable(); - JacksonConfigManager configManager = - new JacksonConfigManager( - getLifecycle().addManagedInstance( - new ConfigManager( - dbi.getDBI(), - getDbTables(), - Suppliers.ofInstance(managerConfig) - ) - ), getJsonMapper() - ); - - initializeHttpClient(); - initializeEmitter(); - initializeMonitors(); - initializeIndexerCoordinatorConfig(); - initializeMergeDBCoordinator(); - initializeJacksonSubtypes(); - initializeJacksonInjections(); - initializeServiceDiscovery(); - initializeTaskStorage(); - initializeTaskLockbox(); - initializeTaskQueue(); - initializeIndexerZkConfig(); - initializeTaskActionClientFactory(); - initializeTaskRunnerFactory(configManager); - initializeResourceManagement(configManager); - initializeTaskMasterLifecycle(); - initializePersistentTaskLogs(); - initializeTaskLogProvider(); - initializeServer(); - - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - final MonitorScheduler monitorScheduler = new MonitorScheduler( - getConfigFactory().build(MonitorSchedulerConfig.class), - globalScheduledExec, - emitter, - monitors - ); - getLifecycle().addManagedInstance(monitorScheduler); - - final Injector injector = Guice.createInjector( - new IndexerCoordinatorServletModule( - getJsonMapper(), - config, - emitter, - taskMaster, - new TaskStorageQueryAdapter(taskStorage), - taskLogStreamer, - configManager - ) - ); - - final ServletContextHandler staticContext = new ServletContextHandler(server, "/static", ServletContextHandler.SESSIONS); - staticContext.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - - ResourceCollection resourceCollection = new ResourceCollection( - new String[]{ - IndexerCoordinatorNode.class.getClassLoader().getResource("static").toExternalForm(), - IndexerCoordinatorNode.class.getClassLoader().getResource("indexer_static").toExternalForm() - } - ); - staticContext.setBaseResource(resourceCollection); - - // If we want to support querying tasks (e.g. for realtime in local mode), we need a QueryServlet here. - - final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/druid/*"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/mmx/*"); // backwards compatibility - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(new FilterHolder(new RedirectFilter(new OverlordRedirectInfo(taskMaster))), "/*", null); - root.addFilter(GuiceFilter.class, "/druid/indexer/v1/*", null); - root.addFilter(GuiceFilter.class, "/mmx/merger/v1/*", null); //backwards compatibility, soon to be removed - - initialized = true; - } - - private void initializeTaskActionClientFactory() - { - if (taskActionClientFactory == null) { - taskActionClientFactory = new LocalTaskActionClientFactory( - taskStorage, - new TaskActionToolbox(taskQueue, taskLockbox, indexerDBCoordinator, emitter) - ); - } - } - - private void initializeTaskMasterLifecycle() - { - if (taskMaster == null) { - final DruidNode nodeConfig = getConfigFactory().build(DruidNode.class); - taskMaster = new TaskMaster( - taskQueue, - taskActionClientFactory, - nodeConfig, - getZkPaths(), - taskRunnerFactory, - resourceManagementSchedulerFactory, - getCuratorFramework(), - serviceAnnouncer, - emitter - ); - getLifecycle().addManagedInstance(taskMaster); - } - } - - private void initializePersistentTaskLogs() throws S3ServiceException - { - if (persistentTaskLogs == null) { - final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class); - if (taskLogConfig.getLogType().equalsIgnoreCase("s3")) { - initializeS3Service(); - persistentTaskLogs = new S3TaskLogs( - null, // TODO: eliminate - s3Service - ); - } else if (taskLogConfig.getLogType().equalsIgnoreCase("noop")) { - persistentTaskLogs = new NoopTaskLogs(); - } else { - throw new IAE("Unknown log type %s", taskLogConfig.getLogType()); - } - } - } - - private void initializeTaskLogProvider() - { - if (taskLogStreamer == null) { - final List providers = Lists.newArrayList(); - - // Use our TaskRunner if it is also a TaskLogStreamer - providers.add(new TaskRunnerTaskLogStreamer(IndexerCoordinatorNode.this.taskMaster)); - - // Use our persistent log storage - providers.add(persistentTaskLogs); - - taskLogStreamer = new SwitchingTaskLogStreamer(providers); - } - } - - @LifecycleStart - public synchronized void start() throws Exception - { - if (!initialized) { - doInit(); - } - - getLifecycle().start(); - } - - @LifecycleStop - public synchronized void stop() - { - getLifecycle().stop(); - } - - private void initializeServer() - { - if (server == null) { - server = Initialization.makeJettyServer(getConfigFactory().build(ServerConfig.class)); - - getLifecycle().addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - log.info("Starting Jetty"); - server.start(); - } - - @Override - public void stop() - { - log.info("Stopping Jetty"); - try { - server.stop(); - } - catch (Exception e) { - log.error(e, "Exception thrown while stopping Jetty"); - } - } - } - ); - } - } - - private void initializeJacksonInjections() - { - InjectableValues.Std injectables = new InjectableValues.Std(); - - injectables.addValue("s3Client", null) - .addValue("segmentPusher", null) - .addValue("chatHandlerProvider", null); - - getJsonMapper().setInjectableValues(injectables); - } - - private void initializeJacksonSubtypes() - { - getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class); - getJsonMapper().registerSubtypes(EventReceiverFirehoseFactory.class); - } - - private void initializeHttpClient() - { - if (httpClient == null) { - httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).withReadTimeout( - new Duration( - PropUtils.getProperty( - getProps(), - "druid.emitter.timeOut" - ) - ) - ).build(), getLifecycle() - ); - } - } - - private void initializeEmitter() - { - if (emitter == null) { - emitter = new ServiceEmitter( - PropUtils.getProperty(getProps(), "druid.service"), - PropUtils.getProperty(getProps(), "druid.host"), - Emitters.create(getProps(), httpClient, getJsonMapper(), getLifecycle()) - ); - } - EmittingLogger.registerEmitter(emitter); - } - - private void initializeMonitors() - { - if (monitors == null) { - monitors = Lists.newArrayList(); - monitors.add(new JvmMonitor()); - monitors.add(new SysMonitor()); - } - } - - private void initializeDB() - { - if (dbConnectorConfig == null) { - dbConnectorConfig = getConfigFactory().build(DbConnectorConfig.class); - } - if (dbi == null) { - dbi = new DbConnector(Suppliers.ofInstance(dbConnectorConfig), null); // TODO - } - } - - private void initializeIndexerCoordinatorConfig() - { - if (config == null) { - config = getConfigFactory().build(IndexerCoordinatorConfig.class); - } - } - - public void initializeS3Service() throws S3ServiceException - { - this.s3Service = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"), - PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey") - ) - ); - } - - public void initializeMergeDBCoordinator() - { - if (indexerDBCoordinator == null) { - indexerDBCoordinator = new IndexerDBCoordinator( - getJsonMapper(), - dbConnectorConfig, - getDbTables().get(), - dbi.getDBI() - ); - } - } - - public void initializeServiceDiscovery() throws Exception - { - final CuratorDiscoveryConfig config = getConfigFactory().build(CuratorDiscoveryConfig.class); - if (serviceDiscovery == null) { - this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - getCuratorFramework(), config, getLifecycle() - ); - } - if (serviceAnnouncer == null) { - this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery); - } - } - - public void initializeTaskQueue() - { - if (taskQueue == null) { - // Don't start it here. The TaskMaster will handle that when it feels like it. - taskQueue = new TaskQueue(taskStorage, taskLockbox); - } - } - - public void initializeTaskLockbox() - { - if (taskLockbox == null) { - taskLockbox = new TaskLockbox(taskStorage); - } - } - - public void initializeIndexerZkConfig() - { - if (indexerZkConfig == null) { - indexerZkConfig = getConfigFactory().build(IndexerZkConfig.class); - } - } - - public void initializeTaskStorage() - { - if (taskStorage == null) { - if (config.getStorageImpl().equals("local")) { - taskStorage = new HeapMemoryTaskStorage(); - } else if (config.getStorageImpl().equals("db")) { - final IndexerDbConnectorConfig dbConnectorConfig = getConfigFactory().build(IndexerDbConnectorConfig.class); - dbi.createTaskTables(); - - taskStorage = new DbTaskStorage(getJsonMapper(), null, dbi.getDBI()); // TODO: eliminate - } else { - throw new ISE("Invalid storage implementation: %s", config.getStorageImpl()); - } - } - } - - private void initializeTaskRunnerFactory(final JacksonConfigManager configManager) - { - if (taskRunnerFactory == null) { - if (config.getRunnerImpl().equals("remote")) { - taskRunnerFactory = new RemoteTaskRunnerFactory( - getCuratorFramework(), - getConfigFactory().build(RemoteTaskRunnerConfig.class), - getZkPaths(), - getJsonMapper(), - DSuppliers.of(configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)), - httpClient - ); - - } else if (config.getRunnerImpl().equals("local")) { - taskRunnerFactory = new ForkingTaskRunnerFactory( - getConfigFactory().build(ForkingTaskRunnerConfig.class), - getProps(), - getJsonMapper(), - persistentTaskLogs, - null // TODO: eliminate - ); - } else { - throw new ISE("Invalid runner implementation: %s", config.getRunnerImpl()); - } - } - } - - private void initializeResourceManagement(final JacksonConfigManager configManager) - { - if (resourceManagementSchedulerFactory == null) { - if (!config.isAutoScalingEnabled()) { - resourceManagementSchedulerFactory = new NoopResourceManagementSchedulerFactory(); - } else { - resourceManagementSchedulerFactory = new WithOpResourceManagementSchedulerFactory(configManager); - } - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMapper(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - return this; - } - - public Builder withLifecycle(Lifecycle lifecycle) - { - this.lifecycle = lifecycle; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public IndexerCoordinatorNode build() - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } else if (jsonMapper == null || smileMapper == null) { - throw new ISE( - "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", - jsonMapper, - smileMapper - ); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new IndexerCoordinatorNode(props, lifecycle, jsonMapper, smileMapper, configFactory); - } - } - - private static class NoopResourceManagementSchedulerFactory implements ResourceManagementSchedulerFactory - { - @Override - public ResourceManagementScheduler build(RemoteTaskRunner runner) - { - return new NoopResourceManagementScheduler(); - } - } - - private class WithOpResourceManagementSchedulerFactory implements ResourceManagementSchedulerFactory - { - private final JacksonConfigManager configManager; - - public WithOpResourceManagementSchedulerFactory(JacksonConfigManager configManager) - { - this.configManager = configManager; - } - - @Override - public ResourceManagementScheduler build(RemoteTaskRunner runner) - { - final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("ScalingExec--%d") - .build() - ); - final AtomicReference workerSetupData = configManager.watch( - WorkerSetupData.CONFIG_KEY, WorkerSetupData.class - ); - - AutoScalingStrategy strategy; - if (config.getAutoScalingImpl().equalsIgnoreCase("ec2")) { - strategy = new EC2AutoScalingStrategy( - getJsonMapper(), - new AmazonEC2Client( - new BasicAWSCredentials( - PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"), - PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey") - ) - ), - null, // TODO: eliminate - DSuppliers.of(workerSetupData) - ); - } else if (config.getAutoScalingImpl().equalsIgnoreCase("noop")) { - strategy = new NoopAutoScalingStrategy(); - } else { - throw new ISE("Invalid strategy implementation: %s", config.getAutoScalingImpl()); - } - - return new ResourceManagementScheduler( - runner, - new SimpleResourceManagementStrategy( - strategy, - getConfigFactory().build(SimpleResourceManagementConfig.class), - DSuppliers.of(workerSetupData) - ), - getConfigFactory().build(ResourceManagementSchedulerConfig.class), - scalingScheduledExec - ); - } - } -} From 55dbda2046c9fd09f938f852e15b2debbe1a828b Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 23 Aug 2013 17:59:48 -0500 Subject: [PATCH 39/92] 1) Worker appears to be running! It's also now known as the MiddleManager --- .../java/com/metamx/druid/QueryableNode.java | 2 +- .../druid/initialization/Initialization.java | 7 +- .../initialization/JettyServerModule.java | 9 ++- .../druid/initialization/ServerConfig.java | 30 ++++--- .../com/metamx/druid/concurrent/Execs.java | 5 ++ .../com/metamx/druid/guice/PolyBindTest.java | 8 +- .../guice/IndexingServiceModuleHelper.java | 35 ++++++++ .../druid/guice/MiddleManagerModule.java | 63 +++++++++++++++ .../metamx/druid/guice/OverlordModule.java | 19 +---- .../metamx/druid/guice/TaskLogsModule.java | 51 ++++++++++++ .../coordinator/ForkingTaskRunner.java | 2 + .../coordinator/ForkingTaskRunnerFactory.java | 6 +- .../coordinator/RemoteTaskRunner.java | 6 +- .../config/RemoteTaskRunnerConfig.java | 8 +- .../metamx/druid/indexing/worker/Worker.java | 13 --- .../worker/WorkerCuratorCoordinator.java | 45 ++++++----- .../indexing/worker/WorkerTaskMonitor.java | 16 ++-- .../indexing/worker/config/WorkerConfig.java | 51 +++++++++--- .../worker/executor/ExecutorNode.java | 4 +- .../indexing/worker/http/WorkerNode.java | 23 ++---- .../indexing/worker/http/WorkerResource.java | 4 - .../coordinator/RemoteTaskRunnerTest.java | 21 ++--- .../guice/{S3Module.java => AWSModule.java} | 2 +- .../main/java/io/druid/cli/CliHistorical.java | 6 +- .../java/io/druid/cli/CliMiddleManager.java | 80 +++++++++++++++++++ .../main/java/io/druid/cli/CliOverlord.java | 10 ++- .../main/java/io/druid/cli/CliRealtime.java | 7 +- services/src/main/java/io/druid/cli/Main.java | 4 +- 28 files changed, 383 insertions(+), 154 deletions(-) create mode 100644 indexing-service/src/main/java/com/metamx/druid/guice/IndexingServiceModuleHelper.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/guice/TaskLogsModule.java rename server/src/main/java/com/metamx/druid/guice/{S3Module.java => AWSModule.java} (98%) create mode 100644 services/src/main/java/io/druid/cli/CliMiddleManager.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java index 23b50bc9597..f8e90092b84 100644 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ b/client/src/main/java/com/metamx/druid/QueryableNode.java @@ -498,7 +498,7 @@ public abstract class QueryableNode extends Registering private void initializeServer() { if (server == null) { - setServer(Initialization.makeJettyServer(configFactory.build(ServerConfig.class))); + setServer(Initialization.makeJettyServer(null, configFactory.build(ServerConfig.class))); // TODO: eliminate lifecycle.addHandler( new Lifecycle.Handler() diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index c1ef6d28a23..3aa13537449 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -25,6 +25,7 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import com.google.common.primitives.Ints; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -181,7 +182,7 @@ public class Initialization return props; } - public static Server makeJettyServer(ServerConfig config) + public static Server makeJettyServer(DruidNode node, ServerConfig config) { final QueuedThreadPool threadPool = new QueuedThreadPool(); threadPool.setMinThreads(config.getNumThreads()); @@ -191,8 +192,8 @@ public class Initialization server.setThreadPool(threadPool); SelectChannelConnector connector = new SelectChannelConnector(); - connector.setPort(config.getPort()); - connector.setMaxIdleTime(config.getMaxIdleTimeMillis()); + connector.setPort(node.getPort()); + connector.setMaxIdleTime(Ints.checkedCast(config.getMaxIdleTime().toStandardDuration().getMillis())); connector.setStatsOn(true); server.setConnectors(new Connector[]{connector}); diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java index abbf3b091b5..73e45fcfe0f 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -17,8 +17,9 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.ConfigProvider; +import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; +import com.metamx.druid.guice.annotations.Self; import com.sun.jersey.api.core.DefaultResourceConfig; import com.sun.jersey.api.core.ResourceConfig; import com.sun.jersey.guice.JerseyServletModule; @@ -58,7 +59,7 @@ public class JettyServerModule extends JerseyServletModule { Binder binder = binder(); - ConfigProvider.bind(binder, ServerConfig.class); + JsonConfigProvider.bind(binder, "druid.server.http", ServerConfig.class); // The Guice servlet extension doesn't actually like requiring explicit bindings, so we do its job for it here. try { @@ -110,9 +111,9 @@ public class JettyServerModule extends JerseyServletModule } @Provides @LazySingleton - public Server getServer(Injector injector, Lifecycle lifecycle, ServerConfig config) + public Server getServer(Injector injector, Lifecycle lifecycle, @Self DruidNode node, ServerConfig config) { - final Server server = Initialization.makeJettyServer(config); + final Server server = Initialization.makeJettyServer(node, config); try { initializer.initialize(server, injector); } diff --git a/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java b/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java index 80c2d18c5e5..5610e476437 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java @@ -19,23 +19,31 @@ package com.metamx.druid.initialization; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; /** */ -public abstract class ServerConfig +public class ServerConfig { - @Config("druid.port") - public abstract int getPort(); + @JsonProperty + @Min(1) + private int numThreads = 10; - @Config("druid.http.numThreads") - @Default("10") - public abstract int getNumThreads(); + @JsonProperty + @NotNull + private Period maxIdleTime = new Period("PT5m"); - @Config("druid.http.maxIdleTimeMillis") - public int getMaxIdleTimeMillis() + public int getNumThreads() { - return 5 * 60 * 1000; // 5 minutes + return numThreads; + } + + public Period getMaxIdleTime() + { + return maxIdleTime; } } diff --git a/common/src/main/java/com/metamx/druid/concurrent/Execs.java b/common/src/main/java/com/metamx/druid/concurrent/Execs.java index 619a8199b00..06be9bdf4dd 100644 --- a/common/src/main/java/com/metamx/druid/concurrent/Execs.java +++ b/common/src/main/java/com/metamx/druid/concurrent/Execs.java @@ -37,6 +37,11 @@ public class Execs ); } + public static ExecutorService multiThreaded(int threads, String nameFormat) + { + return Executors.newFixedThreadPool(threads, makeThreadFactory(nameFormat)); + } + public static ScheduledExecutorService scheduledSingleThreaded(String nameFormat) { return Executors.newSingleThreadScheduledExecutor( diff --git a/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java b/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java index be688e7b560..a5f24e220fe 100644 --- a/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java +++ b/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java @@ -89,16 +89,16 @@ public class PolyBindTest Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); - props.setProperty("billy.type", "b"); + props.setProperty("billy", "b"); Assert.assertEquals("B", injector.getInstance(Gogo.class).go()); Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); - props.setProperty("billy.type", "a"); + props.setProperty("billy", "a"); Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); - props.setProperty("billy.type", "b"); + props.setProperty("billy", "b"); Assert.assertEquals("B", injector.getInstance(Gogo.class).go()); Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); - props.setProperty("billy.type", "c"); + props.setProperty("billy", "c"); Assert.assertEquals("A", injector.getInstance(Gogo.class).go()); Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go()); } diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/IndexingServiceModuleHelper.java b/indexing-service/src/main/java/com/metamx/druid/guice/IndexingServiceModuleHelper.java new file mode 100644 index 00000000000..5a1eedd5d90 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/guice/IndexingServiceModuleHelper.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; + +/** + */ +public class IndexingServiceModuleHelper +{ + public static void configureTaskRunnerConfigs(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.indexer.runner", ForkingTaskRunnerConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.runner", RemoteTaskRunnerConfig.class); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java new file mode 100644 index 00000000000..e8e5bf6a84c --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java @@ -0,0 +1,63 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.guice.annotations.Self; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.TaskRunner; +import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; +import com.metamx.druid.indexing.worker.WorkerTaskMonitor; +import com.metamx.druid.indexing.worker.config.WorkerConfig; +import com.metamx.druid.initialization.DruidNode; + +/** + */ +public class MiddleManagerModule implements Module +{ + @Override + public void configure(Binder binder) + { + IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); + + JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class); + + binder.bind(TaskRunner.class).to(ForkingTaskRunner.class); + binder.bind(ForkingTaskRunner.class).in(LazySingleton.class); + + binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); + binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); + + } + + @Provides @LazySingleton + public Worker getWorker(@Self DruidNode node, WorkerConfig config) + { + return new Worker( + node.getHost(), + config.getIp(), + config.getCapacity(), + config.getVersion() + ); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java index 0b1c908d40c..3d784cc3ba6 100644 --- a/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/guice/OverlordModule.java @@ -29,9 +29,6 @@ import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionToolbox; -import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; -import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; -import com.metamx.druid.indexing.common.tasklogs.S3TaskLogsConfig; import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; import com.metamx.druid.indexing.common.tasklogs.TaskLogs; @@ -47,8 +44,6 @@ import com.metamx.druid.indexing.coordinator.TaskQueue; import com.metamx.druid.indexing.coordinator.TaskRunnerFactory; import com.metamx.druid.indexing.coordinator.TaskStorage; import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.indexing.coordinator.http.OverlordRedirectInfo; import com.metamx.druid.indexing.coordinator.scaling.AutoScalingStrategy; import com.metamx.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy; @@ -111,17 +106,6 @@ public class OverlordModule implements Module storageBinder.addBinding("db").to(DbTaskStorage.class); binder.bind(DbTaskStorage.class).in(LazySingleton.class); - - PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(NoopTaskLogs.class)); - final MapBinder taskLogBinder = PolyBind.optionBinder(binder, Key.get(TaskLogs.class)); - - JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); - taskLogBinder.addBinding("s3").to(S3TaskLogs.class); - binder.bind(S3TaskLogs.class).in(LazySingleton.class); - - taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class); - binder.bind(NoopTaskLogs.class).in(LazySingleton.class); - } private void configureRunners(Binder binder) @@ -131,11 +115,10 @@ public class OverlordModule implements Module ); final MapBinder biddy = PolyBind.optionBinder(binder, Key.get(TaskRunnerFactory.class)); - JsonConfigProvider.bind(binder, "druid.indexer.runner", ForkingTaskRunnerConfig.class); + IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); biddy.addBinding("local").to(ForkingTaskRunnerFactory.class); binder.bind(ForkingTaskRunnerFactory.class).in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.indexer.runner", RemoteTaskRunnerConfig.class); biddy.addBinding("remote").to(RemoteTaskRunnerFactory.class).in(LazySingleton.class); binder.bind(RemoteTaskRunnerFactory.class).in(LazySingleton.class); } diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/TaskLogsModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/TaskLogsModule.java new file mode 100644 index 00000000000..81e7311d634 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/guice/TaskLogsModule.java @@ -0,0 +1,51 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogsConfig; +import com.metamx.druid.indexing.common.tasklogs.TaskLogPusher; +import com.metamx.druid.indexing.common.tasklogs.TaskLogs; + +/** + */ +public class TaskLogsModule implements Module +{ + @Override + public void configure(Binder binder) + { + PolyBind.createChoice(binder, "druid.indexer.logs.type", Key.get(TaskLogs.class), Key.get(NoopTaskLogs.class)); + final MapBinder taskLogBinder = PolyBind.optionBinder(binder, Key.get(TaskLogs.class)); + + JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); + taskLogBinder.addBinding("s3").to(S3TaskLogs.class); + binder.bind(S3TaskLogs.class).in(LazySingleton.class); + + taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class); + binder.bind(NoopTaskLogs.class).in(LazySingleton.class); + + binder.bind(TaskLogPusher.class).to(TaskLogs.class); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 3c223a3f8de..6c55b1b6a95 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -36,6 +36,7 @@ import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.guice.annotations.Self; @@ -80,6 +81,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer private final Map tasks = Maps.newHashMap(); + @Inject public ForkingTaskRunner( ForkingTaskRunnerConfig config, Properties props, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java index 9dfc4cdb093..68be6485310 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunnerFactory.java @@ -22,7 +22,7 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.TaskLogPusher; import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; import com.metamx.druid.initialization.DruidNode; @@ -35,7 +35,7 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory private final ForkingTaskRunnerConfig config; private final Properties props; private final ObjectMapper jsonMapper; - private final TaskLogs persistentTaskLogs; + private final TaskLogPusher persistentTaskLogs; private final DruidNode node; @Inject @@ -43,7 +43,7 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory final ForkingTaskRunnerConfig config, final Properties props, final ObjectMapper jsonMapper, - final TaskLogs persistentTaskLogs, + final TaskLogPusher persistentTaskLogs, @Self DruidNode node ) { this.config = config; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index c12077eb8a2..6912e83be32 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -517,13 +517,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer timeoutStopwatch.start(); synchronized (statusLock) { while (!isWorkerRunningTask(theWorker, task)) { - statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); - if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) { + statusLock.wait(config.getTaskAssignmentTimeout().getMillis()); + if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeout().getMillis()) { log.error( "Something went wrong! %s never ran task %s after %s!", theWorker.getHost(), task.getId(), - config.getTaskAssignmentTimeoutDuration() + config.getTaskAssignmentTimeout() ); taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index fbf3bb1cc8e..0764fee3905 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -20,7 +20,7 @@ package com.metamx.druid.indexing.coordinator.config; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Duration; +import org.joda.time.Period; import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; @@ -31,7 +31,7 @@ public class RemoteTaskRunnerConfig { @JsonProperty @NotNull - private Duration taskAssignmentTimeoutDuration = new Duration("PT5M"); + private Period taskAssignmentTimeout = new Period("PT5M"); @JsonProperty private boolean compressZnodes = false; @@ -43,9 +43,9 @@ public class RemoteTaskRunnerConfig @Min(10 * 1024) private long maxZnodeBytes = 512 * 1024; - public Duration getTaskAssignmentTimeoutDuration() + public Period getTaskAssignmentTimeout() { - return taskAssignmentTimeoutDuration; + return taskAssignmentTimeout; } public boolean isCompressZnodes() diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java index 7749ff8565f..7d60f06fb2a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java @@ -21,7 +21,6 @@ package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.indexing.worker.config.WorkerConfig; /** * A container for worker metadata. @@ -33,18 +32,6 @@ public class Worker private final int capacity; private final String version; - public Worker( - WorkerConfig config - ) - { - this( - config.getHost(), - config.getIp(), - config.getCapacity(), - config.getVersion() - ); - } - @JsonCreator public Worker( @JsonProperty("host") String host, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java index 89d6c313487..572c307a5f1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -24,13 +24,15 @@ import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.announcement.Announcer; import com.metamx.druid.indexing.common.TaskStatus; -import com.metamx.druid.indexing.common.config.IndexerZkConfig; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.CreateMode; import org.joda.time.DateTime; @@ -48,9 +50,9 @@ public class WorkerCuratorCoordinator private final Object lock = new Object(); private final ObjectMapper jsonMapper; + private final RemoteTaskRunnerConfig config; private final CuratorFramework curatorFramework; private final Worker worker; - private final IndexerZkConfig config; private final Announcer announcer; private final String baseAnnouncementsPath; @@ -59,23 +61,25 @@ public class WorkerCuratorCoordinator private volatile boolean started; + @Inject public WorkerCuratorCoordinator( ObjectMapper jsonMapper, - IndexerZkConfig config, + ZkPathsConfig zkPaths, + RemoteTaskRunnerConfig config, CuratorFramework curatorFramework, Worker worker ) { this.jsonMapper = jsonMapper; + this.config = config; this.curatorFramework = curatorFramework; this.worker = worker; - this.config = config; this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor()); - this.baseAnnouncementsPath = getPath(Arrays.asList(config.getIndexerAnnouncementPath(), worker.getHost())); - this.baseTaskPath = getPath(Arrays.asList(config.getIndexerTaskPath(), worker.getHost())); - this.baseStatusPath = getPath(Arrays.asList(config.getIndexerStatusPath(), worker.getHost())); + this.baseAnnouncementsPath = getPath(Arrays.asList(zkPaths.getIndexerAnnouncementPath(), worker.getHost())); + this.baseTaskPath = getPath(Arrays.asList(zkPaths.getIndexerTaskPath(), worker.getHost())); + this.baseStatusPath = getPath(Arrays.asList(zkPaths.getIndexerStatusPath(), worker.getHost())); } @LifecycleStart @@ -125,8 +129,8 @@ public class WorkerCuratorCoordinator if (curatorFramework.checkExists().forPath(path) == null) { try { byte[] rawBytes = jsonMapper.writeValueAsBytes(data); - if (rawBytes.length > config.getMaxNumBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); + if (rawBytes.length > config.getMaxZnodeBytes()) { + throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes()); } curatorFramework.create() @@ -189,15 +193,13 @@ public class WorkerCuratorCoordinator try { byte[] rawBytes = jsonMapper.writeValueAsBytes(status); - if (rawBytes.length > config.getMaxNumBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); + if (rawBytes.length > config.getMaxZnodeBytes()) { + throw new ISE( + "Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes() + ); } - curatorFramework.create() - .withMode(CreateMode.EPHEMERAL) - .forPath( - getStatusPathForId(status.getId()), rawBytes - ); + curatorFramework.create().withMode(CreateMode.EPHEMERAL).forPath(getStatusPathForId(status.getId()), rawBytes); } catch (Exception e) { throw Throwables.propagate(e); @@ -218,14 +220,13 @@ public class WorkerCuratorCoordinator return; } byte[] rawBytes = jsonMapper.writeValueAsBytes(status); - if (rawBytes.length > config.getMaxNumBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes()); + if (rawBytes.length > config.getMaxZnodeBytes()) { + throw new ISE( + "Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes() + ); } - curatorFramework.setData() - .forPath( - getStatusPathForId(status.getId()), rawBytes - ); + curatorFramework.setData().forPath(getStatusPathForId(status.getId()), rawBytes); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java index 584ec0a2ef6..2096f2e4059 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java @@ -20,11 +20,14 @@ package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.concurrent.Execs; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskRunner; +import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; @@ -55,21 +58,24 @@ public class WorkerTaskMonitor private final ExecutorService exec; private final List running = new CopyOnWriteArrayList(); + @Inject public WorkerTaskMonitor( ObjectMapper jsonMapper, - PathChildrenCache pathChildrenCache, CuratorFramework cf, WorkerCuratorCoordinator workerCuratorCoordinator, TaskRunner taskRunner, - ExecutorService exec + WorkerConfig workerConfig ) { this.jsonMapper = jsonMapper; - this.pathChildrenCache = pathChildrenCache; + this.pathChildrenCache = new PathChildrenCache( + cf, workerCuratorCoordinator.getTaskPathForWorker(), false, true, Execs.makeThreadFactory("TaskMonitorCache-%s") + ); this.cf = cf; this.workerCuratorCoordinator = workerCuratorCoordinator; this.taskRunner = taskRunner; - this.exec = exec; + + this.exec = Execs.multiThreaded(workerConfig.getCapacity(), "WorkerTaskMonitor-%d"); } /** @@ -81,7 +87,6 @@ public class WorkerTaskMonitor public void start() { try { - pathChildrenCache.start(); pathChildrenCache.getListenable().addListener( new PathChildrenCacheListener() { @@ -153,6 +158,7 @@ public class WorkerTaskMonitor } } ); + pathChildrenCache.start(); } catch (Exception e) { log.makeAlert(e, "Exception starting WorkerTaskMonitor") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java index 2474704a418..9c29cbb16a3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java @@ -19,27 +19,54 @@ package com.metamx.druid.indexing.worker.config; -import org.skife.config.Config; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; /** */ -public abstract class WorkerConfig +public class WorkerConfig { - @Config("druid.host") - public abstract String getHost(); + @JsonProperty + @NotNull + private String ip = null; - @Config("druid.worker.ip") - public abstract String getIp(); + @JsonProperty + @NotNull + private String version = null; - @Config("druid.worker.version") - public abstract String getVersion(); + @JsonProperty + @NotNull + private String overlordService = null; - @Config("druid.worker.masterService") - public abstract String getMasterService(); + @JsonProperty + @Min(1) + private int capacity = Runtime.getRuntime().availableProcessors() - 1; + + public String getIp() + { + return ip; + } + + public String getVersion() + { + return version; + } + + public String getOverlordService() + { + return overlordService; + } - @Config("druid.worker.capacity") public int getCapacity() { - return Runtime.getRuntime().availableProcessors() - 1; + return capacity; + } + + public WorkerConfig setCapacity(int capacity) + { + this.capacity = capacity; + return this; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 351d617fdcf..53baaec3574 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -218,7 +218,7 @@ public class ExecutorNode extends BaseServerNode private void initializeServer() { if (server == null) { - server = Initialization.makeJettyServer(configFactory.build(ServerConfig.class)); + server = Initialization.makeJettyServer(null, configFactory.build(ServerConfig.class)); lifecycle.addHandler( new Lifecycle.Handler() @@ -361,7 +361,7 @@ public class ExecutorNode extends BaseServerNode } if (coordinatorServiceProvider == null) { this.coordinatorServiceProvider = Initialization.makeServiceProvider( - workerConfig.getMasterService(), + workerConfig.getOverlordService(), serviceDiscovery, lifecycle ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 7391fdf4e93..6b5bfbd6c99 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -37,7 +37,6 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; @@ -46,7 +45,6 @@ import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; import com.metamx.druid.indexing.common.tasklogs.TaskLogs; import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.indexing.worker.Worker; import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; import com.metamx.druid.indexing.worker.WorkerTaskMonitor; import com.metamx.druid.indexing.worker.config.WorkerConfig; @@ -67,7 +65,6 @@ import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; import org.eclipse.jetty.server.Server; @@ -82,8 +79,6 @@ import org.skife.config.ConfigurationObjectFactory; import java.util.List; import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -230,7 +225,7 @@ public class WorkerNode extends QueryableNode private void initializeServer() { if (server == null) { - server = Initialization.makeJettyServer(getConfigFactory().build(ServerConfig.class)); + server = Initialization.makeJettyServer(null, getConfigFactory().build(ServerConfig.class)); getLifecycle().addHandler( new Lifecycle.Handler() @@ -339,7 +334,7 @@ public class WorkerNode extends QueryableNode } if (coordinatorServiceProvider == null) { this.coordinatorServiceProvider = Initialization.makeServiceProvider( - workerConfig.getMasterService(), + workerConfig.getOverlordService(), serviceDiscovery, getLifecycle() ); @@ -351,9 +346,10 @@ public class WorkerNode extends QueryableNode if (workerCuratorCoordinator == null) { workerCuratorCoordinator = new WorkerCuratorCoordinator( getJsonMapper(), - getConfigFactory().build(IndexerZkConfig.class), + getZkPaths(), + null, // TODO: eliminate getCuratorFramework(), - new Worker(workerConfig) + null // TODO: eliminate ); getLifecycle().addManagedInstance(workerCuratorCoordinator); } @@ -391,21 +387,14 @@ public class WorkerNode extends QueryableNode public void initializeWorkerTaskMonitor() { if (workerTaskMonitor == null) { - final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getCapacity()); final CuratorFramework curatorFramework = getCuratorFramework(); - final PathChildrenCache pathChildrenCache = new PathChildrenCache( - curatorFramework, - workerCuratorCoordinator.getTaskPathForWorker(), - false - ); workerTaskMonitor = new WorkerTaskMonitor( getJsonMapper(), - pathChildrenCache, curatorFramework, workerCuratorCoordinator, forkingTaskRunner, - workerExec + workerConfig ); getLifecycle().addManagedInstance(workerTaskMonitor); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java index 5dc5301160e..57a023e3b33 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java @@ -19,7 +19,6 @@ package com.metamx.druid.indexing.worker.http; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableMap; import com.google.common.io.InputSupplier; @@ -44,17 +43,14 @@ public class WorkerResource { private static final Logger log = new Logger(WorkerResource.class); - private final ObjectMapper jsonMapper; private final ForkingTaskRunner taskRunner; @Inject public WorkerResource( - ObjectMapper jsonMapper, ForkingTaskRunner taskRunner ) throws Exception { - this.jsonMapper = jsonMapper; this.taskRunner = taskRunner; } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index cc897a00b9c..c985f6e4aba 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -19,7 +19,6 @@ import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolboxFactory; -import com.metamx.druid.indexing.common.config.IndexerZkConfig; import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.task.TaskResource; @@ -28,20 +27,20 @@ import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; import com.metamx.druid.indexing.worker.WorkerTaskMonitor; +import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingCluster; import org.apache.zookeeper.CreateMode; import org.easymock.EasyMock; import org.joda.time.DateTime; -import org.joda.time.Duration; import org.joda.time.Interval; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -293,20 +292,15 @@ public class RemoteTaskRunnerTest { workerCuratorCoordinator = new WorkerCuratorCoordinator( jsonMapper, - new IndexerZkConfig() + new ZkPathsConfig() { @Override public String getZkBasePath() { return basePath; } - - @Override - public long getMaxNumBytes() - { - return 1000; - } }, + new TestRemoteTaskRunnerConfig(), cf, worker ); @@ -315,7 +309,6 @@ public class RemoteTaskRunnerTest // Start a task monitor workerTaskMonitor = new WorkerTaskMonitor( jsonMapper, - new PathChildrenCache(cf, tasksPath, true), cf, workerCuratorCoordinator, new ThreadPoolTaskRunner( @@ -344,7 +337,7 @@ public class RemoteTaskRunnerTest }, null, null, null, null, null, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() ), - Executors.newSingleThreadExecutor() + new WorkerConfig().setCapacity(1) ); jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test")); jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime")); @@ -397,9 +390,9 @@ public class RemoteTaskRunnerTest } @Override - public Duration getTaskAssignmentTimeoutDuration() + public Period getTaskAssignmentTimeout() { - return new Duration(60000); + return new Period(60000); } @Override diff --git a/server/src/main/java/com/metamx/druid/guice/S3Module.java b/server/src/main/java/com/metamx/druid/guice/AWSModule.java similarity index 98% rename from server/src/main/java/com/metamx/druid/guice/S3Module.java rename to server/src/main/java/com/metamx/druid/guice/AWSModule.java index 7cf04b762e9..63e005475be 100644 --- a/server/src/main/java/com/metamx/druid/guice/S3Module.java +++ b/server/src/main/java/com/metamx/druid/guice/AWSModule.java @@ -33,7 +33,7 @@ import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** */ -public class S3Module implements Module +public class AWSModule implements Module { @Override public void configure(Binder binder) diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 04aca3c58ba..3f30002c577 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -24,17 +24,15 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.AWSModule; import com.metamx.druid.guice.AnnouncerModule; import com.metamx.druid.guice.DataSegmentPullerModule; -import com.metamx.druid.guice.DataSegmentPusherModule; import com.metamx.druid.guice.DruidProcessingModule; import com.metamx.druid.guice.HistoricalModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.S3Module; -import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; @@ -69,7 +67,7 @@ public class CliHistorical extends ServerRunnable CuratorModule.class, AnnouncerModule.class, DruidProcessingModule.class, - S3Module.class, + AWSModule.class, DataSegmentPullerModule.class, new MetricsModule().register(ServerMonitor.class), new StorageNodeModule("historical"), diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java new file mode 100644 index 00000000000..513600f0b10 --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -0,0 +1,80 @@ +package io.druid.cli; + +import com.google.inject.Injector; +import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.AWSModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.MiddleManagerModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.TaskLogsModule; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.indexing.worker.WorkerTaskMonitor; +import com.metamx.druid.indexing.worker.http.WorkerResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerInitializer; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Command; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.DefaultHandler; +import org.eclipse.jetty.server.handler.HandlerList; +import org.eclipse.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.GzipFilter; + +/** + */ +@Command( + name = "middleManager", + description = "Runs a Middle Manager, this is a \"task\" node used as part of the remote indexing service." +) +public class CliMiddleManager extends ServerRunnable +{ + private static final Logger log = new Logger(CliMiddleManager.class); + + public CliMiddleManager() + { + super(log); + } + + @Override + protected Injector getInjector() + { + return Initialization.makeInjector( + new LifecycleModule().register(WorkerTaskMonitor.class), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule(), + ServerModule.class, + new JettyServerModule(new MiddleManagerJettyServerInitializer()) + .addResource(StatusResource.class) + .addResource(WorkerResource.class), + new AWSModule(), + new TaskLogsModule(), + new MiddleManagerModule() + ); + } + + private static class MiddleManagerJettyServerInitializer implements JettyServerInitializer + { + @Override + public void initialize(Server server, Injector injector) + { + final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); + root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); + root.addFilter(GzipFilter.class, "/*", null); + root.addFilter(GuiceFilter.class, "/*", null); + + final HandlerList handlerList = new HandlerList(); + handlerList.setHandlers(new Handler[]{root, new DefaultHandler()}); + server.setHandler(handlerList); + } + } +} diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index c7e43955bae..46bc734138d 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -24,14 +24,16 @@ import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.AWSModule; import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.OverlordModule; -import com.metamx.druid.guice.S3Module; import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.TaskLogsModule; import com.metamx.druid.http.RedirectFilter; +import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.http.IndexerCoordinatorResource; import com.metamx.druid.initialization.EmitterModule; @@ -77,12 +79,14 @@ public class CliOverlord extends ServerRunnable CuratorModule.class, new MetricsModule(), ServerModule.class, - new S3Module(), + new AWSModule(), new DbConnectorModule(), new JacksonConfigManagerModule(), new JettyServerModule(new OverlordJettyServerInitializer()) - .addResource(IndexerCoordinatorResource.class), + .addResource(IndexerCoordinatorResource.class) + .addResource(StatusResource.class), new DiscoveryModule(), + new TaskLogsModule(), new OverlordModule() ); } diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 2defc0a2ae2..165996a4497 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -21,8 +21,8 @@ package io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; -import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.guice.AWSModule; import com.metamx.druid.guice.AnnouncerModule; import com.metamx.druid.guice.DataSegmentPusherModule; import com.metamx.druid.guice.DbConnectorModule; @@ -32,15 +32,12 @@ import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.RealtimeModule; -import com.metamx.druid.guice.S3Module; -import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.ServerViewModule; import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerModule; -import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.metrics.MetricsModule; import com.metamx.druid.realtime.RealtimeManager; import io.airlift.command.Command; @@ -71,7 +68,7 @@ public class CliRealtime extends ServerRunnable CuratorModule.class, AnnouncerModule.class, DruidProcessingModule.class, - S3Module.class, + AWSModule.class, DataSegmentPusherModule.class, new MetricsModule(), new StorageNodeModule("realtime"), diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 0ad07dd5a8b..b2274ffe096 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -39,7 +39,9 @@ public class Main builder.withGroup("server") .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) - .withCommands(CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliOverlord.class); + .withCommands( + CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliOverlord.class, CliMiddleManager.class + ); builder.withGroup("example") .withDescription("Run an example") From 6636ef1ea815c86f4ff3c93b479f7ae87bf498d0 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 23 Aug 2013 18:00:56 -0500 Subject: [PATCH 40/92] Remove unused files again --- .../indexing/worker/http/WorkerMain.java | 50 -- .../indexing/worker/http/WorkerNode.java | 464 ------------------ 2 files changed, 514 deletions(-) delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java deleted file mode 100644 index 9fa2f131979..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java +++ /dev/null @@ -1,50 +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.indexing.worker.http; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; - -/** - */ -public class WorkerMain -{ - private static final Logger log = new Logger(WorkerMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance(WorkerNode.builder().build()); - - try { - lifecycle.start(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - - lifecycle.join(); - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java deleted file mode 100644 index 6b5bfbd6c99..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ /dev/null @@ -1,464 +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.indexing.worker.http; - -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.collect.Lists; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.QueryableNode; -import com.metamx.druid.curator.CuratorConfig; -import com.metamx.druid.http.GuiceServletConfig; -import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.config.TaskLogConfig; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; -import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; -import com.metamx.druid.indexing.common.tasklogs.TaskLogs; -import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; -import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; -import com.metamx.druid.indexing.worker.WorkerTaskMonitor; -import com.metamx.druid.indexing.worker.config.WorkerConfig; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.JvmMonitor; -import com.metamx.metrics.Monitor; -import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceProvider; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; -import org.joda.time.Duration; -import org.skife.config.ConfigurationObjectFactory; - -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ScheduledExecutorService; - -/** - */ -public class WorkerNode extends QueryableNode -{ - private static final EmittingLogger log = new EmittingLogger(WorkerNode.class); - - public static Builder builder() - { - return new Builder(); - } - - private RestS3Service s3Service = null; - private List monitors = null; - private HttpClient httpClient = null; - private ServiceEmitter emitter = null; - private WorkerConfig workerConfig = null; - private ServiceDiscovery serviceDiscovery = null; - private ServiceProvider coordinatorServiceProvider = null; - private WorkerCuratorCoordinator workerCuratorCoordinator = null; - private WorkerTaskMonitor workerTaskMonitor = null; - private TaskLogs persistentTaskLogs = null; - private ForkingTaskRunner forkingTaskRunner = null; - private Server server = null; - - private boolean initialized = false; - - public WorkerNode( - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super("indexer-worker", log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public WorkerNode setHttpClient(HttpClient httpClient) - { - this.httpClient = httpClient; - return this; - } - - public WorkerNode setEmitter(ServiceEmitter emitter) - { - this.emitter = emitter; - return this; - } - - public WorkerNode setS3Service(RestS3Service s3Service) - { - this.s3Service = s3Service; - return this; - } - - public WorkerNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) - { - this.coordinatorServiceProvider = coordinatorServiceProvider; - return this; - } - - public WorkerNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) - { - this.serviceDiscovery = serviceDiscovery; - return this; - } - - public WorkerNode setWorkerCuratorCoordinator(WorkerCuratorCoordinator workerCuratorCoordinator) - { - this.workerCuratorCoordinator = workerCuratorCoordinator; - return this; - } - - public WorkerNode setForkingTaskRunner(ForkingTaskRunner forkingTaskRunner) - { - this.forkingTaskRunner = forkingTaskRunner; - return this; - } - - public WorkerNode setWorkerTaskMonitor(WorkerTaskMonitor workerTaskMonitor) - { - this.workerTaskMonitor = workerTaskMonitor; - return this; - } - - public void doInit() throws Exception - { - initializeHttpClient(); - initializeEmitter(); - initializeMonitors(); - initializeMergerConfig(); - initializeServiceDiscovery(); - initializeJacksonInjections(); - initializeJacksonSubtypes(); - initializeCuratorCoordinator(); - initializePersistentTaskLogs(); - initializeTaskRunner(); - initializeWorkerTaskMonitor(); - initializeServer(); - - final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(getLifecycle()); - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - final MonitorScheduler monitorScheduler = new MonitorScheduler( - getConfigFactory().build(MonitorSchedulerConfig.class), - globalScheduledExec, - emitter, - monitors - ); - getLifecycle().addManagedInstance(monitorScheduler); - - final Injector injector = Guice.createInjector( - new WorkerServletModule( - getJsonMapper(), - emitter, - forkingTaskRunner - ) - ); - final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); - } - - @LifecycleStart - public synchronized void start() throws Exception - { - if (!initialized) { - doInit(); - } - - getLifecycle().start(); - } - - @LifecycleStop - public synchronized void stop() - { - getLifecycle().stop(); - } - - private void initializeServer() - { - if (server == null) { - server = Initialization.makeJettyServer(null, getConfigFactory().build(ServerConfig.class)); - - getLifecycle().addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - log.info("Starting Jetty"); - server.start(); - } - - @Override - public void stop() - { - log.info("Stopping Jetty"); - try { - server.stop(); - } - catch (Exception e) { - log.error(e, "Exception thrown while stopping Jetty"); - } - } - } - ); - } - } - - private void initializeJacksonInjections() - { - InjectableValues.Std injectables = new InjectableValues.Std(); - - injectables.addValue("s3Client", null) - .addValue("segmentPusher", null) - .addValue("chatHandlerProvider", null); - - getJsonMapper().setInjectableValues(injectables); - } - - private void initializeJacksonSubtypes() - { - getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class); - getJsonMapper().registerSubtypes(EventReceiverFirehoseFactory.class); - } - - private void initializeHttpClient() - { - if (httpClient == null) { - httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1) - .withReadTimeout(new Duration(PropUtils.getProperty(getProps(), "druid.emitter.timeOut"))) - .build(), getLifecycle() - ); - } - } - - private void initializeEmitter() - { - if (emitter == null) { - emitter = new ServiceEmitter( - PropUtils.getProperty(getProps(), "druid.service"), - PropUtils.getProperty(getProps(), "druid.host"), - Emitters.create(getProps(), httpClient, getJsonMapper(), getLifecycle()) - ); - } - EmittingLogger.registerEmitter(emitter); - } - - private void initializeS3Service() throws S3ServiceException - { - if (s3Service == null) { - s3Service = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(getProps(), "com.metamx.aws.accessKey"), - PropUtils.getProperty(getProps(), "com.metamx.aws.secretKey") - ) - ); - } - } - - private void initializeMonitors() - { - if (monitors == null) { - monitors = Lists.newArrayList(); - monitors.add(new JvmMonitor()); - monitors.add(new SysMonitor()); - } - } - - private void initializeMergerConfig() - { - if (workerConfig == null) { - workerConfig = getConfigFactory().build(WorkerConfig.class); - } - } - - public void initializeServiceDiscovery() throws Exception - { - if (serviceDiscovery == null) { - final CuratorDiscoveryConfig config = getJsonConfigurator() - .configurate(getProps(), "druid.discovery.curator", CuratorDiscoveryConfig.class); - this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - Initialization.makeCuratorFramework(getConfigFactory().build(CuratorConfig.class), getLifecycle()), - config, - getLifecycle() - ); - } - if (coordinatorServiceProvider == null) { - this.coordinatorServiceProvider = Initialization.makeServiceProvider( - workerConfig.getOverlordService(), - serviceDiscovery, - getLifecycle() - ); - } - } - - public void initializeCuratorCoordinator() - { - if (workerCuratorCoordinator == null) { - workerCuratorCoordinator = new WorkerCuratorCoordinator( - getJsonMapper(), - getZkPaths(), - null, // TODO: eliminate - getCuratorFramework(), - null // TODO: eliminate - ); - getLifecycle().addManagedInstance(workerCuratorCoordinator); - } - } - - private void initializePersistentTaskLogs() throws S3ServiceException - { - if (persistentTaskLogs == null) { - final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class); - if (taskLogConfig.getLogStorageBucket() != null) { - initializeS3Service(); - persistentTaskLogs = new S3TaskLogs( - null, // TODO: eliminate - s3Service - ); - } else { - persistentTaskLogs = new NoopTaskLogs(); - } - } - } - - public void initializeTaskRunner() - { - if (forkingTaskRunner == null) { - forkingTaskRunner = new ForkingTaskRunner( - getConfigFactory().build(ForkingTaskRunnerConfig.class), - getProps(), - persistentTaskLogs, - getJsonMapper(), - null // todo: eliminate - ); - } - } - - public void initializeWorkerTaskMonitor() - { - if (workerTaskMonitor == null) { - final CuratorFramework curatorFramework = getCuratorFramework(); - - workerTaskMonitor = new WorkerTaskMonitor( - getJsonMapper(), - curatorFramework, - workerCuratorCoordinator, - forkingTaskRunner, - workerConfig - ); - getLifecycle().addManagedInstance(workerTaskMonitor); - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMapper(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - return this; - } - - public Builder withLifecycle(Lifecycle lifecycle) - { - this.lifecycle = lifecycle; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public WorkerNode build() - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } else if (jsonMapper == null || smileMapper == null) { - throw new ISE( - "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", - jsonMapper, - smileMapper - ); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new WorkerNode(props, lifecycle, jsonMapper, smileMapper, configFactory); - } - } -} From 269997dc94afaf328b281d33e78a0847702e14fd Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 26 Aug 2013 18:08:41 -0500 Subject: [PATCH 41/92] 1) ExecutorNode is working, except for the running of the task. Need to adjust it to be able to run a task and then everything will be wonderful --- .../metamx/druid/client/selector/Server.java | 1 + .../initialization/JettyServerModule.java | 2 +- .../com/metamx/druid/concurrent/Execs.java | 8 +- .../druid/guice/MiddleManagerModule.java | 1 - .../com/metamx/druid/guice/PeonModule.java | 67 ++++++++++++++ .../druid/indexing/common/RetryPolicy.java | 33 +++---- .../indexing/common/RetryPolicyConfig.java | 70 ++++++++++++++ .../indexing/common/RetryPolicyFactory.java | 3 +- .../indexing/common/TaskToolboxFactory.java | 2 + .../actions/RemoteTaskActionClient.java | 52 +++++------ .../RemoteTaskActionClientFactory.java | 13 ++- .../indexing/common/config/TaskConfig.java | 63 +++++++++---- .../EventReceivingChatHandlerProvider.java | 12 ++- .../coordinator/ThreadPoolTaskRunner.java | 9 +- .../worker/executor/ExecutorNode.java | 20 +--- .../RetryPolicyTest.java | 29 ++---- .../coordinator/RemoteTaskRunnerTest.java | 29 +----- .../coordinator/TaskLifecycleTest.java | 27 +----- .../indexing/IndexingServiceSelector.java | 6 ++ .../IndexingServiceSelectorConfig.java | 26 ++---- .../metamx/druid/guice/CoordinatorModule.java | 43 --------- .../guice/IndexingServiceDiscoveryModule.java | 83 +++++++++++++++++ .../metamx/druid/guice/StorageNodeModule.java | 5 +- .../druid/master/DruidMasterConfig.java | 6 -- .../metamx/druid/master/DruidMasterTest.java | 6 -- .../java/io/druid/cli/CliCoordinator.java | 2 + .../main/java/io/druid/cli/CliHistorical.java | 2 + .../src/main/java/io/druid/cli/CliPeon.java | 92 +++++++++++++++++++ .../main/java/io/druid/cli/CliRealtime.java | 2 + .../java/io/druid/cli/CliRealtimeExample.java | 2 + services/src/main/java/io/druid/cli/Main.java | 3 +- 31 files changed, 469 insertions(+), 250 deletions(-) create mode 100644 indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyConfig.java rename indexing-service/src/test/java/com/metamx/druid/indexing/{coordinator => common}/RetryPolicyTest.java (59%) rename indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java => server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelectorConfig.java (56%) create mode 100644 server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java create mode 100644 services/src/main/java/io/druid/cli/CliPeon.java diff --git a/client/src/main/java/com/metamx/druid/client/selector/Server.java b/client/src/main/java/com/metamx/druid/client/selector/Server.java index 9859c888239..a5f6b910e9a 100644 --- a/client/src/main/java/com/metamx/druid/client/selector/Server.java +++ b/client/src/main/java/com/metamx/druid/client/selector/Server.java @@ -4,6 +4,7 @@ package com.metamx.druid.client.selector; */ public interface Server { + public String getScheme(); public String getHost(); public int getPort(); } diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java index 73e45fcfe0f..7c494754712 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -81,7 +81,7 @@ public class JettyServerModule extends JerseyServletModule .annotatedWith(Names.named("resourceClasses")) .toInstance(theResources); for (Class resource : theResources) { - binder.bind(resource); + binder.bind(resource).in(LazySingleton.class); } binder.bind(Key.get(Server.class, Names.named("ForTheEagerness"))).to(Server.class).asEagerSingleton(); diff --git a/common/src/main/java/com/metamx/druid/concurrent/Execs.java b/common/src/main/java/com/metamx/druid/concurrent/Execs.java index 06be9bdf4dd..38ec4db6416 100644 --- a/common/src/main/java/com/metamx/druid/concurrent/Execs.java +++ b/common/src/main/java/com/metamx/druid/concurrent/Execs.java @@ -32,9 +32,7 @@ public class Execs { public static ExecutorService singleThreaded(String nameFormat) { - return Executors.newSingleThreadExecutor( - makeThreadFactory(nameFormat) - ); + return Executors.newSingleThreadExecutor(makeThreadFactory(nameFormat)); } public static ExecutorService multiThreaded(int threads, String nameFormat) @@ -44,9 +42,7 @@ public class Execs public static ScheduledExecutorService scheduledSingleThreaded(String nameFormat) { - return Executors.newSingleThreadScheduledExecutor( - makeThreadFactory(nameFormat) - ); + return Executors.newSingleThreadScheduledExecutor(makeThreadFactory(nameFormat)); } public static ThreadFactory makeThreadFactory(String nameFormat) diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java index e8e5bf6a84c..2d38f9aa4c2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/guice/MiddleManagerModule.java @@ -47,7 +47,6 @@ public class MiddleManagerModule implements Module binder.bind(WorkerTaskMonitor.class).in(ManageLifecycle.class); binder.bind(WorkerCuratorCoordinator.class).in(ManageLifecycle.class); - } @Provides @LazySingleton diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java new file mode 100644 index 00000000000..1114e425997 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java @@ -0,0 +1,67 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.multibindings.MapBinder; +import com.metamx.druid.indexing.common.RetryPolicyConfig; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; +import com.metamx.druid.indexing.common.index.EventReceivingChatHandlerProvider; +import com.metamx.druid.indexing.common.index.NoopChatHandlerProvider; +import com.metamx.druid.loading.DataSegmentKiller; +import com.metamx.druid.loading.S3DataSegmentKiller; + +/** + */ +public class PeonModule implements Module +{ + @Override + public void configure(Binder binder) + { + PolyBind.createChoice( + binder, + "druid.indexer.task.chathandler.type", + Key.get(ChatHandlerProvider.class), + Key.get(NoopChatHandlerProvider.class) + ); + final MapBinder handlerProviderBinder = PolyBind.optionBinder( + binder, Key.get(ChatHandlerProvider.class) + ); + handlerProviderBinder.addBinding("curator").to(EventReceivingChatHandlerProvider.class); + handlerProviderBinder.addBinding("noop").to(NoopChatHandlerProvider.class); + + binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); + + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); + JsonConfigProvider.bind(binder, "druid.worker.taskActionClient.retry", RetryPolicyConfig.class); + + binder.bind(TaskActionClientFactory.class).to(RemoteTaskActionClientFactory.class).in(LazySingleton.class); + binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); + + binder.bind(DataSegmentKiller.class).to(S3DataSegmentKiller.class).in(LazySingleton.class); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java index e1089990cd0..278af1c108a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java @@ -19,7 +19,6 @@ package com.metamx.druid.indexing.common; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.Duration; @@ -29,41 +28,35 @@ public class RetryPolicy { private static final EmittingLogger log = new EmittingLogger(RetryPolicy.class); - private final long MAX_NUM_RETRIES; - private final Duration MAX_RETRY_DURATION; + private final long maxNumRetries; + private final Duration maxRetryDelay; private volatile Duration currRetryDelay; private volatile int retryCount; public RetryPolicy(RetryPolicyConfig config) { - this.MAX_NUM_RETRIES = config.getMaxRetryCount(); - this.MAX_RETRY_DURATION = config.getRetryMaxDuration(); + this.maxNumRetries = config.getMaxRetryCount(); + this.maxRetryDelay = config.getMaxWait().toStandardDuration(); - this.currRetryDelay = config.getRetryMinDuration(); + this.currRetryDelay = config.getMinWait().toStandardDuration(); this.retryCount = 0; } - public Duration getRetryDelay() - { - return currRetryDelay; - } - public Duration getAndIncrementRetryDelay() { - Duration retVal = new Duration(currRetryDelay); - currRetryDelay = new Duration(Math.min(currRetryDelay.getMillis() * 2, MAX_RETRY_DURATION.getMillis())); - retryCount++; - return retVal; - } + if (hasExceededRetryThreshold()) { + return null; + } - public int getNumRetries() - { - return retryCount; + Duration retVal = currRetryDelay; + currRetryDelay = new Duration(Math.min(currRetryDelay.getMillis() * 2, maxRetryDelay.getMillis())); + ++retryCount; + return retVal; } public boolean hasExceededRetryThreshold() { - return retryCount >= MAX_NUM_RETRIES; + return retryCount >= maxNumRetries; } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyConfig.java new file mode 100644 index 00000000000..de643db2353 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyConfig.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.indexing.common; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Period; + +/** + */ +public class RetryPolicyConfig +{ + @JsonProperty + private Period minWait = new Period("PT1M"); + + @JsonProperty + private Period maxWait = new Period("PT10M"); + + @JsonProperty + private long maxRetryCount = 10; + + public Period getMinWait() + { + return minWait; + } + + RetryPolicyConfig setMinWait(Period minWait) + { + this.minWait = minWait; + return this; + } + + public Period getMaxWait() + { + return maxWait; + } + + RetryPolicyConfig setMaxWait(Period maxWait) + { + this.maxWait = maxWait; + return this; + } + + public long getMaxRetryCount() + { + return maxRetryCount; + } + + RetryPolicyConfig setMaxRetryCount(long maxRetryCount) + { + this.maxRetryCount = maxRetryCount; + return this; + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java index f9dabd54b52..74731e21ff6 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java @@ -19,7 +19,7 @@ package com.metamx.druid.indexing.common; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; +import com.google.inject.Inject; /** */ @@ -27,6 +27,7 @@ public class RetryPolicyFactory { private final RetryPolicyConfig config; + @Inject public RetryPolicyFactory(RetryPolicyConfig config) { this.config = config; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java index 5acd075bee7..09c02ddf0b1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java @@ -20,6 +20,7 @@ package com.metamx.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; import com.metamx.druid.client.ServerView; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; @@ -49,6 +50,7 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final ObjectMapper objectMapper; + @Inject public TaskToolboxFactory( TaskConfig config, TaskActionClientFactory taskActionClientFactory, diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java index 289fcf005d9..531f8653053 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -6,13 +6,13 @@ import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; +import com.metamx.druid.client.indexing.IndexingServiceSelector; +import com.metamx.druid.client.selector.Server; import com.metamx.druid.indexing.common.RetryPolicy; import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; -import org.apache.curator.x.discovery.ServiceInstance; -import org.apache.curator.x.discovery.ServiceProvider; import org.joda.time.Duration; import java.io.IOException; @@ -23,7 +23,7 @@ public class RemoteTaskActionClient implements TaskActionClient { private final Task task; private final HttpClient httpClient; - private final ServiceProvider serviceProvider; + private final IndexingServiceSelector serviceProvider; private final RetryPolicyFactory retryPolicyFactory; private final ObjectMapper jsonMapper; @@ -32,7 +32,7 @@ public class RemoteTaskActionClient implements TaskActionClient public RemoteTaskActionClient( Task task, HttpClient httpClient, - ServiceProvider serviceProvider, + IndexingServiceSelector serviceProvider, RetryPolicyFactory retryPolicyFactory, ObjectMapper jsonMapper ) @@ -79,20 +79,23 @@ public class RemoteTaskActionClient implements TaskActionClient } final Map responseDict = jsonMapper.readValue( - response, - new TypeReference>() {} + response, new TypeReference>() + { + } ); return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference()); - } catch(IOException e) { - log.warn(e, "Exception submitting action for task: %s", task.getId()); + } + catch (IOException e) { + log.warn(e, "Exception submitting action for task[%s]", task.getId()); - if (retryPolicy.hasExceededRetryThreshold()) { + final Duration delay = retryPolicy.getAndIncrementRetryDelay(); + if (delay == null) { throw e; } else { try { - final long sleepTime = retryPolicy.getAndIncrementRetryDelay().getMillis(); - log.info("Will try again in %s.", new Duration(sleepTime).toString()); + final long sleepTime = delay.getMillis(); + log.info("Will try again in [%s].", new Duration(sleepTime).toString()); Thread.sleep(sleepTime); } catch (InterruptedException e2) { @@ -105,26 +108,19 @@ public class RemoteTaskActionClient implements TaskActionClient private URI getServiceUri() throws Exception { - final ServiceInstance instance = serviceProvider.getInstance(); - final String scheme; - final String host; - final int port; - final String path = "/druid/indexer/v1/action"; - + final Server instance = serviceProvider.pick(); if (instance == null) { throw new ISE("Cannot find instance of indexer to talk to!"); } - host = instance.getAddress(); - - if (instance.getSslPort() != null && instance.getSslPort() > 0) { - scheme = "https"; - port = instance.getSslPort(); - } else { - scheme = "http"; - port = instance.getPort(); - } - - return new URI(scheme, null, host, port, path, null, null); + return new URI( + instance.getScheme(), + null, + instance.getHost(), + instance.getPort(), + "/druid/indexer/v1/action", + null, + null + ); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java index c872a2200a4..450ecdaee19 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java @@ -20,23 +20,26 @@ package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.indexing.common.task.Task; +import com.google.inject.Inject; +import com.metamx.druid.client.indexing.IndexingServiceSelector; +import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.http.client.HttpClient; -import org.apache.curator.x.discovery.ServiceProvider; /** */ public class RemoteTaskActionClientFactory implements TaskActionClientFactory { private final HttpClient httpClient; - private final ServiceProvider serviceProvider; + private final IndexingServiceSelector serviceProvider; private final RetryPolicyFactory retryPolicyFactory; private final ObjectMapper jsonMapper; + @Inject public RemoteTaskActionClientFactory( - HttpClient httpClient, - ServiceProvider serviceProvider, + @Global HttpClient httpClient, + IndexingServiceSelector serviceProvider, RetryPolicyFactory retryPolicyFactory, ObjectMapper jsonMapper ) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java index 5e9789e9660..79f43450677 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java @@ -19,38 +19,65 @@ package com.metamx.druid.indexing.common.config; -import com.google.common.base.Joiner; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import java.io.File; -public abstract class TaskConfig +public class TaskConfig { - private static Joiner joiner = Joiner.on("/"); + @JsonProperty + private final String baseDir; - @Config("druid.indexer.baseDir") - @Default("/tmp/") - public abstract String getBaseDir(); + @JsonProperty + private final File baseTaskDir; + + @JsonProperty + private final String hadoopWorkingPath; + + @JsonProperty + private final int defaultRowFlushBoundary; + + @JsonCreator + public TaskConfig( + @JsonProperty("baseDir") String baseDir, + @JsonProperty("baseTaskDir") String baseTaskDir, + @JsonProperty("hadoopWorkingPath") String hadoopWorkingPath, + @JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary + ) + { + this.baseDir = baseDir == null ? "/tmp" : baseDir; + this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task")); + this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing"); + this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary; + } + + public String getBaseDir() + { + return baseDir; + } - @Config("druid.indexer.taskDir") public File getBaseTaskDir() { - return new File(defaultPath("persistent/task")); + return baseTaskDir; } - @Config("druid.indexer.hadoopWorkingPath") public String getHadoopWorkingPath() { - return defaultPath("druid-indexing"); + return hadoopWorkingPath; } - @Config("druid.indexer.rowFlushBoundary") - @Default("500000") - public abstract int getDefaultRowFlushBoundary(); - - private String defaultPath(String subPath) + public int getDefaultRowFlushBoundary() { - return joiner.join(getBaseDir(), subPath); + return defaultRowFlushBoundary; + } + + private String defaultDir(String configParameter, final String defaultVal) + { + if (configParameter == null) { + return String.format("%s/%s", getBaseDir(), defaultVal); + } + + return configParameter; } } \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java index c4ae8cec851..6d0ea613170 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java @@ -21,10 +21,11 @@ package com.metamx.druid.indexing.common.index; import com.google.common.base.Optional; import com.google.common.collect.Maps; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.guice.annotations.Self; import com.metamx.druid.initialization.DruidNode; import java.util.concurrent.ConcurrentMap; @@ -38,16 +39,17 @@ public class EventReceivingChatHandlerProvider implements ChatHandlerProvider { private static final Logger log = new Logger(EventReceivingChatHandlerProvider.class); - private final ChatHandlerProviderConfig config; + private final DruidNode node; private final ServiceAnnouncer serviceAnnouncer; private final ConcurrentMap handlers; + @Inject public EventReceivingChatHandlerProvider( - ChatHandlerProviderConfig config, + @Self DruidNode node, ServiceAnnouncer serviceAnnouncer ) { - this.config = config; + this.node = node; this.serviceAnnouncer = serviceAnnouncer; this.handlers = Maps.newConcurrentMap(); } @@ -100,6 +102,6 @@ public class EventReceivingChatHandlerProvider implements ChatHandlerProvider private DruidNode makeDruidNode(String key) { - return new DruidNode(key, config.getHost(), config.getPort()); + return new DruidNode(key, node.getHost(), node.getPort()); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 202ca0f55f5..dc6601ba1a7 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -28,8 +28,10 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.Query; +import com.metamx.druid.concurrent.Execs; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.TaskToolboxFactory; @@ -48,7 +50,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ExecutorService; /** * Runs tasks in a JVM thread using an ExecutorService. @@ -61,13 +62,13 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private static final EmittingLogger log = new EmittingLogger(ThreadPoolTaskRunner.class); + @Inject public ThreadPoolTaskRunner( - TaskToolboxFactory toolboxFactory, - ExecutorService exec + TaskToolboxFactory toolboxFactory ) { this.toolboxFactory = toolboxFactory; - this.exec = MoreExecutors.listeningDecorator(exec); + this.exec = MoreExecutors.listeningDecorator(Execs.singleThreaded("task-runner-%d")); } @LifecycleStop diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 53baaec3574..ad8a6a26168 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; @@ -36,16 +35,17 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.BaseServerNode; +import com.metamx.druid.client.indexing.IndexingServiceSelector; import com.metamx.druid.curator.CuratorConfig; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.indexing.common.RetryPolicyConfig; import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; @@ -86,7 +86,6 @@ import org.jets3t.service.security.AWSCredentials; import org.skife.config.ConfigurationObjectFactory; import java.util.Properties; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** @@ -322,7 +321,7 @@ public class ExecutorNode extends BaseServerNode taskConfig, new RemoteTaskActionClientFactory( httpClient, - coordinatorServiceProvider, + new IndexingServiceSelector(coordinatorServiceProvider), new RetryPolicyFactory( configFactory.buildWithReplacements( RetryPolicyConfig.class, @@ -371,16 +370,7 @@ public class ExecutorNode extends BaseServerNode public void initializeTaskRunner() { if (taskRunner == null) { - this.taskRunner = lifecycle.addManagedInstance( - new ThreadPoolTaskRunner( - taskToolboxFactory, - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setNameFormat("task-runner-%d") - .build() - ) - ) - ); + this.taskRunner = lifecycle.addManagedInstance(new ThreadPoolTaskRunner(taskToolboxFactory)); } } @@ -389,7 +379,7 @@ public class ExecutorNode extends BaseServerNode if (chatHandlerProvider == null) { final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); if (config.isPublishDiscovery()) { - this.chatHandlerProvider = new EventReceivingChatHandlerProvider(config, serviceAnnouncer); + this.chatHandlerProvider = new EventReceivingChatHandlerProvider(null, serviceAnnouncer); // TODO: eliminate } else { log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); this.chatHandlerProvider = new NoopChatHandlerProvider(); diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/RetryPolicyTest.java similarity index 59% rename from indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/common/RetryPolicyTest.java index cfa08d44401..07aaa66e0bd 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/RetryPolicyTest.java @@ -1,9 +1,8 @@ -package com.metamx.druid.indexing.coordinator; +package com.metamx.druid.indexing.common; -import com.metamx.druid.indexing.common.RetryPolicy; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import junit.framework.Assert; import org.joda.time.Duration; +import org.joda.time.Period; import org.junit.Test; /** @@ -15,25 +14,9 @@ public class RetryPolicyTest { RetryPolicy retryPolicy = new RetryPolicy( new RetryPolicyConfig() - { - @Override - public Duration getRetryMinDuration() - { - return new Duration("PT1S"); - } - - @Override - public Duration getRetryMaxDuration() - { - return new Duration("PT10S"); - } - - @Override - public long getMaxRetryCount() - { - return 10; - } - } + .setMinWait(new Period("PT1S")) + .setMaxWait(new Period("PT10S")) + .setMaxRetryCount(6) ); Assert.assertEquals(new Duration("PT1S"), retryPolicy.getAndIncrementRetryDelay()); @@ -42,5 +25,7 @@ public class RetryPolicyTest Assert.assertEquals(new Duration("PT8S"), retryPolicy.getAndIncrementRetryDelay()); Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); Assert.assertEquals(new Duration("PT10S"), retryPolicy.getAndIncrementRetryDelay()); + Assert.assertEquals(null, retryPolicy.getAndIncrementRetryDelay()); + Assert.assertTrue(retryPolicy.hasExceededRetryThreshold()); } } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index c985f6e4aba..130cfaac383 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -49,7 +49,6 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; import java.util.Set; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -306,6 +305,8 @@ public class RemoteTaskRunnerTest ); workerCuratorCoordinator.start(); + final File tmp = Files.createTempDir(); + // Start a task monitor workerTaskMonitor = new WorkerTaskMonitor( jsonMapper, @@ -313,29 +314,9 @@ public class RemoteTaskRunnerTest workerCuratorCoordinator, new ThreadPoolTaskRunner( new TaskToolboxFactory( - new TaskConfig() - { - @Override - public String getBaseDir() - { - File tmp = Files.createTempDir(); - tmp.deleteOnExit(); - return tmp.toString(); - } - - @Override - public int getDefaultRowFlushBoundary() - { - return 0; - } - - @Override - public String getHadoopWorkingPath() - { - return null; - } - }, null, null, null, null, null, null, null, null, null, jsonMapper - ), Executors.newSingleThreadExecutor() + new TaskConfig(tmp.toString(), null, null, 0), + null, null, null, null, null, null, null, null, null, jsonMapper + ) ), new WorkerConfig().setCapacity(1) ); diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index a67ec0326f6..2ec5bd543f3 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -78,7 +78,6 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Set; -import java.util.concurrent.Executors; public class TaskLifecycleTest { @@ -116,26 +115,7 @@ public class TaskLifecycleTest tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())); tb = new TaskToolboxFactory( - new TaskConfig() - { - @Override - public String getBaseDir() - { - return tmp.toString(); - } - - @Override - public int getDefaultRowFlushBoundary() - { - return 50000; - } - - @Override - public String getHadoopWorkingPath() - { - return null; - } - }, + new TaskConfig(tmp.toString(), null, null, 50000), tac, newMockEmitter(), null, // s3 client @@ -162,10 +142,7 @@ public class TaskLifecycleTest new DefaultObjectMapper() ); - tr = new ThreadPoolTaskRunner( - tb, - Executors.newSingleThreadExecutor() - ); + tr = new ThreadPoolTaskRunner(tb); tc = new TaskConsumer(tq, tr, tac, newMockEmitter()); tsqa = new TaskStorageQueryAdapter(ts); diff --git a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java index 41259c1b28d..6f686832cb6 100644 --- a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java +++ b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java @@ -52,6 +52,12 @@ public class IndexingServiceSelector implements DiscoverySelector { return instance.getPort(); } + + @Override + public String getScheme() + { + return "http"; + } }; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelectorConfig.java similarity index 56% rename from indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java rename to server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelectorConfig.java index b9ad26ad2d4..b34404eebc1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java +++ b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelectorConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,25 +17,19 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexing.common.config; +package com.metamx.druid.client.indexing; -import org.joda.time.Duration; -import org.skife.config.Config; -import org.skife.config.Default; +import com.fasterxml.jackson.annotation.JsonProperty; /** */ -public abstract class RetryPolicyConfig +public class IndexingServiceSelectorConfig { - @Config("${base_path}.retry.minWaitMillis") - @Default("PT1M") // 1 minute - public abstract Duration getRetryMinDuration(); + @JsonProperty + private String serviceName = null; - @Config("${base_path}.retry.maxWaitMillis") - @Default("PT10M") // 10 minutes - public abstract Duration getRetryMaxDuration(); - - @Config("${base_path}.retry.maxRetryCount") - @Default("10") - public abstract long getMaxRetryCount(); + public String getServiceName() + { + return serviceName; + } } diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java index 86d857bcb37..8ae217714ca 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java @@ -4,14 +4,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; -import com.google.inject.TypeLiteral; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.druid.client.ServerInventoryViewConfig; -import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceClient; -import com.metamx.druid.client.indexing.IndexingServiceSelector; -import com.metamx.druid.client.selector.DiscoverySelector; -import com.metamx.druid.client.selector.Server; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseRuleManagerProvider; @@ -26,11 +21,6 @@ import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceInstance; -import org.apache.curator.x.discovery.ServiceProvider; - -import java.io.IOException; /** */ @@ -56,10 +46,6 @@ public class CoordinatorModule implements Module .toProvider(DatabaseRuleManagerProvider.class) .in(ManageLifecycle.class); - binder.bind(new TypeLiteral>(){}) - .annotatedWith(IndexingService.class) - .to(IndexingServiceSelector.class) - .in(ManageLifecycle.class); binder.bind(IndexingServiceClient.class).in(LazySingleton.class); binder.bind(RedirectInfo.class).to(MasterRedirectInfo.class).in(LazySingleton.class); @@ -67,35 +53,6 @@ public class CoordinatorModule implements Module binder.bind(DruidMaster.class); } - @Provides @LazySingleton @IndexingService - public ServiceProvider getServiceProvider(DruidMasterConfig config, ServiceDiscovery serviceDiscovery) - { - // TODO: This service discovery stuff is really really janky. It needs to be reworked. - if (config.getMergerServiceName() == null) { - return new ServiceProvider() - { - @Override - public void start() throws Exception - { - - } - - @Override - public ServiceInstance getInstance() throws Exception - { - return null; - } - - @Override - public void close() throws IOException - { - - } - }; - } - return serviceDiscovery.serviceProviderBuilder().serviceName(config.getMergerServiceName()).build(); - } - @Provides @LazySingleton public LoadQueueTaskMaster getLoadQueueTaskMaster( CuratorFramework curator, ObjectMapper jsonMapper, ScheduledExecutorFactory factory, DruidMasterConfig config diff --git a/server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java b/server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java new file mode 100644 index 00000000000..e39c43dd878 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java @@ -0,0 +1,83 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.TypeLiteral; +import com.metamx.druid.client.indexing.IndexingService; +import com.metamx.druid.client.indexing.IndexingServiceSelector; +import com.metamx.druid.client.indexing.IndexingServiceSelectorConfig; +import com.metamx.druid.client.selector.DiscoverySelector; +import com.metamx.druid.client.selector.Server; +import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceInstance; +import org.apache.curator.x.discovery.ServiceProvider; + +import java.io.IOException; + +/** + */ +public class IndexingServiceDiscoveryModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.selectors.indexing", IndexingServiceSelectorConfig.class); + binder.bind(new TypeLiteral>(){}) + .annotatedWith(IndexingService.class) + .to(IndexingServiceSelector.class); + + binder.bind(IndexingServiceSelector.class).in(ManageLifecycle.class); + } + + @Provides + @LazySingleton @IndexingService + public ServiceProvider getServiceProvider( + IndexingServiceSelectorConfig config, + ServiceDiscovery serviceDiscovery + ) + { + if (config.getServiceName() == null) { + return new ServiceProvider() + { + @Override + public void start() throws Exception + { + + } + + @Override + public ServiceInstance getInstance() throws Exception + { + return null; + } + + @Override + public void close() throws IOException + { + + } + }; + } + return serviceDiscovery.serviceProviderBuilder().serviceName(config.getServiceName()).build(); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java index 72954cec734..33b7688afe7 100644 --- a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java @@ -20,6 +20,7 @@ package com.metamx.druid.guice; import com.google.inject.Binder; +import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.coordination.DruidServerMetadata; @@ -33,7 +34,7 @@ import com.metamx.druid.query.QueryRunnerFactoryConglomerate; /** */ -public class StorageNodeModule extends ServerModule +public class StorageNodeModule implements Module { private final String nodeType; @@ -45,8 +46,6 @@ public class StorageNodeModule extends ServerModule @Override public void configure(Binder binder) { - super.configure(binder); - JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class); JsonConfigProvider.bind(binder, "druid.segmentCache", SegmentLoaderConfig.class); diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java index d514b4d5c4f..13f63d72f70 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java @@ -60,12 +60,6 @@ public abstract class DruidMasterConfig return false; } - @Config("druid.master.merger.service") - public String getMergerServiceName() - { - return null; - } - @Config("druid.master.merge.threshold") public long getMergeBytesLimit() { 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 a49dc85a582..5db44c86363 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -100,12 +100,6 @@ public class DruidMasterTest return super.getMillisToWaitBeforeDeleting(); } - @Override - public String getMergerServiceName() - { - return ""; - } - @Override public int getMaxSegmentsToMove() { diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 7c60be03346..f8466abf057 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -8,6 +8,7 @@ import com.metamx.druid.curator.discovery.DiscoveryModule; import com.metamx.druid.guice.CoordinatorModule; import com.metamx.druid.guice.DbConnectorModule; import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.IndexingServiceDiscoveryModule; import com.metamx.druid.guice.JacksonConfigManagerModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.ServerModule; @@ -68,6 +69,7 @@ public class CliCoordinator extends ServerRunnable .addResource(MasterResource.class) .addResource(StatusResource.class), new ServerViewModule(), + new IndexingServiceDiscoveryModule(), CoordinatorModule.class ); } diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 3f30002c577..2f8d521d5be 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -33,6 +33,7 @@ import com.metamx.druid.guice.HttpClientModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; @@ -70,6 +71,7 @@ public class CliHistorical extends ServerRunnable AWSModule.class, DataSegmentPullerModule.class, new MetricsModule().register(ServerMonitor.class), + new ServerModule(), new StorageNodeModule("historical"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java new file mode 100644 index 00000000000..fb3a43aa73d --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -0,0 +1,92 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli; + +import com.google.inject.Injector; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.CuratorModule; +import com.metamx.druid.curator.discovery.DiscoveryModule; +import com.metamx.druid.guice.AWSModule; +import com.metamx.druid.guice.AnnouncerModule; +import com.metamx.druid.guice.DataSegmentPusherModule; +import com.metamx.druid.guice.DruidProcessingModule; +import com.metamx.druid.guice.HttpClientModule; +import com.metamx.druid.guice.IndexingServiceDiscoveryModule; +import com.metamx.druid.guice.LifecycleModule; +import com.metamx.druid.guice.PeonModule; +import com.metamx.druid.guice.QueryRunnerFactoryModule; +import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.ServerModule; +import com.metamx.druid.guice.ServerViewModule; +import com.metamx.druid.guice.StorageNodeModule; +import com.metamx.druid.http.StatusResource; +import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; +import com.metamx.druid.indexing.worker.executor.ChatHandlerResource; +import com.metamx.druid.initialization.EmitterModule; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Command; + +/** + */ +@Command( + name = "peon", + description = "Runs a Peon, this is an individual forked \"task\" used as part of the indexing service. " + + "This should rarely, if ever, be used directly." +) +public class CliPeon extends ServerRunnable +{ + private static final Logger log = new Logger(CliPeon.class); + + public CliPeon() + { + super(log); + } + + @Override + protected Injector getInjector() + { + // TODO: make it take and run a task + + return Initialization.makeInjector( + new LifecycleModule(), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule(), + new ServerModule(), + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class) + .addResource(ChatHandlerResource.class), + new DiscoveryModule(), + new ServerViewModule(), + new StorageNodeModule("real-time"), + new DataSegmentPusherModule(), + new AnnouncerModule(), + new DruidProcessingModule(), + new QueryableModule(ThreadPoolTaskRunner.class), + new QueryRunnerFactoryModule(), + new IndexingServiceDiscoveryModule(), + new AWSModule(), + new PeonModule() + ); + } +} diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 165996a4497..080397a1a76 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -32,6 +32,7 @@ import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; import com.metamx.druid.guice.RealtimeModule; +import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.ServerViewModule; import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; @@ -71,6 +72,7 @@ public class CliRealtime extends ServerRunnable AWSModule.class, DataSegmentPusherModule.class, new MetricsModule(), + new ServerModule(), new StorageNodeModule("realtime"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 1b11ea07e14..1a6f9f97381 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -25,6 +25,7 @@ import com.metamx.druid.guice.DruidProcessingModule; import com.metamx.druid.guice.LifecycleModule; import com.metamx.druid.guice.QueryRunnerFactoryModule; import com.metamx.druid.guice.QueryableModule; +import com.metamx.druid.guice.ServerModule; import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.initialization.EmitterModule; @@ -56,6 +57,7 @@ public class CliRealtimeExample extends ServerRunnable new LifecycleModule(), EmitterModule.class, DruidProcessingModule.class, + new ServerModule(), new StorageNodeModule("realtime"), new JettyServerModule(new QueryJettyServerInitializer()) .addResource(StatusResource.class), diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index b2274ffe096..934348c7f4d 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -40,7 +40,8 @@ public class Main .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) .withCommands( - CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, CliOverlord.class, CliMiddleManager.class + CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, + CliOverlord.class, CliMiddleManager.class, CliPeon.class ); builder.withGroup("example") From 3617ac17fca1a2cc8712ecbadd20658dfa6f3e5c Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 27 Aug 2013 14:11:05 -0500 Subject: [PATCH 42/92] 1) Eliminate ExecutorMain and have it run using the new Main! --- .../java/com/metamx/druid/QueryableNode.java | 625 ------------------ .../com/metamx/druid/RegisteringNode.java | 16 - .../metamx/druid/curator/CuratorModule.java | 35 +- .../druid/curator/announcement/Announcer.java | 2 +- .../curator/discovery/DiscoveryModule.java | 34 +- .../druid/guice/QueryToolChestModule.java | 2 + .../druid/http/ClientServletModule.java | 70 -- .../com/metamx/druid/http/StatusServlet.java | 57 -- .../druid/initialization/Initialization.java | 315 --------- .../initialization/JettyServerModule.java | 25 +- .../initialization/PropertiesModule.java | 99 +-- .../druid/query/search/SearchQueryConfig.java | 38 ++ .../search/SearchQueryQueryToolChest.java | 45 +- .../com/metamx/druid/guice/PeonModule.java | 19 + .../druid/indexing/common/task/KillTask.java | 9 +- .../druid/indexing/common/task/NoopTask.java | 63 ++ .../druid/indexing/common/task/Task.java | 1 + .../druid/indexing/common/task/TaskUtils.java | 40 ++ .../coordinator/ForkingTaskRunner.java | 32 +- .../indexing/coordinator/TaskMaster.java | 19 +- .../config/ForkingTaskRunnerConfig.java | 15 +- .../http/IndexerCoordinatorServletModule.java | 91 --- .../worker/executor/ExecutorLifecycle.java | 26 +- .../executor/ExecutorLifecycleConfig.java | 88 +++ .../executor/ExecutorLifecycleFactory.java | 9 +- .../worker/executor/ExecutorMain.java | 73 -- .../worker/executor/ExecutorNode.java | 459 ------------- .../executor/ExecutorServletModule.java | 44 -- .../worker/http/WorkerServletModule.java | 70 -- indexing-service/src/test/resources/task.json | 1 + .../java/com/metamx/druid/BaseServerNode.java | 147 ---- .../druid/http/MasterServletModule.java | 91 --- .../druid/initialization/ServerInit.java | 195 ------ .../src/main/java/io/druid/cli/CliPeon.java | 58 +- services/src/main/java/io/druid/cli/Main.java | 9 +- .../java/io/druid/cli/ServerRunnable.java | 1 - 36 files changed, 498 insertions(+), 2425 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/QueryableNode.java delete mode 100644 client/src/main/java/com/metamx/druid/http/ClientServletModule.java delete mode 100644 client/src/main/java/com/metamx/druid/http/StatusServlet.java create mode 100644 client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/task/NoopTask.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskUtils.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java create mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleConfig.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java delete mode 100644 indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java create mode 100644 indexing-service/src/test/resources/task.json delete mode 100644 server/src/main/java/com/metamx/druid/BaseServerNode.java delete mode 100644 server/src/main/java/com/metamx/druid/http/MasterServletModule.java delete mode 100644 server/src/main/java/com/metamx/druid/initialization/ServerInit.java diff --git a/client/src/main/java/com/metamx/druid/QueryableNode.java b/client/src/main/java/com/metamx/druid/QueryableNode.java deleted file mode 100644 index f8e90092b84..00000000000 --- a/client/src/main/java/com/metamx/druid/QueryableNode.java +++ /dev/null @@ -1,625 +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; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.metamx.common.IAE; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.BatchServerInventoryView; -import com.metamx.druid.client.DruidServerConfig; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.ServerInventoryViewConfig; -import com.metamx.druid.client.ServerView; -import com.metamx.druid.client.SingleServerInventoryView; -import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; -import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.coordination.MultipleDataSegmentAnnouncerDataSegmentAnnouncer; -import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; -import com.metamx.druid.curator.CuratorConfig; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.guice.JsonConfigurator; -import com.metamx.druid.http.log.NoopRequestLogger; -import com.metamx.druid.http.log.RequestLogger; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.JvmMonitor; -import com.metamx.metrics.Monitor; -import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; -import org.apache.curator.framework.CuratorFramework; -import org.eclipse.jetty.server.Server; -import org.joda.time.Duration; -import org.skife.config.ConfigurationObjectFactory; - -import javax.validation.Validation; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.Arrays; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; - -/** - */ -public abstract class QueryableNode extends RegisteringNode -{ - private final Logger log; - - private final Lifecycle lifecycle; - private final ObjectMapper jsonMapper; - private final ObjectMapper smileMapper; - private final Properties props; - private final ConfigurationObjectFactory configFactory; - private final String nodeType; - private final JsonConfigurator jsonConfigurator; - - private DruidServerMetadata druidServerMetadata = null; - private ServiceEmitter emitter = null; - private List monitors = null; - private Server server = null; - private CuratorFramework curator = null; - private DataSegmentAnnouncer announcer = null; - private ZkPathsConfig zkPaths = null; - private ScheduledExecutorFactory scheduledExecutorFactory = null; - private RequestLogger requestLogger = null; - private ServerInventoryView serverInventoryView = null; - private ServerView serverView = null; - private InventoryView inventoryView = null; - - private boolean initialized = false; - - public QueryableNode( - String nodeType, - Logger log, - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super(Arrays.asList(jsonMapper, smileMapper)); - - this.log = log; - this.configFactory = configFactory; - this.props = props; - this.jsonMapper = jsonMapper; - this.lifecycle = lifecycle; - this.smileMapper = smileMapper; - - Preconditions.checkNotNull(props, "props"); - Preconditions.checkNotNull(lifecycle, "lifecycle"); - Preconditions.checkNotNull(jsonMapper, "jsonMapper"); - Preconditions.checkNotNull(smileMapper, "smileMapper"); - Preconditions.checkNotNull(configFactory, "configFactory"); - - this.jsonConfigurator = new JsonConfigurator(jsonMapper, Validation.buildDefaultValidatorFactory().getValidator()); - - Preconditions.checkState(smileMapper.getJsonFactory() instanceof SmileFactory, "smileMapper should use smile."); - this.nodeType = nodeType; - } - - public T setDruidServerMetadata(DruidServerMetadata druidServerMetadata) - { - checkFieldNotSetAndSet("druidServerMetadata", druidServerMetadata); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setCuratorFramework(CuratorFramework curator) - { - checkFieldNotSetAndSet("curator", curator); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setAnnouncer(DataSegmentAnnouncer announcer) - { - checkFieldNotSetAndSet("announcer", announcer); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setEmitter(ServiceEmitter emitter) - { - checkFieldNotSetAndSet("emitter", emitter); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setMonitors(List monitors) - { - checkFieldNotSetAndSet("monitors", monitors); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setServer(Server server) - { - checkFieldNotSetAndSet("server", server); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setZkPaths(ZkPathsConfig zkPaths) - { - checkFieldNotSetAndSet("zkPaths", zkPaths); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setScheduledExecutorFactory(ScheduledExecutorFactory factory) - { - checkFieldNotSetAndSet("scheduledExecutorFactory", factory); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setRequestLogger(RequestLogger requestLogger) - { - checkFieldNotSetAndSet("requestLogger", requestLogger); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setInventoryView(InventoryView inventoryView) - { - checkFieldNotSetAndSet("inventoryView", inventoryView); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setServerView(ServerView serverView) - { - checkFieldNotSetAndSet("serverView", serverView); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T registerJacksonSubtype(Class... clazzes) - { - jsonMapper.registerSubtypes(clazzes); - smileMapper.registerSubtypes(clazzes); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T registerJacksonSubtype(NamedType... namedTypes) - { - jsonMapper.registerSubtypes(namedTypes); - smileMapper.registerSubtypes(namedTypes); - return (T) this; - } - - public Lifecycle getLifecycle() - { - return lifecycle; - } - - public ObjectMapper getJsonMapper() - { - return jsonMapper; - } - - public ObjectMapper getSmileMapper() - { - return smileMapper; - } - - public Properties getProps() - { - return props; - } - - public ConfigurationObjectFactory getConfigFactory() - { - return configFactory; - } - - public JsonConfigurator getJsonConfigurator() - { - return jsonConfigurator; - } - - public DruidServerMetadata getDruidServerMetadata() - { - initializeDruidServerMetadata(); - return druidServerMetadata; - } - - public CuratorFramework getCuratorFramework() - { - initializeCuratorFramework(); - return curator; - } - - public DataSegmentAnnouncer getAnnouncer() - { - initializeAnnouncer(); - return announcer; - } - - public ServiceEmitter getEmitter() - { - initializeEmitter(); - return emitter; - } - - public List getMonitors() - { - initializeMonitors(); - return monitors; - } - - public Server getServer() - { - initializeServer(); - return server; - } - - public ZkPathsConfig getZkPaths() - { - initializeZkPaths(); - return zkPaths; - } - - public ScheduledExecutorFactory getScheduledExecutorFactory() - { - initializeScheduledExecutorFactory(); - return scheduledExecutorFactory; - } - - public RequestLogger getRequestLogger() - { - initializeRequestLogger(); - return requestLogger; - } - - public ServerView getServerView() - { - initializeServerView(); - return serverView; - } - - public InventoryView getInventoryView() - { - initializeInventoryView(); - return inventoryView; - } - - private void initializeDruidServerMetadata() - { - if (druidServerMetadata == null) { - final DruidServerConfig serverConfig = getConfigFactory().build(DruidServerConfig.class); - setDruidServerMetadata( - new DruidServerMetadata( - null, // TODO: serverConfig.getServerName(), - null, // TODO: serverConfig.getHost(), - serverConfig.getMaxSize(), - nodeType, - serverConfig.getTier() - ) - ); - } - } - - private void initializeServerView() - { - if (serverView == null) { - initializeServerInventoryView(); - serverView = serverInventoryView; - } - } - - private void initializeInventoryView() - { - if (inventoryView == null) { - initializeServerInventoryView(); - inventoryView = serverInventoryView; - } - } - - private void initializeServerInventoryView() - { - if (serverInventoryView == null) { - final ExecutorService exec = Executors.newFixedThreadPool( - 1, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ServerInventoryView-%s").build() - ); - - final ServerInventoryViewConfig serverInventoryViewConfig = getConfigFactory().build(ServerInventoryViewConfig.class); - final String announcerType = serverInventoryViewConfig.getAnnouncerType(); - - if ("legacy".equalsIgnoreCase(announcerType)) { - serverInventoryView = new SingleServerInventoryView( - getZkPaths(), - getCuratorFramework(), - getJsonMapper() - ); - } else if ("batch".equalsIgnoreCase(announcerType)) { - serverInventoryView = new BatchServerInventoryView( - getZkPaths(), - getCuratorFramework(), - getJsonMapper() - ); - } else { - throw new IAE("Unknown type %s", announcerType); - } - lifecycle.addManagedInstance(serverInventoryView); - } - } - - private void initializeRequestLogger() - { - if (requestLogger == null) { - try { - final String loggingType = props.getProperty("druid.request.logging.type"); - if ("emitter".equals(loggingType)) { - setRequestLogger( - Initialization.makeEmittingRequestLogger( - getProps(), - getEmitter() - ) - ); - } else if ("file".equalsIgnoreCase(loggingType)) { - setRequestLogger( - Initialization.makeFileRequestLogger( - getJsonMapper(), - getScheduledExecutorFactory(), - getProps() - ) - ); - } else { - setRequestLogger(new NoopRequestLogger()); - } - } - catch (IOException e) { - throw Throwables.propagate(e); - } - lifecycle.addManagedInstance(requestLogger); - } - } - - private void initializeZkPaths() - { - if (zkPaths == null) { - setZkPaths(getConfigFactory().build(ZkPathsConfig.class)); - } - } - - private void initializeScheduledExecutorFactory() - { - if (scheduledExecutorFactory == null) { - setScheduledExecutorFactory(ScheduledExecutors.createFactory(getLifecycle())); - } - } - - private void initializeCuratorFramework() - { - if (curator == null) { - try { - setCuratorFramework(Initialization.makeCuratorFramework(configFactory.build(CuratorConfig.class), lifecycle)); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - } - - private void initializeAnnouncer() - { - if (announcer == null) { - final Announcer announcer = new Announcer(getCuratorFramework(), Execs.singleThreaded("Announcer-%s")); - lifecycle.addManagedInstance(announcer); - - final BatchDataSegmentAnnouncerConfig config = getConfigFactory().build(BatchDataSegmentAnnouncerConfig.class); - final String announcerType = "legacy"; - - final DataSegmentAnnouncer dataSegmentAnnouncer; - if ("batch".equalsIgnoreCase(announcerType)) { - dataSegmentAnnouncer = new BatchDataSegmentAnnouncer( - getDruidServerMetadata(), - config, - getZkPaths(), - announcer, - getJsonMapper() - ); - } else if ("legacy".equalsIgnoreCase(announcerType)) { - dataSegmentAnnouncer = new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( - Arrays.asList( - new BatchDataSegmentAnnouncer( - getDruidServerMetadata(), - config, - getZkPaths(), - announcer, - getJsonMapper() - ), - new SingleDataSegmentAnnouncer( - getDruidServerMetadata(), - getZkPaths(), - announcer, - getJsonMapper() - ) - ) - ); - } else { - throw new ISE("Unknown announcer type [%s]", announcerType); - } - - setAnnouncer(dataSegmentAnnouncer); - - lifecycle.addManagedInstance(getAnnouncer(), Lifecycle.Stage.LAST); - } - } - - private void initializeServer() - { - if (server == null) { - setServer(Initialization.makeJettyServer(null, configFactory.build(ServerConfig.class))); // TODO: eliminate - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - log.info("Starting Jetty"); - server.start(); - } - - @Override - public void stop() - { - log.info("Stopping Jetty"); - try { - server.stop(); - } - catch (Exception e) { - log.error(e, "Exception thrown while stopping Jetty"); - } - } - } - ); - } - } - - private void initializeMonitors() - { - if (monitors == null) { - List theMonitors = Lists.newArrayList(); - theMonitors.add(new JvmMonitor()); - if (Boolean.parseBoolean(props.getProperty("druid.monitoring.monitorSystem", "false"))) { - theMonitors.add(new SysMonitor()); - } - - setMonitors(theMonitors); - } - } - - private void initializeEmitter() - { - if (emitter == null) { - final HttpClientConfig.Builder configBuilder = HttpClientConfig.builder().withNumConnections(1); - - final String emitterTimeoutDuration = props.getProperty("druid.emitter.timeOut"); - if (emitterTimeoutDuration != null) { - configBuilder.withReadTimeout(new Duration(emitterTimeoutDuration)); - } - - final HttpClient httpClient = HttpClientInit.createClient(configBuilder.build(), lifecycle); - - setEmitter( - new ServiceEmitter( - PropUtils.getProperty(props, "druid.service"), - PropUtils.getProperty(props, "druid.host"), - Emitters.create(props, httpClient, jsonMapper, lifecycle) - ) - ); - } - EmittingLogger.registerEmitter(emitter); - } - - protected void init() throws Exception - { - doInit(); - initialized = true; - } - - protected abstract void doInit() throws Exception; - - @LifecycleStart - public synchronized void start() throws Exception - { - if (!initialized) { - init(); - } - - lifecycle.start(); - } - - @LifecycleStop - public synchronized void stop() - { - lifecycle.stop(); - } - - protected ScheduledExecutorService startMonitoring(List monitors) - { - final ScheduledExecutorService globalScheduledExec = getScheduledExecutorFactory().create(1, "Global--%d"); - final MonitorScheduler monitorScheduler = new MonitorScheduler( - getConfigFactory().build(MonitorSchedulerConfig.class), - globalScheduledExec, - getEmitter(), - monitors - ); - getLifecycle().addManagedInstance(monitorScheduler); - return globalScheduledExec; - } - - protected void checkFieldNotSetAndSet(String fieldName, Object value) - { - Class theClazz = this.getClass(); - while (theClazz != null && theClazz != Object.class) { - try { - final Field field = theClazz.getDeclaredField(fieldName); - field.setAccessible(true); - Preconditions.checkState(field.get(this) == null, "Cannot set %s once it has already been set.", fieldName); - - field.set(this, value); - return; - } - catch (NoSuchFieldException e) { - // Perhaps it is inherited? - theClazz = theClazz.getSuperclass(); - } - catch (IllegalAccessException e) { - throw Throwables.propagate(e); - } - } - - throw new ISE("Unknown field[%s] on class[%s]", fieldName, this.getClass()); - } -} diff --git a/client/src/main/java/com/metamx/druid/RegisteringNode.java b/client/src/main/java/com/metamx/druid/RegisteringNode.java index 0d09de2f0a3..a0abbf2ecad 100644 --- a/client/src/main/java/com/metamx/druid/RegisteringNode.java +++ b/client/src/main/java/com/metamx/druid/RegisteringNode.java @@ -22,9 +22,6 @@ package com.metamx.druid; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.index.v1.serde.Registererer; -import java.util.Arrays; -import java.util.List; - /** */ public class RegisteringNode @@ -43,17 +40,4 @@ public class RegisteringNode } private static boolean doneRegister = false; - - private final List mappers; - - public RegisteringNode(List mappers) - { - this.mappers = mappers; - } - - public RegisteringNode registerHandlers(Registererer... registererers) - { - registerHandlers(Arrays.asList(registererers), mappers); - return this; - } } diff --git a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java index 94fcee12b42..ee7e59e0e65 100644 --- a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java +++ b/client/src/main/java/com/metamx/druid/curator/CuratorModule.java @@ -4,10 +4,12 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.common.logger.Logger; import com.metamx.druid.guice.ConfigProvider; import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.initialization.Initialization; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; import java.io.IOException; @@ -15,6 +17,8 @@ import java.io.IOException; */ public class CuratorModule implements Module { + private static final Logger log = new Logger(CuratorModule.class); + @Override public void configure(Binder binder) { @@ -24,6 +28,33 @@ public class CuratorModule implements Module @Provides @LazySingleton public CuratorFramework makeCurator(CuratorConfig config, Lifecycle lifecycle) throws IOException { - return Initialization.makeCuratorFramework(config, lifecycle); + final CuratorFramework framework = + CuratorFrameworkFactory.builder() + .connectString(config.getZkHosts()) + .sessionTimeoutMs(config.getZkSessionTimeoutMs()) + .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) + .compressionProvider(new PotentiallyGzippedCompressionProvider(config.enableCompression())) + .build(); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + log.info("Starting Curator"); + framework.start(); + } + + @Override + public void stop() + { + log.info("Stopping Curator"); + framework.close(); + } + } + ); + + return framework; } } diff --git a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java b/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java index 727f7704771..654babfd453 100644 --- a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java +++ b/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java @@ -325,7 +325,7 @@ public class Announcer } if (subPaths.remove(pathAndNode.getNode()) == null) { - throw new IAE("Path[%s] not announced, cannot unannounce.", path); + log.info("Asked to unnanounce path[%s] that is not announced. Was it registered multiple times?.", path); } try { diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java index 16da46a1dd9..9b6bc20af6b 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java +++ b/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java @@ -3,6 +3,7 @@ package com.metamx.druid.curator.discovery; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; @@ -17,9 +18,9 @@ import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.LazySingleton; import com.metamx.druid.initialization.CuratorDiscoveryConfig; import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.initialization.Initialization; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; +import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; import javax.annotation.Nullable; import java.lang.annotation.Annotation; @@ -179,6 +180,35 @@ public class DiscoveryModule implements Module Lifecycle lifecycle ) throws Exception { - return Initialization.makeServiceDiscoveryClient(curator, config.get(), lifecycle); + final ServiceDiscovery serviceDiscovery = + ServiceDiscoveryBuilder.builder(Void.class) + .basePath(config.get().getPath()) + .client(curator) + .build(); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + serviceDiscovery.start(); + } + + @Override + public void stop() + { + try { + serviceDiscovery.close(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + }, + Lifecycle.Stage.LAST + ); + + return serviceDiscovery; } } diff --git a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java index 3a5e39833c5..2f66d6ae8e6 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java @@ -13,6 +13,7 @@ import com.metamx.druid.query.group.GroupByQueryQueryToolChest; import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.metadata.SegmentMetadataQueryQueryToolChest; import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.search.SearchQueryConfig; import com.metamx.druid.query.search.SearchQueryQueryToolChest; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; @@ -47,5 +48,6 @@ public class QueryToolChestModule implements Module } JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); } } diff --git a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java b/client/src/main/java/com/metamx/druid/http/ClientServletModule.java deleted file mode 100644 index 8de99f59496..00000000000 --- a/client/src/main/java/com/metamx/druid/http/ClientServletModule.java +++ /dev/null @@ -1,70 +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.http; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import com.google.inject.Provides; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -/** - */ -public class ClientServletModule extends JerseyServletModule -{ - private final QuerySegmentWalker texasRanger; - private final InventoryView serverInventoryView; - private final ObjectMapper jsonMapper; - - public ClientServletModule( - QuerySegmentWalker texasRanger, - InventoryView serverInventoryView, - ObjectMapper jsonMapper - ) - { - this.texasRanger = texasRanger; - this.serverInventoryView = serverInventoryView; - this.jsonMapper = jsonMapper; - } - - @Override - protected void configureServlets() - { - bind(ClientInfoResource.class); - bind(QuerySegmentWalker.class).toInstance(texasRanger); - bind(InventoryView.class).toInstance(serverInventoryView); - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} - diff --git a/client/src/main/java/com/metamx/druid/http/StatusServlet.java b/client/src/main/java/com/metamx/druid/http/StatusServlet.java deleted file mode 100644 index 95a40c543b8..00000000000 --- a/client/src/main/java/com/metamx/druid/http/StatusServlet.java +++ /dev/null @@ -1,57 +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.http; - -import javax.servlet.ServletException; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; - -/** - */ -public class StatusServlet extends HttpServlet -{ - @Override - protected void doGet(HttpServletRequest req, HttpServletResponse resp) throws ServletException, IOException - { - ByteArrayOutputStream retVal = new ByteArrayOutputStream(); - PrintWriter out = new PrintWriter(new OutputStreamWriter(retVal)); - - Runtime runtime = Runtime.getRuntime(); - long maxMemory = runtime.maxMemory(); - long totalMemory = runtime.totalMemory(); - long freeMemory = runtime.freeMemory(); - - out.printf("Max Memory:\t%,18d\t%1$d%n", maxMemory); - out.printf("Total Memory:\t%,18d\t%1$d%n", totalMemory); - out.printf("Free Memory:\t%,18d\t%1$d%n", freeMemory); - out.printf("Used Memory:\t%,18d\t%1$d%n", totalMemory - freeMemory); - - out.flush(); - - resp.setStatus(HttpServletResponse.SC_OK); - resp.setContentType("text/plain"); - resp.getOutputStream().write(retVal.toByteArray()); - } -} diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 3aa13537449..ad81c4500e3 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -20,343 +20,28 @@ package com.metamx.druid.initialization; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Charsets; import com.google.common.base.Function; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.io.Closeables; -import com.google.common.primitives.Ints; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorConfig; -import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.guice.DruidGuiceExtensions; import com.metamx.druid.guice.DruidSecondaryModule; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; -import com.metamx.druid.http.log.EmittingRequestLogger; -import com.metamx.druid.http.log.FileRequestLogger; -import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.jackson.JacksonModule; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.service.ServiceEmitter; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.BoundedExponentialBackoffRetry; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; -import org.apache.curator.x.discovery.ServiceProvider; -import org.apache.zookeeper.data.Stat; -import org.eclipse.jetty.server.Connector; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.server.nio.SelectChannelConnector; -import org.eclipse.jetty.util.thread.QueuedThreadPool; -import org.skife.config.ConfigurationObjectFactory; import javax.annotation.Nullable; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; import java.util.Arrays; import java.util.List; -import java.util.Properties; /** */ public class Initialization { - private static final Logger log = new Logger(Initialization.class); - - private static final String PROPERTIES_FILE = "runtime.properties"; - private static final Properties zkProps = new Properties(); - private static final Properties fileProps = new Properties(zkProps); - private static Properties props = null; - - /** - * Load properties. - * Properties are layered: - *

- * # stored in zookeeper - * # runtime.properties file, - * # cmdLine -D - *

- * command line overrides runtime.properties which overrides zookeeper - *

- * Idempotent. Thread-safe. Properties are only loaded once. - * If property druid.zk.service.host is not set then do not load properties from zookeeper. - * - * @return Properties ready to use. - */ - public synchronized static Properties loadProperties() - { - if (props != null) { - return props; - } - - // Note that zookeeper coordinates must be either in cmdLine or in runtime.properties - Properties sp = System.getProperties(); - - Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain - tmp_props.putAll(sp); - - final InputStream stream = ClassLoader.getSystemResourceAsStream(PROPERTIES_FILE); - if (stream == null) { - log.info("%s not found on classpath, relying only on system properties and zookeeper.", PROPERTIES_FILE); - } else { - log.info("Loading properties from %s", PROPERTIES_FILE); - try { - try { - fileProps.load(stream); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - finally { - Closeables.closeQuietly(stream); - } - } - - // log properties from file; stringPropertyNames() would normally cascade down into the sub Properties objects, but - // zkProps (the parent level) is empty at this point so it will only log properties from runtime.properties - for (String prop : fileProps.stringPropertyNames()) { - log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop)); - } - - final String zkHostsProperty = "druid.zk.service.host"; - - if (tmp_props.getProperty(zkHostsProperty) != null) { - final ConfigurationObjectFactory factory = Config.createFactory(tmp_props); - - ZkPathsConfig config; - try { - config = factory.build(ZkPathsConfig.class); - } - catch (IllegalArgumentException e) { - log.warn(e, "Unable to build ZkPathsConfig. Cannot load properties from ZK."); - config = null; - } - - if (config != null) { - Lifecycle lifecycle = new Lifecycle(); - try { - CuratorFramework curator = makeCuratorFramework(factory.build(CuratorConfig.class), lifecycle); - - lifecycle.start(); - - final Stat stat = curator.checkExists().forPath(config.getPropertiesPath()); - if (stat != null) { - final byte[] data = curator.getData().forPath(config.getPropertiesPath()); - zkProps.load(new InputStreamReader(new ByteArrayInputStream(data), Charsets.UTF_8)); - } - - // log properties from zk - for (String prop : zkProps.stringPropertyNames()) { - log.info("Loaded(zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - lifecycle.stop(); - } - } - } else { - log.warn("property[%s] not set, skipping ZK-specified properties.", zkHostsProperty); - } - - props = tmp_props; - - return props; - } - - public static Server makeJettyServer(DruidNode node, ServerConfig config) - { - final QueuedThreadPool threadPool = new QueuedThreadPool(); - threadPool.setMinThreads(config.getNumThreads()); - threadPool.setMaxThreads(config.getNumThreads()); - - final Server server = new Server(); - server.setThreadPool(threadPool); - - SelectChannelConnector connector = new SelectChannelConnector(); - connector.setPort(node.getPort()); - connector.setMaxIdleTime(Ints.checkedCast(config.getMaxIdleTime().toStandardDuration().getMillis())); - connector.setStatsOn(true); - - server.setConnectors(new Connector[]{connector}); - - return server; - } - - public static CuratorFramework makeCuratorFramework( - CuratorConfig curatorConfig, - Lifecycle lifecycle - ) throws IOException - { - final CuratorFramework framework = - CuratorFrameworkFactory.builder() - .connectString(curatorConfig.getZkHosts()) - .sessionTimeoutMs(curatorConfig.getZkSessionTimeoutMs()) - .retryPolicy(new BoundedExponentialBackoffRetry(1000, 45000, 30)) - .compressionProvider(new PotentiallyGzippedCompressionProvider(curatorConfig.enableCompression())) - .build(); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - log.info("Starting Curator"); - framework.start(); - } - - @Override - public void stop() - { - log.info("Stopping Curator"); - framework.close(); - } - } - ); - - return framework; - } - - public static ServiceDiscovery makeServiceDiscoveryClient( - CuratorFramework discoveryClient, - CuratorDiscoveryConfig config, - Lifecycle lifecycle - ) - throws Exception - { - final ServiceDiscovery serviceDiscovery = - ServiceDiscoveryBuilder.builder(Void.class) - .basePath(config.getPath()) - .client(discoveryClient) - .build(); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - serviceDiscovery.start(); - } - - @Override - public void stop() - { - try { - serviceDiscovery.close(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - }, - Lifecycle.Stage.LAST - ); - - return serviceDiscovery; - } - - public static void announceDefaultService( - final DruidNode nodeConfig, - final ServiceAnnouncer serviceAnnouncer, - final Lifecycle lifecycle - ) throws Exception - { - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - serviceAnnouncer.announce(nodeConfig); - } - - @Override - public void stop() - { - try { - serviceAnnouncer.unannounce(nodeConfig); - } - catch (Exception e) { - log.warn(e, "Failed to unannouce default service[%s]", nodeConfig.getServiceName()); - } - } - } - ); - } - - public static ServiceProvider makeServiceProvider( - String serviceName, - ServiceDiscovery serviceDiscovery, - Lifecycle lifecycle - ) - { - final ServiceProvider serviceProvider = serviceDiscovery.serviceProviderBuilder() - .serviceName(serviceName) - .build(); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - serviceProvider.start(); - } - - @Override - public void stop() - { - try { - serviceProvider.close(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - - return serviceProvider; - } - - public static RequestLogger makeFileRequestLogger( - ObjectMapper objectMapper, - ScheduledExecutorFactory factory, - Properties props - ) throws IOException - { - return new FileRequestLogger( - objectMapper, - factory.create(1, "RequestLogger-%s"), - new File(PropUtils.getProperty(props, "druid.request.logging.dir")) - ); - } - - public static RequestLogger makeEmittingRequestLogger(Properties props, ServiceEmitter emitter) - { - return new EmittingRequestLogger(emitter, PropUtils.getProperty(props, "druid.request.logging.feed")); - } - public static Injector makeInjector(final Object... modules) { final Injector baseInjector = Guice.createInjector( diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java index 7c494754712..cf7da29b23e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java @@ -4,6 +4,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.primitives.Ints; import com.google.inject.Binder; import com.google.inject.ConfigurationException; import com.google.inject.Inject; @@ -25,7 +26,10 @@ import com.sun.jersey.api.core.ResourceConfig; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.spi.container.servlet.WebConfig; +import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.nio.SelectChannelConnector; +import org.eclipse.jetty.util.thread.QueuedThreadPool; import javax.servlet.ServletException; import java.util.List; @@ -113,7 +117,7 @@ public class JettyServerModule extends JerseyServletModule @Provides @LazySingleton public Server getServer(Injector injector, Lifecycle lifecycle, @Self DruidNode node, ServerConfig config) { - final Server server = Initialization.makeJettyServer(node, config); + final Server server = makeJettyServer(node, config); try { initializer.initialize(server, injector); } @@ -145,4 +149,23 @@ public class JettyServerModule extends JerseyServletModule ); return server; } + + private static Server makeJettyServer(@Self DruidNode node, ServerConfig config) + { + final QueuedThreadPool threadPool = new QueuedThreadPool(); + threadPool.setMinThreads(config.getNumThreads()); + threadPool.setMaxThreads(config.getNumThreads()); + + final Server server = new Server(); + server.setThreadPool(threadPool); + + SelectChannelConnector connector = new SelectChannelConnector(); + connector.setPort(node.getPort()); + connector.setMaxIdleTime(Ints.checkedCast(config.getMaxIdleTime().toStandardDuration().getMillis())); + connector.setStatsOn(true); + + server.setConnectors(new Connector[]{connector}); + + return server; + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java index 3e1eb3efa1a..b9ae29c2d90 100644 --- a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java +++ b/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java @@ -19,23 +19,18 @@ package com.metamx.druid.initialization; -import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.inject.Binder; import com.google.inject.Module; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorConfig; -import org.apache.curator.framework.CuratorFramework; -import org.apache.zookeeper.data.Stat; -import org.skife.config.ConfigurationObjectFactory; -import java.io.ByteArrayInputStream; +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.util.Properties; /** @@ -54,21 +49,23 @@ public class PropertiesModule implements Module @Override public void configure(Binder binder) { - final Properties zkProps = new Properties(); - final Properties fileProps = new Properties(zkProps); + final Properties fileProps = new Properties(); + Properties systemProps = System.getProperties(); - // Note that zookeeper coordinates must be either in cmdLine or in runtime.properties - Properties sp = System.getProperties(); + Properties props = new Properties(fileProps); + props.putAll(systemProps); - Properties tmp_props = new Properties(fileProps); // the head of the 3 level Properties chain - tmp_props.putAll(sp); + InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); + try { + if (stream == null) { + File workingDirectoryFile = new File(systemProps.getProperty("druid.properties.file", propertiesFile)); + if (workingDirectoryFile.exists()) { + stream = new BufferedInputStream(new FileInputStream(workingDirectoryFile)); + } + } - final InputStream stream = ClassLoader.getSystemResourceAsStream(propertiesFile); - if (stream == null) { - log.info("%s not found on classpath, relying only on system properties and zookeeper.", propertiesFile); - } else { - log.info("Loading properties from %s", propertiesFile); - try { + if (stream != null) { + log.info("Loading properties from %s", propertiesFile); try { fileProps.load(stream); } @@ -76,60 +73,14 @@ public class PropertiesModule implements Module throw Throwables.propagate(e); } } - finally { - Closeables.closeQuietly(stream); - } + } + catch (FileNotFoundException e) { + log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); + } + finally { + Closeables.closeQuietly(stream); } - // log properties from file; stringPropertyNames() would normally cascade down into the sub Properties objects, but - // zkProps (the parent level) is empty at this point so it will only log properties from runtime.properties - for (String prop : fileProps.stringPropertyNames()) { - log.info("Loaded(runtime.properties) Property[%s] as [%s]", prop, fileProps.getProperty(prop)); - } - - final String zkHostsProperty = "druid.zk.service.host"; - - if (tmp_props.getProperty(zkHostsProperty) != null) { - final ConfigurationObjectFactory factory = Config.createFactory(tmp_props); - - ZkPathsConfig config; - try { - config = factory.build(ZkPathsConfig.class); - } - catch (IllegalArgumentException e) { - log.warn(e, "Unable to build ZkPathsConfig. Cannot load properties from ZK."); - config = null; - } - - if (config != null) { - Lifecycle lifecycle = new Lifecycle(); - try { - CuratorFramework curator = Initialization.makeCuratorFramework(factory.build(CuratorConfig.class), lifecycle); - - lifecycle.start(); - - final Stat stat = curator.checkExists().forPath(config.getPropertiesPath()); - if (stat != null) { - final byte[] data = curator.getData().forPath(config.getPropertiesPath()); - zkProps.load(new InputStreamReader(new ByteArrayInputStream(data), Charsets.UTF_8)); - } - - // log properties from zk - for (String prop : zkProps.stringPropertyNames()) { - log.info("Loaded(zk) Property[%s] as [%s]", prop, zkProps.getProperty(prop)); - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - lifecycle.stop(); - } - } - } else { - log.warn("property[%s] not set, skipping ZK-specified properties.", zkHostsProperty); - } - - binder.bind(Properties.class).toInstance(tmp_props); + binder.bind(Properties.class).toInstance(props); } } diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java new file mode 100644 index 00000000000..f9b06988262 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java @@ -0,0 +1,38 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.search; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Min; + +/** + */ +public class SearchQueryConfig +{ + @JsonProperty + @Min(1) + private int maxSearchLimit = 1000; + + public int getMaxSearchLimit() + { + return maxSearchLimit; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index f58f31ebad7..612a89a0ecd 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -27,6 +27,7 @@ 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.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.guava.MergeSequence; @@ -37,7 +38,6 @@ import com.metamx.druid.Query; import com.metamx.druid.ResultGranularTimestampComparator; import com.metamx.druid.SearchBinaryFn; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.initialization.Initialization; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.IntervalChunkingQueryRunner; import com.metamx.druid.query.MetricManipulationFn; @@ -48,9 +48,7 @@ import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.BySegmentSearchResultValue; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceMetricEvent; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; @@ -60,7 +58,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; /** @@ -70,23 +67,19 @@ public class SearchQueryQueryToolChest extends QueryToolChest> TYPE_REFERENCE = new TypeReference>() + private static final TypeReference> TYPE_REFERENCE = new TypeReference>(){}; + + private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference(){}; + private final SearchQueryConfig config; + + @Inject + public SearchQueryQueryToolChest( + SearchQueryConfig config + ) { - }; - - private static final int maxSearchLimit; - - static { - // I dislike this static loading of properies, but it's the only mechanism available right now. - Properties props = Initialization.loadProperties(); - - maxSearchLimit = PropUtils.getPropertyAsInt(props, "com.metamx.query.search.maxSearchLimit", 1000); + this.config = config; } - private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() - { - }; - @Override public QueryRunner> mergeResults(QueryRunner> runner) { @@ -259,15 +252,23 @@ public class SearchQueryQueryToolChest extends QueryToolChest> preMergeQueryDecoration(QueryRunner> runner) { return new SearchThresholdAdjustingQueryRunner( - new IntervalChunkingQueryRunner>(runner, Period.months(1)) + new IntervalChunkingQueryRunner>(runner, Period.months(1)), + config ); } private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; + private final SearchQueryConfig config; - public SearchThresholdAdjustingQueryRunner(QueryRunner> runner) {this.runner = runner;} + public SearchThresholdAdjustingQueryRunner( + QueryRunner> runner, + SearchQueryConfig config + ) { + this.runner = runner; + this.config = config; + } @Override public Sequence> run(Query> input) @@ -277,14 +278,14 @@ public class SearchQueryQueryToolChest extends QueryToolChest, Result>() { @Override diff --git a/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java b/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java index 1114e425997..3b38dc869f0 100644 --- a/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/guice/PeonModule.java @@ -32,6 +32,10 @@ import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.metamx.druid.indexing.common.index.EventReceivingChatHandlerProvider; import com.metamx.druid.indexing.common.index.NoopChatHandlerProvider; +import com.metamx.druid.indexing.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; +import com.metamx.druid.indexing.worker.executor.ExecutorLifecycle; +import com.metamx.druid.indexing.worker.executor.ExecutorLifecycleConfig; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.S3DataSegmentKiller; @@ -39,6 +43,15 @@ import com.metamx.druid.loading.S3DataSegmentKiller; */ public class PeonModule implements Module { + private final ExecutorLifecycleConfig config; + + public PeonModule( + ExecutorLifecycleConfig config + ) + { + this.config = config; + } + @Override public void configure(Binder binder) { @@ -63,5 +76,11 @@ public class PeonModule implements Module binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); binder.bind(DataSegmentKiller.class).to(S3DataSegmentKiller.class).in(LazySingleton.class); + + binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); + binder.bind(ExecutorLifecycleConfig.class).toInstance(config); + + binder.bind(TaskRunner.class).to(ThreadPoolTaskRunner.class).in(LazySingleton.class); + binder.bind(ThreadPoolTaskRunner.class).in(ManageLifecycle.class); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java index 2b5784f0081..f5cf055ffe4 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java @@ -32,7 +32,6 @@ import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.LockListAction; import com.metamx.druid.indexing.common.actions.SegmentListUnusedAction; import com.metamx.druid.indexing.common.actions.SegmentNukeAction; -import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; @@ -51,13 +50,7 @@ public class KillTask extends AbstractTask ) { super( - id != null ? id : String.format( - "kill_%s_%s_%s_%s", - dataSource, - interval.getStart(), - interval.getEnd(), - new DateTime().toString() - ), + TaskUtils.makeId(id, "kill", dataSource, interval), dataSource, interval ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/NoopTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/NoopTask.java new file mode 100644 index 00000000000..8ff42b792d2 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/NoopTask.java @@ -0,0 +1,63 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.logger.Logger; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +/** + */ +public class NoopTask extends AbstractTask +{ + private static final Logger log = new Logger(NoopTask.class); + + @JsonCreator + public NoopTask( + @JsonProperty("id") String id, + @JsonProperty("interval") Interval interval + ) + { + super( + id == null ? String.format("noop_%s", new DateTime()) : id, + "none", + interval == null ? new Interval(Period.days(1), new DateTime()) : interval + ); + } + + @Override + public String getType() + { + return "noop"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + log.info("Running noop task[%s]", getId()); + Thread.sleep(2500); + return TaskStatus.success(getId()); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index 25f8b6425cd..fdff1ed2305 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -55,6 +55,7 @@ import org.joda.time.Interval; @JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), + @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) }) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskUtils.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskUtils.java new file mode 100644 index 00000000000..6a17ef9099a --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskUtils.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.common.task; + +import org.joda.time.DateTime; +import org.joda.time.Interval; + +/** + */ +public class TaskUtils +{ + public static String makeId(String id, final String typeName, String dataSource, Interval interval) + { + return id != null ? id : String.format( + "%s_%s_%s_%s_%s", + typeName, + dataSource, + interval.getStart(), + interval.getEnd(), + new DateTime().toString() + ); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 6c55b1b6a95..0e72bf53461 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -45,7 +45,6 @@ import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.tasklogs.TaskLogPusher; import com.metamx.druid.indexing.common.tasklogs.TaskLogStreamer; import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.indexing.worker.executor.ExecutorMain; import com.metamx.druid.initialization.DruidNode; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; @@ -65,12 +64,13 @@ import java.util.concurrent.Callable; import java.util.concurrent.Executors; /** - * Runs tasks in separate processes using {@link ExecutorMain}. + * Runs tasks in separate processes using the "internal peon" verb. */ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer { private static final EmittingLogger log = new EmittingLogger(ForkingTaskRunner.class); private static final String CHILD_PROPERTY_PREFIX = "druid.indexer.fork.property."; + private static final Splitter whiteSpaceSplitter = Splitter.on(CharMatcher.WHITESPACE).omitEmptyStrings(); private final ForkingTaskRunnerConfig config; private final Properties props; @@ -145,11 +145,15 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer } if (taskInfo == null) { - throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId()); + log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); + throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); } if (taskInfo.processHolder != null) { - throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId()); + log.makeAlert("WTF?! TaskInfo already has a processHolder") + .addData("task", task.getId()) + .emit(); + throw new ISE("TaskInfo already has processHolder for task[%s]!", task.getId()); } final List command = Lists.newArrayList(); @@ -160,12 +164,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer command.add("-cp"); command.add(config.getClasspath()); - Iterables.addAll( - command, - Splitter.on(CharMatcher.WHITESPACE) - .omitEmptyStrings() - .split(config.getJavaOpts()) - ); + Iterables.addAll(command, whiteSpaceSplitter.split(config.getJavaOpts())); for (String propName : props.stringPropertyNames()) { for (String allowedPrefix : config.getAllowedPrefixes()) { @@ -194,15 +193,16 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer } } - String nodeType = task.getNodeType(); - if (nodeType != null) { - command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType)); - } - command.add(String.format("-Ddruid.host=%s", childHost)); command.add(String.format("-Ddruid.port=%d", childPort)); - command.add(config.getMainClass()); + command.add("io.druid.cli.Main"); + command.add("internal"); + command.add("peon"); + String nodeType = task.getNodeType(); + if (nodeType != null) { + command.add(String.format("--nodeType %s", nodeType)); + } command.add(taskFile.toString()); command.add(statusFile.toString()); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java index f8ef078f771..e3f8a9d60ae 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMaster.java @@ -34,7 +34,6 @@ import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; @@ -129,7 +128,23 @@ public class TaskMaster } ); leaderLifecycle.addManagedInstance(taskQueue); - Initialization.announceDefaultService(node, serviceAnnouncer, leaderLifecycle); + + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + serviceAnnouncer.announce(node); + } + + @Override + public void stop() + { + serviceAnnouncer.unannounce(node); + } + } + ); leaderLifecycle.addManagedInstance(taskConsumer); if (taskRunner instanceof RemoteTaskRunner) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index b0a69e9a14a..d6f99d94e25 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -2,7 +2,6 @@ package com.metamx.druid.indexing.coordinator.config; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; -import com.metamx.druid.indexing.worker.executor.ExecutorMain; import javax.validation.constraints.Max; import javax.validation.constraints.Min; @@ -23,6 +22,11 @@ public class ForkingTaskRunnerConfig @NotNull private String javaCommand = "java"; + /** + * This is intended for setting -X parameters on the underlying java. It is used by first splitting on whitespace, + * so it cannot handle properties that have whitespace in the value. Those properties should be set via a + * druid.indexer.fork.property. property instead. + */ @JsonProperty @NotNull private String javaOpts = ""; @@ -31,10 +35,6 @@ public class ForkingTaskRunnerConfig @NotNull private String classpath = System.getProperty("java.class.path"); - @JsonProperty - @NotNull - private String mainClass = ExecutorMain.class.getName(); - @JsonProperty @Min(1024) @Max(65535) private int startPort = 8080; @@ -68,11 +68,6 @@ public class ForkingTaskRunnerConfig return classpath; } - public String getMainClass() - { - return mainClass; - } - public int getStartPort() { return startPort; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java deleted file mode 100644 index 7eb8ce7e436..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java +++ /dev/null @@ -1,91 +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.indexing.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.indexing.common.tasklogs.TaskLogStreamer; -import com.metamx.druid.indexing.coordinator.TaskMaster; -import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.emitter.service.ServiceEmitter; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -/** - */ -public class IndexerCoordinatorServletModule extends JerseyServletModule -{ - private final ObjectMapper jsonMapper; - private final IndexerCoordinatorConfig indexerCoordinatorConfig; - private final ServiceEmitter emitter; - private final TaskMaster taskMaster; - private final TaskStorageQueryAdapter taskStorageQueryAdapter; - private final TaskLogStreamer taskLogStreamer; - private final JacksonConfigManager configManager; - - public IndexerCoordinatorServletModule( - ObjectMapper jsonMapper, - IndexerCoordinatorConfig indexerCoordinatorConfig, - ServiceEmitter emitter, - TaskMaster taskMaster, - TaskStorageQueryAdapter taskStorageQueryAdapter, - TaskLogStreamer taskLogStreamer, - JacksonConfigManager configManager - ) - { - this.jsonMapper = jsonMapper; - this.indexerCoordinatorConfig = indexerCoordinatorConfig; - this.emitter = emitter; - this.taskMaster = taskMaster; - this.taskStorageQueryAdapter = taskStorageQueryAdapter; - this.taskLogStreamer = taskLogStreamer; - this.configManager = configManager; - } - - @Override - protected void configureServlets() - { - bind(IndexerCoordinatorResource.class); - bind(OldIndexerCoordinatorResource.class); - bind(ObjectMapper.class).toInstance(jsonMapper); - bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); - bind(ServiceEmitter.class).toInstance(emitter); - bind(TaskMaster.class).toInstance(taskMaster); - bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); - bind(TaskLogStreamer.class).toInstance(taskLogStreamer); - bind(JacksonConfigManager.class).toInstance(configManager); - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java index 5d5543b9de7..4e998825656 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -5,9 +5,10 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.druid.concurrent.Execs; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskRunner; @@ -17,7 +18,6 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; /** * Encapsulates the lifecycle of a task executor. Loads one task, runs it, writes its status, and all the while @@ -27,37 +27,33 @@ public class ExecutorLifecycle { private static final EmittingLogger log = new EmittingLogger(ExecutorLifecycle.class); - private final File taskFile; - private final File statusFile; + private final ExecutorLifecycleConfig config; private final TaskRunner taskRunner; - private final InputStream parentStream; private final ObjectMapper jsonMapper; - private final ExecutorService parentMonitorExec = Executors.newFixedThreadPool( - 1, - new ThreadFactoryBuilder().setNameFormat("parent-monitor-%d").setDaemon(true).build() - ); + private final ExecutorService parentMonitorExec = Execs.singleThreaded("parent-monitor-%d"); private volatile ListenableFuture statusFuture = null; + @Inject public ExecutorLifecycle( - File taskFile, - File statusFile, + ExecutorLifecycleConfig config, TaskRunner taskRunner, - InputStream parentStream, ObjectMapper jsonMapper ) { - this.taskFile = taskFile; - this.statusFile = statusFile; + this.config = config; this.taskRunner = taskRunner; - this.parentStream = parentStream; this.jsonMapper = jsonMapper; } @LifecycleStart public void start() { + final File taskFile = config.getTaskFile(); + final File statusFile = config.getStatusFile(); + final InputStream parentStream = config.getParentStream(); + final Task task; try { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleConfig.java new file mode 100644 index 00000000000..614a21fb2ac --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleConfig.java @@ -0,0 +1,88 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexing.worker.executor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.ISE; + +import javax.validation.constraints.NotNull; +import javax.validation.constraints.Pattern; +import java.io.File; +import java.io.InputStream; + +/** + */ +public class ExecutorLifecycleConfig +{ + @JsonProperty + @NotNull + private File taskFile = null; + + @JsonProperty + @NotNull + private File statusFile = null; + + @JsonProperty + @Pattern(regexp = "\\{stdin\\}") + private String parentStreamName = "stdin"; + + public File getTaskFile() + { + return taskFile; + } + + public ExecutorLifecycleConfig setTaskFile(File taskFile) + { + this.taskFile = taskFile; + return this; + } + + public File getStatusFile() + { + return statusFile; + } + + public ExecutorLifecycleConfig setStatusFile(File statusFile) + { + this.statusFile = statusFile; + return this; + } + + public String getParentStreamName() + { + return parentStreamName; + } + + public ExecutorLifecycleConfig setParentStreamName(String parentStreamName) + { + this.parentStreamName = parentStreamName; + return this; + } + + public InputStream getParentStream() + { + if ("stdin".equals(parentStreamName)) { + return System.in; + } + else { + throw new ISE("Unknown stream name[%s]", parentStreamName); + } + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java index eec2fff2929..079e9909bc8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java @@ -4,23 +4,22 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.indexing.coordinator.TaskRunner; import java.io.File; -import java.io.InputStream; public class ExecutorLifecycleFactory { private final File taskFile; private final File statusFile; - private final InputStream parentStream; - public ExecutorLifecycleFactory(File taskFile, File statusFile, InputStream parentStream) + public ExecutorLifecycleFactory(File taskFile, File statusFile) { this.taskFile = taskFile; this.statusFile = statusFile; - this.parentStream = parentStream; } public ExecutorLifecycle build(TaskRunner taskRunner, ObjectMapper jsonMapper) { - return new ExecutorLifecycle(taskFile, statusFile, taskRunner, parentStream, jsonMapper); + return new ExecutorLifecycle( + new ExecutorLifecycleConfig().setTaskFile(taskFile).setStatusFile(statusFile), taskRunner, jsonMapper + ); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java deleted file mode 100644 index f4471b61ab4..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java +++ /dev/null @@ -1,73 +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.indexing.worker.executor; - -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.log.LogLevelAdjuster; - -import java.io.File; -import java.util.Arrays; -import java.util.Iterator; - -/** - */ -public class ExecutorMain -{ - private static final Logger log = new Logger(ExecutorMain.class); - - public static void main(String[] args) throws Exception - { - LogLevelAdjuster.register(); - - if (args.length != 2) { - log.info("Usage: ExecutorMain "); - System.exit(2); - } - - Iterator arguments = Arrays.asList(args).iterator(); - final String taskJsonFile = arguments.next(); - final String statusJsonFile = arguments.next(); - - final ExecutorNode node = ExecutorNode.builder() - .build( - System.getProperty("druid.executor.nodeType", "indexer-executor"), - new ExecutorLifecycleFactory( - new File(taskJsonFile), - new File(statusJsonFile), - System.in - ) - ); - - final Lifecycle lifecycle = new Lifecycle(); - - lifecycle.addManagedInstance(node); - - try { - lifecycle.start(); - node.join(); - lifecycle.stop(); - } - catch (Throwable t) { - log.info(t, "Throwable caught at startup, committing seppuku"); - System.exit(2); - } - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java deleted file mode 100644 index ad8a6a26168..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ /dev/null @@ -1,459 +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.indexing.worker.executor; - -import com.fasterxml.jackson.databind.InjectableValues; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.servlet.GuiceFilter; -import com.metamx.common.ISE; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.config.Config; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.lifecycle.LifecycleStart; -import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.BaseServerNode; -import com.metamx.druid.client.indexing.IndexingServiceSelector; -import com.metamx.druid.curator.CuratorConfig; -import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.http.GuiceServletConfig; -import com.metamx.druid.http.QueryServlet; -import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.RetryPolicyConfig; -import com.metamx.druid.indexing.common.RetryPolicyFactory; -import com.metamx.druid.indexing.common.TaskToolboxFactory; -import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; -import com.metamx.druid.indexing.common.config.TaskConfig; -import com.metamx.druid.indexing.common.index.ChatHandlerProvider; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.EventReceivingChatHandlerProvider; -import com.metamx.druid.indexing.common.index.NoopChatHandlerProvider; -import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; -import com.metamx.druid.indexing.worker.config.WorkerConfig; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServerConfig; -import com.metamx.druid.initialization.ServerInit; -import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.DataSegmentKiller; -import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.S3DataSegmentKiller; -import com.metamx.druid.utils.PropUtils; -import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Emitters; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.Monitor; -import com.metamx.metrics.MonitorScheduler; -import com.metamx.metrics.MonitorSchedulerConfig; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.x.discovery.ServiceDiscovery; -import org.apache.curator.x.discovery.ServiceProvider; -import org.eclipse.jetty.server.Server; -import org.eclipse.jetty.servlet.DefaultServlet; -import org.eclipse.jetty.servlet.ServletContextHandler; -import org.eclipse.jetty.servlet.ServletHolder; -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.util.Properties; -import java.util.concurrent.ScheduledExecutorService; - -/** - */ -public class ExecutorNode extends BaseServerNode -{ - private static final EmittingLogger log = new EmittingLogger(ExecutorNode.class); - - public static Builder builder() - { - return new Builder(); - } - - private final Lifecycle lifecycle; - private final Properties props; - private final ConfigurationObjectFactory configFactory; - private final ExecutorLifecycleFactory executorLifecycleFactory; - - private RestS3Service s3Service = null; - private MonitorScheduler monitorScheduler = null; - private HttpClient httpClient = null; - private ServiceEmitter emitter = null; - private TaskConfig taskConfig = null; - private WorkerConfig workerConfig = null; - private DataSegmentPusher segmentPusher = null; - private TaskToolboxFactory taskToolboxFactory = null; - private ServiceDiscovery serviceDiscovery = null; - private ServiceAnnouncer serviceAnnouncer = null; - private ServiceProvider coordinatorServiceProvider = null; - private Server server = null; - private ThreadPoolTaskRunner taskRunner = null; - private ExecutorLifecycle executorLifecycle = null; - private ChatHandlerProvider chatHandlerProvider = null; - - public ExecutorNode( - String nodeType, - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory, - ExecutorLifecycleFactory executorLifecycleFactory - ) - { - super(nodeType, log, props, lifecycle, jsonMapper, smileMapper, configFactory); - - this.lifecycle = lifecycle; - this.props = props; - this.configFactory = configFactory; - this.executorLifecycleFactory = executorLifecycleFactory; - } - - @Override - public void doInit() throws Exception - { - initializeHttpClient(); - initializeEmitter(); - initializeS3Service(); - initializeMergerConfig(); - initializeServiceDiscovery(); - initializeDataSegmentPusher(); - initializeMonitorScheduler(); - initializeTaskToolbox(); - initializeTaskRunner(); - initializeChatHandlerProvider(); - initializeJacksonInjections(); - initializeJacksonSubtypes(); - initializeServer(); - - executorLifecycle = executorLifecycleFactory.build(taskRunner, getJsonMapper()); - lifecycle.addManagedInstance(executorLifecycle); - - final Injector injector = Guice.createInjector( - new ExecutorServletModule( - getJsonMapper(), - chatHandlerProvider - ) - ); - final ServletContextHandler root = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS); - - root.addServlet(new ServletHolder(new StatusServlet()), "/status"); - root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", null); - root.addServlet( - new ServletHolder( - new QueryServlet(getJsonMapper(), getSmileMapper(), taskRunner, emitter, getRequestLogger()) - ), - "/druid/v2/*" - ); - } - - private void initializeMonitorScheduler() - { - if (monitorScheduler == null) - { - final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - this.monitorScheduler = new MonitorScheduler( - configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, ImmutableList.of() - ); - lifecycle.addManagedInstance(monitorScheduler); - } - } - - @LifecycleStart - public synchronized void start() throws Exception - { - init(); - lifecycle.start(); - } - - @LifecycleStop - public synchronized void stop() - { - lifecycle.stop(); - } - - public void join() - { - executorLifecycle.join(); - } - - public ThreadPoolTaskRunner getTaskRunner() - { - return taskRunner; - } - - private void initializeServer() - { - if (server == null) { - server = Initialization.makeJettyServer(null, configFactory.build(ServerConfig.class)); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - log.info("Starting Jetty"); - server.start(); - } - - @Override - public void stop() - { - log.info("Stopping Jetty"); - try { - server.stop(); - } - catch (Exception e) { - log.error(e, "Exception thrown while stopping Jetty"); - } - } - } - ); - } - } - - private void initializeJacksonInjections() - { - InjectableValues.Std injectables = new InjectableValues.Std(); - - injectables.addValue("s3Client", s3Service) - .addValue("segmentPusher", segmentPusher) - .addValue("chatHandlerProvider", chatHandlerProvider); - - getJsonMapper().setInjectableValues(injectables); - } - - private void initializeJacksonSubtypes() - { - getJsonMapper().registerSubtypes(StaticS3FirehoseFactory.class); - getJsonMapper().registerSubtypes(EventReceiverFirehoseFactory.class); - } - - private void initializeHttpClient() - { - if (httpClient == null) { - httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), lifecycle - ); - } - } - - private void initializeEmitter() - { - if (emitter == null) { - emitter = new ServiceEmitter( - PropUtils.getProperty(props, "druid.service"), - PropUtils.getProperty(props, "druid.host"), - Emitters.create(props, httpClient, getJsonMapper(), lifecycle) - ); - } - EmittingLogger.registerEmitter(emitter); - } - - private void initializeS3Service() throws S3ServiceException - { - if (s3Service == null) { - s3Service = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) - ); - } - } - - private void initializeMergerConfig() - { - if (taskConfig == null) { - taskConfig = configFactory.build(TaskConfig.class); - } - - if (workerConfig == null) { - workerConfig = configFactory.build(WorkerConfig.class); - } - } - - public void initializeDataSegmentPusher() - { - if (segmentPusher == null) { - segmentPusher = ServerInit.getSegmentPusher(props, configFactory, getJsonMapper()); - } - } - - public void initializeTaskToolbox() throws S3ServiceException - { - if (taskToolboxFactory == null) { - final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service); - taskToolboxFactory = new TaskToolboxFactory( - taskConfig, - new RemoteTaskActionClientFactory( - httpClient, - new IndexingServiceSelector(coordinatorServiceProvider), - new RetryPolicyFactory( - configFactory.buildWithReplacements( - RetryPolicyConfig.class, - ImmutableMap.of("base_path", "druid.worker.taskActionClient") - ) - ), - getJsonMapper() - ), - emitter, - s3Service, - segmentPusher, - dataSegmentKiller, - getAnnouncer(), - getServerView(), - getConglomerate(), - monitorScheduler, - getJsonMapper() - ); - } - } - - public void initializeServiceDiscovery() throws Exception - { - final CuratorConfig config = configFactory.build(CuratorConfig.class); - if (serviceDiscovery == null) { - final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(config, lifecycle); - CuratorDiscoveryConfig discoveryConfig = getJsonConfigurator() - .configurate(getProps(), "druid.discovery.curator", CuratorDiscoveryConfig.class); - - this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - serviceDiscoveryCuratorFramework, discoveryConfig, lifecycle - ); - } - if (serviceAnnouncer == null) { - this.serviceAnnouncer = new CuratorServiceAnnouncer(serviceDiscovery); - } - if (coordinatorServiceProvider == null) { - this.coordinatorServiceProvider = Initialization.makeServiceProvider( - workerConfig.getOverlordService(), - serviceDiscovery, - lifecycle - ); - } - } - - public void initializeTaskRunner() - { - if (taskRunner == null) { - this.taskRunner = lifecycle.addManagedInstance(new ThreadPoolTaskRunner(taskToolboxFactory)); - } - } - - public void initializeChatHandlerProvider() - { - if (chatHandlerProvider == null) { - final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); - if (config.isPublishDiscovery()) { - this.chatHandlerProvider = new EventReceivingChatHandlerProvider(null, serviceAnnouncer); // TODO: eliminate - } else { - log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); - this.chatHandlerProvider = new NoopChatHandlerProvider(); - } - } - } - - public static class Builder - { - private ObjectMapper jsonMapper = null; - private ObjectMapper smileMapper = null; - private Lifecycle lifecycle = null; - private Properties props = null; - private ConfigurationObjectFactory configFactory = null; - - public Builder withMapper(ObjectMapper jsonMapper) - { - this.jsonMapper = jsonMapper; - return this; - } - - public Builder withLifecycle(Lifecycle lifecycle) - { - this.lifecycle = lifecycle; - return this; - } - - public Builder withProps(Properties props) - { - this.props = props; - return this; - } - - public Builder withConfigFactory(ConfigurationObjectFactory configFactory) - { - this.configFactory = configFactory; - return this; - } - - public ExecutorNode build(String nodeType, ExecutorLifecycleFactory executorLifecycleFactory) - { - if (jsonMapper == null && smileMapper == null) { - jsonMapper = new DefaultObjectMapper(); - smileMapper = new DefaultObjectMapper(new SmileFactory()); - smileMapper.getJsonFactory().setCodec(smileMapper); - } else if (jsonMapper == null || smileMapper == null) { - throw new ISE( - "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", - jsonMapper, - smileMapper - ); - } - - if (lifecycle == null) { - lifecycle = new Lifecycle(); - } - - if (props == null) { - props = Initialization.loadProperties(); - } - - if (configFactory == null) { - configFactory = Config.createFactory(props); - } - - return new ExecutorNode( - nodeType, - props, - lifecycle, - jsonMapper, - smileMapper, - configFactory, - executorLifecycleFactory - ); - } - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java deleted file mode 100644 index 1350094420c..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java +++ /dev/null @@ -1,44 +0,0 @@ -package com.metamx.druid.indexing.worker.executor; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import com.google.inject.Provides; -import com.metamx.druid.indexing.common.index.ChatHandlerProvider; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -public class ExecutorServletModule extends JerseyServletModule -{ - private final ObjectMapper jsonMapper; - private final ChatHandlerProvider receivers; - - public ExecutorServletModule( - ObjectMapper jsonMapper, - ChatHandlerProvider receivers - ) - { - this.jsonMapper = jsonMapper; - this.receivers = receivers; - } - - @Override - protected void configureServlets() - { - bind(ChatHandlerResource.class); - bind(ObjectMapper.class).toInstance(jsonMapper); - bind(ChatHandlerProvider.class).toInstance(receivers); - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java deleted file mode 100644 index b135af7dcb6..00000000000 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java +++ /dev/null @@ -1,70 +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.indexing.worker.http; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import com.google.inject.Provides; -import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; -import com.metamx.emitter.service.ServiceEmitter; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -/** - */ -public class WorkerServletModule extends JerseyServletModule -{ - private final ObjectMapper jsonMapper; - private final ServiceEmitter emitter; - private final ForkingTaskRunner forkingTaskRunner; - - public WorkerServletModule( - ObjectMapper jsonMapper, - ServiceEmitter emitter, - ForkingTaskRunner forkingTaskRunner - ) - { - this.jsonMapper = jsonMapper; - this.emitter = emitter; - this.forkingTaskRunner = forkingTaskRunner; - } - - @Override - protected void configureServlets() - { - bind(WorkerResource.class); - bind(ObjectMapper.class).toInstance(jsonMapper); - bind(ServiceEmitter.class).toInstance(emitter); - bind(ForkingTaskRunner.class).toInstance(forkingTaskRunner); - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} diff --git a/indexing-service/src/test/resources/task.json b/indexing-service/src/test/resources/task.json new file mode 100644 index 00000000000..62098bbe7bd --- /dev/null +++ b/indexing-service/src/test/resources/task.json @@ -0,0 +1 @@ +{ "type": "noop" } \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/BaseServerNode.java b/server/src/main/java/com/metamx/druid/BaseServerNode.java deleted file mode 100644 index efe2f2a19c6..00000000000 --- a/server/src/main/java/com/metamx/druid/BaseServerNode.java +++ /dev/null @@ -1,147 +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; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.common.logger.Logger; -import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.initialization.ServerInit; -import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryRunnerFactoryConglomerate; - -import org.skife.config.ConfigurationObjectFactory; - -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.Properties; - -/** - */ -public abstract class BaseServerNode extends QueryableNode -{ - private final Map, QueryRunnerFactory> additionalFactories = Maps.newLinkedHashMap(); - private DruidProcessingConfig processingConfig = null; - private QueryRunnerFactoryConglomerate conglomerate = null; - private StupidPool computeScratchPool = null; - - public BaseServerNode( - String nodeType, - Logger log, - Properties props, - Lifecycle lifecycle, - ObjectMapper jsonMapper, - ObjectMapper smileMapper, - ConfigurationObjectFactory configFactory - ) - { - super(nodeType, log, props, lifecycle, jsonMapper, smileMapper, configFactory); - } - - public QueryRunnerFactoryConglomerate getConglomerate() - { - initializeQueryRunnerFactoryConglomerate(); - return conglomerate; - } - - public StupidPool getComputeScratchPool() - { - initializeComputeScratchPool(); - return computeScratchPool; - } - - public DruidProcessingConfig getProcessingConfig() - { - initializeProcessingConfig(); - return processingConfig; - } - - @SuppressWarnings("unchecked") - public T setConglomerate(QueryRunnerFactoryConglomerate conglomerate) - { - checkFieldNotSetAndSet("conglomerate", conglomerate); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setComputeScratchPool(StupidPool computeScratchPool) - { - checkFieldNotSetAndSet("computeScratchPool", computeScratchPool); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T setProcessingConfig(DruidProcessingConfig processingConfig) - { - checkFieldNotSetAndSet("processingConfig", processingConfig); - return (T) this; - } - - @SuppressWarnings("unchecked") - public T registerQueryRunnerFactory(Class queryClazz, QueryRunnerFactory factory) - { - Preconditions.checkState( - conglomerate == null, - "Registering a QueryRunnerFactory only works when a separate conglomerate is not specified." - ); - Preconditions.checkState( - !additionalFactories.containsKey(queryClazz), "Registered factory for class[%s] multiple times", queryClazz - ); - additionalFactories.put(queryClazz, factory); - return (T) this; - } - - private void initializeComputeScratchPool() - { - if (computeScratchPool == null) { - setComputeScratchPool(ServerInit.makeComputeScratchPool(getProcessingConfig())); - } - } - - private void initializeQueryRunnerFactoryConglomerate() - { - if (conglomerate == null) { - final Map, QueryRunnerFactory> factories = ServerInit.initDefaultQueryTypes( - getConfigFactory(), getComputeScratchPool() - ); - - for (Map.Entry, QueryRunnerFactory> entry : additionalFactories.entrySet()) { - factories.put(entry.getKey(), entry.getValue()); - } - - setConglomerate(new DefaultQueryRunnerFactoryConglomerate(factories)); - } - } - - private void initializeProcessingConfig() - { - if (processingConfig == null) { - setProcessingConfig( - getConfigFactory().buildWithReplacements( - DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing") - ) - ); - } - } -} diff --git a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java b/server/src/main/java/com/metamx/druid/http/MasterServletModule.java deleted file mode 100644 index 26d9ad15459..00000000000 --- a/server/src/main/java/com/metamx/druid/http/MasterServletModule.java +++ /dev/null @@ -1,91 +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.http; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; -import com.google.inject.Provides; -import com.google.inject.util.Providers; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.indexing.IndexingServiceClient; -import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.master.DruidMaster; -import com.sun.jersey.guice.JerseyServletModule; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; - -import javax.inject.Singleton; - -/** - */ -public class MasterServletModule extends JerseyServletModule -{ - private final InventoryView serverInventoryView; - private final DatabaseSegmentManager segmentInventoryManager; - private final DatabaseRuleManager databaseRuleManager; - private final DruidMaster master; - private final ObjectMapper jsonMapper; - private final IndexingServiceClient indexingServiceClient; - - public MasterServletModule( - InventoryView serverInventoryView, - DatabaseSegmentManager segmentInventoryManager, - DatabaseRuleManager databaseRuleManager, - DruidMaster master, - ObjectMapper jsonMapper, - IndexingServiceClient indexingServiceClient - ) - { - this.serverInventoryView = serverInventoryView; - this.segmentInventoryManager = segmentInventoryManager; - this.databaseRuleManager = databaseRuleManager; - this.master = master; - this.jsonMapper = jsonMapper; - this.indexingServiceClient = indexingServiceClient; - } - - @Override - protected void configureServlets() - { - bind(InfoResource.class); - bind(MasterResource.class); - bind(InventoryView.class).toInstance(serverInventoryView); - bind(DatabaseSegmentManager.class).toInstance(segmentInventoryManager); - bind(DatabaseRuleManager.class).toInstance(databaseRuleManager); - bind(DruidMaster.class).toInstance(master); - if (indexingServiceClient == null) { - bind(IndexingServiceClient.class).toProvider(Providers.of(null)); - } - else { - bind(IndexingServiceClient.class).toInstance(indexingServiceClient); - } - - serve("/*").with(GuiceContainer.class); - } - - @Provides - @Singleton - public JacksonJsonProvider getJacksonJsonProvider() - { - final JacksonJsonProvider provider = new JacksonJsonProvider(); - provider.setMapper(jsonMapper); - return provider; - } -} diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java deleted file mode 100644 index 96f5d3a5bda..00000000000 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ /dev/null @@ -1,195 +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.initialization; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import com.google.common.base.Throwables; -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.Query; -import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.loading.HdfsDataSegmentPusher; -import com.metamx.druid.loading.HdfsDataSegmentPusherConfig; -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.loading.cassandra.CassandraDataSegmentConfig; -import com.metamx.druid.loading.cassandra.CassandraDataSegmentPusher; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.group.GroupByQueryConfig; -import com.metamx.druid.query.group.GroupByQueryEngine; -import com.metamx.druid.query.group.GroupByQueryQueryToolChest; -import com.metamx.druid.query.group.GroupByQueryRunnerFactory; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQueryRunnerFactory; -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.apache.hadoop.conf.Configuration; -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; - -/** - */ -public class ServerInit -{ - private static Logger log = new Logger(ServerInit.class); - - public static StupidPool makeComputeScratchPool(DruidProcessingConfig config) - { - try { - Class vmClass = Class.forName("sun.misc.VM"); - Object maxDirectMemoryObj = vmClass.getMethod("maxDirectMemory").invoke(null); - - if (maxDirectMemoryObj == null || !(maxDirectMemoryObj instanceof Number)) { - log.info("Cannot determine maxDirectMemory from[%s]", maxDirectMemoryObj); - } else { - long maxDirectMemory = ((Number) maxDirectMemoryObj).longValue(); - - final long memoryNeeded = (long) config.intermediateComputeSizeBytes() * (config.getNumThreads() + 1); - if (maxDirectMemory < memoryNeeded) { - throw new ISE( - "Not enough direct memory. Please adjust -XX:MaxDirectMemorySize or druid.computation.buffer.size: " - + "maxDirectMemory[%,d], memoryNeeded[%,d], druid.computation.buffer.size[%,d], numThreads[%,d]", - maxDirectMemory, memoryNeeded, config.intermediateComputeSizeBytes(), config.getNumThreads() - ); - } - } - } - catch (ClassNotFoundException e) { - log.info("No VM class, cannot do memory check."); - } - catch (NoSuchMethodException e) { - log.info("VM.maxDirectMemory doesn't exist, cannot do memory check."); - } - catch (InvocationTargetException e) { - log.warn(e, "static method shouldn't throw this"); - } - catch (IllegalAccessException e) { - log.warn(e, "public method, shouldn't throw this"); - } - - return new ComputeScratchPool(config.intermediateComputeSizeBytes()); - } - - // TODO: Get rid of this method - public static Map, QueryRunnerFactory> initDefaultQueryTypes( - ConfigurationObjectFactory configFactory, - StupidPool computationBufferPool - ) - { - Map, QueryRunnerFactory> queryRunners = Maps.newLinkedHashMap(); - queryRunners.put(TimeseriesQuery.class, new TimeseriesQueryRunnerFactory()); - final Supplier configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); - queryRunners.put( - GroupByQuery.class, - new GroupByQueryRunnerFactory( - new GroupByQueryEngine(configSupplier, computationBufferPool), - configSupplier, - new GroupByQueryQueryToolChest(configSupplier) - ) - ); - queryRunners.put(SearchQuery.class, new SearchQueryRunnerFactory()); - queryRunners.put(TimeBoundaryQuery.class, new TimeBoundaryQueryRunnerFactory()); - queryRunners.put(SegmentMetadataQuery.class, new SegmentMetadataQueryRunnerFactory()); - 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 if (Boolean.parseBoolean(props.getProperty("druid.pusher.cassandra", "false"))) { - final CassandraDataSegmentConfig config = configFactory.build(CassandraDataSegmentConfig.class); - - return new CassandraDataSegmentPusher(config, jsonMapper); - } - else if (Boolean.parseBoolean(props.getProperty("druid.pusher.hdfs", "false"))) { - final HdfsDataSegmentPusherConfig config = configFactory.build(HdfsDataSegmentPusherConfig.class); - - return new HdfsDataSegmentPusher(config, new Configuration(), 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); - - public ComputeScratchPool(final int computationBufferSize) - { - super( - new Supplier() - { - final AtomicLong count = new AtomicLong(0); - - @Override - public ByteBuffer get() - { - log.info( - "Allocating new computeScratchPool[%,d] of size[%,d]", count.getAndIncrement(), computationBufferSize - ); - return ByteBuffer.allocateDirect(computationBufferSize); - } - } - ); - } - } -} diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index fb3a43aa73d..167c019c075 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -19,7 +19,9 @@ package io.druid.cli; +import com.google.common.base.Throwables; import com.google.inject.Injector; +import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.CuratorModule; import com.metamx.druid.curator.discovery.DiscoveryModule; @@ -39,11 +41,19 @@ import com.metamx.druid.guice.StorageNodeModule; import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; import com.metamx.druid.indexing.worker.executor.ChatHandlerResource; +import com.metamx.druid.indexing.worker.executor.ExecutorLifecycle; +import com.metamx.druid.indexing.worker.executor.ExecutorLifecycleConfig; import com.metamx.druid.initialization.EmitterModule; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.JettyServerModule; +import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.metrics.MetricsModule; +import io.airlift.command.Arguments; import io.airlift.command.Command; +import io.airlift.command.Option; + +import java.io.File; +import java.util.List; /** */ @@ -52,20 +62,18 @@ import io.airlift.command.Command; description = "Runs a Peon, this is an individual forked \"task\" used as part of the indexing service. " + "This should rarely, if ever, be used directly." ) -public class CliPeon extends ServerRunnable +public class CliPeon implements Runnable { + @Arguments(description = "task.json status.json", required = true) + public List taskAndStatusFile; + + @Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK") + public String nodeType = "indexer-executor"; + private static final Logger log = new Logger(CliPeon.class); - public CliPeon() - { - super(log); - } - - @Override protected Injector getInjector() { - // TODO: make it take and run a task - return Initialization.makeInjector( new LifecycleModule(), EmitterModule.class, @@ -78,7 +86,7 @@ public class CliPeon extends ServerRunnable .addResource(ChatHandlerResource.class), new DiscoveryModule(), new ServerViewModule(), - new StorageNodeModule("real-time"), + new StorageNodeModule(nodeType), new DataSegmentPusherModule(), new AnnouncerModule(), new DruidProcessingModule(), @@ -86,7 +94,35 @@ public class CliPeon extends ServerRunnable new QueryRunnerFactoryModule(), new IndexingServiceDiscoveryModule(), new AWSModule(), - new PeonModule() + new PeonModule( + new ExecutorLifecycleConfig() + .setTaskFile(new File(taskAndStatusFile.get(0))) + .setStatusFile(new File(taskAndStatusFile.get(1))) + ) ); } + + @Override + public void run() + { + try { + LogLevelAdjuster.register(); + + final Injector injector = getInjector(); + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); + + try { + lifecycle.start(); + injector.getInstance(ExecutorLifecycle.class).join(); + lifecycle.stop(); + } + catch (Throwable t) { + log.error(t, "Error when starting up. Failing."); + System.exit(1); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } } diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 934348c7f4d..f7a264f6b4d 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -40,8 +40,8 @@ public class Main .withDescription("Run one of the Druid server types.") .withDefaultCommand(Help.class) .withCommands( - CliCoordinator.class, CliHistorical.class, CliBroker.class, CliRealtime.class, - CliOverlord.class, CliMiddleManager.class, CliPeon.class + CliCoordinator.class, CliHistorical.class, CliBroker.class, + CliRealtime.class, CliOverlord.class, CliMiddleManager.class ); builder.withGroup("example") @@ -49,6 +49,11 @@ public class Main .withDefaultCommand(Help.class) .withCommands(CliRealtimeExample.class); + builder.withGroup("internal") + .withDescription("Processes that Druid runs \"internally\", you should rarely use these directly") + .withDefaultCommand(Help.class) + .withCommands(CliPeon.class); + final Cli cli = builder.build(); try { cli.parse(args).run(); diff --git a/services/src/main/java/io/druid/cli/ServerRunnable.java b/services/src/main/java/io/druid/cli/ServerRunnable.java index 1ebdc23ae27..a7b6f6c6d4e 100644 --- a/services/src/main/java/io/druid/cli/ServerRunnable.java +++ b/services/src/main/java/io/druid/cli/ServerRunnable.java @@ -40,6 +40,5 @@ public abstract class ServerRunnable implements Runnable catch (Exception e) { throw Throwables.propagate(e); } - } } From 8097450d8c9ed80debbe0d7fa33215719347b88e Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 27 Aug 2013 14:29:03 -0500 Subject: [PATCH 43/92] Some things that didn't get committed with the merge for some reason!? --- .../com/metamx/druid/index/v1/IndexMerger.java | 1 - .../indexer/data/ByteBufferInputRowParser.java | 4 ++-- .../indexer/data/ProtoBufInputRowParser.java | 18 +++++++++--------- .../data/ProtoBufInputRowParserTest.java | 12 +++++------- .../druid/indexing/common/TaskStatus.java | 1 - .../coordinator/ForkingTaskRunner.java | 5 ++--- .../indexing/coordinator/RemoteTaskRunner.java | 7 ++++--- .../druid/indexing/coordinator/ZkWorker.java | 1 - .../worker/WorkerCuratorCoordinator.java | 1 - .../realtime/firehose/IrcFirehoseFactory.java | 1 - .../druid/index/ReferenceCountingSequence.java | 2 -- .../druid/master/CostBalancerStrategy.java | 2 -- 12 files changed, 22 insertions(+), 33 deletions(-) diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index e662bf0224a..bf454f5077e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -22,7 +22,6 @@ package com.metamx.druid.index.v1; import com.google.common.base.Function; import com.google.common.base.Objects; import com.google.common.base.Splitter; -import com.google.common.base.Stopwatch; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java index bc3721bf719..d0764e72a26 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java @@ -1,10 +1,10 @@ package com.metamx.druid.indexer.data; -import java.nio.ByteBuffer; - import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.nio.ByteBuffer; + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java index b5cb4e2e0c9..3130a86a1b4 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java @@ -1,14 +1,5 @@ package com.metamx.druid.indexer.data; -import static com.google.protobuf.DescriptorProtos.FileDescriptorSet; -import static com.google.protobuf.Descriptors.Descriptor; -import static com.google.protobuf.Descriptors.FileDescriptor; - -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Throwables; @@ -20,6 +11,15 @@ import com.google.protobuf.InvalidProtocolBufferException; import com.metamx.common.exception.FormattedException; import com.metamx.druid.input.InputRow; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import static com.google.protobuf.DescriptorProtos.FileDescriptorSet; +import static com.google.protobuf.Descriptors.Descriptor; +import static com.google.protobuf.Descriptors.FileDescriptor; + public class ProtoBufInputRowParser implements ByteBufferInputRowParser { diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java index 237f89edf14..fd402c56b41 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java @@ -1,18 +1,16 @@ package com.metamx.druid.indexer.data; -import static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE; -import static org.junit.Assert.assertEquals; +import com.metamx.druid.input.InputRow; +import org.joda.time.DateTime; +import org.junit.Test; import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; -import com.google.protobuf.ByteString; -import org.joda.time.DateTime; -import org.junit.Test; - -import com.metamx.druid.input.InputRow; +import static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE; +import static org.junit.Assert.assertEquals; public class ProtoBufInputRowParserTest { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java index bb7c6c4a10f..68952a9bc1f 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import com.metamx.druid.indexing.common.task.TaskResource; /** * Represents the status of a task from the perspective of the coordinator. The task may be ongoing diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index dfb9cbc94c4..41abb0a036c 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -62,7 +62,6 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.Executors; -import java.util.concurrent.ExecutorService; /** * Runs tasks in separate processes using the "internal peon" verb. @@ -146,12 +145,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer throw new IllegalStateException("Task has been shut down!"); } - if (taskInfo == null) { + if (taskWorkItem == null) { log.makeAlert("WTF?! TaskInfo disappeared!").addData("task", task.getId()).emit(); throw new ISE("TaskInfo disappeared for task[%s]!", task.getId()); } - if (taskInfo.processHolder != null) { + if (taskWorkItem.processHolder != null) { log.makeAlert("WTF?! TaskInfo already has a processHolder") .addData("task", task.getId()) .emit(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 3392f471849..66af95d5316 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -20,7 +20,6 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.api.client.util.Maps; import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -29,6 +28,7 @@ import com.google.common.base.Stopwatch; import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.InputSupplier; import com.google.common.primitives.Ints; @@ -272,7 +272,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer runningTasks.put( task.getId(), new RemoteTaskRunnerWorkItem( - task, SettableFuture.create(), + task, + SettableFuture.create(), worker ) ); @@ -680,7 +681,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer if (zkWorker != null) { try { List tasksToFail = Lists.newArrayList( - cf.getChildren().forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost())) + cf.getChildren().forPath(JOINER.join(zkPaths.getIndexerTaskPath(), worker.getHost())) ); log.info("[%s]: Found %d tasks assigned", worker.getHost(), tasksToFail.size()); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index c1e33525e23..3ee77c5f820 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -26,7 +26,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.worker.TaskAnnouncement; import com.metamx.druid.indexing.worker.Worker; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java index b62d383a4ae..ae0cde96201 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -30,7 +30,6 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/IrcFirehoseFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/IrcFirehoseFactory.java index 5e3efa7af4e..b67a246fb24 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/IrcFirehoseFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/firehose/IrcFirehoseFactory.java @@ -35,7 +35,6 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import org.joda.time.DateTime; -import java.io.File; import java.io.IOException; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java index 58aeb068f90..ca7066ececb 100644 --- a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java +++ b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java @@ -25,8 +25,6 @@ import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingSequenceBase; -import java.io.Closeable; - /** */ public class ReferenceCountingSequence extends YieldingSequenceBase diff --git a/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java b/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java index 0da949b9a7c..8c31261d5bb 100644 --- a/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java +++ b/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java @@ -19,14 +19,12 @@ package com.metamx.druid.master; -import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.common.Pair; import com.metamx.druid.client.DataSegment; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.Comparator; import java.util.List; public class CostBalancerStrategy implements BalancerStrategy From ee1e73cfa18144085e928b01b90435562a1ea33c Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 27 Aug 2013 14:36:01 -0500 Subject: [PATCH 44/92] 1) Make it compile again after the merge --- .../druid/indexing/common/tasklogs/S3TaskLogs.java | 2 +- .../indexing/coordinator/RemoteTaskRunnerTest.java | 1 + .../realtime/plumber/RealtimePlumberSchoolTest.java | 2 +- .../druid/query/search/SearchQueryRunnerFactory.java | 11 ++++++++++- .../druid/query/group/GroupByQueryRunnerTest.java | 12 ++++++++++-- .../druid/query/search/SearchQueryRunnerTest.java | 3 +-- 6 files changed, 24 insertions(+), 7 deletions(-) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java index 5928976b4fa..d5b6f2c7639 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java @@ -101,7 +101,7 @@ public class S3TaskLogs implements TaskLogs { final StorageObject object = new StorageObject(logFile); object.setKey(taskKey); - service.putObject(bucket, object); + service.putObject(config.getS3Bucket(), object); return null; } } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 7a26fbe629c..607f93b820b 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -49,6 +49,7 @@ import org.junit.Test; import java.io.File; import java.util.Arrays; import java.util.Set; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; diff --git a/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java b/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java index a93308c4fa6..22721866ce5 100644 --- a/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java @@ -113,7 +113,7 @@ public class RealtimePlumberSchoolTest realtimePlumberSchool.setVersioningPolicy(new IntervalStartVersioningPolicy()); realtimePlumberSchool.setDataSegmentPusher(dataSegmentPusher); realtimePlumberSchool.setServerView(serverView); - realtimePlumberSchool.setServiceEmitter(emitter); + realtimePlumberSchool.setEmitter(emitter); plumber = realtimePlumberSchool.findPlumber(schema, new FireDepartmentMetrics()); } diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java index 516742ba59b..2373f0da4d9 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java @@ -20,6 +20,7 @@ package com.metamx.druid.query.search; import com.google.common.collect.Iterators; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; @@ -42,7 +43,15 @@ import java.util.concurrent.ExecutorService; */ public class SearchQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> { - private static final SearchQueryQueryToolChest toolChest = new SearchQueryQueryToolChest(); + private final SearchQueryQueryToolChest toolChest; + + @Inject + public SearchQueryRunnerFactory( + SearchQueryQueryToolChest toolChest + ) + { + this.toolChest = toolChest; + } @Override public QueryRunner> createRunner(final Segment segment) diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index ca92b92ff8f..e5620304d9b 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -56,6 +56,7 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; import org.joda.time.Period; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -74,6 +75,13 @@ public class GroupByQueryRunnerTest { private final QueryRunner runner; private GroupByQueryRunnerFactory factory; + private Supplier configSupplier; + + @Before + public void setUp() throws Exception + { + configSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); + } @Parameterized.Parameters public static Collection constructorFeeder() throws IOException @@ -653,7 +661,7 @@ public class GroupByQueryRunnerTest createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -688,7 +696,7 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } diff --git a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java index 0802a99bb9a..ae846787f45 100644 --- a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java @@ -43,7 +43,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.TreeMap; /** */ @@ -54,7 +53,7 @@ public class SearchQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - new SearchQueryRunnerFactory() + new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig())) ); } From fc60158273b57912613909f26c800b374a475077 Mon Sep 17 00:00:00 2001 From: cheddar Date: Tue, 27 Aug 2013 14:37:12 -0500 Subject: [PATCH 45/92] 1) What is an .Rhistory file and why is it checked in? It's not! (anymore) --- server/src/main/resources/static/js/.Rhistory | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 server/src/main/resources/static/js/.Rhistory diff --git a/server/src/main/resources/static/js/.Rhistory b/server/src/main/resources/static/js/.Rhistory deleted file mode 100644 index e69de29bb2d..00000000000 From 9c30ced5ea22372b113854bcccda9a75d3fdad2b Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 28 Aug 2013 15:50:59 -0500 Subject: [PATCH 46/92] 1) Move various "api" classes to io.druid packages and make sure things compile and stuff --- .../main/java/com/metamx/druid/BaseQuery.java | 6 +- .../main/java/com/metamx/druid/Druids.java | 7 +- .../ResultGranularTimestampComparator.java | 1 + .../java/com/metamx/druid/SearchBinaryFn.java | 2 + .../com/metamx/druid/TimeseriesBinaryFn.java | 6 +- .../metamx/druid/client/BrokerServerView.java | 2 +- .../druid/client/CachingClusteredClient.java | 16 +- .../com/metamx/druid/client/DataSegment.java | 3 +- .../druid/client/DirectDruidClient.java | 10 +- .../druid/client/TimelineServerView.java | 2 +- .../client/indexing/ClientMergeQuery.java | 2 +- .../druid/guice/QueryToolChestModule.java | 4 +- .../metamx/druid/guice/QueryableModule.java | 2 +- .../druid/http/ClientQuerySegmentWalker.java | 8 +- .../http/DirectClientQuerySegmentWalker.java | 6 +- .../com/metamx/druid/http/QueryServlet.java | 2 +- .../com/metamx/druid/http/RequestLogLine.java | 2 +- .../druid/http/log/EmittingRequestLogger.java | 2 +- .../druid/initialization/DruidModule.java | 10 -- .../druid/initialization/Initialization.java | 1 + .../druid/query/BySegmentQueryRunner.java | 3 +- .../query/BySegmentSkippingQueryRunner.java | 3 +- .../query/ChainedExecutionQueryRunner.java | 3 +- .../metamx/druid/query/ConcatQueryRunner.java | 3 +- .../query/FinalizeResultsQueryRunner.java | 7 +- .../query/IntervalChunkingQueryRunner.java | 7 +- .../query/MapQueryToolChestWarehouse.java | 3 +- .../query/MetricsEmittingQueryRunner.java | 3 +- .../metamx/druid/query/NoopQueryRunner.java | 3 +- .../java/com/metamx/druid/query/Queries.java | 2 +- .../metamx/druid/query/QueryCacheHelper.java | 2 +- .../druid/query/QueryToolChestWarehouse.java | 3 +- .../ReflectionQueryToolChestWarehouse.java | 3 +- .../druid/query/ResultMergeQueryRunner.java | 3 +- .../druid/query/group/GroupByQuery.java | 10 +- .../group/GroupByQueryQueryToolChest.java | 12 +- .../query/group/orderby/DefaultLimitSpec.java | 2 +- .../druid/query/group/orderby/LimitSpec.java | 2 +- .../query/group/orderby/NoopLimitSpec.java | 2 +- .../druid/query/metadata/ColumnAnalysis.java | 2 +- .../query/metadata/SegmentMetadataQuery.java | 4 +- .../SegmentMetadataQueryQueryToolChest.java | 10 +- .../druid/query/search/SearchQuery.java | 6 +- .../search/SearchQueryQueryToolChest.java | 10 +- .../query/segment/QuerySegmentSpecs.java | 3 + .../query/segment/QuerySegmentWalker.java | 5 +- .../segment/SpecificSegmentQueryRunner.java | 5 +- .../query/segment/SpecificSegmentSpec.java | 6 +- .../query/timeboundary/TimeBoundaryQuery.java | 7 +- .../TimeBoundaryQueryQueryToolChest.java | 12 +- .../query/timeseries/TimeseriesQuery.java | 8 +- .../TimeseriesQueryQueryToolChest.java | 15 +- .../java/com/metamx/druid/sql/SQLRunner.java | 4 +- .../druid/query/CacheStrategy.java | 3 +- .../druid => io/druid/query}/Query.java | 6 +- .../druid/query/QueryRunner.java | 3 +- .../druid/query/QueryToolChest.java | 7 +- .../druid/query}/SegmentDescriptor.java | 2 +- .../aggregation}/MetricManipulationFn.java | 4 +- .../druid/query/spec}/LegacySegmentSpec.java | 3 +- .../spec}/MultipleIntervalSegmentSpec.java | 8 +- .../spec}/MultipleSpecificSegmentSpec.java | 9 +- .../druid/query/spec}/QuerySegmentSpec.java | 8 +- ...ResultGranularTimestampComparatorTest.java | 1 + .../com/metamx/druid/SearchBinaryFnTest.java | 1 + .../metamx/druid/TimeseriesBinaryFnTest.java | 3 +- .../com/metamx/druid/query/QueriesTest.java | 2 +- .../druid/query/QuerySegmentSpecTest.java | 11 +- .../TimeBoundaryQueryQueryToolChestTest.java | 2 +- common/pom.xml | 7 + .../metamx/druid/TimelineObjectHolder.java | 1 + .../druid/aggregation/CountAggregator.java | 2 + .../aggregation/CountAggregatorFactory.java | 9 +- .../aggregation/CountBufferAggregator.java | 2 + .../aggregation/DoubleSumAggregator.java | 3 +- .../DoubleSumAggregatorFactory.java | 9 +- .../DoubleSumBufferAggregator.java | 3 +- .../aggregation/HistogramAggregator.java | 3 +- .../HistogramAggregatorFactory.java | 9 +- .../HistogramBufferAggregator.java | 3 +- .../aggregation/JavaScriptAggregator.java | 11 +- .../JavaScriptAggregatorFactory.java | 23 +-- .../JavaScriptBufferAggregator.java | 11 +- .../druid/aggregation/LongSumAggregator.java | 3 +- .../aggregation/LongSumAggregatorFactory.java | 9 +- .../aggregation/LongSumBufferAggregator.java | 3 +- .../druid/aggregation/MaxAggregator.java | 3 +- .../aggregation/MaxAggregatorFactory.java | 9 +- .../aggregation/MaxBufferAggregator.java | 3 +- .../druid/aggregation/MinAggregator.java | 3 +- .../aggregation/MinAggregatorFactory.java | 9 +- .../aggregation/MinBufferAggregator.java | 3 +- .../druid/aggregation/NoopAggregator.java | 2 + .../aggregation/NoopBufferAggregator.java | 2 + .../ToLowerCaseAggregatorFactory.java | 9 +- .../jackson/AggregatorFactoriesModule.java | 56 +++++++ .../druid/jackson/DefaultObjectMapper.java | 119 +-------------- .../DruidDefaultSerializersModule.java | 144 ++++++++++++++++++ .../com/metamx/druid/jackson/JodaStuff.java | 9 +- .../druid/jackson/QueryGranularityModule.java | 50 ++++++ .../druid/granularity}/AllGranularity.java | 2 +- .../granularity}/BaseQueryGranularity.java | 2 +- .../granularity}/DurationGranularity.java | 3 +- .../druid/granularity}/NoneGranularity.java | 2 +- .../druid/granularity}/PeriodGranularity.java | 2 +- .../druid/granularity}/QueryGranularity.java | 3 +- .../io/druid/initialization/DruidModule.java | 31 ++++ .../druid/query}/aggregation/Aggregator.java | 2 +- .../query}/aggregation/AggregatorFactory.java | 17 ++- .../query}/aggregation/BufferAggregator.java | 4 +- .../druid/segment}/ComplexMetricSelector.java | 2 +- .../druid/segment}/FloatMetricSelector.java | 2 +- .../druid/segment}/LogicalSegment.java | 2 +- .../druid/segment/MetricSelectorFactory.java} | 6 +- .../druid/segment/ObjectMetricSelector.java} | 5 +- .../metamx/druid/QueryGranularityTest.java | 4 +- .../aggregation/HistogramAggregatorTest.java | 2 + .../aggregation/JavaScriptAggregatorTest.java | 10 +- .../aggregation/MetricSelectorUtils.java | 9 +- .../aggregation/TestFloatMetricSelector.java | 2 +- .../examples/guice/RealtimeExampleModule.java | 2 +- .../druid/index/SimpleQueryableIndex.java | 5 +- .../druid/index/column/AbstractColumn.java | 9 ++ .../druid/index/column/ColumnBuilder.java | 8 + .../index/column/ColumnCapabilitiesImpl.java | 2 + .../druid/index/column/ColumnDescriptor.java | 2 + .../druid/index/column/ComplexColumnImpl.java | 5 +- .../druid/index/column/FloatColumn.java | 3 + .../index/column/IndexedComplexColumn.java | 3 +- .../column/IndexedFloatsGenericColumn.java | 8 +- .../column/IndexedLongsGenericColumn.java | 8 +- .../metamx/druid/index/column/LongColumn.java | 3 + .../druid/index/column/SimpleColumn.java | 8 + .../column/SimpleDictionaryEncodedColumn.java | 3 +- .../serde/BitmapIndexColumnPartSupplier.java | 2 +- .../serde/ComplexColumnPartSupplier.java | 2 +- .../DictionaryEncodedColumnPartSerde.java | 2 +- .../DictionaryEncodedColumnSupplier.java | 2 +- .../serde/FloatGenericColumnPartSerde.java | 2 +- .../serde/FloatGenericColumnSupplier.java | 2 +- .../serde/LongGenericColumnPartSerde.java | 2 +- .../serde/LongGenericColumnSupplier.java | 2 +- .../serde/SpatialIndexColumnPartSupplier.java | 3 +- .../v1/CompressedFloatsIndexedSupplier.java | 2 +- .../v1/CompressedLongsIndexedSupplier.java | 2 +- .../index/v1/InMemoryCompressedFloats.java | 2 +- .../index/v1/InMemoryCompressedLongs.java | 2 +- .../druid/index/v1/IncrementalIndex.java | 22 +-- .../index/v1/IncrementalIndexAdapter.java | 6 +- .../index/v1/IncrementalIndexSchema.java | 4 +- .../com/metamx/druid/index/v1/IndexIO.java | 6 +- .../metamx/druid/index/v1/IndexMerger.java | 8 +- .../druid/index/v1/IndexableAdapter.java | 4 +- .../metamx/druid/index/v1/MMappedIndex.java | 4 +- .../druid/index/v1/MMappedIndexAdapter.java | 8 +- .../metamx/druid/index/v1/MetricHolder.java | 6 +- .../v1/QueryableIndexIndexableAdapter.java | 18 +-- .../druid/kv/ArrayBasedIndexedInts.java | 2 + .../com/metamx/druid/kv/ArrayIndexed.java | 2 + .../kv/ConciseCompressedIndexedInts.java | 1 + .../com/metamx/druid/kv/EmptyIndexedInts.java | 1 + .../com/metamx/druid/kv/GenericIndexed.java | 1 + .../metamx/druid/kv/IndexedIntsIterator.java | 2 + .../com/metamx/druid/kv/IndexedIterable.java | 2 + .../java/com/metamx/druid/kv/IndexedList.java | 2 + .../java/com/metamx/druid/kv/Indexedids.java | 2 + .../metamx/druid/kv/IntBufferIndexedInts.java | 1 + .../java/com/metamx/druid/kv/ListIndexed.java | 2 + .../metamx/druid/kv/SingleIndexedInts.java | 1 + .../com/metamx/druid/kv/VSizeIndexed.java | 1 + .../com/metamx/druid/kv/VSizeIndexedInts.java | 1 + .../druid/kv => io/druid/data}/Indexed.java | 2 +- .../kv => io/druid/data}/IndexedFloats.java | 2 +- .../kv => io/druid/data}/IndexedInts.java | 4 +- .../kv => io/druid/data}/IndexedLongs.java | 2 +- .../druid/segment}/ColumnSelector.java | 4 +- .../druid/segment}/QueryableIndex.java | 5 +- .../druid/segment}/column/BitmapIndex.java | 2 +- .../druid/segment}/column/Column.java | 2 +- .../segment}/column/ColumnCapabilities.java | 2 +- .../druid/segment}/column/ComplexColumn.java | 2 +- .../column/DictionaryEncodedColumn.java | 4 +- .../druid/segment}/column/GenericColumn.java | 8 +- .../segment}/column/RunLengthColumn.java | 2 +- .../druid/segment}/column/SpatialIndex.java | 2 +- .../druid/segment}/column/ValueType.java | 2 +- .../CompressedFloatsIndexedSupplierTest.java | 2 +- ...ompressedFloatsSupplierSerializerTest.java | 2 +- .../CompressedLongsIndexedSupplierTest.java | 3 +- ...CompressedLongsSupplierSerializerTest.java | 2 +- .../v1/InMemoryCompressedFloatsTest.java | 2 +- .../index/v1/InMemoryCompressedLongsTest.java | 2 +- .../druid/index/v1/IncrementalIndexTest.java | 4 +- .../metamx/druid/kv/GenericIndexedTest.java | 1 + .../com/metamx/druid/kv/IndexedIntsTest.java | 1 + .../druid/indexer/DeterminePartitionsJob.java | 4 +- .../indexer/HadoopDruidIndexerConfig.java | 2 +- .../druid/indexer/IndexGeneratorJob.java | 4 +- .../druid/indexer/rollup/DataRollupSpec.java | 4 +- .../common/index/YeOldePlumberSchool.java | 9 +- .../indexing/common/task/AbstractTask.java | 4 +- .../indexing/common/task/DeleteTask.java | 4 +- .../druid/indexing/common/task/IndexTask.java | 5 +- .../druid/indexing/common/task/MergeTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 8 +- .../druid/indexing/common/task/Task.java | 4 +- .../coordinator/ThreadPoolTaskRunner.java | 6 +- .../scaling/ResourceManagementScheduler.java | 2 +- .../com/metamx/druid/indexing/TestTask.java | 2 +- .../indexing/common/task/TaskSerdeTest.java | 4 +- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../coordinator/TaskLifecycleTest.java | 4 +- .../coordinator/TestRealtimeTask.java | 4 +- .../SimpleResourceManagementStrategyTest.java | 2 +- .../indexing/worker/TaskAnnouncementTest.java | 4 +- pom.xml | 18 ++- .../metamx/druid/realtime/FireHydrant.java | 2 +- .../druid/realtime/RealtimeManager.java | 10 +- .../com/metamx/druid/realtime/Schema.java | 4 +- .../druid/realtime/plumber/Plumber.java | 4 +- .../plumber/RealtimePlumberSchool.java | 14 +- .../metamx/druid/realtime/plumber/Sink.java | 2 +- .../druid/realtime/RealtimeManagerTest.java | 8 +- .../plumber/RealtimePlumberSchoolTest.java | 8 +- .../druid/realtime/plumber/SinkTest.java | 4 +- .../com/metamx/druid/BaseStorageAdapter.java | 5 +- .../metamx/druid/TimeseriesResultBuilder.java | 2 +- .../druid/coordination/ServerManager.java | 14 +- .../druid/guice/QueryRunnerFactoryModule.java | 13 +- .../druid/index/IncrementalIndexSegment.java | 4 +- .../druid/index/QueryableIndexSegment.java | 4 +- .../druid/index/ReferenceCountingSegment.java | 4 +- .../metamx/druid/index/brita/AndFilter.java | 4 + .../index/brita/BooleanValueMatcher.java | 2 + .../index/brita/DimensionPredicateFilter.java | 6 +- .../druid/index/brita/ExtractionFilter.java | 6 +- .../com/metamx/druid/index/brita/Filters.java | 1 + .../druid/index/brita/JavaScriptFilter.java | 4 + .../metamx/druid/index/brita/NotFilter.java | 4 + .../metamx/druid/index/brita/OrFilter.java | 4 + .../druid/index/brita/SelectorFilter.java | 4 + .../druid/index/brita/SpatialFilter.java | 7 +- .../v1/IncrementalIndexStorageAdapter.java | 32 ++-- .../druid/index/v1/IndexGranularity.java | 2 +- .../druid/index/v1/IndexedIntsOffset.java | 2 +- .../v1/QueryableIndexStorageAdapter.java | 62 ++++---- .../v1/RowboatFilteringIndexAdapter.java | 4 +- .../com/metamx/druid/index/v1/Searchable.java | 2 +- .../v1/SegmentIdAttachedStorageAdapter.java | 10 +- .../loading/DelegatingSegmentLoader.java | 2 +- .../loading/MMappedQueryableIndexFactory.java | 2 +- .../druid/loading/OmniSegmentLoader.java | 4 +- .../druid/loading/QueryableIndexFactory.java | 2 +- .../metamx/druid/loading/SegmentLoader.java | 2 +- .../druid/loading/SingleSegmentLoader.java | 4 +- ...DefaultQueryRunnerFactoryConglomerate.java | 3 +- .../query/QueryRunnerFactoryConglomerate.java | 3 +- .../metamx/druid/query/QueryRunnerHelper.java | 13 +- .../ReferenceCountingSegmentQueryRunner.java | 4 +- .../druid/query/group/GroupByQueryEngine.java | 10 +- .../group/GroupByQueryRunnerFactory.java | 12 +- .../druid/query/metadata/SegmentAnalyzer.java | 12 +- .../SegmentMetadataQueryRunnerFactory.java | 12 +- .../search/SearchQueryRunnerFactory.java | 12 +- .../TimeBoundaryQueryRunnerFactory.java | 12 +- .../timeseries/TimeseriesQueryEngine.java | 8 +- .../TimeseriesQueryRunnerFactory.java | 12 +- .../java/io/druid/initialization/Binders.java | 38 +++++ .../druid/query/QueryRunnerFactory.java | 5 +- .../aggregation}/ValueMatcherFactory.java | 3 +- .../query/filter}/BitmapIndexSelector.java | 4 +- .../druid/query/filter}/Filter.java | 3 +- .../druid/query/filter}/ValueMatcher.java | 2 +- .../druid/segment}/Capabilities.java | 2 +- .../druid/segment}/Cursor.java | 5 +- .../druid/segment}/CursorFactory.java | 6 +- .../druid/segment}/DimensionSelector.java | 4 +- .../segment}/DimensionSelectorFactory.java | 2 +- .../index => io/druid/segment}/Segment.java | 3 +- .../druid/segment}/StorageAdapter.java | 3 +- .../druid/coordination/ServerManagerTest.java | 18 +-- .../index/ReferenceCountingSegmentTest.java | 4 +- .../index/brita/SpatialFilterBonusTest.java | 10 +- .../druid/index/brita/SpatialFilterTest.java | 10 +- .../metamx/druid/index/v1/EmptyIndexTest.java | 6 +- .../druid/index/v1/IndexMergerTest.java | 6 +- .../com/metamx/druid/index/v1/TestIndex.java | 11 +- .../druid/loading/CacheTestSegmentLoader.java | 6 +- .../NoopQueryRunnerFactoryConglomerate.java | 3 +- .../druid/query/QueryRunnerTestHelper.java | 16 +- .../query/group/GroupByQueryRunnerTest.java | 12 +- .../GroupByTimeseriesQueryRunnerTest.java | 4 +- .../query/metadata/SegmentAnalyzerTest.java | 8 +- .../query/search/SearchQueryRunnerTest.java | 2 +- .../TimeBoundaryQueryRunnerTest.java | 2 +- .../TimeseriesQueryRunnerBonusTest.java | 12 +- .../timeseries/TimeseriesQueryRunnerTest.java | 11 +- 297 files changed, 1176 insertions(+), 783 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/initialization/DruidModule.java rename client/src/main/java/{com/metamx => io}/druid/query/CacheStrategy.java (95%) rename client/src/main/java/{com/metamx/druid => io/druid/query}/Query.java (95%) rename client/src/main/java/{com/metamx => io}/druid/query/QueryRunner.java (93%) rename client/src/main/java/{com/metamx => io}/druid/query/QueryToolChest.java (95%) rename client/src/main/java/{com/metamx/druid/query/segment => io/druid/query}/SegmentDescriptor.java (98%) rename client/src/main/java/{com/metamx/druid/query => io/druid/query/aggregation}/MetricManipulationFn.java (91%) rename client/src/main/java/{com/metamx/druid/query/segment => io/druid/query/spec}/LegacySegmentSpec.java (98%) rename client/src/main/java/{com/metamx/druid/query/segment => io/druid/query/spec}/MultipleIntervalSegmentSpec.java (92%) rename client/src/main/java/{com/metamx/druid/query/segment => io/druid/query/spec}/MultipleSpecificSegmentSpec.java (92%) rename client/src/main/java/{com/metamx/druid/query/segment => io/druid/query/spec}/QuerySegmentSpec.java (90%) create mode 100644 common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/DruidDefaultSerializersModule.java create mode 100644 common/src/main/java/com/metamx/druid/jackson/QueryGranularityModule.java rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/AllGranularity.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/BaseQueryGranularity.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/DurationGranularity.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/NoneGranularity.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/PeriodGranularity.java (99%) rename common/src/main/java/{com/metamx/druid => io/druid/granularity}/QueryGranularity.java (99%) create mode 100644 common/src/main/java/io/druid/initialization/DruidModule.java rename common/src/main/java/{com/metamx/druid => io/druid/query}/aggregation/Aggregator.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/query}/aggregation/AggregatorFactory.java (86%) rename common/src/main/java/{com/metamx/druid => io/druid/query}/aggregation/BufferAggregator.java (96%) rename common/src/main/java/{com/metamx/druid/processing => io/druid/segment}/ComplexMetricSelector.java (96%) rename common/src/main/java/{com/metamx/druid/processing => io/druid/segment}/FloatMetricSelector.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/segment}/LogicalSegment.java (79%) rename common/src/main/java/{com/metamx/druid/processing/ColumnSelectorFactory.java => io/druid/segment/MetricSelectorFactory.java} (87%) rename common/src/main/java/{com/metamx/druid/processing/ObjectColumnSelector.java => io/druid/segment/ObjectMetricSelector.java} (91%) rename indexing-common/src/main/java/{com/metamx/druid/kv => io/druid/data}/Indexed.java (97%) rename indexing-common/src/main/java/{com/metamx/druid/kv => io/druid/data}/IndexedFloats.java (97%) rename indexing-common/src/main/java/{com/metamx/druid/kv => io/druid/data}/IndexedInts.java (91%) rename indexing-common/src/main/java/{com/metamx/druid/kv => io/druid/data}/IndexedLongs.java (97%) rename indexing-common/src/main/java/{com/metamx/druid/index/column => io/druid/segment}/ColumnSelector.java (93%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/QueryableIndex.java (91%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/BitmapIndex.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/Column.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/ColumnCapabilities.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/ComplexColumn.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/DictionaryEncodedColumn.java (93%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/GenericColumn.java (89%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/RunLengthColumn.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/SpatialIndex.java (96%) rename indexing-common/src/main/java/{com/metamx/druid/index => io/druid/segment}/column/ValueType.java (64%) create mode 100644 server/src/main/java/io/druid/initialization/Binders.java rename server/src/main/java/{com/metamx => io}/druid/query/QueryRunnerFactory.java (91%) rename server/src/main/java/{com/metamx/druid/index/brita => io/druid/query/aggregation}/ValueMatcherFactory.java (93%) rename server/src/main/java/{com/metamx/druid/index/brita => io/druid/query/filter}/BitmapIndexSelector.java (94%) rename server/src/main/java/{com/metamx/druid/index/brita => io/druid/query/filter}/Filter.java (92%) rename server/src/main/java/{com/metamx/druid/index/brita => io/druid/query/filter}/ValueMatcher.java (95%) rename server/src/main/java/{com/metamx/druid => io/druid/segment}/Capabilities.java (98%) rename server/src/main/java/{com/metamx/druid/index/v1/processing => io/druid/segment}/Cursor.java (85%) rename server/src/main/java/{com/metamx/druid/index/v1/processing => io/druid/segment}/CursorFactory.java (88%) rename server/src/main/java/{com/metamx/druid/index/v1/processing => io/druid/segment}/DimensionSelector.java (95%) rename server/src/main/java/{com/metamx/druid/index/v1/processing => io/druid/segment}/DimensionSelectorFactory.java (95%) rename server/src/main/java/{com/metamx/druid/index => io/druid/segment}/Segment.java (93%) rename server/src/main/java/{com/metamx/druid => io/druid/segment}/StorageAdapter.java (93%) diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java index 9f3ae8d30ed..e3e01b331f7 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; - +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/Druids.java b/client/src/main/java/com/metamx/druid/Druids.java index 03401211db7..dd02ad85d32 100644 --- a/client/src/main/java/com/metamx/druid/Druids.java +++ b/client/src/main/java/com/metamx/druid/Druids.java @@ -20,7 +20,6 @@ package com.metamx.druid; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; @@ -31,13 +30,15 @@ import com.metamx.druid.query.filter.SelectorDimFilter; import com.metamx.druid.query.search.InsensitiveContainsSearchQuerySpec; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.query.search.SearchQuerySpec; -import com.metamx.druid.query.segment.LegacySegmentSpec; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java b/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java index 972d98b3214..a66ae16bf6f 100644 --- a/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java +++ b/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java @@ -21,6 +21,7 @@ package com.metamx.druid; import com.google.common.primitives.Longs; import com.metamx.druid.result.Result; +import io.druid.granularity.QueryGranularity; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/SearchBinaryFn.java b/client/src/main/java/com/metamx/druid/SearchBinaryFn.java index b9ebe74b705..b459ead8cd3 100644 --- a/client/src/main/java/com/metamx/druid/SearchBinaryFn.java +++ b/client/src/main/java/com/metamx/druid/SearchBinaryFn.java @@ -26,6 +26,8 @@ import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchSortSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; +import io.druid.granularity.AllGranularity; +import io.druid.granularity.QueryGranularity; import java.util.TreeSet; diff --git a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java b/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java index 0824366f259..345a18a5f28 100644 --- a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java +++ b/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java @@ -20,12 +20,12 @@ package com.metamx.druid; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.AllGranularity; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.AllGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import java.util.LinkedHashMap; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java index 0f55560449b..1e4ed76f23b 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/client/src/main/java/com/metamx/druid/client/BrokerServerView.java @@ -30,9 +30,9 @@ import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.guice.annotations.Client; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.http.client.HttpClient; +import io.druid.query.QueryRunner; import java.util.Iterator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 3bbd8146ebe..6c985cf1ab6 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -38,24 +38,24 @@ import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.LazySequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.cache.Cache; import com.metamx.druid.client.selector.QueryableDruidServer; import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.druid.query.CacheStrategy; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; import com.metamx.emitter.EmittingLogger; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.spec.MultipleSpecificSegmentSpec; import org.joda.time.DateTime; import org.joda.time.Interval; 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 fec47128cf7..3d048e430e4 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/client/src/main/java/com/metamx/druid/client/DataSegment.java @@ -31,10 +31,9 @@ import com.google.common.collect.Iterables; import com.metamx.common.Granularity; import com.metamx.druid.jackson.CommaListJoinDeserializer; import com.metamx.druid.jackson.CommaListJoinSerializer; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; - +import io.druid.query.SegmentDescriptor; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java index 92a4e5b9426..5f82a8bf5be 100644 --- a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java +++ b/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java @@ -35,11 +35,6 @@ import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import com.metamx.druid.Query; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; @@ -47,6 +42,11 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; diff --git a/client/src/main/java/com/metamx/druid/client/TimelineServerView.java b/client/src/main/java/com/metamx/druid/client/TimelineServerView.java index 33747f18af1..adbf863a6b5 100644 --- a/client/src/main/java/com/metamx/druid/client/TimelineServerView.java +++ b/client/src/main/java/com/metamx/druid/client/TimelineServerView.java @@ -21,7 +21,7 @@ package com.metamx.druid.client; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.selector.ServerSelector; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java index e363618a107..493e9588f5a 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java @@ -21,8 +21,8 @@ package com.metamx.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; +import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java index 2f66d6ae8e6..ffdf639ea7a 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java @@ -5,8 +5,6 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQueryConfig; import com.metamx.druid.query.group.GroupByQueryQueryToolChest; @@ -19,6 +17,8 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.Query; +import io.druid.query.QueryToolChest; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java index b28f7bd4021..2c00162e2d0 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java @@ -8,8 +8,8 @@ import com.metamx.druid.http.log.EmittingRequestLoggerProvider; import com.metamx.druid.http.log.FileRequestLoggerProvider; import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.http.log.RequestLoggerProvider; -import com.metamx.druid.initialization.DruidModule; import com.metamx.druid.query.segment.QuerySegmentWalker; +import io.druid.initialization.DruidModule; import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java index e7c187e5545..86bd39e6522 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java @@ -21,17 +21,17 @@ package com.metamx.druid.http; import com.google.common.base.Function; import com.google.inject.Inject; -import com.metamx.druid.Query; import com.metamx.druid.client.CachingClusteredClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.MetricsEmittingQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java index e1cac8a90c3..228d03a3a9e 100644 --- a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java @@ -19,13 +19,13 @@ package com.metamx.druid.http; -import com.metamx.druid.Query; import com.metamx.druid.client.DirectDruidClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SegmentDescriptor; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 5c78bc3aad5..6e11d8c2825 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -29,7 +29,6 @@ import com.google.inject.Inject; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import com.metamx.druid.Query; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.http.log.RequestLogger; @@ -37,6 +36,7 @@ import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.Query; import org.eclipse.jetty.server.Request; import org.joda.time.DateTime; diff --git a/client/src/main/java/com/metamx/druid/http/RequestLogLine.java b/client/src/main/java/com/metamx/druid/http/RequestLogLine.java index 0dbecb03ec8..faf403690ac 100644 --- a/client/src/main/java/com/metamx/druid/http/RequestLogLine.java +++ b/client/src/main/java/com/metamx/druid/http/RequestLogLine.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; -import com.metamx.druid.Query; +import io.druid.query.Query; import org.joda.time.DateTime; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java index 08691a3b637..c551542374f 100644 --- a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java +++ b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java @@ -22,11 +22,11 @@ package com.metamx.druid.http.log; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.Query; import com.metamx.druid.http.RequestLogLine; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; +import io.druid.query.Query; import org.joda.time.DateTime; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidModule.java b/client/src/main/java/com/metamx/druid/initialization/DruidModule.java deleted file mode 100644 index 0677d7110cf..00000000000 --- a/client/src/main/java/com/metamx/druid/initialization/DruidModule.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.metamx.druid.initialization; - -import java.util.List; - -/** - */ -public interface DruidModule extends com.google.inject.Module -{ - public List getJacksonModules(); -} diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index ad81c4500e3..a310c9c8184 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -33,6 +33,7 @@ import com.metamx.druid.guice.DruidSecondaryModule; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.jackson.JacksonModule; +import io.druid.initialization.DruidModule; import javax.annotation.Nullable; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java b/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java index 4713fc8207a..43b2f7be7c4 100644 --- a/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java @@ -26,9 +26,10 @@ import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.Yielders; import com.metamx.common.guava.YieldingAccumulator; -import com.metamx.druid.Query; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.joda.time.DateTime; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java b/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java index 3bf8cc2a661..004481a4a90 100644 --- a/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java @@ -20,7 +20,8 @@ package com.metamx.druid.query; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java b/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java index 8acc43a7585..f29642e5cce 100644 --- a/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java @@ -30,7 +30,8 @@ import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import java.util.Arrays; import java.util.Iterator; diff --git a/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java b/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java index 0a19840384a..95bc6dceea0 100644 --- a/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java @@ -22,7 +22,8 @@ package com.metamx.druid.query; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java b/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java index e54dd9d172a..cd0e2a3108b 100644 --- a/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java @@ -24,10 +24,13 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.result.BySegmentResultValueClass; import com.metamx.druid.result.Result; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java b/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java index 9c0a32eac3f..bbb12ce7e01 100644 --- a/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java @@ -24,9 +24,10 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.PeriodGranularity; -import com.metamx.druid.Query; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; +import io.druid.granularity.PeriodGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java b/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java index 9b0700f6dc8..f6838425939 100644 --- a/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java +++ b/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java @@ -1,7 +1,8 @@ package com.metamx.druid.query; import com.google.inject.Inject; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryToolChest; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java b/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java index 9a7111526de..9b346099a53 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java @@ -24,9 +24,10 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; -import com.metamx.druid.Query; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java b/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java index 57b2f27e46c..a143a43641d 100644 --- a/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java @@ -21,7 +21,8 @@ package com.metamx.druid.query; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/client/src/main/java/com/metamx/druid/query/Queries.java index 0ebc18e64fd..8cb56b92942 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/client/src/main/java/com/metamx/druid/query/Queries.java @@ -23,8 +23,8 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; +import io.druid.query.aggregation.AggregatorFactory; import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java b/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java index dd293fbdf5c..16b3a0836c1 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java +++ b/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java @@ -20,7 +20,7 @@ package com.metamx.druid.query; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorFactory; import java.nio.ByteBuffer; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java b/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java index 0eec4114eee..a7a61b2aede 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java @@ -19,7 +19,8 @@ package com.metamx.druid.query; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryToolChest; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java b/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java index 89f193bbf52..75db6b8fb2f 100644 --- a/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java +++ b/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java @@ -19,7 +19,8 @@ package com.metamx.druid.query; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryToolChest; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java b/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java index 77d9eeda07d..5bcb3edc278 100644 --- a/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java @@ -22,8 +22,9 @@ package com.metamx.druid.query; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.Query; import com.metamx.druid.guava.CombiningSequence; +import io.druid.query.Query; +import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index 3c48c4cecbc..b705deac2bf 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -31,9 +31,6 @@ import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.BaseQuery; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.Queries; @@ -45,8 +42,11 @@ import com.metamx.druid.query.group.orderby.DefaultLimitSpec; import com.metamx.druid.query.group.orderby.LimitSpec; import com.metamx.druid.query.group.orderby.NoopLimitSpec; import com.metamx.druid.query.group.orderby.OrderByColumnSpec; -import com.metamx.druid.query.segment.LegacySegmentSpec; -import com.metamx.druid.query.segment.QuerySegmentSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index bf3c7c0eaee..7ff15dc9371 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -32,18 +32,18 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java index 66049f925f3..cbe5efa90b0 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java @@ -31,10 +31,10 @@ import com.google.common.primitives.Longs; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.query.aggregation.AggregatorFactory; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java index cee3c76e55d..bcefbf82663 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java index 3f383430b12..fddb6ce1904 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java @@ -3,10 +3,10 @@ package com.metamx.druid.query.group.orderby; import com.google.common.base.Function; import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java index 3c98dd514d6..8c353964a30 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java @@ -21,7 +21,7 @@ package com.metamx.druid.query.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.index.column.ValueType; +import io.druid.segment.column.ValueType; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index 4be24fdaf95..cc375e57433 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -22,8 +22,8 @@ package com.metamx.druid.query.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.BaseQuery; -import com.metamx.druid.Query; -import com.metamx.druid.query.segment.QuerySegmentSpec; +import io.druid.query.Query; +import io.druid.query.spec.QuerySegmentSpec; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 9b210c5aa30..7d09f7e9d5c 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -31,15 +31,15 @@ import com.metamx.common.ISE; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.Query; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.CacheStrategy; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.MetricManipulationFn; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index 9a0d57cb82d..b81e5e54737 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -25,12 +25,12 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.druid.BaseQuery; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index f7d01b563ee..efcdfe7e465 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -35,21 +35,21 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.Query; import com.metamx.druid.ResultGranularTimestampComparator; import com.metamx.druid.SearchBinaryFn; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.IntervalChunkingQueryRunner; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.BySegmentSearchResultValue; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.MetricManipulationFn; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java index 9516db4c508..7c61493f056 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java @@ -19,6 +19,9 @@ package com.metamx.druid.query.segment; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java index efb7667a370..3bf3b37bea6 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java @@ -19,8 +19,9 @@ package com.metamx.druid.query.segment; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java index f5289185646..d074f45a948 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java @@ -24,8 +24,9 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.spec.QuerySegmentSpec; import java.io.IOException; import java.util.concurrent.Callable; diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java index c85ef86aa2a..55973aff8f4 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java @@ -19,8 +19,10 @@ package com.metamx.druid.query.segment; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index de846108e40..ecded61c0e9 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -25,12 +25,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.metamx.druid.BaseQuery; -import com.metamx.druid.Query; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; - +import io.druid.query.Query; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a5409b60887..e3e4924c418 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -29,17 +29,17 @@ import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.LogicalSegment; -import com.metamx.druid.Query; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.BySegmentSkippingQueryRunner; -import com.metamx.druid.query.CacheStrategy; -import com.metamx.druid.query.MetricManipulationFn; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.segment.LogicalSegment; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index 10fd403ef83..ef06fd4480b 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -24,15 +24,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import com.metamx.druid.BaseQuery; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.Queries; import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.QuerySegmentSpec; import java.util.List; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c731bf1b166..3eb48759864 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -28,25 +28,24 @@ import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.ResultGranularTimestampComparator; import com.metamx.druid.TimeseriesBinaryFn; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.IntervalChunkingQueryRunner; -import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.QueryCacheHelper; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.emitter.service.ServiceMetricEvent; - +import io.druid.granularity.QueryGranularity; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MetricManipulationFn; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java index e2111060b01..5667583e25d 100644 --- a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java +++ b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java @@ -9,8 +9,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; import com.metamx.druid.Druids; -import com.metamx.druid.Query; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.input.Row; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.query.dimension.DimensionSpec; @@ -19,6 +17,8 @@ import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.druid.sql.antlr4.DruidSQLLexer; import com.metamx.druid.sql.antlr4.DruidSQLParser; +import io.druid.query.Query; +import io.druid.query.aggregation.AggregatorFactory; import org.antlr.v4.runtime.ANTLRInputStream; import org.antlr.v4.runtime.CharStream; import org.antlr.v4.runtime.CommonTokenStream; diff --git a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java b/client/src/main/java/io/druid/query/CacheStrategy.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/CacheStrategy.java rename to client/src/main/java/io/druid/query/CacheStrategy.java index 1e73ada3f3b..c8d5217841f 100644 --- a/client/src/main/java/com/metamx/druid/query/CacheStrategy.java +++ b/client/src/main/java/io/druid/query/CacheStrategy.java @@ -17,12 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; /** */ diff --git a/client/src/main/java/com/metamx/druid/Query.java b/client/src/main/java/io/druid/query/Query.java similarity index 95% rename from client/src/main/java/com/metamx/druid/Query.java rename to client/src/main/java/io/druid/query/Query.java index 02b81bc6586..47f38d6e61f 100644 --- a/client/src/main/java/com/metamx/druid/Query.java +++ b/client/src/main/java/io/druid/query/Query.java @@ -17,20 +17,18 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery; - +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/QueryRunner.java b/client/src/main/java/io/druid/query/QueryRunner.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/QueryRunner.java rename to client/src/main/java/io/druid/query/QueryRunner.java index 5dd3cb9db15..4d20739412a 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryRunner.java +++ b/client/src/main/java/io/druid/query/QueryRunner.java @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/io/druid/query/QueryToolChest.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/QueryToolChest.java rename to client/src/main/java/io/druid/query/QueryToolChest.java index e2d227729d3..37fbd41f3ee 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/io/druid/query/QueryToolChest.java @@ -17,14 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; -import com.metamx.druid.LogicalSegment; -import com.metamx.druid.Query; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.segment.LogicalSegment; + import java.util.List; /** diff --git a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java b/client/src/main/java/io/druid/query/SegmentDescriptor.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java rename to client/src/main/java/io/druid/query/SegmentDescriptor.java index 1e2ac83484a..f66b7f66d6a 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SegmentDescriptor.java +++ b/client/src/main/java/io/druid/query/SegmentDescriptor.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/MetricManipulationFn.java b/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/MetricManipulationFn.java rename to client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java index 1ae354f8714..0da9c9f63cc 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricManipulationFn.java +++ b/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; - -import com.metamx.druid.aggregation.AggregatorFactory; +package io.druid.query.aggregation; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java b/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java rename to client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java index f4e62ff77c2..505760aa762 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/LegacySegmentSpec.java +++ b/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Function; import com.google.common.collect.Lists; import com.metamx.common.IAE; - import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java similarity index 92% rename from client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java rename to client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java index 51b77c7efc6..7c9c7703492 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleIntervalSegmentSpec.java +++ b/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.utils.JodaUtils; - +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.joda.time.Interval; import java.util.Collections; diff --git a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java similarity index 92% rename from client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java rename to client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java index 281f3222676..37e8b6c338d 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/MultipleSpecificSegmentSpec.java +++ b/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java @@ -17,16 +17,17 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.utils.JodaUtils; - +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java b/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java similarity index 90% rename from client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java rename to client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java index 0498deb16f1..126b1d99b58 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpec.java +++ b/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; - +import com.metamx.druid.query.segment.QuerySegmentWalker; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java b/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java index 13f99f601ac..ad43fd1a31a 100644 --- a/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java +++ b/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java @@ -20,6 +20,7 @@ package com.metamx.druid; import com.metamx.druid.result.Result; +import io.druid.granularity.QueryGranularity; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java b/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java index 4b6b4181d26..3f9aaa4cbaf 100644 --- a/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java +++ b/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java @@ -25,6 +25,7 @@ import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.StrlenSearchSortSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; +import io.druid.granularity.QueryGranularity; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java b/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java index f278153a643..ad72b669bcc 100644 --- a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java +++ b/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java @@ -21,7 +21,6 @@ package com.metamx.druid; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; @@ -30,6 +29,8 @@ import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/QueriesTest.java b/client/src/test/java/com/metamx/druid/query/QueriesTest.java index 02acaa71834..a9ee0e18d42 100644 --- a/client/src/test/java/com/metamx/druid/query/QueriesTest.java +++ b/client/src/test/java/com/metamx/druid/query/QueriesTest.java @@ -19,13 +19,13 @@ package com.metamx.druid.query; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.aggregation.post.PostAggregator; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java index 654ea2e6b79..7e6c8e06894 100644 --- a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java +++ b/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java @@ -23,12 +23,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.query.segment.LegacySegmentSpec; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; -import com.metamx.druid.query.segment.MultipleSpecificSegmentSpec; -import com.metamx.druid.query.segment.QuerySegmentSpec; -import com.metamx.druid.query.segment.SegmentDescriptor; - +import io.druid.query.SegmentDescriptor; +import io.druid.query.spec.LegacySegmentSpec; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.MultipleSpecificSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java b/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java index 03492e32716..2a80bab2309 100644 --- a/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java +++ b/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java @@ -19,7 +19,7 @@ package com.metamx.druid.query.timeboundary; -import com.metamx.druid.LogicalSegment; +import io.druid.segment.LogicalSegment; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; diff --git a/common/pom.xml b/common/pom.xml index 7a411d101b0..7138012d675 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -36,6 +36,13 @@ com.metamx java-util + commons-codec diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index d93e5ed8f6b..80e0e415bf1 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -20,6 +20,7 @@ package com.metamx.druid; import com.metamx.druid.partition.PartitionHolder; +import io.druid.segment.LogicalSegment; import org.joda.time.Interval; /** diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java index e806aa78e1c..6c307fecb64 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java @@ -19,6 +19,8 @@ package com.metamx.druid.aggregation; +import io.druid.query.aggregation.Aggregator; + import java.util.Comparator; /** diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java index 55900394860..dcc389084b6 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java @@ -24,7 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.util.Comparator; import java.util.List; @@ -47,13 +50,13 @@ public class CountAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new CountAggregator(name); } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new CountBufferAggregator(); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java index 0e6f06c786f..9cc0e41c42c 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java @@ -19,6 +19,8 @@ package com.metamx.druid.aggregation; +import io.druid.query.aggregation.BufferAggregator; + import java.nio.ByteBuffer; /** diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java index 278cd686a3b..318c1d8fbf2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java @@ -21,7 +21,8 @@ package com.metamx.druid.aggregation; import com.google.common.collect.Ordering; import com.google.common.primitives.Doubles; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatMetricSelector; import java.util.Comparator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java index ec89a79f39d..e130f9d90a4 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java @@ -23,7 +23,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.nio.ByteBuffer; import java.util.Arrays; @@ -53,7 +56,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new DoubleSumAggregator( name, @@ -62,7 +65,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new DoubleSumBufferAggregator(metricFactory.makeFloatMetricSelector(fieldName)); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java index 62f4e75f553..72436c51b6e 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatMetricSelector; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java index 23ac98f48a9..cf2f24e2c30 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java @@ -20,7 +20,8 @@ package com.metamx.druid.aggregation; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatMetricSelector; import java.util.Comparator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java index 6c65cd95f28..f4d4f00dafc 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java @@ -26,7 +26,10 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import org.apache.commons.codec.binary.Base64; import java.nio.ByteBuffer; @@ -64,7 +67,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new HistogramAggregator( name, @@ -74,7 +77,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new HistogramBufferAggregator( metricFactory.makeFloatMetricSelector(fieldName), diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java index 01196bd6298..e312f187593 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java @@ -21,7 +21,8 @@ package com.metamx.druid.aggregation; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatMetricSelector; import java.nio.ByteBuffer; import java.util.Arrays; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java index 6ce103e2af4..fb6de96192a 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java @@ -20,7 +20,8 @@ package com.metamx.druid.aggregation; import com.google.common.collect.Lists; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.ObjectMetricSelector; import java.util.List; @@ -28,7 +29,7 @@ public class JavaScriptAggregator implements Aggregator { static interface ScriptAggregator { - public double aggregate(double current, ObjectColumnSelector[] selectorList); + public double aggregate(double current, ObjectMetricSelector[] selectorList); public double combine(double a, double b); @@ -38,15 +39,15 @@ public class JavaScriptAggregator implements Aggregator } private final String name; - private final ObjectColumnSelector[] selectorList; + private final ObjectMetricSelector[] selectorList; private final ScriptAggregator script; private volatile double current; - public JavaScriptAggregator(String name, List selectorList, ScriptAggregator script) + public JavaScriptAggregator(String name, List selectorList, ScriptAggregator script) { this.name = name; - this.selectorList = Lists.newArrayList(selectorList).toArray(new ObjectColumnSelector[]{}); + this.selectorList = Lists.newArrayList(selectorList).toArray(new ObjectMetricSelector[]{}); this.script = script; this.current = script.reset(); diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java index a8375f294fa..ae2375069b5 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java @@ -26,8 +26,11 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.primitives.Doubles; -import com.metamx.druid.processing.ColumnSelectorFactory; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; +import io.druid.segment.ObjectMetricSelector; import org.mozilla.javascript.Context; import org.mozilla.javascript.ContextAction; import org.mozilla.javascript.ContextFactory; @@ -80,16 +83,16 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(final ColumnSelectorFactory columnFactory) + public Aggregator factorize(final MetricSelectorFactory columnFactory) { return new JavaScriptAggregator( name, Lists.transform( fieldNames, - new com.google.common.base.Function() + new com.google.common.base.Function() { @Override - public ObjectColumnSelector apply(@Nullable String s) + public ObjectMetricSelector apply(@Nullable String s) { return columnFactory.makeObjectColumnSelector(s); } @@ -100,17 +103,17 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory } @Override - public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory) + public BufferAggregator factorizeBuffered(final MetricSelectorFactory metricSelectorFactory) { return new JavaScriptBufferAggregator( Lists.transform( fieldNames, - new com.google.common.base.Function() + new com.google.common.base.Function() { @Override - public ObjectColumnSelector apply(@Nullable String s) + public ObjectMetricSelector apply(@Nullable String s) { - return columnSelectorFactory.makeObjectColumnSelector(s); + return metricSelectorFactory.makeObjectColumnSelector(s); } } ), @@ -254,7 +257,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory return new JavaScriptAggregator.ScriptAggregator() { @Override - public double aggregate(final double current, final ObjectColumnSelector[] selectorList) + public double aggregate(final double current, final ObjectMetricSelector[] selectorList) { Context cx = Context.getCurrentContext(); if (cx == null) { diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java index 90cb171d972..cf55c32ce07 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java @@ -20,24 +20,23 @@ package com.metamx.druid.aggregation; import com.google.common.collect.Lists; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ObjectMetricSelector; import java.nio.ByteBuffer; import java.util.List; public class JavaScriptBufferAggregator implements BufferAggregator { - private final ObjectColumnSelector[] selectorList; + private final ObjectMetricSelector[] selectorList; private final JavaScriptAggregator.ScriptAggregator script; public JavaScriptBufferAggregator( - List selectorList, + List selectorList, JavaScriptAggregator.ScriptAggregator script ) { - this.selectorList = Lists.newArrayList(selectorList).toArray(new ObjectColumnSelector[]{}); + this.selectorList = Lists.newArrayList(selectorList).toArray(new ObjectMetricSelector[]{}); this.script = script; } diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java index 9c7aadfa4f1..3defff46019 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java @@ -20,7 +20,8 @@ package com.metamx.druid.aggregation; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatMetricSelector; import java.util.Comparator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java index 07e04254f76..7ff27a47fdd 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java @@ -23,7 +23,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.nio.ByteBuffer; import java.util.Arrays; @@ -53,7 +56,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new LongSumAggregator( name, @@ -62,7 +65,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new LongSumBufferAggregator(metricFactory.makeFloatMetricSelector(fieldName)); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java index 747d383fcb7..f0cbca7098b 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatMetricSelector; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java index 8761c001da4..337615730d8 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatMetricSelector; import java.util.Comparator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java index 45cd85257f7..e9e85ae45cc 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java @@ -23,7 +23,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.nio.ByteBuffer; import java.util.Arrays; @@ -53,13 +56,13 @@ public class MaxAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new MaxAggregator(name, metricFactory.makeFloatMetricSelector(fieldName)); } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new MaxBufferAggregator(metricFactory.makeFloatMetricSelector(fieldName)); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java index 4bfcd2f2bbf..b5a07cd288c 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatMetricSelector; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java index a42ba758ed0..586556915c8 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatMetricSelector; import java.util.Comparator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java index a6d19ebd8e5..ad31b46142e 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java @@ -23,7 +23,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.primitives.Doubles; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.nio.ByteBuffer; import java.util.Arrays; @@ -53,13 +56,13 @@ public class MinAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return new MinAggregator(name, metricFactory.makeFloatMetricSelector(fieldName)); } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return new MinBufferAggregator(metricFactory.makeFloatMetricSelector(fieldName)); } diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java index 7bd29477b97..7eab06f06c7 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java @@ -19,7 +19,8 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatMetricSelector; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java index 5f7ddb2e78b..f5907f100eb 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java @@ -19,6 +19,8 @@ package com.metamx.druid.aggregation; +import io.druid.query.aggregation.Aggregator; + /** */ public class NoopAggregator implements Aggregator diff --git a/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java index a486fbcad2e..51ceafe04c8 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java @@ -19,6 +19,8 @@ package com.metamx.druid.aggregation; +import io.druid.query.aggregation.BufferAggregator; + import java.nio.ByteBuffer; /** diff --git a/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java b/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java index fcd782a9f01..a48b33c4f77 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java +++ b/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java @@ -19,7 +19,10 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.ColumnSelectorFactory; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.MetricSelectorFactory; import java.util.Comparator; import java.util.List; @@ -36,13 +39,13 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory } @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(MetricSelectorFactory metricFactory) { return baseAggregatorFactory.factorize(metricFactory); } @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory) { return baseAggregatorFactory.factorizeBuffered(metricFactory); } diff --git a/common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java b/common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java new file mode 100644 index 00000000000..d566642f5ac --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.metamx.druid.aggregation.CountAggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.aggregation.HistogramAggregatorFactory; +import com.metamx.druid.aggregation.JavaScriptAggregatorFactory; +import com.metamx.druid.aggregation.LongSumAggregatorFactory; +import com.metamx.druid.aggregation.MaxAggregatorFactory; +import com.metamx.druid.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.AggregatorFactory; + +/** + */ +public class AggregatorFactoriesModule extends SimpleModule +{ + public AggregatorFactoriesModule() + { + super("AggregatorFactories"); + + setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); + } + + @JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") + @JsonSubTypes(value={ + @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), + @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), + @JsonSubTypes.Type(name="doubleSum", value=DoubleSumAggregatorFactory.class), + @JsonSubTypes.Type(name="max", value=MaxAggregatorFactory.class), + @JsonSubTypes.Type(name="min", value=MinAggregatorFactory.class), + @JsonSubTypes.Type(name="javascript", value=JavaScriptAggregatorFactory.class), + @JsonSubTypes.Type(name="histogram", value=HistogramAggregatorFactory.class) + }) + public static interface AggregatorFactoryMixin {} +} diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index c8e7ab0ab40..1c6e4b18636 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -20,31 +20,11 @@ package com.metamx.druid.jackson; import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.Version; -import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; import com.fasterxml.jackson.datatype.guava.GuavaModule; -import com.google.common.base.Throwables; -import com.metamx.common.Granularity; -import com.metamx.common.guava.Accumulator; -import com.metamx.common.guava.Sequence; - -import org.joda.time.DateTimeZone; - -import java.io.IOException; -import java.nio.ByteOrder; -import java.util.TimeZone; /** */ @@ -58,103 +38,10 @@ public class DefaultObjectMapper extends ObjectMapper public DefaultObjectMapper(JsonFactory factory) { super(factory); - SimpleModule serializerModule = new SimpleModule("Druid default serializers", new Version(1, 0, 0, null)); - JodaStuff.register(serializerModule); - serializerModule.addDeserializer( - Granularity.class, - new JsonDeserializer() - { - @Override - public Granularity deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException - { - return Granularity.valueOf(jp.getText().toUpperCase()); - } - } - ); - serializerModule.addDeserializer( - DateTimeZone.class, - new JsonDeserializer() - { - @Override - public DateTimeZone deserialize(JsonParser jp, DeserializationContext ctxt) - throws IOException - { - String tzId = jp.getText(); - try { - return DateTimeZone.forID(tzId); - } catch(IllegalArgumentException e) { - // also support Java timezone strings - return DateTimeZone.forTimeZone(TimeZone.getTimeZone(tzId)); - } - } - } - ); - serializerModule.addSerializer( - DateTimeZone.class, - new JsonSerializer() - { - @Override - public void serialize( - DateTimeZone dateTimeZone, - JsonGenerator jsonGenerator, - SerializerProvider serializerProvider - ) - throws IOException, JsonProcessingException - { - jsonGenerator.writeString(dateTimeZone.getID()); - } - } - ); - serializerModule.addSerializer( - Sequence.class, - new JsonSerializer() - { - @Override - public void serialize(Sequence value, final JsonGenerator jgen, SerializerProvider provider) - throws IOException, JsonProcessingException - { - jgen.writeStartArray(); - value.accumulate( - null, - new Accumulator() - { - @Override - public Object accumulate(Object o, Object o1) - { - try { - jgen.writeObject(o1); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - return o; - } - } - ); - jgen.writeEndArray(); - } - } - ); - serializerModule.addSerializer(ByteOrder.class, ToStringSerializer.instance); - serializerModule.addDeserializer( - ByteOrder.class, - new JsonDeserializer() - { - @Override - public ByteOrder deserialize( - JsonParser jp, DeserializationContext ctxt - ) throws IOException, JsonProcessingException - { - if (ByteOrder.BIG_ENDIAN.toString().equals(jp.getText())) { - return ByteOrder.BIG_ENDIAN; - } - return ByteOrder.LITTLE_ENDIAN; - } - } - ); - registerModule(serializerModule); + registerModule(new DruidDefaultSerializersModule()); registerModule(new GuavaModule()); + registerModule(new QueryGranularityModule()); + registerModule(new AggregatorFactoriesModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/common/src/main/java/com/metamx/druid/jackson/DruidDefaultSerializersModule.java b/common/src/main/java/com/metamx/druid/jackson/DruidDefaultSerializersModule.java new file mode 100644 index 00000000000..04add157247 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/DruidDefaultSerializersModule.java @@ -0,0 +1,144 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.google.common.base.Throwables; +import com.metamx.common.Granularity; +import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.Sequence; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.nio.ByteOrder; +import java.util.TimeZone; + +/** + */ +public class DruidDefaultSerializersModule extends SimpleModule +{ + public DruidDefaultSerializersModule() + { + super("Druid default serializers"); + + JodaStuff.register(this); + addDeserializer( + Granularity.class, + new JsonDeserializer() + { + @Override + public Granularity deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException + { + return Granularity.valueOf(jp.getText().toUpperCase()); + } + } + ); + addDeserializer( + DateTimeZone.class, + new JsonDeserializer() + { + @Override + public DateTimeZone deserialize(JsonParser jp, DeserializationContext ctxt) + throws IOException + { + String tzId = jp.getText(); + try { + return DateTimeZone.forID(tzId); + } catch(IllegalArgumentException e) { + // also support Java timezone strings + return DateTimeZone.forTimeZone(TimeZone.getTimeZone(tzId)); + } + } + } + ); + addSerializer( + DateTimeZone.class, + new JsonSerializer() + { + @Override + public void serialize( + DateTimeZone dateTimeZone, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider + ) + throws IOException, JsonProcessingException + { + jsonGenerator.writeString(dateTimeZone.getID()); + } + } + ); + addSerializer( + Sequence.class, + new JsonSerializer() + { + @Override + public void serialize(Sequence value, final JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException + { + jgen.writeStartArray(); + value.accumulate( + null, + new Accumulator() + { + @Override + public Object accumulate(Object o, Object o1) + { + try { + jgen.writeObject(o1); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + return o; + } + } + ); + jgen.writeEndArray(); + } + } + ); + addSerializer(ByteOrder.class, ToStringSerializer.instance); + addDeserializer( + ByteOrder.class, + new JsonDeserializer() + { + @Override + public ByteOrder deserialize( + JsonParser jp, DeserializationContext ctxt + ) throws IOException, JsonProcessingException + { + if (ByteOrder.BIG_ENDIAN.toString().equals(jp.getText())) { + return ByteOrder.BIG_ENDIAN; + } + return ByteOrder.LITTLE_ENDIAN; + } + } + ); + } +} diff --git a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java index 13a5243be1e..321daa9bb78 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java +++ b/common/src/main/java/com/metamx/druid/jackson/JodaStuff.java @@ -39,9 +39,9 @@ import java.io.IOException; /** */ -public class JodaStuff +class JodaStuff { - public static SimpleModule register(SimpleModule module) + static SimpleModule register(SimpleModule module) { module.addKeyDeserializer(DateTime.class, new DateTimeKeyDeserializer()); module.addDeserializer(DateTime.class, new DateTimeDeserializer()); @@ -58,7 +58,7 @@ public class JodaStuff /** */ - public static class IntervalDeserializer extends StdDeserializer + private static class IntervalDeserializer extends StdDeserializer { public IntervalDeserializer() { @@ -82,7 +82,7 @@ public class JodaStuff } } - public static class DateTimeDeserializer extends StdDeserializer + private static class DateTimeDeserializer extends StdDeserializer { public DateTimeDeserializer() { super(DateTime.class); @@ -109,5 +109,4 @@ public class JodaStuff throw ctxt.mappingException(getValueClass()); } } - } diff --git a/common/src/main/java/com/metamx/druid/jackson/QueryGranularityModule.java b/common/src/main/java/com/metamx/druid/jackson/QueryGranularityModule.java new file mode 100644 index 00000000000..d40ca57c497 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/jackson/QueryGranularityModule.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import io.druid.granularity.AllGranularity; +import io.druid.granularity.DurationGranularity; +import io.druid.granularity.NoneGranularity; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; + +/** + */ +public class QueryGranularityModule extends SimpleModule +{ + public QueryGranularityModule() + { + super("QueryGranularityModule"); + + setMixInAnnotation(QueryGranularity.class, QueryGranularityMixin.class); + registerSubtypes( + new NamedType(PeriodGranularity.class, "period"), + new NamedType(DurationGranularity.class, "duration"), + new NamedType(AllGranularity.class, "all"), + new NamedType(NoneGranularity.class, "none") + ); + } + + @JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) + public static interface QueryGranularityMixin {} +} diff --git a/common/src/main/java/com/metamx/druid/AllGranularity.java b/common/src/main/java/io/druid/granularity/AllGranularity.java similarity index 98% rename from common/src/main/java/com/metamx/druid/AllGranularity.java rename to common/src/main/java/io/druid/granularity/AllGranularity.java index a35b632f5c2..164cbe30f1a 100644 --- a/common/src/main/java/com/metamx/druid/AllGranularity.java +++ b/common/src/main/java/io/druid/granularity/AllGranularity.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; import com.google.common.collect.ImmutableList; diff --git a/common/src/main/java/com/metamx/druid/BaseQueryGranularity.java b/common/src/main/java/io/druid/granularity/BaseQueryGranularity.java similarity index 98% rename from common/src/main/java/com/metamx/druid/BaseQueryGranularity.java rename to common/src/main/java/io/druid/granularity/BaseQueryGranularity.java index bf9b8ce1f21..24b8e90e8cf 100644 --- a/common/src/main/java/com/metamx/druid/BaseQueryGranularity.java +++ b/common/src/main/java/io/druid/granularity/BaseQueryGranularity.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/common/src/main/java/com/metamx/druid/DurationGranularity.java b/common/src/main/java/io/druid/granularity/DurationGranularity.java similarity index 98% rename from common/src/main/java/com/metamx/druid/DurationGranularity.java rename to common/src/main/java/io/druid/granularity/DurationGranularity.java index 5849f834103..8af4eccb8f8 100644 --- a/common/src/main/java/com/metamx/druid/DurationGranularity.java +++ b/common/src/main/java/io/druid/granularity/DurationGranularity.java @@ -17,12 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Longs; - import org.joda.time.DateTime; import java.nio.ByteBuffer; diff --git a/common/src/main/java/com/metamx/druid/NoneGranularity.java b/common/src/main/java/io/druid/granularity/NoneGranularity.java similarity index 97% rename from common/src/main/java/com/metamx/druid/NoneGranularity.java rename to common/src/main/java/io/druid/granularity/NoneGranularity.java index d3a0afeb087..3185fba9ad4 100644 --- a/common/src/main/java/com/metamx/druid/NoneGranularity.java +++ b/common/src/main/java/io/druid/granularity/NoneGranularity.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; public final class NoneGranularity extends BaseQueryGranularity { diff --git a/common/src/main/java/com/metamx/druid/PeriodGranularity.java b/common/src/main/java/io/druid/granularity/PeriodGranularity.java similarity index 99% rename from common/src/main/java/com/metamx/druid/PeriodGranularity.java rename to common/src/main/java/io/druid/granularity/PeriodGranularity.java index 5c8e594013a..02cce4119c4 100644 --- a/common/src/main/java/com/metamx/druid/PeriodGranularity.java +++ b/common/src/main/java/io/druid/granularity/PeriodGranularity.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/QueryGranularity.java b/common/src/main/java/io/druid/granularity/QueryGranularity.java similarity index 99% rename from common/src/main/java/com/metamx/druid/QueryGranularity.java rename to common/src/main/java/io/druid/granularity/QueryGranularity.java index ee745d3dadb..635a19a0bbc 100644 --- a/common/src/main/java/com/metamx/druid/QueryGranularity.java +++ b/common/src/main/java/io/druid/granularity/QueryGranularity.java @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.granularity; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.IAE; - import org.joda.time.DateTime; import org.joda.time.ReadableDuration; diff --git a/common/src/main/java/io/druid/initialization/DruidModule.java b/common/src/main/java/io/druid/initialization/DruidModule.java new file mode 100644 index 00000000000..7891baf5cc6 --- /dev/null +++ b/common/src/main/java/io/druid/initialization/DruidModule.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.initialization; + +import com.fasterxml.jackson.databind.Module; + +import java.util.List; + +/** + */ +public interface DruidModule extends com.google.inject.Module +{ + public List getJacksonModules(); +} diff --git a/common/src/main/java/com/metamx/druid/aggregation/Aggregator.java b/common/src/main/java/io/druid/query/aggregation/Aggregator.java similarity index 98% rename from common/src/main/java/com/metamx/druid/aggregation/Aggregator.java rename to common/src/main/java/io/druid/query/aggregation/Aggregator.java index fd42549971e..08673b5a48c 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Aggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation; /** * An Aggregator is an object that can aggregate metrics. Its aggregation-related methods (namely, aggregate() and get()) diff --git a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java similarity index 86% rename from common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index 400217d0bc4..48c7f2b8753 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/AggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -17,11 +17,18 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.processing.ColumnSelectorFactory; +import com.metamx.druid.aggregation.CountAggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.aggregation.HistogramAggregatorFactory; +import com.metamx.druid.aggregation.JavaScriptAggregatorFactory; +import com.metamx.druid.aggregation.LongSumAggregatorFactory; +import com.metamx.druid.aggregation.MaxAggregatorFactory; +import com.metamx.druid.aggregation.MinAggregatorFactory; +import io.druid.segment.MetricSelectorFactory; import java.util.Comparator; import java.util.List; @@ -29,7 +36,7 @@ import java.util.List; /** * Processing related interface * - * An AggregatorFactory is an object that knows how to generate an Aggregator using a ColumnSelectorFactory. + * An AggregatorFactory is an object that knows how to generate an Aggregator using a MetricSelectorFactory. * * This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects * without making any assumptions about how they are pulling values out of the base data. That is, the data is @@ -48,8 +55,8 @@ import java.util.List; }) public interface AggregatorFactory { - public Aggregator factorize(ColumnSelectorFactory metricFactory); - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory); + public Aggregator factorize(MetricSelectorFactory metricFactory); + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory); public Comparator getComparator(); /** diff --git a/common/src/main/java/com/metamx/druid/aggregation/BufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/BufferAggregator.java similarity index 96% rename from common/src/main/java/com/metamx/druid/aggregation/BufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/BufferAggregator.java index 6a010f7a3db..0cd6dec6be0 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/BufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation; import java.nio.ByteBuffer; @@ -80,7 +80,7 @@ public interface BufferAggregator * Implementations must not change the position, limit or mark of the given buffer * * Implementations are only required to support this method if they are aggregations which - * have an {@link com.metamx.druid.aggregation.AggregatorFactory#getTypeName()} of "float". + * have an {@link AggregatorFactory#getTypeName()} of "float". * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. * * @param buf byte buffer storing the byte array representation of the aggregate diff --git a/common/src/main/java/com/metamx/druid/processing/ComplexMetricSelector.java b/common/src/main/java/io/druid/segment/ComplexMetricSelector.java similarity index 96% rename from common/src/main/java/com/metamx/druid/processing/ComplexMetricSelector.java rename to common/src/main/java/io/druid/segment/ComplexMetricSelector.java index db14990c837..b27e0395f8c 100644 --- a/common/src/main/java/com/metamx/druid/processing/ComplexMetricSelector.java +++ b/common/src/main/java/io/druid/segment/ComplexMetricSelector.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.processing; +package io.druid.segment; /** */ diff --git a/common/src/main/java/com/metamx/druid/processing/FloatMetricSelector.java b/common/src/main/java/io/druid/segment/FloatMetricSelector.java similarity index 97% rename from common/src/main/java/com/metamx/druid/processing/FloatMetricSelector.java rename to common/src/main/java/io/druid/segment/FloatMetricSelector.java index d9a79603f33..714b4efa39f 100644 --- a/common/src/main/java/com/metamx/druid/processing/FloatMetricSelector.java +++ b/common/src/main/java/io/druid/segment/FloatMetricSelector.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.processing; +package io.druid.segment; /** * An object that gets a metric value. Metric values are always floats and there is an assumption that the diff --git a/common/src/main/java/com/metamx/druid/LogicalSegment.java b/common/src/main/java/io/druid/segment/LogicalSegment.java similarity index 79% rename from common/src/main/java/com/metamx/druid/LogicalSegment.java rename to common/src/main/java/io/druid/segment/LogicalSegment.java index 5550fb44966..8add65414fa 100644 --- a/common/src/main/java/com/metamx/druid/LogicalSegment.java +++ b/common/src/main/java/io/druid/segment/LogicalSegment.java @@ -1,4 +1,4 @@ -package com.metamx.druid; +package io.druid.segment; import org.joda.time.Interval; diff --git a/common/src/main/java/com/metamx/druid/processing/ColumnSelectorFactory.java b/common/src/main/java/io/druid/segment/MetricSelectorFactory.java similarity index 87% rename from common/src/main/java/com/metamx/druid/processing/ColumnSelectorFactory.java rename to common/src/main/java/io/druid/segment/MetricSelectorFactory.java index 250ab22ab11..4851c3e0437 100644 --- a/common/src/main/java/com/metamx/druid/processing/ColumnSelectorFactory.java +++ b/common/src/main/java/io/druid/segment/MetricSelectorFactory.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.processing; +package io.druid.segment; /** * Factory class for MetricSelectors */ -public interface ColumnSelectorFactory +public interface MetricSelectorFactory { public FloatMetricSelector makeFloatMetricSelector(String metricName); public ComplexMetricSelector makeComplexMetricSelector(String metricName); - public ObjectColumnSelector makeObjectColumnSelector(String columnName); + public ObjectMetricSelector makeObjectColumnSelector(String columnName); } diff --git a/common/src/main/java/com/metamx/druid/processing/ObjectColumnSelector.java b/common/src/main/java/io/druid/segment/ObjectMetricSelector.java similarity index 91% rename from common/src/main/java/com/metamx/druid/processing/ObjectColumnSelector.java rename to common/src/main/java/io/druid/segment/ObjectMetricSelector.java index b91a3e0b4d1..6bd28607323 100644 --- a/common/src/main/java/com/metamx/druid/processing/ObjectColumnSelector.java +++ b/common/src/main/java/io/druid/segment/ObjectMetricSelector.java @@ -17,10 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.processing; +package io.druid.segment; -public interface ObjectColumnSelector +public interface ObjectMetricSelector { public Class classOfObject(); public T get(); } + diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index 84c1aed555e..ad54b418d01 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; - +import io.druid.granularity.DurationGranularity; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java b/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java index d72ca031efc..93079506b81 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java +++ b/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java @@ -20,6 +20,8 @@ package com.metamx.druid.aggregation; import com.google.common.collect.Lists; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.BufferAggregator; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java b/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java index 98e4576a731..bf287cce344 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java +++ b/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java @@ -21,7 +21,9 @@ package com.metamx.druid.aggregation; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ObjectMetricSelector; import org.junit.Assert; import org.junit.Test; @@ -77,7 +79,7 @@ public class JavaScriptAggregatorTest JavaScriptAggregator agg = new JavaScriptAggregator( "billy", - Arrays.asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)), + Arrays.asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)), JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"), script.get("fnReset"), script.get("fnCombine")) @@ -113,7 +115,7 @@ public class JavaScriptAggregatorTest Map script = sumLogATimesBPlusTen; JavaScriptBufferAggregator agg = new JavaScriptBufferAggregator( - Arrays.asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)), + Arrays.asList(MetricSelectorUtils.wrap(selector1), MetricSelectorUtils.wrap(selector2)), JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"), script.get("fnReset"), script.get("fnCombine")) @@ -164,7 +166,7 @@ public class JavaScriptAggregatorTest Map script = scriptDoubleSum; JavaScriptAggregator aggRhino = new JavaScriptAggregator( "billy", - Lists.asList(MetricSelectorUtils.wrap(selector), new ObjectColumnSelector[]{}), + Lists.asList(MetricSelectorUtils.wrap(selector), new ObjectMetricSelector[]{}), JavaScriptAggregatorFactory.compileScript(script.get("fnAggregate"), script.get("fnReset"), script.get("fnCombine")) diff --git a/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java b/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java index da49af820e2..3cb856fdd2d 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java +++ b/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java @@ -19,15 +19,14 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.segment.FloatMetricSelector; +import io.druid.segment.ObjectMetricSelector; public class MetricSelectorUtils { - public static ObjectColumnSelector wrap(final FloatMetricSelector selector) + public static ObjectMetricSelector wrap(final FloatMetricSelector selector) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() diff --git a/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java b/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java index 616b3c5ecca..759a33a9902 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java +++ b/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java @@ -19,7 +19,7 @@ package com.metamx.druid.aggregation; -import com.metamx.druid.processing.FloatMetricSelector; +import io.druid.segment.FloatMetricSelector; /** */ diff --git a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java index 45e06db05b1..d2ebf350968 100644 --- a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java +++ b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java @@ -16,7 +16,6 @@ import com.metamx.druid.guice.JsonConfigProvider; import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.guice.NoopSegmentPublisherProvider; import com.metamx.druid.guice.RealtimeManagerConfig; -import com.metamx.druid.initialization.DruidModule; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.RealtimeManager; @@ -25,6 +24,7 @@ import druid.examples.flights.FlightsFirehoseFactory; import druid.examples.rand.RandomFirehoseFactory; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; import druid.examples.web.WebFirehoseFactory; +import io.druid.initialization.DruidModule; import java.io.File; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java index f9898d3a607..98ef7afd409 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java @@ -20,8 +20,9 @@ package com.metamx.druid.index; import com.metamx.common.io.smoosh.SmooshedFileMapper; -import com.metamx.druid.index.column.Column; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.Column; import org.joda.time.Interval; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java index c44c59a5e47..6929bc8c428 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java @@ -19,6 +19,15 @@ package com.metamx.druid.index.column; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.RunLengthColumn; +import io.druid.segment.column.SpatialIndex; + /** */ public abstract class AbstractColumn implements Column diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java index 180099f108f..3dc3e865cbc 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java @@ -21,6 +21,14 @@ package com.metamx.druid.index.column; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.RunLengthColumn; +import io.druid.segment.column.SpatialIndex; +import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java index 1e1554c030d..fce82246ce2 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java @@ -20,6 +20,8 @@ package com.metamx.druid.index.column; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java index 72918aee9fa..8324a930488 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java @@ -25,6 +25,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; import com.metamx.druid.index.serde.ColumnPartSerde; +import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java index 46f665c57c6..caccdd20787 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java @@ -19,7 +19,10 @@ package com.metamx.druid.index.column; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java index 44ffd7e970a..fa2f759eea8 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java @@ -20,6 +20,9 @@ package com.metamx.druid.index.column; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java index bafb6977dd0..241ca019a48 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java @@ -19,7 +19,8 @@ package com.metamx.druid.index.column; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; +import io.druid.segment.column.ComplexColumn; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java index 5df228435c1..20e13d5b652 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java @@ -19,9 +19,11 @@ package com.metamx.druid.index.column; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; +import io.druid.data.IndexedLongs; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java index 211dab35a19..ee9767c98ad 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java @@ -19,9 +19,11 @@ package com.metamx.druid.index.column; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; +import io.druid.data.IndexedLongs; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java index 1ec297ea61c..6bbfb87cfe4 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java @@ -20,6 +20,9 @@ package com.metamx.druid.index.column; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 5fd9f1c381e..72a6dce582d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -21,6 +21,14 @@ package com.metamx.druid.index.column; import com.google.common.base.Supplier; import com.google.common.io.Closeables; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.RunLengthColumn; +import io.druid.segment.column.SpatialIndex; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java index fbc3877c5fe..71133d618c4 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java @@ -20,9 +20,10 @@ package com.metamx.druid.index.column; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; +import io.druid.data.IndexedInts; +import io.druid.segment.column.DictionaryEncodedColumn; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java index 831fc887eb8..f5ada9c15fb 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java @@ -20,8 +20,8 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.BitmapIndex; import com.metamx.druid.kv.GenericIndexed; +import io.druid.segment.column.BitmapIndex; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java index 4a4cfce4f77..092a3820a7b 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java @@ -20,9 +20,9 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.ComplexColumn; import com.metamx.druid.index.column.IndexedComplexColumn; import com.metamx.druid.kv.GenericIndexed; +import io.druid.segment.column.ComplexColumn; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java index 82fe3cd5cea..b8fa919a35f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java @@ -25,13 +25,13 @@ import com.google.common.primitives.Ints; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.kv.ByteBufferSerializer; import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.IndexedRTree; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; +import io.druid.segment.column.ValueType; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java index 40285cf550d..c1a3cc0f077 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java @@ -20,11 +20,11 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.DictionaryEncodedColumn; import com.metamx.druid.index.column.SimpleDictionaryEncodedColumn; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; +import io.druid.segment.column.DictionaryEncodedColumn; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java index 13f86775f30..b18a06c21d1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java @@ -22,8 +22,8 @@ package com.metamx.druid.index.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; +import io.druid.segment.column.ValueType; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java index c13a5c11942..3cfad0184f1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java @@ -20,9 +20,9 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.GenericColumn; import com.metamx.druid.index.column.IndexedFloatsGenericColumn; import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; +import io.druid.segment.column.GenericColumn; import java.nio.ByteOrder; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java index e37f08dff81..32e0703dacc 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java @@ -22,8 +22,8 @@ package com.metamx.druid.index.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; +import io.druid.segment.column.ValueType; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java index 0f080d74564..3c1401a18a6 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java @@ -20,9 +20,9 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.GenericColumn; import com.metamx.druid.index.column.IndexedLongsGenericColumn; import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; +import io.druid.segment.column.GenericColumn; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java index 246da53ab7b..ced6fcf65fa 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java @@ -20,8 +20,7 @@ package com.metamx.druid.index.serde; import com.google.common.base.Supplier; import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.druid.index.column.SpatialIndex; -import com.metamx.druid.kv.GenericIndexed; +import io.druid.segment.column.SpatialIndex; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java index c93d09d4725..5f561620222 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java @@ -28,7 +28,7 @@ import com.metamx.common.IAE; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedFloats; +import io.druid.data.IndexedFloats; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java index ee752e4a20f..6ec7fdc2ac1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java @@ -28,7 +28,7 @@ import com.metamx.common.IAE; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.IndexedLongs; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java index a46b990a2ba..4557c5bda6d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java @@ -26,7 +26,7 @@ import com.google.common.io.Closeables; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedFloats; +import io.druid.data.IndexedFloats; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java index 9a408b46a4c..811de5118df 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java @@ -26,7 +26,7 @@ import com.google.common.io.Closeables; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.IndexedLongs; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java index 4141ce9d3a1..b3b08ee8166 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java @@ -33,9 +33,6 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.Aggregator; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.index.v1.serde.ComplexMetricExtractor; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; @@ -43,10 +40,13 @@ import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; -import com.metamx.druid.processing.ColumnSelectorFactory; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.ComplexMetricSelector; +import io.druid.segment.FloatMetricSelector; +import io.druid.segment.MetricSelectorFactory; +import io.druid.segment.ObjectMetricSelector; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -194,7 +194,7 @@ public class IncrementalIndex implements Iterable for (int i = 0; i < metrics.length; ++i) { final AggregatorFactory agg = metrics[i]; aggs[i] = agg.factorize( - new ColumnSelectorFactory() + new MetricSelectorFactory() { @Override public FloatMetricSelector makeFloatMetricSelector(String metric) @@ -241,13 +241,13 @@ public class IncrementalIndex implements Iterable } @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) + public ObjectMetricSelector makeObjectColumnSelector(String column) { final String typeName = agg.getTypeName(); final String columnName = column.toLowerCase(); if (typeName.equals("float")) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -271,7 +271,7 @@ public class IncrementalIndex implements Iterable final ComplexMetricExtractor extractor = serde.getExtractor(); - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java index 6831f17fdda..9968484f386 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java @@ -23,12 +23,12 @@ import com.google.common.base.Function; import com.google.common.collect.Maps; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import com.metamx.druid.aggregation.Aggregator; import com.metamx.druid.kv.EmptyIndexedInts; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.ListIndexed; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; +import io.druid.query.aggregation.Aggregator; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.IntSet; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java index 5bb0b63103b..54d94ac74a2 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java @@ -3,8 +3,8 @@ package com.metamx.druid.index.v1; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import java.util.Collections; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 07008e0a42b..f95803831c8 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -39,12 +39,9 @@ import com.metamx.common.io.smoosh.Smoosh; import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.SimpleQueryableIndex; -import com.metamx.druid.index.column.Column; import com.metamx.druid.index.column.ColumnBuilder; import com.metamx.druid.index.column.ColumnDescriptor; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.serde.BitmapIndexColumnPartSupplier; import com.metamx.druid.index.serde.ComplexColumnPartSerde; import com.metamx.druid.index.serde.ComplexColumnPartSupplier; @@ -65,6 +62,9 @@ import com.metamx.druid.kv.IndexedRTree; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ValueType; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index bf454f5077e..b03fbbbbcaf 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -44,11 +44,9 @@ import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.io.smoosh.Smoosh; import com.metamx.common.logger.Logger; import com.metamx.druid.CombiningIterable; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.ToLowerCaseAggregatorFactory; import com.metamx.druid.guava.FileOutputSupplier; import com.metamx.druid.guava.GuavaUtils; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.kv.ByteBufferWriter; @@ -56,14 +54,16 @@ import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.GenericIndexedWriter; import com.metamx.druid.kv.IOPeon; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.IndexedRTree; import com.metamx.druid.kv.TmpFileIOPeon; import com.metamx.druid.kv.VSizeIndexedWriter; import com.metamx.druid.utils.JodaUtils; import com.metamx.druid.utils.SerializerUtils; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.apache.commons.io.FileUtils; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java index 5c65a969957..2b0f51b1325 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java @@ -19,8 +19,8 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; import org.joda.time.Interval; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java index d792d80de01..83a32dedfbe 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java @@ -28,11 +28,11 @@ import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.logger.Logger; import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.Indexed; import com.metamx.druid.kv.IndexedList; -import com.metamx.druid.kv.IndexedLongs; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; +import io.druid.data.Indexed; +import io.druid.data.IndexedLongs; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java index 8ea0dc21572..fffc26fee0f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java @@ -22,10 +22,10 @@ package com.metamx.druid.index.v1; import com.google.common.collect.Maps; import com.google.common.io.Closeables; import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedInts; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; +import io.druid.data.IndexedInts; +import io.druid.data.IndexedLongs; import org.joda.time.Interval; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java index dc7002af24b..5196af561ce 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java @@ -27,12 +27,12 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.GenericIndexedWriter; import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; +import com.metamx.druid.kv.GenericIndexedWriter; import com.metamx.druid.kv.ObjectStrategy; import com.metamx.druid.utils.SerializerUtils; +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; import java.io.IOException; import java.io.InputStream; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java index 4f5633543b7..2a346774315 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java @@ -25,20 +25,20 @@ import com.google.common.collect.Sets; import com.google.common.io.Closeables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.column.BitmapIndex; -import com.metamx.druid.index.column.Column; -import com.metamx.druid.index.column.ComplexColumn; -import com.metamx.druid.index.column.DictionaryEncodedColumn; -import com.metamx.druid.index.column.GenericColumn; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.kv.ArrayBasedIndexedInts; import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.EmptyIndexedInts; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.ListIndexed; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; import org.joda.time.Interval; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java index 9e38b4a84db..db3a1a7dd37 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.IndexedInts; + import java.util.Iterator; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java index 753db6cb5a3..d008e53a2a3 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.Indexed; + import java.util.Arrays; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java index 2fdaeda30c0..02b78854a5d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java @@ -20,6 +20,7 @@ package com.metamx.druid.kv; import com.google.common.collect.Ordering; +import io.druid.data.IndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.IntSet; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java index cd83f0b481d..dcdf68dbdd0 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java @@ -20,6 +20,7 @@ package com.metamx.druid.kv; import com.google.common.collect.ImmutableList; +import io.druid.data.IndexedInts; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java index 99e003e844c..52e087d741e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java @@ -24,6 +24,7 @@ import com.google.common.collect.Ordering; import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.metamx.common.IAE; +import io.druid.data.Indexed; import java.io.ByteArrayOutputStream; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java index 02d5c47f663..4b72d0dc7f2 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.IndexedInts; + import java.util.Iterator; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java index 6f5efd7fbb2..07b349f528c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.Indexed; + import java.util.Iterator; import java.util.NoSuchElementException; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java index c65556cc50d..544d64f7d68 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.Indexed; + import java.util.AbstractList; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java b/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java index 0b3f3a2c000..4a87a0eb42c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.IndexedInts; + /** */ public class Indexedids diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java index a6c7e42b049..d59b642b56d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java @@ -22,6 +22,7 @@ package com.metamx.druid.kv; import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.metamx.druid.IntList; +import io.druid.data.IndexedInts; import java.nio.ByteBuffer; import java.nio.IntBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java index df939fe88f8..c5b52f54b0d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java @@ -19,6 +19,8 @@ package com.metamx.druid.kv; +import io.druid.data.Indexed; + import java.util.Iterator; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java index cba9195e0f6..57bd0d957bb 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java @@ -1,6 +1,7 @@ package com.metamx.druid.kv; import com.google.common.collect.Iterators; +import io.druid.data.IndexedInts; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java index 9ef63f39ac0..8a9f3620549 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java @@ -22,6 +22,7 @@ package com.metamx.druid.kv; import com.google.common.primitives.Ints; import com.metamx.common.IAE; import com.metamx.common.ISE; +import io.druid.data.Indexed; import java.io.ByteArrayOutputStream; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java index 616065d163c..02d7e2a5d75 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java +++ b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java @@ -21,6 +21,7 @@ package com.metamx.druid.kv; import com.google.common.primitives.Ints; import com.metamx.common.IAE; +import io.druid.data.IndexedInts; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/Indexed.java b/indexing-common/src/main/java/io/druid/data/Indexed.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/kv/Indexed.java rename to indexing-common/src/main/java/io/druid/data/Indexed.java index 44ef9dc103a..43a4aad25af 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/Indexed.java +++ b/indexing-common/src/main/java/io/druid/data/Indexed.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.data; /** * An Integer-indexed random-access collection. diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java b/indexing-common/src/main/java/io/druid/data/IndexedFloats.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java rename to indexing-common/src/main/java/io/druid/data/IndexedFloats.java index 5bfffda48cf..74b337f0796 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java +++ b/indexing-common/src/main/java/io/druid/data/IndexedFloats.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.data; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedInts.java b/indexing-common/src/main/java/io/druid/data/IndexedInts.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedInts.java rename to indexing-common/src/main/java/io/druid/data/IndexedInts.java index a868f33838b..287eb30832d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedInts.java +++ b/indexing-common/src/main/java/io/druid/data/IndexedInts.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.data; /** * Get a int an index (array or list lookup abstraction without boxing). - * Typically wraps an {@link com.metamx.druid.kv.Indexed}. + * Typically wraps an {@link Indexed}. */ public interface IndexedInts extends Iterable { diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java b/indexing-common/src/main/java/io/druid/data/IndexedLongs.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java rename to indexing-common/src/main/java/io/druid/data/IndexedLongs.java index fbdebb63693..e777718060f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java +++ b/indexing-common/src/main/java/io/druid/data/IndexedLongs.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.data; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java b/indexing-common/src/main/java/io/druid/segment/ColumnSelector.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java rename to indexing-common/src/main/java/io/druid/segment/ColumnSelector.java index 0ed66850dcb..b23655364d2 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java +++ b/indexing-common/src/main/java/io/druid/segment/ColumnSelector.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment; + +import io.druid.segment.column.Column; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/indexing-common/src/main/java/io/druid/segment/QueryableIndex.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java rename to indexing-common/src/main/java/io/druid/segment/QueryableIndex.java index 963c62dc964..6d44382b047 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java +++ b/indexing-common/src/main/java/io/druid/segment/QueryableIndex.java @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; -import com.metamx.druid.index.column.ColumnSelector; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; import org.joda.time.Interval; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java b/indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java rename to indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java index ae9ae8a17ed..a1a39666362 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java +++ b/indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/Column.java b/indexing-common/src/main/java/io/druid/segment/column/Column.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/Column.java rename to indexing-common/src/main/java/io/druid/segment/column/Column.java index 8de06439af7..972dbb705f1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/Column.java +++ b/indexing-common/src/main/java/io/druid/segment/column/Column.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java b/indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java rename to indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java index 3fbd439ffca..53efd3fe7d7 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java +++ b/indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java b/indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java rename to indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java index f7cfb706e44..f68a2ebdf59 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java +++ b/indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java b/indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java rename to indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java index 62057cb8e10..ba8953057fe 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java +++ b/indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import com.metamx.druid.kv.IndexedInts; +import io.druid.data.IndexedInts; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java b/indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java similarity index 89% rename from indexing-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java rename to indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java index 530eb2fc232..63a53a7d15c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java +++ b/indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedFloats; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; +import io.druid.data.IndexedLongs; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java b/indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java rename to indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java index 5242e994008..83339aa9c05 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java +++ b/indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java b/indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java rename to indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java index 27502e13f35..b6607ff8162 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java +++ b/indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import com.metamx.collections.spatial.ImmutableRTree; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ValueType.java b/indexing-common/src/main/java/io/druid/segment/column/ValueType.java similarity index 64% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ValueType.java rename to indexing-common/src/main/java/io/druid/segment/column/ValueType.java index 245ff682bd7..8d4eae81e9c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ValueType.java +++ b/indexing-common/src/main/java/io/druid/segment/column/ValueType.java @@ -1,4 +1,4 @@ -package com.metamx.druid.index.column; +package io.druid.segment.column; /** */ diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java index 125aa7618f8..cf49cb1c61f 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java @@ -21,7 +21,7 @@ package com.metamx.druid.index.v1; import com.google.common.io.Closeables; import com.google.common.primitives.Floats; -import com.metamx.druid.kv.IndexedFloats; +import io.druid.data.IndexedFloats; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java index 8ddf982fc7c..f59a8958d35 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java @@ -22,7 +22,7 @@ package com.metamx.druid.index.v1; import com.google.common.io.OutputSupplier; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IndexedFloats; +import io.druid.data.IndexedFloats; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java index 0f06cff2b64..36e847f78bd 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java @@ -21,7 +21,7 @@ package com.metamx.druid.index.v1; import com.google.common.io.Closeables; import com.google.common.primitives.Longs; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.IndexedLongs; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -31,7 +31,6 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.FloatBuffer; import java.nio.LongBuffer; import java.nio.channels.Channels; import java.util.concurrent.CountDownLatch; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java index c387db96618..d2988459c4b 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java @@ -22,7 +22,7 @@ package com.metamx.druid.index.v1; import com.google.common.io.OutputSupplier; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.IndexedLongs; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java index bb52785b910..1b3d64bb384 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java @@ -19,7 +19,7 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.kv.IndexedFloats; +import io.druid.data.IndexedFloats; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java index 00c01ac52be..103253809f1 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java @@ -19,7 +19,7 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.kv.IndexedLongs; +import io.druid.data.IndexedLongs; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java index 17f089ab3ee..410c4c6b751 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java @@ -20,10 +20,10 @@ package com.metamx.druid.index.v1; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.input.Row; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java index fad903abc96..9b126feaf65 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java @@ -20,6 +20,7 @@ package com.metamx.druid.kv; import com.google.common.collect.Maps; +import io.druid.data.Indexed; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java index d7341cb5c28..a130ea15d3e 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java @@ -19,6 +19,7 @@ package com.metamx.druid.kv; +import io.druid.data.IndexedInts; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index c45b141ac0b..3ad9780669b 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -20,7 +20,6 @@ package com.metamx.druid.indexer; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -40,11 +39,11 @@ import com.metamx.common.ISE; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.logger.Logger; import com.metamx.druid.CombiningIterable; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.input.InputRow; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; +import io.druid.granularity.QueryGranularity; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -66,7 +65,6 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; - import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; import org.joda.time.Interval; diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 8a6b17f9c33..334210f0015 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -39,7 +39,6 @@ import com.metamx.common.MapUtils; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import com.metamx.druid.RegisteringNode; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.serde.Registererer; import com.metamx.druid.indexer.data.DataSpec; @@ -56,6 +55,7 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.utils.JodaUtils; +import io.druid.query.aggregation.AggregatorFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index b2052bad739..23cf4c05fec 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -32,9 +32,7 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; 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.index.QueryableIndex; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexSchema; import com.metamx.druid.index.v1.IndexIO; @@ -42,6 +40,8 @@ import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.input.InputRow; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java index e3c82bfbc03..6823f47b094 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java @@ -20,8 +20,8 @@ package com.metamx.druid.indexer.rollup; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java index a3c49057c85..62c6b0ca878 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java @@ -30,20 +30,19 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.logger.Logger; -import com.metamx.druid.Query; 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 com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.FireHydrant; +import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.PlumberSchool; -import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.plumber.Sink; - +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.segment.QueryableIndex; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java index 7a7fa791412..db1972b0f73 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java @@ -25,11 +25,11 @@ 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.Query; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.actions.SegmentListUsedAction; import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.joda.time.Interval; public abstract class AbstractTask implements Task diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java index 7c9f8d53801..ba8cbcff8b2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java @@ -26,8 +26,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexAdapter; @@ -39,6 +37,8 @@ import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.LockListAction; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java index 8c584040e86..3c2611fde7a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java @@ -25,18 +25,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.indexer.granularity.GranularitySpec; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.SpawnTasksAction; import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java index 95e79c37f25..5f0501fe767 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java @@ -26,11 +26,11 @@ 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 io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import javax.annotation.Nullable; import java.io.File; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index 808d5e3fad1..f2fc09f6601 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.io.Closeables; import com.metamx.common.exception.FormattedException; -import com.metamx.druid.Query; import com.metamx.druid.client.DataSegment; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.index.v1.IndexGranularity; @@ -40,10 +39,7 @@ import com.metamx.druid.indexing.common.actions.LockReleaseAction; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.input.InputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.FireDepartmentConfig; import com.metamx.druid.realtime.RealtimeMetricsMonitor; @@ -57,6 +53,10 @@ import com.metamx.druid.realtime.plumber.RejectionPolicyFactory; import com.metamx.druid.realtime.plumber.Sink; import com.metamx.druid.realtime.plumber.VersioningPolicy; import com.metamx.emitter.EmittingLogger; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index fdff1ed2305..af7b22f4890 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -22,11 +22,11 @@ package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Optional; -import com.metamx.druid.Query; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.actions.TaskActionClient; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.joda.time.Interval; /** diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index dc6601ba1a7..e41f0aabb50 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -30,17 +30,17 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.Query; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.NoopQueryRunner; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.emitter.EmittingLogger; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java index 0e359635202..cefcc702860 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java @@ -23,9 +23,9 @@ 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.PeriodGranularity; import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunner; +import io.druid.granularity.PeriodGranularity; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java index 588e8fc2c6d..8109c753bff 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java @@ -22,11 +22,11 @@ package com.metamx.druid.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.task.MergeTask; +import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java index 9e05c58e60e..c53cbb701f3 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java @@ -4,8 +4,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.metamx.common.Granularity; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.DataSegment; @@ -18,6 +16,8 @@ import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 607f93b820b..82df7c55bcf 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -11,7 +11,6 @@ import com.google.common.collect.Sets; import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.ISE; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; @@ -32,6 +31,7 @@ import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.query.aggregation.AggregatorFactory; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index 2ec5bd543f3..f2019323f28 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -31,8 +31,6 @@ import com.google.common.io.Files; import com.metamx.common.Granularity; import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexer.granularity.UniformGranularitySpec; @@ -64,6 +62,8 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import org.apache.commons.io.FileUtils; import org.easymock.EasyMock; import org.joda.time.DateTime; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java index 9c30cf690d0..cf9bcb233b0 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java @@ -22,14 +22,14 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.task.RealtimeIndexTask; import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; /** */ diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index b3389f3842b..7b7b769183d 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -22,7 +22,6 @@ package com.metamx.druid.indexing.coordinator.scaling; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.guava.DSuppliers; import com.metamx.druid.indexing.TestTask; @@ -37,6 +36,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/worker/TaskAnnouncementTest.java index 5202609f42d..9300585e245 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/worker/TaskAnnouncementTest.java @@ -1,8 +1,6 @@ package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.RealtimeIndexTask; @@ -11,6 +9,8 @@ import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.joda.time.Period; import org.junit.Test; diff --git a/pom.xml b/pom.xml index cf65d6b36cd..7af362cd5b6 100644 --- a/pom.xml +++ b/pom.xml @@ -38,7 +38,7 @@ UTF-8 - 0.24.0 + 0.25.0-SNAPSHOT 2.1.0-incubating @@ -56,6 +56,12 @@ + + io.druid + druid-api + 1.0.0-SNAPSHOT + + com.metamx @@ -378,6 +384,16 @@ protobuf-java 2.4.0a + + io.tesla.aether + tesla-aether + 0.0.2 + + + org.eclipse.aether + aether-api + 0.9.0.M2 + diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java b/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java index 366d3e51a28..12d309fb13e 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireHydrant.java @@ -20,8 +20,8 @@ package com.metamx.druid.realtime; import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; +import io.druid.segment.Segment; /** */ diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java index 2189f0604c8..89288d5ab83 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java @@ -27,20 +27,20 @@ import com.google.inject.Inject; import com.metamx.common.exception.FormattedException; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.Query; import com.metamx.druid.input.InputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.NoopQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.Sink; import com.metamx.emitter.EmittingLogger; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java index 37b9fa2ae79..c5c4664b2f6 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/Schema.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/Schema.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import java.util.Arrays; import java.util.List; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Plumber.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Plumber.java index 3487c655efb..e1a01699fb6 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Plumber.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Plumber.java @@ -19,8 +19,8 @@ package com.metamx.druid.realtime.plumber; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunner; +import io.druid.query.Query; +import io.druid.query.QueryRunner; public interface Plumber { diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index e425a70bec8..92d905a213d 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -36,7 +36,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; @@ -45,20 +44,14 @@ import com.metamx.druid.client.ServerView; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.guava.ThreadRenamingCallable; import com.metamx.druid.guava.ThreadRenamingRunnable; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.partition.SingleElementPartitionChunk; import com.metamx.druid.query.MetricsEmittingQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.druid.realtime.FireDepartmentMetrics; @@ -68,6 +61,13 @@ import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java index bc0bc194f99..a86bfc48edb 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java @@ -27,13 +27,13 @@ import com.google.common.collect.Lists; import com.metamx.common.IAE; 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.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexSchema; import com.metamx.druid.input.InputRow; import com.metamx.druid.realtime.FireHydrant; import com.metamx.druid.realtime.Schema; +import io.druid.query.aggregation.AggregatorFactory; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/realtime/src/test/java/com/metamx/druid/realtime/RealtimeManagerTest.java b/realtime/src/test/java/com/metamx/druid/realtime/RealtimeManagerTest.java index 1203a0e6ba4..d098b47b6ec 100644 --- a/realtime/src/test/java/com/metamx/druid/realtime/RealtimeManagerTest.java +++ b/realtime/src/test/java/com/metamx/druid/realtime/RealtimeManagerTest.java @@ -22,20 +22,20 @@ package com.metamx.druid.realtime; import com.google.common.base.Stopwatch; import com.google.common.collect.Lists; import com.metamx.common.ISE; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.guava.Runnables; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.InputRow; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.PlumberSchool; import com.metamx.druid.realtime.plumber.Sink; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java b/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java index 22721866ce5..b0444f283f5 100644 --- a/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/realtime/src/test/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchoolTest.java @@ -24,9 +24,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; import com.metamx.common.ISE; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.ServerView; @@ -35,12 +32,15 @@ import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.SegmentPublisher; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.apache.commons.lang.mutable.MutableBoolean; import org.easymock.EasyMock; diff --git a/realtime/src/test/java/com/metamx/druid/realtime/plumber/SinkTest.java b/realtime/src/test/java/com/metamx/druid/realtime/plumber/SinkTest.java index 27b06609a0d..239f161dd91 100644 --- a/realtime/src/test/java/com/metamx/druid/realtime/plumber/SinkTest.java +++ b/realtime/src/test/java/com/metamx/druid/realtime/plumber/SinkTest.java @@ -21,14 +21,14 @@ package com.metamx.druid.realtime.plumber; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.InputRow; import com.metamx.druid.realtime.FireHydrant; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java b/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java index 2aa7e4f59bf..34ec2986b63 100644 --- a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java @@ -21,14 +21,15 @@ package com.metamx.druid; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.ConciseOffset; import com.metamx.druid.index.v1.processing.IntersectingOffset; import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.Indexed; import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.data.Indexed; +import io.druid.query.filter.Filter; +import io.druid.segment.StorageAdapter; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java index 076f40413bd..40aad21b5cb 100644 --- a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java +++ b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java @@ -19,10 +19,10 @@ package com.metamx.druid; -import com.metamx.druid.aggregation.Aggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.aggregation.Aggregator; import org.joda.time.DateTime; import java.util.HashMap; diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 3b65374bce5..138aab556af 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -25,14 +25,12 @@ import com.google.common.collect.Ordering; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.guice.annotations.Processing; import com.metamx.druid.index.ReferenceCountingSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; @@ -41,19 +39,21 @@ import com.metamx.druid.query.BySegmentQueryRunner; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.NoopQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner; -import com.metamx.druid.query.segment.QuerySegmentSpec; import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SegmentDescriptor; import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.SegmentDescriptor; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.Segment; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java index 81078b56595..6abcb9a9414 100644 --- a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java @@ -2,10 +2,7 @@ package com.metamx.druid.guice; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; -import com.metamx.druid.Query; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryRunnerFactory; @@ -17,6 +14,9 @@ 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 io.druid.initialization.Binders; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; import java.util.Map; @@ -24,7 +24,7 @@ import java.util.Map; */ public class QueryRunnerFactoryModule extends QueryToolChestModule { - final Map, Class> mappings = + private static final Map, Class> mappings = ImmutableMap., Class>builder() .put(TimeseriesQuery.class, TimeseriesQueryRunnerFactory.class) .put(SearchQuery.class, SearchQueryRunnerFactory.class) @@ -38,9 +38,8 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule { super.configure(binder); - final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( - binder, new TypeLiteral>(){}, new TypeLiteral(){} - ); + + final MapBinder, QueryRunnerFactory> queryFactoryBinder = Binders.queryFactoryBinder(binder); for (Map.Entry, Class> entry : mappings.entrySet()) { queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue()); diff --git a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java index 6e2d312aa75..2675d864f60 100644 --- a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java @@ -19,9 +19,11 @@ package com.metamx.druid.index; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java index 118dd9ab285..1f3945c2965 100644 --- a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java @@ -19,8 +19,10 @@ package com.metamx.druid.index; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java index 967537a1dbb..ed4e732182b 100644 --- a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java +++ b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java @@ -19,8 +19,10 @@ package com.metamx.druid.index; -import com.metamx.druid.StorageAdapter; import com.metamx.emitter.EmittingLogger; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java b/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java index 2de8823d9ca..1c50dc1837d 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java @@ -20,6 +20,10 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java b/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java index 0f7d7adc71d..fe99271d0a6 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java +++ b/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java @@ -19,6 +19,8 @@ package com.metamx.druid.index.brita; +import io.druid.query.filter.ValueMatcher; + /** */ public class BooleanValueMatcher implements ValueMatcher diff --git a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java b/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java index d88dd06d3dc..9e75e68c779 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java @@ -22,7 +22,11 @@ package com.metamx.druid.index.brita; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java index 883e19226e8..a12da1b358d 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java @@ -20,8 +20,12 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; -import com.metamx.druid.kv.Indexed; import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.data.Indexed; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/Filters.java b/server/src/main/java/com/metamx/druid/index/brita/Filters.java index c27940b06fd..1c3f0785301 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/Filters.java +++ b/server/src/main/java/com/metamx/druid/index/brita/Filters.java @@ -31,6 +31,7 @@ import com.metamx.druid.query.filter.RegexDimFilter; import com.metamx.druid.query.filter.SearchQueryDimFilter; import com.metamx.druid.query.filter.SelectorDimFilter; import com.metamx.druid.query.filter.SpatialDimFilter; +import io.druid.query.filter.Filter; import javax.annotation.Nullable; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java b/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java index 7c465d35a79..1f67d72f970 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java @@ -3,6 +3,10 @@ package com.metamx.druid.index.brita; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.metamx.common.guava.FunctionalIterable; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; diff --git a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java b/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java index d4d335add0c..e3fe4d6bd52 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java @@ -19,6 +19,10 @@ package com.metamx.druid.index.brita; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java b/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java index c9312b0b6f1..5afe92a5ceb 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java @@ -20,6 +20,10 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java index a43c7fa5afc..1258fdb3dc7 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java @@ -19,6 +19,10 @@ package com.metamx.druid.index.brita; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java index 4d7e4e36f6a..a7ea0696368 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java @@ -19,10 +19,11 @@ package com.metamx.druid.index.brita; import com.metamx.collections.spatial.search.Bound; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import it.uniroma3.mat.extendedset.intset.IntSet; - -import java.util.Iterator; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java index 695b116c5b0..dbb96def335 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java @@ -30,25 +30,25 @@ import com.metamx.collections.spatial.search.Bound; import com.metamx.common.IAE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterator; -import com.metamx.druid.Capabilities; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.aggregation.Aggregator; import com.metamx.druid.index.brita.BooleanValueMatcher; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.brita.ValueMatcher; -import com.metamx.druid.index.brita.ValueMatcherFactory; -import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.index.v1.processing.DimensionSelector; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.IndexedInts; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import com.metamx.druid.processing.ObjectColumnSelector; import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.data.IndexedInts; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.ValueMatcherFactory; +import io.druid.query.filter.Filter; +import io.druid.query.filter.ValueMatcher; +import io.druid.segment.Capabilities; +import io.druid.segment.ComplexMetricSelector; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatMetricSelector; +import io.druid.segment.ObjectMetricSelector; +import io.druid.segment.StorageAdapter; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -367,7 +367,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) + public ObjectMetricSelector makeObjectColumnSelector(String column) { final String columnName = column.toLowerCase(); final Integer metricIndexInt = index.getMetricIndex(columnName); @@ -377,7 +377,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(index.getMetricType(columnName)); - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -397,7 +397,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter if (dimensionIndexInt != null) { final int dimensionIndex = dimensionIndexInt; - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java index 67ac3810534..2900d479611 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java @@ -20,7 +20,7 @@ package com.metamx.druid.index.v1; import com.fasterxml.jackson.annotation.JsonCreator; -import com.metamx.druid.QueryGranularity; +import io.druid.granularity.QueryGranularity; import org.joda.time.DateTime; import org.joda.time.Days; import org.joda.time.Hours; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java b/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java index b258313c972..e55c581567b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java @@ -20,7 +20,7 @@ package com.metamx.druid.index.v1; import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.IndexedInts; +import io.druid.data.IndexedInts; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java index 9c95eb27ed4..8129b57c368 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java @@ -29,28 +29,28 @@ import com.metamx.common.collect.MoreIterators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterator; import com.metamx.druid.BaseStorageAdapter; -import com.metamx.druid.Capabilities; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.brita.BitmapIndexSelector; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.column.Column; -import com.metamx.druid.index.column.ColumnCapabilities; -import com.metamx.druid.index.column.ColumnSelector; -import com.metamx.druid.index.column.ComplexColumn; -import com.metamx.druid.index.column.DictionaryEncodedColumn; -import com.metamx.druid.index.column.GenericColumn; -import com.metamx.druid.index.column.ValueType; -import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.index.v1.processing.DimensionSelector; import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.SingleIndexedInts; -import com.metamx.druid.processing.ComplexMetricSelector; -import com.metamx.druid.processing.FloatMetricSelector; -import com.metamx.druid.processing.ObjectColumnSelector; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; +import io.druid.granularity.QueryGranularity; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.query.filter.Filter; +import io.druid.segment.Capabilities; +import io.druid.segment.ColumnSelector; +import io.druid.segment.ComplexMetricSelector; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.FloatMetricSelector; +import io.druid.segment.ObjectMetricSelector; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.ValueType; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -483,7 +483,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter } @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) + public ObjectMetricSelector makeObjectColumnSelector(String column) { final String columnName = column.toLowerCase(); @@ -524,7 +524,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter final ValueType type = columnVals.getType(); if (type == ValueType.FLOAT) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -540,7 +540,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter }; } if (type == ValueType.LONG) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -556,7 +556,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter }; } if (type == ValueType.STRING) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -575,7 +575,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter if (cachedColumnVals instanceof DictionaryEncodedColumn) { final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals; - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -592,7 +592,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter } final ComplexColumn columnVals = (ComplexColumn) cachedColumnVals; - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -902,7 +902,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter } @Override - public ObjectColumnSelector makeObjectColumnSelector(String column) + public ObjectMetricSelector makeObjectColumnSelector(String column) { final String columnName = column.toLowerCase(); @@ -942,7 +942,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter final ValueType type = columnVals.getType(); if (type == ValueType.FLOAT) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -958,7 +958,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter }; } if (type == ValueType.LONG) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -974,7 +974,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter }; } if (type == ValueType.STRING) { - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -993,7 +993,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter if (cachedColumnVals instanceof DictionaryEncodedColumn) { final DictionaryEncodedColumn columnVals = (DictionaryEncodedColumn) cachedColumnVals; - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() @@ -1010,7 +1010,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter } final ComplexColumn columnVals = (ComplexColumn) cachedColumnVals; - return new ObjectColumnSelector() + return new ObjectMetricSelector() { @Override public Class classOfObject() diff --git a/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java index 95061ae39a8..3fa346f341b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java @@ -21,8 +21,8 @@ package com.metamx.druid.index.v1; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; -import com.metamx.druid.kv.Indexed; -import com.metamx.druid.kv.IndexedInts; +import io.druid.data.Indexed; +import io.druid.data.IndexedInts; import org.joda.time.Interval; /** diff --git a/server/src/main/java/com/metamx/druid/index/v1/Searchable.java b/server/src/main/java/com/metamx/druid/index/v1/Searchable.java index a4f6ad70c78..ef222745079 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/Searchable.java +++ b/server/src/main/java/com/metamx/druid/index/v1/Searchable.java @@ -19,9 +19,9 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; +import io.druid.query.filter.Filter; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java index 1e47528d801..5842cd136bd 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java @@ -19,13 +19,13 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.Capabilities; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; +import io.druid.granularity.QueryGranularity; +import io.druid.query.filter.Filter; +import io.druid.segment.Capabilities; +import io.druid.segment.Cursor; +import io.druid.segment.StorageAdapter; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index 135a14ccb90..cced8f28add 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -21,7 +21,7 @@ package com.metamx.druid.loading; import com.metamx.common.MapUtils; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.Segment; +import io.druid.segment.Segment; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index d2cef9b76b3..8b530ea061f 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -20,8 +20,8 @@ package com.metamx.druid.loading; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.v1.IndexIO; +import io.druid.segment.QueryableIndex; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java index ffe419246ed..421676a108d 100644 --- a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java @@ -10,9 +10,9 @@ import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index 276bbc2028a..208f9d5e161 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -19,7 +19,7 @@ package com.metamx.druid.loading; -import com.metamx.druid.index.QueryableIndex; +import io.druid.segment.QueryableIndex; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 75c7591a395..3a303ab9076 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.Segment; +import io.druid.segment.Segment; /** */ diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 345b880004c..31be1e8ef57 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -28,9 +28,9 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java b/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java index 53208308719..b897f7933b0 100644 --- a/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java +++ b/server/src/main/java/com/metamx/druid/query/DefaultQueryRunnerFactoryConglomerate.java @@ -20,7 +20,8 @@ package com.metamx.druid.query; import com.google.inject.Inject; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactoryConglomerate.java b/server/src/main/java/com/metamx/druid/query/QueryRunnerFactoryConglomerate.java index f1e7621fd77..2195349ca26 100644 --- a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactoryConglomerate.java +++ b/server/src/main/java/com/metamx/druid/query/QueryRunnerFactoryConglomerate.java @@ -19,7 +19,8 @@ package com.metamx.druid.query; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; /** */ diff --git a/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java b/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java index 411542ffe4d..513fbd31756 100644 --- a/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java +++ b/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java @@ -21,16 +21,15 @@ package com.metamx.druid.query; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.primitives.Longs; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.aggregation.Aggregator; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.index.brita.Filter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.result.Result; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.filter.Filter; +import io.druid.segment.Cursor; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java b/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java index 8b2879dd493..6d6b68acd07 100644 --- a/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -3,8 +3,10 @@ package com.metamx.druid.query; import com.google.common.io.Closeables; import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; import com.metamx.druid.index.ReferenceCountingSegment; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 01f5f6cf450..600461d0974 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -35,19 +35,19 @@ import com.metamx.common.guava.FunctionalIterator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.parsers.CloseableIterator; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.aggregation.BufferAggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.index.brita.Filters; -import com.metamx.druid.index.v1.processing.Cursor; -import com.metamx.druid.index.v1.processing.DimensionSelector; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.StorageAdapter; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java index 9cc502ff1e6..23de48bb9bc 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java @@ -29,15 +29,15 @@ import com.metamx.common.ISE; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.Segment; import com.metamx.druid.input.Row; import com.metamx.druid.query.ChainedExecutionQueryRunner; import com.metamx.druid.query.ConcatQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryToolChest; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java index 8d5a6136998..8839a985278 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java @@ -25,14 +25,14 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.primitives.Longs; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.column.BitmapIndex; -import com.metamx.druid.index.column.Column; -import com.metamx.druid.index.column.ColumnCapabilities; -import com.metamx.druid.index.column.ComplexColumn; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; +import io.druid.segment.QueryableIndex; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.ValueType; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index f44110fcf14..dc6f6b7c89b 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -25,13 +25,13 @@ import com.google.common.collect.Maps; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.Segment; import com.metamx.druid.query.ConcatQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryToolChest; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import java.util.Arrays; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java index 2373f0da4d9..53f5b560a17 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java @@ -24,17 +24,17 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; import com.metamx.druid.SearchResultBuilder; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.brita.Filters; import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import java.util.Iterator; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 6c40e6774d9..9742380e4fd 100644 --- a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -22,15 +22,15 @@ package com.metamx.druid.query.timeboundary; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.Segment; import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import java.util.Iterator; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java index c95b7acf5df..8f64b0321f6 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java @@ -22,16 +22,16 @@ package com.metamx.druid.query.timeseries; import com.google.common.base.Function; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.TimeseriesResultBuilder; -import com.metamx.druid.aggregation.Aggregator; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.index.brita.Filters; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.query.QueryRunnerHelper; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.Cursor; +import io.druid.segment.StorageAdapter; import java.util.Iterator; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 9e60d1d49fa..7ddc4947625 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -21,15 +21,15 @@ package com.metamx.druid.query.timeseries; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; -import com.metamx.druid.Query; -import com.metamx.druid.StorageAdapter; -import com.metamx.druid.index.Segment; import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/io/druid/initialization/Binders.java b/server/src/main/java/io/druid/initialization/Binders.java new file mode 100644 index 00000000000..b4f9ce82d8c --- /dev/null +++ b/server/src/main/java/io/druid/initialization/Binders.java @@ -0,0 +1,38 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.initialization; + +import com.google.inject.Binder; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; + +/** + */ +public class Binders +{ + public static MapBinder, QueryRunnerFactory> queryFactoryBinder(Binder binder) + { + return MapBinder.newMapBinder( + binder, new TypeLiteral>(){}, new TypeLiteral(){} + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java b/server/src/main/java/io/druid/query/QueryRunnerFactory.java similarity index 91% rename from server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java rename to server/src/main/java/io/druid/query/QueryRunnerFactory.java index 8dade405e49..0eee7a579aa 100644 --- a/server/src/main/java/com/metamx/druid/query/QueryRunnerFactory.java +++ b/server/src/main/java/io/druid/query/QueryRunnerFactory.java @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; -import com.metamx.druid.Query; -import com.metamx.druid.index.Segment; +import io.druid.segment.Segment; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ValueMatcherFactory.java b/server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java similarity index 93% rename from server/src/main/java/com/metamx/druid/index/brita/ValueMatcherFactory.java rename to server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java index cc9971b23b8..6d5e5ba931a 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ValueMatcherFactory.java +++ b/server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java @@ -17,10 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.query.aggregation; import com.google.common.base.Predicate; import com.metamx.collections.spatial.search.Bound; +import io.druid.query.filter.ValueMatcher; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/brita/BitmapIndexSelector.java b/server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java similarity index 94% rename from server/src/main/java/com/metamx/druid/index/brita/BitmapIndexSelector.java rename to server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java index fb8837de1f5..9a7cb2e70df 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/BitmapIndexSelector.java +++ b/server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.query.filter; import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.druid.kv.Indexed; +import io.druid.data.Indexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/Filter.java b/server/src/main/java/io/druid/query/filter/Filter.java similarity index 92% rename from server/src/main/java/com/metamx/druid/index/brita/Filter.java rename to server/src/main/java/io/druid/query/filter/Filter.java index fcfde0be18a..08446a1ee2f 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/Filter.java +++ b/server/src/main/java/io/druid/query/filter/Filter.java @@ -17,8 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.query.filter; +import io.druid.query.aggregation.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/ValueMatcher.java b/server/src/main/java/io/druid/query/filter/ValueMatcher.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/brita/ValueMatcher.java rename to server/src/main/java/io/druid/query/filter/ValueMatcher.java index 5d68a203bfa..9ecc8b32a9e 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ValueMatcher.java +++ b/server/src/main/java/io/druid/query/filter/ValueMatcher.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.query.filter; /** */ diff --git a/server/src/main/java/com/metamx/druid/Capabilities.java b/server/src/main/java/io/druid/segment/Capabilities.java similarity index 98% rename from server/src/main/java/com/metamx/druid/Capabilities.java rename to server/src/main/java/io/druid/segment/Capabilities.java index 7b70a8fbc1d..10bcda912a5 100644 --- a/server/src/main/java/com/metamx/druid/Capabilities.java +++ b/server/src/main/java/io/druid/segment/Capabilities.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.segment; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/Cursor.java b/server/src/main/java/io/druid/segment/Cursor.java similarity index 85% rename from server/src/main/java/com/metamx/druid/index/v1/processing/Cursor.java rename to server/src/main/java/io/druid/segment/Cursor.java index c304d66b0d6..a5b144c07d1 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/Cursor.java +++ b/server/src/main/java/io/druid/segment/Cursor.java @@ -17,14 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment; -import com.metamx.druid.processing.ColumnSelectorFactory; import org.joda.time.DateTime; /** */ -public interface Cursor extends ColumnSelectorFactory, DimensionSelectorFactory +public interface Cursor extends MetricSelectorFactory, DimensionSelectorFactory { public DateTime getTime(); public void advance(); diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/CursorFactory.java b/server/src/main/java/io/druid/segment/CursorFactory.java similarity index 88% rename from server/src/main/java/com/metamx/druid/index/v1/processing/CursorFactory.java rename to server/src/main/java/io/druid/segment/CursorFactory.java index b18fec4b322..fe8c722053a 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/CursorFactory.java +++ b/server/src/main/java/io/druid/segment/CursorFactory.java @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.index.brita.Filter; +import io.druid.granularity.QueryGranularity; +import io.druid.query.filter.Filter; import org.joda.time.Interval; /** diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java b/server/src/main/java/io/druid/segment/DimensionSelector.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java rename to server/src/main/java/io/druid/segment/DimensionSelector.java index 8ced8f388dd..fb24be75f7e 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelector.java +++ b/server/src/main/java/io/druid/segment/DimensionSelector.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment; -import com.metamx.druid.kv.IndexedInts; +import io.druid.data.IndexedInts; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelectorFactory.java b/server/src/main/java/io/druid/segment/DimensionSelectorFactory.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelectorFactory.java rename to server/src/main/java/io/druid/segment/DimensionSelectorFactory.java index 9af5988a3ec..a3ad9d2da91 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/DimensionSelectorFactory.java +++ b/server/src/main/java/io/druid/segment/DimensionSelectorFactory.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment; /** */ diff --git a/server/src/main/java/com/metamx/druid/index/Segment.java b/server/src/main/java/io/druid/segment/Segment.java similarity index 93% rename from server/src/main/java/com/metamx/druid/index/Segment.java rename to server/src/main/java/io/druid/segment/Segment.java index 9b9e6c94cd1..70bb88aa743 100644 --- a/server/src/main/java/com/metamx/druid/index/Segment.java +++ b/server/src/main/java/io/druid/segment/Segment.java @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; -import com.metamx.druid.StorageAdapter; import org.joda.time.Interval; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/StorageAdapter.java b/server/src/main/java/io/druid/segment/StorageAdapter.java similarity index 93% rename from server/src/main/java/com/metamx/druid/StorageAdapter.java rename to server/src/main/java/io/druid/segment/StorageAdapter.java index 285d17b5b05..307212a7b98 100644 --- a/server/src/main/java/com/metamx/druid/StorageAdapter.java +++ b/server/src/main/java/io/druid/segment/StorageAdapter.java @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.segment; import com.metamx.druid.index.v1.Searchable; -import com.metamx.druid.index.v1.processing.CursorFactory; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index f2cd54f4551..d375ea370b0 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -36,30 +36,30 @@ import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingSequenceBase; import com.metamx.druid.Druids; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.ReferenceCountingSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.ConcatQueryRunner; -import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.NoopQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.QueryToolChest; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; diff --git a/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java index ef986166cb1..dee1229e388 100644 --- a/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java +++ b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java @@ -20,7 +20,9 @@ package com.metamx.druid.index; import com.google.common.base.Throwables; -import com.metamx.druid.StorageAdapter; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Days; diff --git a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java index f4ef6249b39..d83321bb93c 100644 --- a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java +++ b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java @@ -25,15 +25,11 @@ import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import com.metamx.druid.Druids; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexSchema; import com.metamx.druid.index.v1.IndexIO; @@ -41,12 +37,16 @@ import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.filter.SpatialDimFilter; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.QueryRunner; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java index 5261c19d861..dea44e0c44f 100644 --- a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java +++ b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java @@ -25,15 +25,11 @@ import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import com.metamx.druid.Druids; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexSchema; import com.metamx.druid.index.v1.IndexIO; @@ -41,12 +37,16 @@ import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.filter.SpatialDimFilter; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.QueryRunner; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java index 55c4b7be6e5..6bbfd6fbbba 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java @@ -21,9 +21,9 @@ package com.metamx.druid.index.v1; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.index.QueryableIndex; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 8af66f30eb4..e5f3182a12c 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -23,10 +23,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.Files; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import junit.framework.Assert; import org.apache.commons.io.FileUtils; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index df5761681af..3f82a58c491 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -20,31 +20,26 @@ package com.metamx.druid.index.v1; import com.google.common.base.Charsets; -import com.google.common.base.Function; import com.google.common.base.Throwables; -import com.google.common.collect.Maps; import com.google.common.io.CharStreams; import com.google.common.io.InputSupplier; import com.google.common.io.LineProcessor; import com.metamx.common.logger.Logger; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.client.RangeIterable; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.indexer.data.DelimitedDataSpec; import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.indexer.data.TimestampSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.QueryableIndex; import org.joda.time.DateTime; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.URL; import java.util.Arrays; -import java.util.Map; import java.util.concurrent.atomic.AtomicLong; /** diff --git a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java index ebf6ae09b10..0b80a7fc5b6 100644 --- a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java @@ -20,10 +20,10 @@ package com.metamx.druid.loading; import com.metamx.common.MapUtils; -import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndex; -import com.metamx.druid.index.Segment; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.File; diff --git a/server/src/test/java/com/metamx/druid/query/NoopQueryRunnerFactoryConglomerate.java b/server/src/test/java/com/metamx/druid/query/NoopQueryRunnerFactoryConglomerate.java index 6960b921ad7..00fd36c6498 100644 --- a/server/src/test/java/com/metamx/druid/query/NoopQueryRunnerFactoryConglomerate.java +++ b/server/src/test/java/com/metamx/druid/query/NoopQueryRunnerFactoryConglomerate.java @@ -19,7 +19,8 @@ package com.metamx.druid.query; -import com.metamx.druid.Query; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; /** */ diff --git a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java index 18c647c1c70..8d46099a016 100644 --- a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java +++ b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java @@ -20,9 +20,6 @@ package com.metamx.druid.query; import com.google.common.collect.Lists; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; @@ -30,13 +27,18 @@ import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.TestIndex; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; -import com.metamx.druid.query.segment.QuerySegmentSpec; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index e5620304d9b..a06161e5b4c 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -30,17 +30,12 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.PeriodGranularity; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DimensionSpec; @@ -51,7 +46,12 @@ import com.metamx.druid.query.group.having.OrHavingSpec; import com.metamx.druid.query.group.orderby.DefaultLimitSpec; import com.metamx.druid.query.group.orderby.LimitSpec; import com.metamx.druid.query.group.orderby.OrderByColumnSpec; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index 4c4a801487c..b8fe4b60d72 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -25,16 +25,16 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerTest; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.Query; +import io.druid.query.QueryRunner; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java index 6805bb6c23b..94f64540149 100644 --- a/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java +++ b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java @@ -23,13 +23,13 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import com.metamx.druid.index.IncrementalIndexSegment; import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.Segment; -import com.metamx.druid.index.column.ValueType; import com.metamx.druid.index.v1.TestIndex; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.segment.QuerySegmentSpecs; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.segment.Segment; +import io.druid.segment.column.ValueType; import junit.framework.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java index ae846787f45..59184d4da60 100644 --- a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java @@ -25,11 +25,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.guava.Sequences; import com.metamx.druid.Druids; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; +import io.druid.query.QueryRunner; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 999c4202073..ed45fee9508 100644 --- a/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -22,10 +22,10 @@ package com.metamx.druid.query.timeboundary; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import com.metamx.druid.Druids; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.query.QueryRunner; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index 91865f47c7d..5325212d2f3 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -5,19 +5,19 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import com.metamx.druid.Druids; -import com.metamx.druid.Query; -import com.metamx.druid.QueryGranularity; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.CountAggregatorFactory; import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryRunner; -import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.Segment; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index c1eb9a249eb..612108fdd71 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -23,22 +23,22 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import com.metamx.druid.Druids; -import com.metamx.druid.PeriodGranularity; -import com.metamx.druid.QueryGranularity; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.MaxAggregatorFactory; import com.metamx.druid.aggregation.MinAggregatorFactory; import com.metamx.druid.aggregation.post.PostAggregator; -import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.RegexDimFilter; -import com.metamx.druid.query.segment.MultipleIntervalSegmentSpec; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.query.QueryRunner; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; @@ -52,7 +52,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Map; /** */ From bfe96739a3c445e15e16c8c0dc60af2d663b357d Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 28 Aug 2013 15:52:55 -0500 Subject: [PATCH 47/92] 1) Move various "api" classes to io.druid packages and make sure things compile and stuff -- continued! Come classes didn't get committed? --- .../src/main/java/io/druid/granularity/DurationGranularity.java | 2 +- common/src/main/java/io/druid/granularity/QueryGranularity.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/io/druid/granularity/DurationGranularity.java b/common/src/main/java/io/druid/granularity/DurationGranularity.java index 8af4eccb8f8..a4e8393eabd 100644 --- a/common/src/main/java/io/druid/granularity/DurationGranularity.java +++ b/common/src/main/java/io/druid/granularity/DurationGranularity.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 diff --git a/common/src/main/java/io/druid/granularity/QueryGranularity.java b/common/src/main/java/io/druid/granularity/QueryGranularity.java index 635a19a0bbc..90bd2103b01 100644 --- a/common/src/main/java/io/druid/granularity/QueryGranularity.java +++ b/common/src/main/java/io/druid/granularity/QueryGranularity.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 From 1f37e962f65dc59978b0de0389c51c929d235ad6 Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 28 Aug 2013 16:18:40 -0500 Subject: [PATCH 48/92] 1) Delete the files that have moved to druid-api and get them through the druid-api dependency. Still need to fix Searchable on StorageAdapter, but other than that, everything should compile. --- common/pom.xml | 2 - .../io/druid/granularity/AllGranularity.java | 74 ---- .../granularity/BaseQueryGranularity.java | 83 ---- .../granularity/DurationGranularity.java | 128 ------ .../io/druid/granularity/NoneGranularity.java | 66 --- .../druid/granularity/PeriodGranularity.java | 375 ------------------ .../druid/granularity/QueryGranularity.java | 100 ----- .../io/druid/initialization/DruidModule.java | 31 -- .../druid/query/aggregation/Aggregator.java | 42 -- .../query/aggregation/AggregatorFactory.java | 119 ------ .../query/aggregation/BufferAggregator.java | 96 ----- .../druid/segment/ComplexMetricSelector.java | 28 -- .../io/druid/segment/FloatMetricSelector.java | 30 -- .../java/io/druid/segment/LogicalSegment.java | 8 - .../druid/segment/MetricSelectorFactory.java | 30 -- .../druid/segment/ObjectMetricSelector.java | 27 -- .../src/main/java/io/druid/data/Indexed.java | 33 -- .../java/io/druid/data/IndexedFloats.java | 32 -- .../main/java/io/druid/data/IndexedInts.java | 30 -- .../main/java/io/druid/data/IndexedLongs.java | 34 -- .../java/io/druid/segment/ColumnSelector.java | 30 -- .../java/io/druid/segment/QueryableIndex.java | 42 -- .../io/druid/segment/column/BitmapIndex.java | 33 -- .../java/io/druid/segment/column/Column.java | 35 -- .../segment/column/ColumnCapabilities.java | 33 -- .../druid/segment/column/ComplexColumn.java | 31 -- .../column/DictionaryEncodedColumn.java | 35 -- .../druid/segment/column/GenericColumn.java | 42 -- .../druid/segment/column/RunLengthColumn.java | 27 -- .../io/druid/segment/column/SpatialIndex.java | 28 -- .../io/druid/segment/column/ValueType.java | 11 - .../metamx/druid/index/brita/AndFilter.java | 2 +- .../index/brita/DimensionPredicateFilter.java | 2 +- .../druid/index/brita/ExtractionFilter.java | 2 +- .../druid/index/brita/JavaScriptFilter.java | 2 +- .../metamx/druid/index/brita/NotFilter.java | 2 +- .../metamx/druid/index/brita/OrFilter.java | 2 +- .../druid/index/brita/SelectorFilter.java | 2 +- .../druid/index/brita/SpatialFilter.java | 2 +- .../v1/IncrementalIndexStorageAdapter.java | 2 +- .../java/io/druid/initialization/Binders.java | 38 -- .../io/druid/query/QueryRunnerFactory.java | 33 -- .../aggregation/ValueMatcherFactory.java | 33 -- .../query/filter/BitmapIndexSelector.java | 35 -- .../java/io/druid/query/filter/Filter.java | 31 -- .../io/druid/query/filter/ValueMatcher.java | 27 -- .../java/io/druid/segment/Capabilities.java | 64 --- .../main/java/io/druid/segment/Cursor.java | 32 -- .../java/io/druid/segment/CursorFactory.java | 31 -- .../io/druid/segment/DimensionSelector.java | 84 ---- .../segment/DimensionSelectorFactory.java | 27 -- .../main/java/io/druid/segment/Segment.java | 34 -- .../java/io/druid/segment/StorageAdapter.java | 36 -- 53 files changed, 9 insertions(+), 2229 deletions(-) delete mode 100644 common/src/main/java/io/druid/granularity/AllGranularity.java delete mode 100644 common/src/main/java/io/druid/granularity/BaseQueryGranularity.java delete mode 100644 common/src/main/java/io/druid/granularity/DurationGranularity.java delete mode 100644 common/src/main/java/io/druid/granularity/NoneGranularity.java delete mode 100644 common/src/main/java/io/druid/granularity/PeriodGranularity.java delete mode 100644 common/src/main/java/io/druid/granularity/QueryGranularity.java delete mode 100644 common/src/main/java/io/druid/initialization/DruidModule.java delete mode 100644 common/src/main/java/io/druid/query/aggregation/Aggregator.java delete mode 100644 common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java delete mode 100644 common/src/main/java/io/druid/query/aggregation/BufferAggregator.java delete mode 100644 common/src/main/java/io/druid/segment/ComplexMetricSelector.java delete mode 100644 common/src/main/java/io/druid/segment/FloatMetricSelector.java delete mode 100644 common/src/main/java/io/druid/segment/LogicalSegment.java delete mode 100644 common/src/main/java/io/druid/segment/MetricSelectorFactory.java delete mode 100644 common/src/main/java/io/druid/segment/ObjectMetricSelector.java delete mode 100644 indexing-common/src/main/java/io/druid/data/Indexed.java delete mode 100644 indexing-common/src/main/java/io/druid/data/IndexedFloats.java delete mode 100644 indexing-common/src/main/java/io/druid/data/IndexedInts.java delete mode 100644 indexing-common/src/main/java/io/druid/data/IndexedLongs.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/ColumnSelector.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/QueryableIndex.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/Column.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java delete mode 100644 indexing-common/src/main/java/io/druid/segment/column/ValueType.java delete mode 100644 server/src/main/java/io/druid/initialization/Binders.java delete mode 100644 server/src/main/java/io/druid/query/QueryRunnerFactory.java delete mode 100644 server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java delete mode 100644 server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java delete mode 100644 server/src/main/java/io/druid/query/filter/Filter.java delete mode 100644 server/src/main/java/io/druid/query/filter/ValueMatcher.java delete mode 100644 server/src/main/java/io/druid/segment/Capabilities.java delete mode 100644 server/src/main/java/io/druid/segment/Cursor.java delete mode 100644 server/src/main/java/io/druid/segment/CursorFactory.java delete mode 100644 server/src/main/java/io/druid/segment/DimensionSelector.java delete mode 100644 server/src/main/java/io/druid/segment/DimensionSelectorFactory.java delete mode 100644 server/src/main/java/io/druid/segment/Segment.java delete mode 100644 server/src/main/java/io/druid/segment/StorageAdapter.java diff --git a/common/pom.xml b/common/pom.xml index 7138012d675..2c513ff5fb0 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -36,13 +36,11 @@ com.metamx java-util - commons-codec diff --git a/common/src/main/java/io/druid/granularity/AllGranularity.java b/common/src/main/java/io/druid/granularity/AllGranularity.java deleted file mode 100644 index 164cbe30f1a..00000000000 --- a/common/src/main/java/io/druid/granularity/AllGranularity.java +++ /dev/null @@ -1,74 +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 io.druid.granularity; - -import com.google.common.collect.ImmutableList; - -public final class AllGranularity extends BaseQueryGranularity -{ - @Override - public long next(long offset) - { - return Long.MAX_VALUE; - } - - @Override - public long truncate(long offset) - { - return Long.MIN_VALUE; - } - - @Override - public byte[] cacheKey() - { - return new byte[]{0x7f}; - } - - @Override - public Iterable iterable(long start, long end) - { - return ImmutableList.of(start); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - return 0; - } - - @Override - public String toString() - { - return "AllGranularity"; - } -} diff --git a/common/src/main/java/io/druid/granularity/BaseQueryGranularity.java b/common/src/main/java/io/druid/granularity/BaseQueryGranularity.java deleted file mode 100644 index 24b8e90e8cf..00000000000 --- a/common/src/main/java/io/druid/granularity/BaseQueryGranularity.java +++ /dev/null @@ -1,83 +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 io.druid.granularity; - -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -import java.util.Iterator; -import java.util.NoSuchElementException; - -public abstract class BaseQueryGranularity extends QueryGranularity -{ - public abstract long next(long offset); - - public abstract long truncate(long offset); - - public abstract byte[] cacheKey(); - - public DateTime toDateTime(long offset) - { - return new DateTime(offset, DateTimeZone.UTC); - } - - public Iterable iterable(final long start, final long end) - { - return new Iterable() - { - @Override - public Iterator iterator() - { - return new Iterator() - { - long curr = truncate(start); - long next = BaseQueryGranularity.this.next(curr); - - @Override - public boolean hasNext() - { - return curr < end; - } - - @Override - public Long next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - long retVal = curr; - - curr = next; - next = BaseQueryGranularity.this.next(curr); - - return retVal; - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - }; - } -} diff --git a/common/src/main/java/io/druid/granularity/DurationGranularity.java b/common/src/main/java/io/druid/granularity/DurationGranularity.java deleted file mode 100644 index a4e8393eabd..00000000000 --- a/common/src/main/java/io/druid/granularity/DurationGranularity.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.granularity; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.primitives.Longs; -import org.joda.time.DateTime; - -import java.nio.ByteBuffer; - -public class DurationGranularity extends BaseQueryGranularity -{ - private final long length; - private final long origin; - - @JsonCreator - public DurationGranularity( - @JsonProperty("duration") long duration, - @JsonProperty("origin") DateTime origin - ) - { - this(duration, origin == null ? 0 : origin.getMillis()); - } - - public DurationGranularity(long millis, long origin) - { - this.length = millis; - this.origin = origin % length; - } - - @JsonProperty("duration") - public long getDuration() - { - return length; - } - - @JsonProperty("origin") - public DateTime getOrigin() - { - return new DateTime(origin); - } - - public long getOriginMillis() - { - return origin; - } - - @Override - public long next(long t) - { - return t + getDurationMillis(); - } - - @Override - public long truncate(long t) - { - final long duration = getDurationMillis(); - return t - t % duration + origin; - } - - @Override - public byte[] cacheKey() - { - return ByteBuffer.allocate(2 * Longs.BYTES).putLong(length).putLong(origin).array(); - } - - public long getDurationMillis() - { - return length; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - DurationGranularity that = (DurationGranularity) o; - - if (length != that.length) { - return false; - } - if (origin != that.origin) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - int result = (int) (length ^ (length >>> 32)); - result = 31 * result + (int) (origin ^ (origin >>> 32)); - return result; - } - - @Override - public String toString() - { - return "DurationGranularity{" + - "length=" + length + - ", origin=" + origin + - '}'; - } -} diff --git a/common/src/main/java/io/druid/granularity/NoneGranularity.java b/common/src/main/java/io/druid/granularity/NoneGranularity.java deleted file mode 100644 index 3185fba9ad4..00000000000 --- a/common/src/main/java/io/druid/granularity/NoneGranularity.java +++ /dev/null @@ -1,66 +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 io.druid.granularity; - -public final class NoneGranularity extends BaseQueryGranularity -{ - @Override - public long next(long offset) - { - return offset + 1; - } - - @Override - public long truncate(long offset) - { - return offset; - } - - @Override - public byte[] cacheKey() - { - return new byte[]{0x0}; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - return 0; - } - - @Override - public String toString() - { - return "NoneGranularity"; - } -} diff --git a/common/src/main/java/io/druid/granularity/PeriodGranularity.java b/common/src/main/java/io/druid/granularity/PeriodGranularity.java deleted file mode 100644 index 02cce4119c4..00000000000 --- a/common/src/main/java/io/druid/granularity/PeriodGranularity.java +++ /dev/null @@ -1,375 +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 io.druid.granularity; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Chronology; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Period; -import org.joda.time.chrono.ISOChronology; - -public class PeriodGranularity extends BaseQueryGranularity -{ - private final Period period; - private final Chronology chronology; - private final long origin; - private final boolean hasOrigin; - - private final boolean isCompound; - - @JsonCreator - public PeriodGranularity( - @JsonProperty("period") Period period, - @JsonProperty("origin") DateTime origin, - @JsonProperty("timeZone") DateTimeZone tz - ) - { - this.period = period; - this.chronology = tz == null ? ISOChronology.getInstanceUTC() : ISOChronology.getInstance(tz); - if(origin == null) - { - // default to origin in given time zone when aligning multi-period granularities - this.origin = new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(chronology.getZone()).getMillis(); - this.hasOrigin = false; - } - else - { - this.origin = origin.getMillis(); - this.hasOrigin = true; - } - this.isCompound = isCompoundPeriod(period); - } - - @JsonProperty("period") - public Period getPeriod() - { - return period; - } - - @JsonProperty("timeZone") - public DateTimeZone getTimeZone() - { - return chronology.getZone(); - } - - @JsonProperty("origin") - public DateTime getOrigin() - { - return hasOrigin ? new DateTime(origin) : null; - } - - @Override - public DateTime toDateTime(long t) - { - return new DateTime(t, chronology.getZone()); - } - - @Override - public long next(long t) - { - return chronology.add(period, t, 1); - } - - @Override - public long truncate(long t) - { - if(isCompound) - { - try { - return truncateMillisPeriod(t); - } catch(UnsupportedOperationException e) { - return truncateCompoundPeriod(t); - } - } - - final int years = period.getYears(); - if(years > 0) - { - if(years > 1 || hasOrigin) - { - int y = chronology.years().getDifference(t, origin); - y -= y % years; - long tt = chronology.years().add(origin, y); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.years().add(tt, -years); - else t = tt; - return t; - } - else - { - return chronology.year().roundFloor(t); - } - } - - final int months = period.getMonths(); - if(months > 0) - { - if(months > 1 || hasOrigin) - { - int m = chronology.months().getDifference(t, origin); - m -= m % months; - long tt = chronology.months().add(origin, m); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.months().add(tt, -months); - else t = tt; - return t; - } - else - { - return chronology.monthOfYear().roundFloor(t); - } - } - - final int weeks = period.getWeeks(); - if(weeks > 0) - { - if(weeks > 1 || hasOrigin) - { - // align on multiples from origin - int w = chronology.weeks().getDifference(t, origin); - w -= w % weeks; - long tt = chronology.weeks().add(origin, w); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.weeks().add(tt, -weeks); - else t = tt; - return t; - } - else - { - t = chronology.dayOfWeek().roundFloor(t); - // default to Monday as beginning of the week - return chronology.dayOfWeek().set(t, 1); - } - } - - final int days = period.getDays(); - if(days > 0) - { - if(days > 1 || hasOrigin) - { - // align on multiples from origin - int d = chronology.days().getDifference(t, origin); - d -= d % days; - long tt = chronology.days().add(origin, d); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.days().add(tt, -days); - else t = tt; - return t; - } - else - { - t = chronology.hourOfDay().roundFloor(t); - return chronology.hourOfDay().set(t, 0); - } - } - - final int hours = period.getHours(); - if(hours > 0) - { - if(hours > 1 || hasOrigin) - { - // align on multiples from origin - long h = chronology.hours().getDifferenceAsLong(t, origin); - h -= h % hours; - long tt = chronology.hours().add(origin, h); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.hours().add(tt, -hours); - else t = tt; - return t; - } - else - { - t = chronology.minuteOfHour().roundFloor(t); - return chronology.minuteOfHour().set(t, 0); - } - } - - final int minutes = period.getMinutes(); - if(minutes > 0) - { - // align on multiples from origin - if(minutes > 1 || hasOrigin) - { - long m = chronology.minutes().getDifferenceAsLong(t, origin); - m -= m % minutes; - long tt = chronology.minutes().add(origin, m); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.minutes().add(tt, -minutes); - else t = tt; - return t; - } - else - { - t = chronology.secondOfMinute().roundFloor(t); - return chronology.secondOfMinute().set(t, 0); - } - } - - final int seconds = period.getSeconds(); - if(seconds > 0) - { - // align on multiples from origin - if(seconds > 1 || hasOrigin) - { - long s = chronology.seconds().getDifferenceAsLong(t, origin); - s -= s % seconds; - long tt = chronology.seconds().add(origin, s); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.seconds().add(tt, -seconds); - else t = tt; - return t; - } - else - { - return chronology.millisOfSecond().set(t, 0); - } - } - - final int millis = period.getMillis(); - if(millis > 0) - { - if(millis > 1) - { - long ms = chronology.millis().getDifferenceAsLong(t, origin); - ms -= ms % millis; - long tt = chronology.millis().add(origin, ms); - // always round down to the previous period (for timestamps prior to origin) - if(t < tt) t = chronology.millis().add(tt, -millis); - else t = tt; - return t; - } - else { - return t; - } - } - - return t; - } - - private static boolean isCompoundPeriod(Period period) - { - int[] values = period.getValues(); - boolean single = false; - for(int v : values) - { - if(v > 0) - { - if(single) return true; - single = true; - } - } - return false; - } - - private long truncateCompoundPeriod(long t) - { - long current; - if(t >= origin) - { - long next = origin; - do { - current = next; - next = chronology.add(period, current, 1); - } while(t >= next); - } - else - { - current = origin; - do { - current = chronology.add(period, current, -1); - } while(t < current); - } - return current; - } - - private long truncateMillisPeriod(long t) - { - // toStandardDuration assumes days are always 24h, and hours are always 60 minutes, - // which may not always be the case, e.g if there are daylight saving changes. - if(chronology.days().isPrecise() && chronology.hours().isPrecise()) { - final long millis = period.toStandardDuration().getMillis(); - t -= t % millis + origin % millis; - return t; - } - else - { - throw new UnsupportedOperationException( - "Period cannot be converted to milliseconds as some fields mays vary in length with chronology " - + chronology.toString() - ); - } - } - - @Override - public byte[] cacheKey() - { - return (period.toString() + ":" + chronology.getZone().toString()).getBytes(); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - PeriodGranularity that = (PeriodGranularity) o; - - if (hasOrigin != that.hasOrigin) { - return false; - } - if (origin != that.origin) { - return false; - } - if (!chronology.equals(that.chronology)) { - return false; - } - if (!period.equals(that.period)) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - int result = period.hashCode(); - result = 31 * result + chronology.hashCode(); - result = 31 * result + (int) (origin ^ (origin >>> 32)); - result = 31 * result + (hasOrigin ? 1 : 0); - return result; - } - - @Override - public String toString() - { - return "PeriodGranularity{" + - "period=" + period + - ", timeZone=" + chronology .getZone() + - ", origin=" + (hasOrigin ? origin : "null") + - '}'; - } -} diff --git a/common/src/main/java/io/druid/granularity/QueryGranularity.java b/common/src/main/java/io/druid/granularity/QueryGranularity.java deleted file mode 100644 index 90bd2103b01..00000000000 --- a/common/src/main/java/io/druid/granularity/QueryGranularity.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.granularity; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.common.IAE; -import org.joda.time.DateTime; -import org.joda.time.ReadableDuration; - -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property = "type", defaultImpl = QueryGranularity.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "period", value = PeriodGranularity.class), - @JsonSubTypes.Type(name = "duration", value = DurationGranularity.class), - @JsonSubTypes.Type(name = "all", value = AllGranularity.class), - @JsonSubTypes.Type(name = "none", value = NoneGranularity.class) -}) -public abstract class QueryGranularity -{ - public abstract long next(long offset); - - public abstract long truncate(long offset); - - public abstract byte[] cacheKey(); - - public abstract DateTime toDateTime(long offset); - - public abstract Iterable iterable(final long start, final long end); - - public static final QueryGranularity ALL = new AllGranularity(); - public static final QueryGranularity NONE = new NoneGranularity(); - - public static final QueryGranularity MINUTE = fromString("MINUTE"); - public static final QueryGranularity HOUR = fromString("HOUR"); - public static final QueryGranularity DAY = fromString("DAY"); - public static final QueryGranularity SECOND = fromString("SECOND"); - - @JsonCreator - public static QueryGranularity fromString(String str) - { - String name = str.toUpperCase(); - if(name.equals("ALL")) - { - return QueryGranularity.ALL; - } - else if(name.equals("NONE")) - { - return QueryGranularity.NONE; - } - return new DurationGranularity(convertValue(str), 0); - } - - private static enum MillisIn - { - SECOND ( 1000), - MINUTE ( 60 * 1000), - FIFTEEN_MINUTE (15 * 60 * 1000), - THIRTY_MINUTE (30 * 60 * 1000), - HOUR ( 3600 * 1000), - DAY (24 * 3600 * 1000); - - private final long millis; - MillisIn(final long millis) { this.millis = millis; } - } - - private static long convertValue(Object o) - { - if(o instanceof String) - { - return MillisIn.valueOf(((String) o).toUpperCase()).millis; - } - else if(o instanceof ReadableDuration) - { - return ((ReadableDuration)o).getMillis(); - } - else if(o instanceof Number) - { - return ((Number)o).longValue(); - } - throw new IAE("Cannot convert [%s] to QueryGranularity", o.getClass()); - } -} diff --git a/common/src/main/java/io/druid/initialization/DruidModule.java b/common/src/main/java/io/druid/initialization/DruidModule.java deleted file mode 100644 index 7891baf5cc6..00000000000 --- a/common/src/main/java/io/druid/initialization/DruidModule.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.initialization; - -import com.fasterxml.jackson.databind.Module; - -import java.util.List; - -/** - */ -public interface DruidModule extends com.google.inject.Module -{ - public List getJacksonModules(); -} diff --git a/common/src/main/java/io/druid/query/aggregation/Aggregator.java b/common/src/main/java/io/druid/query/aggregation/Aggregator.java deleted file mode 100644 index 08673b5a48c..00000000000 --- a/common/src/main/java/io/druid/query/aggregation/Aggregator.java +++ /dev/null @@ -1,42 +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 io.druid.query.aggregation; - -/** - * An Aggregator is an object that can aggregate metrics. Its aggregation-related methods (namely, aggregate() and get()) - * do not take any arguments as the assumption is that the Aggregator was given something in its constructor that - * it can use to get at the next bit of data. - * - * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls - * to aggregate(). This is currently (as of this documentation) implemented through the use of Offset and - * FloatMetricSelector objects. The Aggregator has a handle on a FloatMetricSelector object which has a handle on an Offset. - * QueryableIndex has both the Aggregators and the Offset object and iterates through the Offset calling the aggregate() - * method on the Aggregators for each applicable row. - * - * This interface is old and going away. It is being replaced by BufferAggregator - */ -public interface Aggregator { - void aggregate(); - void reset(); - Object get(); - float getFloat(); - String getName(); - void close(); -} diff --git a/common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java deleted file mode 100644 index 48c7f2b8753..00000000000 --- a/common/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ /dev/null @@ -1,119 +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 io.druid.query.aggregation; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.HistogramAggregatorFactory; -import com.metamx.druid.aggregation.JavaScriptAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.aggregation.MaxAggregatorFactory; -import com.metamx.druid.aggregation.MinAggregatorFactory; -import io.druid.segment.MetricSelectorFactory; - -import java.util.Comparator; -import java.util.List; - -/** - * Processing related interface - * - * An AggregatorFactory is an object that knows how to generate an Aggregator using a MetricSelectorFactory. - * - * This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects - * without making any assumptions about how they are pulling values out of the base data. That is, the data is - * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how - * the data is actually stored and accessed. - */ -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="count", value=CountAggregatorFactory.class), - @JsonSubTypes.Type(name="longSum", value=LongSumAggregatorFactory.class), - @JsonSubTypes.Type(name="doubleSum", value=DoubleSumAggregatorFactory.class), - @JsonSubTypes.Type(name="max", value=MaxAggregatorFactory.class), - @JsonSubTypes.Type(name="min", value=MinAggregatorFactory.class), - @JsonSubTypes.Type(name="javascript", value=JavaScriptAggregatorFactory.class), - @JsonSubTypes.Type(name="histogram", value=HistogramAggregatorFactory.class) -}) -public interface AggregatorFactory -{ - public Aggregator factorize(MetricSelectorFactory metricFactory); - public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory); - public Comparator getComparator(); - - /** - * A method that knows how to combine the outputs of the getIntermediate() method from the Aggregators - * produced via factorize(). Note, even though this is called combine, this method's contract *does* - * allow for mutation of the input objects. Thus, any use of lhs or rhs after calling this method is - * highly discouraged. - * - * @param lhs The left hand side of the combine - * @param rhs The right hand side of the combine - * @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs - */ - public Object combine(Object lhs, Object rhs); - - /** - * Returns an AggregatorFactory that can be used to combine the output of aggregators from this factory. This - * generally amounts to simply creating a new factory that is the same as the current except with its input - * column renamed to the same as the output column. - * - * @return a new Factory that can be used for operations on top of data output from the current factory. - */ - public AggregatorFactory getCombiningFactory(); - - /** - * A method that knows how to "deserialize" the object from whatever form it might have been put into - * in order to transfer via JSON. - * - * @param object the object to deserialize - * @return the deserialized object - */ - public Object deserialize(Object object); - - /** - * "Finalizes" the computation of an object. Primarily useful for complex types that have a different mergeable - * intermediate format than their final resultant output. - * - * @param object the object to be finalized - * @return the finalized value that should be returned for the initial query - */ - public Object finalizeComputation(Object object); - - public String getName(); - public List requiredFields(); - public byte[] getCacheKey(); - public String getTypeName(); - - /** - * Returns the maximum size that this aggregator will require in bytes for intermediate storage of results. - * - * @return the maximum number of bytes that an aggregator of this type will require for intermediate result storage. - */ - public int getMaxIntermediateSize(); - - /** - * Returns the starting value for a corresponding aggregator. For example, 0 for sums, - Infinity for max, an empty mogrifier - * - * @return the starting value for a corresponding aggregator. - */ - public Object getAggregatorStartValue(); -} diff --git a/common/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/BufferAggregator.java deleted file mode 100644 index 0cd6dec6be0..00000000000 --- a/common/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ /dev/null @@ -1,96 +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 io.druid.query.aggregation; - -import java.nio.ByteBuffer; - -/** - * A BufferAggregator is an object that can aggregate metrics into a ByteBuffer. Its aggregation-related methods - * (namely, aggregate(...) and get(...)) only take the ByteBuffer and position because it is assumed that the Aggregator - * was given something (one or more MetricSelector(s)) in its constructor that it can use to get at the next bit of data. - * - * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls - * to aggregate(...). - */ -public interface BufferAggregator -{ - /** - * Initializes the buffer location - * - * Implementations of this method must initialize the byte buffer at the given position - * - * Implementations must not change the position, limit or mark of the given buffer - * - * This method must not exceed the number of bytes returned by {@link AggregatorFactory#getMaxIntermediateSize()} - * in the corresponding {@link AggregatorFactory} - * - * @param buf byte buffer to initialize - * @param position offset within the byte buffer for initialization - */ - void init(ByteBuffer buf, int position); - - /** - * Aggregates metric values into the given aggregate byte representation - * - * Implementations of this method must read in the aggregate value from the buffer at the given position, - * aggregate the next element of data and write the updated aggregate value back into the buffer. - * - * Implementations must not change the position, limit or mark of the given buffer - * - * @param buf byte buffer storing the byte array representation of the aggregate - * @param position offset within the byte buffer at which the current aggregate value is stored - */ - void aggregate(ByteBuffer buf, int position); - - /** - * Returns the intermediate object representation of the given aggregate. - * - * Converts the given byte buffer representation into an intermediate aggregate Object - * - * Implementations must not change the position, limit or mark of the given buffer - * - * @param buf byte buffer storing the byte array representation of the aggregate - * @param position offset within the byte buffer at which the aggregate value is stored - * @return the Object representation of the aggregate - */ - Object get(ByteBuffer buf, int position); - - /** - * Returns the float representation of the given aggregate byte array - * - * Converts the given byte buffer representation into the intermediate aggregate value. - * - * Implementations must not change the position, limit or mark of the given buffer - * - * Implementations are only required to support this method if they are aggregations which - * have an {@link AggregatorFactory#getTypeName()} of "float". - * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. - * - * @param buf byte buffer storing the byte array representation of the aggregate - * @param position offset within the byte buffer at which the aggregate value is stored - * @return the float representation of the aggregate - */ - float getFloat(ByteBuffer buf, int position); - - /** - * Release any resources used by the aggregator - */ - void close(); -} diff --git a/common/src/main/java/io/druid/segment/ComplexMetricSelector.java b/common/src/main/java/io/druid/segment/ComplexMetricSelector.java deleted file mode 100644 index b27e0395f8c..00000000000 --- a/common/src/main/java/io/druid/segment/ComplexMetricSelector.java +++ /dev/null @@ -1,28 +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 io.druid.segment; - -/** - */ -public interface ComplexMetricSelector -{ - public Class classOfObject(); - public T get(); -} diff --git a/common/src/main/java/io/druid/segment/FloatMetricSelector.java b/common/src/main/java/io/druid/segment/FloatMetricSelector.java deleted file mode 100644 index 714b4efa39f..00000000000 --- a/common/src/main/java/io/druid/segment/FloatMetricSelector.java +++ /dev/null @@ -1,30 +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 io.druid.segment; - -/** - * An object that gets a metric value. Metric values are always floats and there is an assumption that the - * FloatMetricSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls - * to get() (though, that doesn't have to be the case if you always want the same value...). - */ -public interface FloatMetricSelector -{ - public float get(); -} diff --git a/common/src/main/java/io/druid/segment/LogicalSegment.java b/common/src/main/java/io/druid/segment/LogicalSegment.java deleted file mode 100644 index 8add65414fa..00000000000 --- a/common/src/main/java/io/druid/segment/LogicalSegment.java +++ /dev/null @@ -1,8 +0,0 @@ -package io.druid.segment; - -import org.joda.time.Interval; - -public interface LogicalSegment -{ - public Interval getInterval(); -} diff --git a/common/src/main/java/io/druid/segment/MetricSelectorFactory.java b/common/src/main/java/io/druid/segment/MetricSelectorFactory.java deleted file mode 100644 index 4851c3e0437..00000000000 --- a/common/src/main/java/io/druid/segment/MetricSelectorFactory.java +++ /dev/null @@ -1,30 +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 io.druid.segment; - -/** - * Factory class for MetricSelectors - */ -public interface MetricSelectorFactory -{ - public FloatMetricSelector makeFloatMetricSelector(String metricName); - public ComplexMetricSelector makeComplexMetricSelector(String metricName); - public ObjectMetricSelector makeObjectColumnSelector(String columnName); -} diff --git a/common/src/main/java/io/druid/segment/ObjectMetricSelector.java b/common/src/main/java/io/druid/segment/ObjectMetricSelector.java deleted file mode 100644 index 6bd28607323..00000000000 --- a/common/src/main/java/io/druid/segment/ObjectMetricSelector.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2013 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 io.druid.segment; - -public interface ObjectMetricSelector -{ - public Class classOfObject(); - public T get(); -} - diff --git a/indexing-common/src/main/java/io/druid/data/Indexed.java b/indexing-common/src/main/java/io/druid/data/Indexed.java deleted file mode 100644 index 43a4aad25af..00000000000 --- a/indexing-common/src/main/java/io/druid/data/Indexed.java +++ /dev/null @@ -1,33 +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 io.druid.data; - -/** - * An Integer-indexed random-access collection. - * Typically wraps an {@link Indexed}. - * - * @param - */ -public interface Indexed extends Iterable { - Class getClazz(); - int size(); - T get(int index); - int indexOf(T value); -} diff --git a/indexing-common/src/main/java/io/druid/data/IndexedFloats.java b/indexing-common/src/main/java/io/druid/data/IndexedFloats.java deleted file mode 100644 index 74b337f0796..00000000000 --- a/indexing-common/src/main/java/io/druid/data/IndexedFloats.java +++ /dev/null @@ -1,32 +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 io.druid.data; - -import java.io.Closeable; - -/** - * Get a float at an index (array or list lookup abstraction without boxing). - */ -public interface IndexedFloats extends Closeable -{ - public int size(); - public float get(int index); - public void fill(int index, float[] toFill); -} diff --git a/indexing-common/src/main/java/io/druid/data/IndexedInts.java b/indexing-common/src/main/java/io/druid/data/IndexedInts.java deleted file mode 100644 index 287eb30832d..00000000000 --- a/indexing-common/src/main/java/io/druid/data/IndexedInts.java +++ /dev/null @@ -1,30 +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 io.druid.data; - -/** - * Get a int an index (array or list lookup abstraction without boxing). - * Typically wraps an {@link Indexed}. - */ -public interface IndexedInts extends Iterable -{ - int size(); - int get(int index); -} diff --git a/indexing-common/src/main/java/io/druid/data/IndexedLongs.java b/indexing-common/src/main/java/io/druid/data/IndexedLongs.java deleted file mode 100644 index e777718060f..00000000000 --- a/indexing-common/src/main/java/io/druid/data/IndexedLongs.java +++ /dev/null @@ -1,34 +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 io.druid.data; - -import java.io.Closeable; - -/** - * Get a long at an index (array or list lookup abstraction without boxing). - */ -public interface IndexedLongs extends Closeable -{ - public int size(); - public long get(int index); - public void fill(int index, long[] toFill); - int binarySearch(long key); - int binarySearch(long key, int from, int to); -} diff --git a/indexing-common/src/main/java/io/druid/segment/ColumnSelector.java b/indexing-common/src/main/java/io/druid/segment/ColumnSelector.java deleted file mode 100644 index b23655364d2..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/ColumnSelector.java +++ /dev/null @@ -1,30 +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 io.druid.segment; - -import io.druid.segment.column.Column; - -/** - */ -public interface ColumnSelector -{ - public Column getTimeColumn(); - public Column getColumn(String columnName); -} diff --git a/indexing-common/src/main/java/io/druid/segment/QueryableIndex.java b/indexing-common/src/main/java/io/druid/segment/QueryableIndex.java deleted file mode 100644 index 6d44382b047..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/QueryableIndex.java +++ /dev/null @@ -1,42 +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 io.druid.segment; - -import io.druid.data.Indexed; -import org.joda.time.Interval; - -import java.io.IOException; - -/** - */ -public interface QueryableIndex extends ColumnSelector -{ - public Interval getDataInterval(); - public int getNumRows(); - public Indexed getColumnNames(); - public Indexed getAvailableDimensions(); - - /** - * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. - * @throws IOException - */ - @Deprecated - public void close() throws IOException; -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java b/indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java deleted file mode 100644 index a1a39666362..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/BitmapIndex.java +++ /dev/null @@ -1,33 +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 io.druid.segment.column; - -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; - -/** - */ -public interface BitmapIndex -{ - public int getCardinality(); - public String getValue(int index); - public boolean hasNulls(); - public ImmutableConciseSet getConciseSet(String value); - public ImmutableConciseSet getConciseSet(int idx); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/Column.java b/indexing-common/src/main/java/io/druid/segment/column/Column.java deleted file mode 100644 index 972dbb705f1..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/Column.java +++ /dev/null @@ -1,35 +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 io.druid.segment.column; - -/** - */ -public interface Column -{ - public ColumnCapabilities getCapabilities(); - - public int getLength(); - public DictionaryEncodedColumn getDictionaryEncoding(); - public RunLengthColumn getRunLengthColumn(); - public GenericColumn getGenericColumn(); - public ComplexColumn getComplexColumn(); - public BitmapIndex getBitmapIndex(); - public SpatialIndex getSpatialIndex(); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java deleted file mode 100644 index 53efd3fe7d7..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/ColumnCapabilities.java +++ /dev/null @@ -1,33 +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 io.druid.segment.column; - -/** - */ -public interface ColumnCapabilities -{ - public ValueType getType(); - - public boolean isDictionaryEncoded(); - public boolean isRunLengthEncoded(); - public boolean hasBitmapIndexes(); - public boolean hasSpatialIndexes(); - public boolean hasMultipleValues(); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java b/indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java deleted file mode 100644 index f68a2ebdf59..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/ComplexColumn.java +++ /dev/null @@ -1,31 +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 io.druid.segment.column; - -import java.io.Closeable; - -/** - */ -public interface ComplexColumn extends Closeable -{ - public Class getClazz(); - public String getTypeName(); - public Object getRowValue(int rowNum); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java deleted file mode 100644 index ba8953057fe..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ /dev/null @@ -1,35 +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 io.druid.segment.column; - -import io.druid.data.IndexedInts; - -/** - */ -public interface DictionaryEncodedColumn -{ - public int length(); - public boolean hasMultipleValues(); - public int getSingleValueRow(int rowNum); - public IndexedInts getMultiValueRow(int rowNum); - public String lookupName(int id); - public int lookupId(String name); - public int getCardinality(); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java b/indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java deleted file mode 100644 index 63a53a7d15c..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/GenericColumn.java +++ /dev/null @@ -1,42 +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 io.druid.segment.column; - -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; -import io.druid.data.IndexedLongs; - -import java.io.Closeable; - -/** - */ -public interface GenericColumn extends Closeable -{ - public int length(); - public ValueType getType(); - public boolean hasMultipleValues(); - - public String getStringSingleValueRow(int rowNum); - public Indexed getStringMultiValueRow(int rowNum); - public float getFloatSingleValueRow(int rowNum); - public IndexedFloats getFloatMultiValueRow(int rowNum); - public long getLongSingleValueRow(int rowNum); - public IndexedLongs getLongMultiValueRow(int rowNum); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java b/indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java deleted file mode 100644 index 83339aa9c05..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/RunLengthColumn.java +++ /dev/null @@ -1,27 +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 io.druid.segment.column; - -/** - */ -public interface RunLengthColumn -{ - public void thisIsAFictionalInterfaceThatWillHopefullyMeanSomethingSometime(); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java b/indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java deleted file mode 100644 index b6607ff8162..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/SpatialIndex.java +++ /dev/null @@ -1,28 +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 io.druid.segment.column; - -import com.metamx.collections.spatial.ImmutableRTree; - -/** - */ -public interface SpatialIndex -{ - public ImmutableRTree getRTree(); -} diff --git a/indexing-common/src/main/java/io/druid/segment/column/ValueType.java b/indexing-common/src/main/java/io/druid/segment/column/ValueType.java deleted file mode 100644 index 8d4eae81e9c..00000000000 --- a/indexing-common/src/main/java/io/druid/segment/column/ValueType.java +++ /dev/null @@ -1,11 +0,0 @@ -package io.druid.segment.column; - -/** -*/ -public enum ValueType -{ - FLOAT, - LONG, - STRING, - COMPLEX -} diff --git a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java b/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java index 1c50dc1837d..d5acb100f05 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java @@ -20,10 +20,10 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java b/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java index 9e75e68c779..87e40e2a715 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java @@ -23,10 +23,10 @@ import com.google.common.base.Function; import com.google.common.base.Predicate; import com.metamx.common.guava.FunctionalIterable; import io.druid.data.Indexed; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java index a12da1b358d..e12f067e211 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java @@ -22,10 +22,10 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.data.Indexed; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java b/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java index 1f67d72f970..1288b35bfa9 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java @@ -3,10 +3,10 @@ package com.metamx.druid.index.brita; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.metamx.common.guava.FunctionalIterable; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.mozilla.javascript.Context; import org.mozilla.javascript.Function; diff --git a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java b/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java index e3fe4d6bd52..39f5f9d5f4e 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java @@ -19,10 +19,10 @@ package com.metamx.druid.index.brita; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java b/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java index 5afe92a5ceb..31a861258b4 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java @@ -20,10 +20,10 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java index 1258fdb3dc7..13b0cdd6c07 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java @@ -19,10 +19,10 @@ package com.metamx.druid.index.brita; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java index a7ea0696368..bfe74c0e329 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java @@ -19,10 +19,10 @@ package com.metamx.druid.index.brita; import com.metamx.collections.spatial.search.Bound; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java index dbb96def335..88fb9dc406f 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java @@ -39,9 +39,9 @@ import com.metamx.druid.query.search.SearchQuerySpec; import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.Aggregator; -import io.druid.query.aggregation.ValueMatcherFactory; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; +import io.druid.query.filter.ValueMatcherFactory; import io.druid.segment.Capabilities; import io.druid.segment.ComplexMetricSelector; import io.druid.segment.Cursor; diff --git a/server/src/main/java/io/druid/initialization/Binders.java b/server/src/main/java/io/druid/initialization/Binders.java deleted file mode 100644 index b4f9ce82d8c..00000000000 --- a/server/src/main/java/io/druid/initialization/Binders.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.initialization; - -import com.google.inject.Binder; -import com.google.inject.TypeLiteral; -import com.google.inject.multibindings.MapBinder; -import io.druid.query.Query; -import io.druid.query.QueryRunnerFactory; - -/** - */ -public class Binders -{ - public static MapBinder, QueryRunnerFactory> queryFactoryBinder(Binder binder) - { - return MapBinder.newMapBinder( - binder, new TypeLiteral>(){}, new TypeLiteral(){} - ); - } -} diff --git a/server/src/main/java/io/druid/query/QueryRunnerFactory.java b/server/src/main/java/io/druid/query/QueryRunnerFactory.java deleted file mode 100644 index 0eee7a579aa..00000000000 --- a/server/src/main/java/io/druid/query/QueryRunnerFactory.java +++ /dev/null @@ -1,33 +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 io.druid.query; - -import io.druid.segment.Segment; - -import java.util.concurrent.ExecutorService; - -/** - */ -public interface QueryRunnerFactory> -{ - public QueryRunner createRunner(Segment segment); - public QueryRunner mergeRunners(ExecutorService queryExecutor, Iterable> queryRunners); - public QueryToolChest getToolchest(); -} diff --git a/server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java b/server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java deleted file mode 100644 index 6d5e5ba931a..00000000000 --- a/server/src/main/java/io/druid/query/aggregation/ValueMatcherFactory.java +++ /dev/null @@ -1,33 +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 io.druid.query.aggregation; - -import com.google.common.base.Predicate; -import com.metamx.collections.spatial.search.Bound; -import io.druid.query.filter.ValueMatcher; - -/** - */ -public interface ValueMatcherFactory -{ - public ValueMatcher makeValueMatcher(String dimension, String value); - public ValueMatcher makeValueMatcher(String dimension, Predicate value); - public ValueMatcher makeValueMatcher(String dimension, Bound bound); -} diff --git a/server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java b/server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java deleted file mode 100644 index 9a7cb2e70df..00000000000 --- a/server/src/main/java/io/druid/query/filter/BitmapIndexSelector.java +++ /dev/null @@ -1,35 +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 io.druid.query.filter; - -import com.metamx.collections.spatial.ImmutableRTree; -import io.druid.data.Indexed; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; - -/** - */ -public interface BitmapIndexSelector -{ - public Indexed getDimensionValues(String dimension); - public int getNumRows(); - public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value); - public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx); - public ImmutableRTree getSpatialIndex(String dimension); -} diff --git a/server/src/main/java/io/druid/query/filter/Filter.java b/server/src/main/java/io/druid/query/filter/Filter.java deleted file mode 100644 index 08446a1ee2f..00000000000 --- a/server/src/main/java/io/druid/query/filter/Filter.java +++ /dev/null @@ -1,31 +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 io.druid.query.filter; - -import io.druid.query.aggregation.ValueMatcherFactory; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; - -/** - */ -public interface Filter -{ - public ImmutableConciseSet goConcise(BitmapIndexSelector selector); - public ValueMatcher makeMatcher(ValueMatcherFactory factory); -} diff --git a/server/src/main/java/io/druid/query/filter/ValueMatcher.java b/server/src/main/java/io/druid/query/filter/ValueMatcher.java deleted file mode 100644 index 9ecc8b32a9e..00000000000 --- a/server/src/main/java/io/druid/query/filter/ValueMatcher.java +++ /dev/null @@ -1,27 +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 io.druid.query.filter; - -/** - */ -public interface ValueMatcher -{ - public boolean matches(); -} diff --git a/server/src/main/java/io/druid/segment/Capabilities.java b/server/src/main/java/io/druid/segment/Capabilities.java deleted file mode 100644 index 10bcda912a5..00000000000 --- a/server/src/main/java/io/druid/segment/Capabilities.java +++ /dev/null @@ -1,64 +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 io.druid.segment; - -/** - */ -public class Capabilities -{ - private final boolean dimensionValuesSorted; - - public static CapabilitiesBuilder builder() - { - return new CapabilitiesBuilder(); - } - - private Capabilities( - boolean dimensionValuesSorted - ) - { - this.dimensionValuesSorted = dimensionValuesSorted; - } - - public boolean dimensionValuesSorted() - { - return dimensionValuesSorted; - } - - public static class CapabilitiesBuilder - { - private boolean dimensionValuesSorted = false; - - private CapabilitiesBuilder() {} - - public CapabilitiesBuilder dimensionValuesSorted(boolean value) - { - dimensionValuesSorted = value; - return this; - } - - public Capabilities build() - { - return new Capabilities( - dimensionValuesSorted - ); - } - } -} diff --git a/server/src/main/java/io/druid/segment/Cursor.java b/server/src/main/java/io/druid/segment/Cursor.java deleted file mode 100644 index a5b144c07d1..00000000000 --- a/server/src/main/java/io/druid/segment/Cursor.java +++ /dev/null @@ -1,32 +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 io.druid.segment; - -import org.joda.time.DateTime; - -/** - */ -public interface Cursor extends MetricSelectorFactory, DimensionSelectorFactory -{ - public DateTime getTime(); - public void advance(); - public boolean isDone(); - public void reset(); -} diff --git a/server/src/main/java/io/druid/segment/CursorFactory.java b/server/src/main/java/io/druid/segment/CursorFactory.java deleted file mode 100644 index fe8c722053a..00000000000 --- a/server/src/main/java/io/druid/segment/CursorFactory.java +++ /dev/null @@ -1,31 +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 io.druid.segment; - -import io.druid.granularity.QueryGranularity; -import io.druid.query.filter.Filter; -import org.joda.time.Interval; - -/** - */ -public interface CursorFactory -{ - public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran); -} diff --git a/server/src/main/java/io/druid/segment/DimensionSelector.java b/server/src/main/java/io/druid/segment/DimensionSelector.java deleted file mode 100644 index fb24be75f7e..00000000000 --- a/server/src/main/java/io/druid/segment/DimensionSelector.java +++ /dev/null @@ -1,84 +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 io.druid.segment; - -import io.druid.data.IndexedInts; - -/** - */ -public interface DimensionSelector -{ - /** - * Gets all values for the row inside of an IntBuffer. I.e. one possible implementation could be - * - * return IntBuffer.wrap(lookupExpansion(get()); - * - * @return all values for the row as an IntBuffer - */ - public IndexedInts getRow(); - - /** - * Value cardinality is the cardinality of the different occurring values. If there were 4 rows: - * - * A,B - * A - * B - * A - * - * Value cardinality would be 2. - * - * @return - */ - public int getValueCardinality(); - - /** - * The Name is the String name of the actual field. It is assumed that storage layers convert names - * into id values which can then be used to get the string value. For example - * - * A,B - * A - * A,B - * B - * - * getRow() would return - * - * getRow(0) => [0 1] - * getRow(1) => [0] - * getRow(2) => [0 1] - * getRow(3) => [1] - * - * and then lookupName would return: - * - * lookupName(0) => A - * lookupName(1) => B - * - * @param id - * @return - */ - public String lookupName(int id); - - /** - * The ID is the int id value of the field. - * - * @param name - * @return - */ - public int lookupId(String name); -} diff --git a/server/src/main/java/io/druid/segment/DimensionSelectorFactory.java b/server/src/main/java/io/druid/segment/DimensionSelectorFactory.java deleted file mode 100644 index a3ad9d2da91..00000000000 --- a/server/src/main/java/io/druid/segment/DimensionSelectorFactory.java +++ /dev/null @@ -1,27 +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 io.druid.segment; - -/** - */ -public interface DimensionSelectorFactory -{ - public DimensionSelector makeDimensionSelector(String dimensionName); -} diff --git a/server/src/main/java/io/druid/segment/Segment.java b/server/src/main/java/io/druid/segment/Segment.java deleted file mode 100644 index 70bb88aa743..00000000000 --- a/server/src/main/java/io/druid/segment/Segment.java +++ /dev/null @@ -1,34 +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 io.druid.segment; - -import org.joda.time.Interval; - -import java.io.Closeable; - -/** - */ -public interface Segment extends Closeable -{ - public String getIdentifier(); - public Interval getDataInterval(); - public QueryableIndex asQueryableIndex(); - public StorageAdapter asStorageAdapter(); -} diff --git a/server/src/main/java/io/druid/segment/StorageAdapter.java b/server/src/main/java/io/druid/segment/StorageAdapter.java deleted file mode 100644 index 307212a7b98..00000000000 --- a/server/src/main/java/io/druid/segment/StorageAdapter.java +++ /dev/null @@ -1,36 +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 io.druid.segment; - -import com.metamx.druid.index.v1.Searchable; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -/** - */ -public interface StorageAdapter extends CursorFactory, Searchable -{ - public String getSegmentIdentifier(); - public Interval getInterval(); - public int getDimensionCardinality(String dimension); - public DateTime getMinTime(); - public DateTime getMaxTime(); - public Capabilities getCapabilities(); -} From c02d887cfe919381da7709477fc133ce107d5a62 Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 28 Aug 2013 18:50:40 -0500 Subject: [PATCH 49/92] 1) Rewrite SearchQueryRunner to not require StorageAdapter to be "Searchable" 2) Extract SearchQueryRunner out of SearchQueryRunnerFactory 3) Extract ColumnSelectorBitmapIndexSelector out to make it reusable --- .../com/metamx/druid/sql/antlr4/DruidSQL.g4 | 52 ++++- .../main/java/com/metamx/druid/Druids.java | 6 +- .../query/search/FragmentSearchQuerySpec.java | 13 +- .../InsensitiveContainsSearchQuerySpec.java | 13 +- .../druid/query/search/SearchQuery.java | 4 +- .../druid/query/search/SearchQuerySpec.java | 12 -- .../SearchQuerySpecDimExtractionFnTest.java | 3 +- .../com/metamx/druid/BaseStorageAdapter.java | 93 --------- .../v1/ColumnSelectorBitmapIndexSelector.java | 143 +++++++++++++ .../v1/IncrementalIndexStorageAdapter.java | 101 +-------- .../v1/QueryableIndexStorageAdapter.java | 195 +----------------- .../v1/SegmentIdAttachedStorageAdapter.java | 100 --------- .../druid/query/search/SearchQueryRunner.java | 165 +++++++++++++++ .../search/SearchQueryRunnerFactory.java | 54 ----- .../query/search/SearchQueryRunnerTest.java | 12 +- 15 files changed, 381 insertions(+), 585 deletions(-) delete mode 100644 server/src/main/java/com/metamx/druid/BaseStorageAdapter.java create mode 100644 server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java delete mode 100644 server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java create mode 100644 server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java diff --git a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 index 9dd297dc8c0..d1ce41a1cc5 100644 --- a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 +++ b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 @@ -1,18 +1,50 @@ grammar DruidSQL; @header { -import com.metamx.druid.aggregation.post.*; -import com.metamx.druid.aggregation.*; -import com.metamx.druid.query.filter.*; -import com.metamx.druid.query.dimension.*; -import com.metamx.druid.*; - -import com.google.common.base.*; +import com.google.common.base.Joiner; import com.google.common.collect.Lists; -import org.joda.time.*; +import com.metamx.druid.aggregation.CountAggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.aggregation.MaxAggregatorFactory; +import com.metamx.druid.aggregation.MinAggregatorFactory; +import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; +import com.metamx.druid.aggregation.post.ConstantPostAggregator; +import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; +import com.metamx.druid.aggregation.post.PostAggregator; +import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; +import com.metamx.druid.query.filter.AndDimFilter; +import com.metamx.druid.query.filter.DimFilter; +import com.metamx.druid.query.filter.NotDimFilter; +import com.metamx.druid.query.filter.OrDimFilter; +import com.metamx.druid.query.filter.RegexDimFilter; +import com.metamx.druid.query.filter.SelectorDimFilter; +import io.druid.granularity.PeriodGranularity; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import org.antlr.v4.runtime.NoViableAltException; +import org.antlr.v4.runtime.Parser; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.RecognitionException; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenStream; +import org.antlr.v4.runtime.atn.ATN; +import org.antlr.v4.runtime.atn.ATNSimulator; +import org.antlr.v4.runtime.atn.ParserATNSimulator; +import org.antlr.v4.runtime.atn.PredictionContextCache; +import org.antlr.v4.runtime.dfa.DFA; +import org.antlr.v4.runtime.tree.ParseTreeListener; +import org.antlr.v4.runtime.tree.TerminalNode; +import org.joda.time.DateTime; +import org.joda.time.Period; -import java.text.*; -import java.util.*; +import java.text.NumberFormat; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; } @parser::members { diff --git a/client/src/main/java/com/metamx/druid/Druids.java b/client/src/main/java/com/metamx/druid/Druids.java index dd02ad85d32..53ad282a036 100644 --- a/client/src/main/java/com/metamx/druid/Druids.java +++ b/client/src/main/java/com/metamx/druid/Druids.java @@ -524,7 +524,7 @@ public class Druids querySegmentSpec, dimensions, querySpec, - querySpec.getSearchSortSpec(), + null, context ); } @@ -639,13 +639,13 @@ public class Druids public SearchQueryBuilder query(String q) { - querySpec = new InsensitiveContainsSearchQuerySpec(q, null); + querySpec = new InsensitiveContainsSearchQuerySpec(q); return this; } public SearchQueryBuilder query(Map q) { - querySpec = new InsensitiveContainsSearchQuerySpec((String) q.get("value"), null); + querySpec = new InsensitiveContainsSearchQuerySpec((String) q.get("value")); return this; } diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index c4a7b707712..3abf19d5124 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -34,12 +34,10 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec private static final byte CACHE_TYPE_ID = 0x2; private final List values; - private final SearchSortSpec sortSpec; @JsonCreator public FragmentSearchQuerySpec( - @JsonProperty("values") List values, - @JsonProperty("sort") SearchSortSpec sortSpec + @JsonProperty("values") List values ) { this.values = Lists.transform( @@ -53,7 +51,6 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec } } ); - this.sortSpec = (sortSpec == null) ? new LexicographicSearchSortSpec() : sortSpec; } @JsonProperty @@ -62,13 +59,6 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec return values; } - @JsonProperty("sort") - @Override - public SearchSortSpec getSearchSortSpec() - { - return sortSpec; - } - @Override public boolean accept(String dimVal) { @@ -107,7 +97,6 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec { return "FragmentSearchQuerySpec{" + "values=" + values + - ", sortSpec=" + sortSpec + "}"; } } diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 80c609cae76..44b0ee07197 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -31,16 +31,13 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec private static final byte CACHE_TYPE_ID = 0x1; private final String value; - private final SearchSortSpec sortSpec; @JsonCreator public InsensitiveContainsSearchQuerySpec( - @JsonProperty("value") String value, - @JsonProperty("sort") SearchSortSpec sortSpec + @JsonProperty("value") String value ) { this.value = value.toLowerCase(); - this.sortSpec = (sortSpec == null) ? new LexicographicSearchSortSpec() : sortSpec; } @JsonProperty @@ -49,13 +46,6 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec return value; } - @JsonProperty("sort") - @Override - public SearchSortSpec getSearchSortSpec() - { - return sortSpec; - } - @Override public boolean accept(String dimVal) { @@ -81,7 +71,6 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec { return "InsensitiveContainsSearchQuerySpec{" + "value=" + value + - ", sortSpec=" + sortSpec + "}"; } } diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index b81e5e54737..8702156598b 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -62,7 +62,7 @@ public class SearchQuery extends BaseQuery> { super(dataSource, querySegmentSpec, context); this.dimFilter = dimFilter; - this.sortSpec = sortSpec; + this.sortSpec = sortSpec == null ? new LexicographicSearchSortSpec() : sortSpec; this.granularity = granularity == null ? QueryGranularity.ALL : granularity; this.limit = (limit == 0) ? 1000 : limit; this.dimensions = (dimensions == null) ? null : Lists.transform( @@ -159,7 +159,7 @@ public class SearchQuery extends BaseQuery> @JsonProperty("sort") public SearchSortSpec getSort() { - return sortSpec == null ? querySpec.getSearchSortSpec() : sortSpec; + return sortSpec; } public SearchQuery withLimit(int newLimit) diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java index 68eb86dfb73..a54aae35ea8 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java @@ -31,18 +31,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; }) public interface SearchQuerySpec { - /** - * Deprecated! - * - * This has been moved to the SearchQuery and is only still here for backwards compatibility purposes. Search - * queries should be adjusted to use the sort parameter on the SearchQuery object itself rather than on this - * object. This method will eventually go away. - * - * @return - */ - @Deprecated - public SearchSortSpec getSearchSortSpec(); - public boolean accept(String dimVal); public byte[] getCacheKey(); diff --git a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java index 808c3603b6b..06e9cde0ffd 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; import com.metamx.druid.query.search.FragmentSearchQuerySpec; -import com.metamx.druid.query.search.LexicographicSearchSortSpec; import com.metamx.druid.query.search.SearchQuerySpec; import org.junit.Assert; import org.junit.Test; @@ -49,7 +48,7 @@ public class SearchQuerySpecDimExtractionFnTest public void testExtraction() { SearchQuerySpec spec = new FragmentSearchQuerySpec( - Arrays.asList("to", "yo"), new LexicographicSearchSortSpec() + Arrays.asList("to", "yo") ); DimExtractionFn dimExtractionFn = new SearchQuerySpecDimExtractionFn(spec); List expected = Arrays.asList("Kyoto", "Tokyo", "Toyokawa", "Yorktown"); diff --git a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java b/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java deleted file mode 100644 index 34ec2986b63..00000000000 --- a/server/src/main/java/com/metamx/druid/BaseStorageAdapter.java +++ /dev/null @@ -1,93 +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; - -import com.google.common.collect.Sets; -import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.index.v1.ConciseOffset; -import com.metamx.druid.index.v1.processing.IntersectingOffset; -import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQuerySpec; -import io.druid.data.Indexed; -import io.druid.query.filter.Filter; -import io.druid.segment.StorageAdapter; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; - -import java.util.List; -import java.util.TreeSet; - -/** - */ -public abstract class BaseStorageAdapter implements StorageAdapter -{ - public abstract Indexed getAvailableDimensions(); - - public abstract Indexed getDimValueLookup(String dimension); - - public abstract ImmutableConciseSet getInvertedIndex(String dimension, String dimVal); - - public abstract ImmutableConciseSet getInvertedIndex(String dimension, int idx); - - public abstract Offset getFilterOffset(Filter filter); - - @Override - public Iterable searchDimensions(final SearchQuery query, final Filter filter) - { - final List dimensions = query.getDimensions(); - final SearchQuerySpec searchQuerySpec = query.getQuery(); - - final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); - - Iterable dimsToSearch; - if (dimensions == null || dimensions.isEmpty()) { - dimsToSearch = getAvailableDimensions(); - } else { - dimsToSearch = dimensions; - } - - Offset filterOffset = (filter == null) ? null : getFilterOffset(filter); - - for (String dimension : dimsToSearch) { - Iterable dims = getDimValueLookup(dimension); - if (dims != null) { - for (String dimVal : dims) { - dimVal = dimVal == null ? "" : dimVal; - if (searchQuerySpec.accept(dimVal)) { - if (filterOffset != null) { - Offset lhs = new ConciseOffset(getInvertedIndex(dimension, dimVal)); - Offset rhs = filterOffset.clone(); - - if (new IntersectingOffset(lhs, rhs).withinBounds()) { - retVal.add(new SearchHit(dimension, dimVal)); - } - } else { - retVal.add(new SearchHit(dimension, dimVal)); - } - } - } - } - } - - return new FunctionalIterable(retVal).limit(query.getLimit()); - } -} - diff --git a/server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java b/server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java new file mode 100644 index 00000000000..6489655a59a --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java @@ -0,0 +1,143 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.index.v1; + +import com.google.common.io.Closeables; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.druid.kv.IndexedIterable; +import io.druid.data.Indexed; +import io.druid.query.filter.BitmapIndexSelector; +import io.druid.segment.ColumnSelector; +import io.druid.segment.column.Column; +import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.GenericColumn; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +import java.util.Iterator; + +/** +*/ +public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector +{ + private final ColumnSelector index; + + public ColumnSelectorBitmapIndexSelector( + final ColumnSelector index + ) + { + this.index = index; + } + + @Override + public Indexed getDimensionValues(String dimension) + { + final Column columnDesc = index.getColumn(dimension.toLowerCase()); + if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) { + return null; + } + final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + return new Indexed() + { + @Override + public Class getClazz() + { + return String.class; + } + + @Override + public int size() + { + return column.getCardinality(); + } + + @Override + public String get(int index) + { + return column.lookupName(index); + } + + @Override + public int indexOf(String value) + { + return column.lookupId(value); + } + + @Override + public Iterator iterator() + { + return IndexedIterable.create(this).iterator(); + } + }; + } + + @Override + public int getNumRows() + { + GenericColumn column = null; + try { + column = index.getTimeColumn().getGenericColumn(); + return column.length(); + } + finally { + Closeables.closeQuietly(column); + } + } + + @Override + public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value) + { + final Column column = index.getColumn(dimension.toLowerCase()); + if (column == null) { + return new ImmutableConciseSet(); + } + if (!column.getCapabilities().hasBitmapIndexes()) { + return new ImmutableConciseSet(); + } + + return column.getBitmapIndex().getConciseSet(value); + } + + @Override + public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx) + { + final Column column = index.getColumn(dimension.toLowerCase()); + if (column == null) { + return new ImmutableConciseSet(); + } + if (!column.getCapabilities().hasBitmapIndexes()) { + return new ImmutableConciseSet(); + } + // This is a workaround given the current state of indexing, I feel shame + final int index1 = column.getBitmapIndex().hasNulls() ? idx + 1 : idx; + + return column.getBitmapIndex().getConciseSet(index1); + } + + @Override + public ImmutableRTree getSpatialIndex(String dimension) + { + final Column column = index.getColumn(dimension.toLowerCase()); + if (column == null || !column.getCapabilities().hasSpatialIndexes()) { + return new ImmutableRTree(); + } + + return column.getSpatialIndex().getRTree(); + } +} diff --git a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java index 88fb9dc406f..80a734fe8ea 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java @@ -25,17 +25,13 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.metamx.collections.spatial.search.Bound; -import com.metamx.common.IAE; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterator; import com.metamx.druid.index.brita.BooleanValueMatcher; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQuerySpec; +import com.metamx.druid.kv.ListIndexed; +import io.druid.data.Indexed; import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.Aggregator; @@ -57,7 +53,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.TreeSet; import java.util.concurrent.ConcurrentNavigableMap; /** @@ -87,6 +82,12 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter return index.getInterval(); } + @Override + public Indexed getAvailableDimensions() + { + return new ListIndexed(index.getDimensions(), String.class); + } + @Override public int getDimensionCardinality(String dimension) { @@ -432,92 +433,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter }; } - @Override - public Iterable searchDimensions(final SearchQuery query, final Filter filter) - { - final List dimensions = query.getDimensions(); - final int[] dimensionIndexes; - final String[] dimensionNames; - final List dimensionOrder = index.getDimensions(); - if (dimensions == null || dimensions.isEmpty()) { - dimensionIndexes = new int[dimensionOrder.size()]; - dimensionNames = new String[dimensionIndexes.length]; - - Iterator dimensionOrderIter = dimensionOrder.iterator(); - for (int i = 0; i < dimensionIndexes.length; ++i) { - dimensionNames[i] = dimensionOrderIter.next(); - dimensionIndexes[i] = index.getDimensionIndex(dimensionNames[i]); - } - } else { - int[] tmpDimensionIndexes = new int[dimensions.size()]; - String[] tmpDimensionNames = new String[dimensions.size()]; - int i = 0; - for (String dimension : dimensions) { - Integer dimIndex = index.getDimensionIndex(dimension.toLowerCase()); - if (dimIndex != null) { - tmpDimensionNames[i] = dimension; - tmpDimensionIndexes[i] = dimIndex; - ++i; - } - } - - if (i != tmpDimensionIndexes.length) { - dimensionIndexes = new int[i]; - dimensionNames = new String[i]; - System.arraycopy(tmpDimensionIndexes, 0, dimensionIndexes, 0, i); - System.arraycopy(tmpDimensionNames, 0, dimensionNames, 0, i); - } else { - dimensionIndexes = tmpDimensionIndexes; - dimensionNames = tmpDimensionNames; - } - } - - final List queryIntervals = query.getIntervals(); - if (queryIntervals.size() != 1) { - throw new IAE("Can only handle one interval, got query[%s]", query); - } - - final Interval queryInterval = queryIntervals.get(0); - final long intervalStart = queryInterval.getStartMillis(); - final long intervalEnd = queryInterval.getEndMillis(); - - final EntryHolder holder = new EntryHolder(); - final ValueMatcher theMatcher = makeFilterMatcher(filter, holder); - final SearchQuerySpec searchQuerySpec = query.getQuery(); - final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); - - ConcurrentNavigableMap facts = index.getSubMap( - new IncrementalIndex.TimeAndDims(intervalStart, new String[][]{}), - new IncrementalIndex.TimeAndDims(intervalEnd, new String[][]{}) - ); - - for (Map.Entry entry : facts.entrySet()) { - holder.set(entry); - final IncrementalIndex.TimeAndDims key = holder.getKey(); - final long timestamp = key.getTimestamp(); - - if (timestamp >= intervalStart && timestamp < intervalEnd && theMatcher.matches()) { - final String[][] dims = key.getDims(); - - for (int i = 0; i < dimensionIndexes.length; ++i) { - if (dimensionIndexes[i] < dims.length) { - final String[] dimVals = dims[dimensionIndexes[i]]; - if (dimVals != null) { - for (int j = 0; j < dimVals.length; ++j) { - if (searchQuerySpec.accept(dimVals[j])) { - retVal.add(new SearchHit(dimensionNames[i], dimVals[j])); - } - } - } - } - } - } - } - - - return new FunctionalIterable(retVal).limit(query.getLimit()); - } - private ValueMatcher makeFilterMatcher(final Filter filter, final EntryHolder holder) { return filter == null diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java index 8129b57c368..817ac8a9dec 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java +++ b/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java @@ -24,18 +24,14 @@ import com.google.common.base.Functions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.io.Closeables; -import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.collect.MoreIterators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterator; -import com.metamx.druid.BaseStorageAdapter; import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.IndexedIterable; import com.metamx.druid.kv.SingleIndexedInts; import io.druid.data.Indexed; import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; -import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.segment.Capabilities; import io.druid.segment.ColumnSelector; @@ -45,13 +41,13 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatMetricSelector; import io.druid.segment.ObjectMetricSelector; import io.druid.segment.QueryableIndex; +import io.druid.segment.StorageAdapter; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.ValueType; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -61,7 +57,7 @@ import java.util.Map; /** */ -public class QueryableIndexStorageAdapter extends BaseStorageAdapter +public class QueryableIndexStorageAdapter implements StorageAdapter { private final QueryableIndex index; @@ -84,6 +80,12 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter return index.getDataInterval(); } + @Override + public Indexed getAvailableDimensions() + { + return index.getAvailableDimensions(); + } + @Override public int getDimensionCardinality(String dimension) { @@ -155,7 +157,7 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter if (filter == null) { iterable = new NoFilterCursorIterable(index, actualInterval, gran); } else { - Offset offset = new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index))); + Offset offset = new ConciseOffset(filter.goConcise(new ColumnSelectorBitmapIndexSelector(index))); iterable = new CursorIterable(index, actualInterval, gran, offset); } @@ -163,102 +165,6 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter return FunctionalIterable.create(iterable).keep(Functions.identity()); } - @Override - public Indexed getAvailableDimensions() - { - return index.getAvailableDimensions(); - } - - @Override - public Indexed getDimValueLookup(String dimension) - { - final Column column = index.getColumn(dimension.toLowerCase()); - - if (column == null || !column.getCapabilities().isDictionaryEncoded()) { - return null; - } - - final DictionaryEncodedColumn dictionary = column.getDictionaryEncoding(); - return new Indexed() - { - @Override - public Class getClazz() - { - return String.class; - } - - @Override - public int size() - { - return dictionary.getCardinality(); - } - - @Override - public String get(int index) - { - return dictionary.lookupName(index); - } - - @Override - public int indexOf(String value) - { - return dictionary.lookupId(value); - } - - @Override - public Iterator iterator() - { - return IndexedIterable.create(this).iterator(); - } - }; - } - - @Override - public ImmutableConciseSet getInvertedIndex(String dimension, String dimVal) - { - final Column column = index.getColumn(dimension.toLowerCase()); - if (column == null) { - return new ImmutableConciseSet(); - } - if (!column.getCapabilities().hasBitmapIndexes()) { - return new ImmutableConciseSet(); - } - - return column.getBitmapIndex().getConciseSet(dimVal); - } - - @Override - public ImmutableConciseSet getInvertedIndex(String dimension, int idx) - { - final Column column = index.getColumn(dimension.toLowerCase()); - if (column == null) { - return new ImmutableConciseSet(); - } - if (!column.getCapabilities().hasBitmapIndexes()) { - return new ImmutableConciseSet(); - } - // This is a workaround given the current state of indexing, I feel shame - final int index = column.getBitmapIndex().hasNulls() ? idx + 1 : idx; - - return column.getBitmapIndex().getConciseSet(index); - } - - public ImmutableRTree getRTreeSpatialIndex(String dimension) - { - final Column column = index.getColumn(dimension.toLowerCase()); - if (column == null || !column.getCapabilities().hasSpatialIndexes()) { - return new ImmutableRTree(); - } - - return column.getSpatialIndex().getRTree(); - } - - @Override - public Offset getFilterOffset(Filter filter) - { - return new ConciseOffset(filter.goConcise(new MMappedBitmapIndexSelector(index))); - } - private static class CursorIterable implements Iterable { private final ColumnSelector index; @@ -1081,87 +987,4 @@ public class QueryableIndexStorageAdapter extends BaseStorageAdapter return 0; } } - - private class MMappedBitmapIndexSelector implements BitmapIndexSelector - { - private final ColumnSelector index; - - public MMappedBitmapIndexSelector(final ColumnSelector index) - { - this.index = index; - } - - @Override - public Indexed getDimensionValues(String dimension) - { - final Column columnDesc = index.getColumn(dimension.toLowerCase()); - if (columnDesc == null || !columnDesc.getCapabilities().isDictionaryEncoded()) { - return null; - } - final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); - return new Indexed() - { - @Override - public Class getClazz() - { - return String.class; - } - - @Override - public int size() - { - return column.getCardinality(); - } - - @Override - public String get(int index) - { - return column.lookupName(index); - } - - @Override - public int indexOf(String value) - { - return column.lookupId(value); - } - - @Override - public Iterator iterator() - { - return IndexedIterable.create(this).iterator(); - } - }; - } - - @Override - public int getNumRows() - { - GenericColumn column = null; - try { - column = index.getTimeColumn().getGenericColumn(); - return column.length(); - } - finally { - Closeables.closeQuietly(column); - } - } - - @Override - public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value) - { - return getInvertedIndex(dimension, value); - } - - @Override - public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx) - { - return getInvertedIndex(dimension, idx); - } - - @Override - public ImmutableRTree getSpatialIndex(String dimension) - { - return getRTreeSpatialIndex(dimension); - } - } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java b/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java deleted file mode 100644 index 5842cd136bd..00000000000 --- a/server/src/main/java/com/metamx/druid/index/v1/SegmentIdAttachedStorageAdapter.java +++ /dev/null @@ -1,100 +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.index.v1; - -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchQuery; -import io.druid.granularity.QueryGranularity; -import io.druid.query.filter.Filter; -import io.druid.segment.Capabilities; -import io.druid.segment.Cursor; -import io.druid.segment.StorageAdapter; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -/** - */ -public class SegmentIdAttachedStorageAdapter implements StorageAdapter -{ - private final String segmentId; - private final StorageAdapter delegate; - - public SegmentIdAttachedStorageAdapter( - String segmentId, - StorageAdapter delegate - ) - { - this.segmentId = segmentId; - this.delegate = delegate; - } - - @Override - public String getSegmentIdentifier() - { - return segmentId; - } - - @Override - public Interval getInterval() - { - return delegate.getInterval(); - } - - @Override - public Iterable searchDimensions(SearchQuery query, Filter filter) - { - return delegate.searchDimensions(query, filter); - } - - @Override - public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran) - { - return delegate.makeCursors(filter, interval, gran); - } - - @Override - public Capabilities getCapabilities() - { - return delegate.getCapabilities(); - } - - @Override - public DateTime getMaxTime() - { - return delegate.getMaxTime(); - } - - @Override - public DateTime getMinTime() - { - return delegate.getMinTime(); - } - - @Override - public int getDimensionCardinality(String dimension) - { - return delegate.getDimensionCardinality(dimension); - } - - public StorageAdapter getDelegate() - { - return delegate; - } -} diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java new file mode 100644 index 00000000000..2e25720122f --- /dev/null +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java @@ -0,0 +1,165 @@ +package com.metamx.druid.query.search; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.druid.index.brita.Filters; +import com.metamx.druid.index.v1.ColumnSelectorBitmapIndexSelector; +import com.metamx.druid.result.Result; +import com.metamx.druid.result.SearchResultValue; +import com.metamx.emitter.EmittingLogger; +import io.druid.data.IndexedInts; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.filter.Filter; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.QueryableIndex; +import io.druid.segment.Segment; +import io.druid.segment.StorageAdapter; +import io.druid.segment.column.BitmapIndex; +import io.druid.segment.column.Column; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +import java.util.List; +import java.util.Map; +import java.util.TreeSet; + +/** +*/ +public class SearchQueryRunner implements QueryRunner> +{ + private static final EmittingLogger log = new EmittingLogger(SearchQueryRunner.class); + + private final Segment segment; + + public SearchQueryRunner(Segment segment) + { + this.segment = segment; + } + + @Override + public Sequence> run(final Query> input) + { + if (!(input instanceof SearchQuery)) { + throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); + } + + final SearchQuery query = (SearchQuery) input; + final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter()); + final List dimensions = query.getDimensions(); + final SearchQuerySpec searchQuerySpec = query.getQuery(); + final int limit = query.getLimit(); + + final QueryableIndex index = segment.asQueryableIndex(); + if (index != null) { + final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); + + Iterable dimsToSearch; + if (dimensions == null || dimensions.isEmpty()) { + dimsToSearch = index.getAvailableDimensions(); + } else { + dimsToSearch = dimensions; + } + + + final ImmutableConciseSet baseFilter; + if (filter == null) { + // Accept all + baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); + } + else { + baseFilter = filter.goConcise(new ColumnSelectorBitmapIndexSelector(index)); + } + + for (String dimension : dimsToSearch) { + final Column column = index.getColumn(dimension.toLowerCase()); + if (column == null) { + continue; + } + + final BitmapIndex bitmapIndex = column.getBitmapIndex(); + if (bitmapIndex != null) { + for (int i = 0; i < bitmapIndex.getCardinality(); ++i) { + String dimVal = Strings.nullToEmpty(bitmapIndex.getValue(i)); + if (searchQuerySpec.accept(dimVal) && + ImmutableConciseSet.intersection(baseFilter, bitmapIndex.getConciseSet(i)).size() > 0) { + retVal.add(new SearchHit(dimension, dimVal)); + if (retVal.size() >= limit) { + return makeReturnResult(limit, retVal); + } + } + } + } + } + + return makeReturnResult(limit, retVal); + } + + final StorageAdapter adapter = segment.asStorageAdapter(); + if (adapter != null) { + Iterable dimsToSearch; + if (dimensions == null || dimensions.isEmpty()) { + dimsToSearch = adapter.getAvailableDimensions(); + } else { + dimsToSearch = dimensions; + } + + final TreeSet retVal = Sets.newTreeSet(query.getSort().getComparator()); + + final Iterable cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularity.ALL); + for (Cursor cursor : cursors) { + Map dimSelectors = Maps.newHashMap(); + for (String dim : dimsToSearch) { + dimSelectors.put(dim, cursor.makeDimensionSelector(dim)); + } + + while (!cursor.isDone()) { + for (Map.Entry entry : dimSelectors.entrySet()) { + final DimensionSelector selector = entry.getValue(); + final IndexedInts vals = selector.getRow(); + for (int i = 0; i < vals.size(); ++i) { + final String dimVal = selector.lookupName(vals.get(i)); + if (searchQuerySpec.accept(dimVal)) { + retVal.add(new SearchHit(entry.getKey(), dimVal)); + if (retVal.size() >= limit) { + return makeReturnResult(limit, retVal); + } + } + } + } + + cursor.advance(); + } + } + + return makeReturnResult(limit, retVal); + } + + log.makeAlert("WTF!? Unable to process search query on segment.") + .addData("segment", segment.getIdentifier()) + .addData("query", query); + return Sequences.empty(); + } + + private Sequence> makeReturnResult(int limit, TreeSet retVal) + { + return Sequences.simple( + ImmutableList.of( + new Result( + segment.getDataInterval().getStart(), + new SearchResultValue( + Lists.newArrayList(new FunctionalIterable(retVal).limit(limit)) + ) + ) + ) + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java index 53f5b560a17..9df93768f2c 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java @@ -19,24 +19,15 @@ package com.metamx.druid.query.search; -import com.google.common.collect.Iterators; import com.google.inject.Inject; -import com.metamx.common.ISE; -import com.metamx.common.guava.BaseSequence; -import com.metamx.common.guava.Sequence; -import com.metamx.druid.SearchResultBuilder; -import com.metamx.druid.index.brita.Filters; import com.metamx.druid.query.ChainedExecutionQueryRunner; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; import io.druid.segment.Segment; -import io.druid.segment.StorageAdapter; -import java.util.Iterator; import java.util.concurrent.ExecutorService; /** @@ -74,49 +65,4 @@ public class SearchQueryRunnerFactory implements QueryRunnerFactory> - { - private final StorageAdapter adapter; - - public SearchQueryRunner(Segment segment) - { - this.adapter = segment.asStorageAdapter(); - } - - @Override - public Sequence> run(final Query> input) - { - if (!(input instanceof SearchQuery)) { - throw new ISE("Got a [%s] which isn't a %s", input.getClass(), SearchQuery.class); - } - - final SearchQuery query = (SearchQuery) input; - - return new BaseSequence, Iterator>>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return Iterators.singletonIterator( - new SearchResultBuilder( - adapter.getInterval().getStart(), - adapter.searchDimensions( - query, - Filters.convertDimensionFilters(query.getDimensionsFilter()) - ) - ).build() - ); - } - - @Override - public void cleanup(Iterator> toClean) - { - - } - } - ); - } - } } diff --git a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java index 59184d4da60..b0512978b77 100644 --- a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java @@ -94,7 +94,7 @@ public class SearchQueryRunnerTest .dataSource(QueryRunnerTestHelper.dataSource) .granularity(QueryRunnerTestHelper.allGran) .intervals(QueryRunnerTestHelper.fullOnInterval) - .query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve"), null)) + .query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve"))) .build(); Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); @@ -104,7 +104,7 @@ public class SearchQueryRunnerTest } @Test - public void testSearchWithDimension1() + public void testSearchWithDimensionQuality() { Map> expectedResults = new HashMap>(); expectedResults.put( @@ -128,7 +128,7 @@ public class SearchQueryRunnerTest } @Test - public void testSearchWithDimension2() + public void testSearchWithDimensionProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); @@ -146,7 +146,7 @@ public class SearchQueryRunnerTest } @Test - public void testSearchWithDimensions1() + public void testSearchWithDimensionsQualityAndProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); expectedResults.putAll( @@ -182,7 +182,7 @@ public class SearchQueryRunnerTest } @Test - public void testSearchWithDimensions2() + public void testSearchWithDimensionsPlacementAndProvider() { Map> expectedResults = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); expectedResults.put(QueryRunnerTestHelper.providerDimension, new HashSet(Arrays.asList("total_market"))); @@ -390,7 +390,7 @@ public class SearchQueryRunnerTest for (Map.Entry> entry : expectedResults.entrySet()) { Assert.assertTrue( String.format( - "Dimension %s should have had everything removed, still has[%s]", entry.getKey(), entry.getValue() + "Dimension[%s] should have had everything removed, still has[%s]", entry.getKey(), entry.getValue() ), entry.getValue().isEmpty() ); From 15ec8e7054054c4d104f80a9a46ce5e4b53a95a0 Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 28 Aug 2013 18:51:19 -0500 Subject: [PATCH 50/92] 1) Remove the Searchable interface --- .../com/metamx/druid/index/v1/Searchable.java | 31 ------------------- 1 file changed, 31 deletions(-) delete mode 100644 server/src/main/java/com/metamx/druid/index/v1/Searchable.java diff --git a/server/src/main/java/com/metamx/druid/index/v1/Searchable.java b/server/src/main/java/com/metamx/druid/index/v1/Searchable.java deleted file mode 100644 index ef222745079..00000000000 --- a/server/src/main/java/com/metamx/druid/index/v1/Searchable.java +++ /dev/null @@ -1,31 +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.index.v1; - -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchQuery; -import io.druid.query.filter.Filter; - -/** - */ -public interface Searchable -{ - public Iterable searchDimensions(SearchQuery query, Filter filter); -} From 7834ffe6fc4d0fde325fae57b5ba52bd548cc98c Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 14:00:28 -0500 Subject: [PATCH 51/92] Make LinearShardSpec require a partitionNum --- .../com/metamx/druid/shard/LinearShardSpec.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java index 4d77a19086d..3a3e6f0aa94 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java @@ -1,6 +1,8 @@ package com.metamx.druid.shard; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.LinearPartitionChunk; import com.metamx.druid.partition.PartitionChunk; @@ -10,12 +12,11 @@ import java.util.Map; public class LinearShardSpec implements ShardSpec { private int partitionNum; - public LinearShardSpec() { - this(-1); - } - - public LinearShardSpec(int partitionNum) { - this.partitionNum = partitionNum; + @JsonCreator + public LinearShardSpec( + @JsonProperty("partitionNum") Integer partitionNum + ) { + this.partitionNum = Preconditions.checkNotNull(partitionNum, "Must set partitionNum on LinearShardSpec"); } @JsonProperty("partitionNum") From c5760f43e477e0b5b1ac9a2d58618331ab5d2ee9 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 14:02:46 -0500 Subject: [PATCH 52/92] Whitespace! --- .../src/main/java/com/metamx/druid/shard/LinearShardSpec.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java index 3a3e6f0aa94..4eb9c963672 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java @@ -15,7 +15,8 @@ public class LinearShardSpec implements ShardSpec { @JsonCreator public LinearShardSpec( @JsonProperty("partitionNum") Integer partitionNum - ) { + ) + { this.partitionNum = Preconditions.checkNotNull(partitionNum, "Must set partitionNum on LinearShardSpec"); } From 2a46086e2077892a0c47826e38378b02441b60c0 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 15:25:36 -0500 Subject: [PATCH 53/92] 1) Didn't remove the io.druid files from client. Remove those and make sure things compile 2) Switch DefaultObjectMapper to CommonObjectMapper 3) Create new DefaultObjectMapper in client that has Query stuff registered on it by default --- .../com/metamx/druid/sql/antlr4/DruidSQL.g4 | 2 +- .../main/java/com/metamx/druid/BaseQuery.java | 137 ------------------ .../metamx/druid/guice/QueryableModule.java | 21 ++- .../druid/http/ClientQuerySegmentWalker.java | 4 +- .../http/DirectClientQuerySegmentWalker.java | 4 +- .../com/metamx/druid/http/QueryServlet.java | 4 +- .../druid/jackson/DefaultObjectMapper.java} | 19 ++- .../druid/jackson/QueryRegisteringModule.java | 85 +++++++++++ .../query/dimension/DefaultDimensionSpec.java | 5 +- .../druid/query/dimension/DimensionSpec.java | 39 ----- .../dimension/ExtractionDimensionSpec.java | 5 +- .../query/extraction/DimExtractionFn.java | 38 ----- .../extraction/PartialDimExtractionFn.java | 3 +- .../extraction/RegexDimExtractionFn.java | 3 +- .../SearchQuerySpecDimExtractionFn.java | 3 +- .../query/extraction/TimeDimExtractionFn.java | 3 +- .../query/filter/ExtractionDimFilter.java | 4 +- .../druid/query/group/GroupByQuery.java | 9 +- .../group/GroupByQueryQueryToolChest.java | 4 +- .../query/group/orderby/DefaultLimitSpec.java | 4 +- .../druid/query/group/orderby/LimitSpec.java | 4 +- .../query/group/orderby/NoopLimitSpec.java | 21 ++- .../query/metadata/SegmentMetadataQuery.java | 6 +- .../druid/query/search/SearchQuery.java | 7 +- .../query/segment/QuerySegmentWalker.java | 47 ------ .../query/segment/SpecificSegmentSpec.java | 3 +- .../query/timeboundary/TimeBoundaryQuery.java | 7 +- .../query/timeseries/TimeseriesQuery.java | 7 +- .../java/com/metamx/druid/sql/SQLRunner.java | 21 ++- .../java/io/druid/query/CacheStrategy.java | 39 ----- .../src/main/java/io/druid/query/Query.java | 75 ---------- .../java/io/druid/query/QueryToolChest.java | 66 --------- .../io/druid/query/SegmentDescriptor.java | 106 -------------- .../aggregation/MetricManipulationFn.java | 27 ---- .../druid/query/spec/LegacySegmentSpec.java | 69 --------- .../spec/MultipleIntervalSegmentSpec.java | 67 --------- .../spec/MultipleSpecificSegmentSpec.java | 94 ------------ .../io/druid/query/spec/QuerySegmentSpec.java | 43 ------ .../metamx/druid/client/DataSegmentTest.java | 3 +- .../PartialDimExtractionFnTest.java | 3 +- .../extraction/RegexDimExtractionFnTest.java | 3 +- .../SearchQuerySpecDimExtractionFnTest.java | 3 +- .../extraction/TimeDimExtractionFnTest.java | 3 +- ...ectMapper.java => CommonObjectMapper.java} | 8 +- .../metamx/druid/jackson/JacksonModule.java | 4 +- .../metamx/druid/QueryGranularityTest.java | 12 +- .../metamx/druid/histogram/HistogramTest.java | 6 +- ...rTest.java => CommonObjectMapperTest.java} | 6 +- .../com/metamx/druid/index/v1/IndexIO.java | 8 +- .../indexer/data/InputRowParserSerdeTest.java | 23 ++- .../indexer/HadoopDruidIndexerAzkWrapper.java | 87 ----------- .../coordinator/ThreadPoolTaskRunner.java | 4 +- .../indexing/worker/WorkerTaskMonitor.java | 4 +- .../druid/realtime/RealtimeManager.java | 4 +- .../druid/coordination/ServerManager.java | 4 +- .../druid/index/brita/ExtractionFilter.java | 4 +- .../druid/query/group/GroupByQueryEngine.java | 4 +- .../query/group/GroupByQueryRunnerTest.java | 6 +- 58 files changed, 272 insertions(+), 1032 deletions(-) delete mode 100644 client/src/main/java/com/metamx/druid/BaseQuery.java rename client/src/main/java/{io/druid/query/QueryRunner.java => com/metamx/druid/jackson/DefaultObjectMapper.java} (67%) create mode 100644 client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java delete mode 100644 client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java delete mode 100644 client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java delete mode 100644 client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java delete mode 100644 client/src/main/java/io/druid/query/CacheStrategy.java delete mode 100644 client/src/main/java/io/druid/query/Query.java delete mode 100644 client/src/main/java/io/druid/query/QueryToolChest.java delete mode 100644 client/src/main/java/io/druid/query/SegmentDescriptor.java delete mode 100644 client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java delete mode 100644 client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java delete mode 100644 client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java delete mode 100644 client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java delete mode 100644 client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java rename common/src/main/java/com/metamx/druid/jackson/{DefaultObjectMapper.java => CommonObjectMapper.java} (91%) rename common/src/test/java/com/metamx/druid/jackson/{DefaultObjectMapperTest.java => CommonObjectMapperTest.java} (89%) delete mode 100644 indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java diff --git a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 index d1ce41a1cc5..825e0728afa 100644 --- a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 +++ b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 @@ -12,7 +12,7 @@ import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.query.spec.DimensionSpec; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.NotDimFilter; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java deleted file mode 100644 index e3e01b331f7..00000000000 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ /dev/null @@ -1,137 +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; - -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.collect.Maps; -import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.spec.QuerySegmentSpec; -import org.joda.time.Duration; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Map; - -/** - */ -public abstract class BaseQuery implements Query -{ - private final String dataSource; - private final Map context; - private final QuerySegmentSpec querySegmentSpec; - - private volatile Duration duration; - - public BaseQuery( - String dataSource, - QuerySegmentSpec querySegmentSpec, - Map context - ) - { - Preconditions.checkNotNull(dataSource, "dataSource can't be null"); - Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); - - this.dataSource = dataSource.toLowerCase(); - this.context = context; - this.querySegmentSpec = querySegmentSpec; - - } - - @JsonProperty - @Override - public String getDataSource() - { - return dataSource; - } - - @JsonProperty("intervals") - public QuerySegmentSpec getQuerySegmentSpec() - { - return querySegmentSpec; - } - - @Override - public Sequence run(QuerySegmentWalker walker) - { - return run(querySegmentSpec.lookup(this, walker)); - } - - public Sequence run(QueryRunner runner) - { - return runner.run(this); - } - - @Override - public List getIntervals() - { - return querySegmentSpec.getIntervals(); - } - - @Override - public Duration getDuration() - { - if (duration == null) { - Duration totalDuration = new Duration(0); - for (Interval interval : querySegmentSpec.getIntervals()) { - if (interval != null) { - totalDuration = totalDuration.plus(interval.toDuration()); - } - } - duration = totalDuration; - } - - return duration; - } - - @JsonProperty - public Map getContext() - { - return context; - } - - @Override - public String getContextValue(String key) - { - return context == null ? null : context.get(key); - } - - @Override - public String getContextValue(String key, String defaultValue) - { - String retVal = getContextValue(key); - return retVal == null ? defaultValue : retVal; - } - - protected Map computeOverridenContext(Map overrides) - { - Map overridden = Maps.newTreeMap(); - final Map context = getContext(); - if (context != null) { - overridden.putAll(context); - } - overridden.putAll(overrides); - - return overridden; - } -} diff --git a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java index 2c00162e2d0..29a65462d36 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.guice; import com.fasterxml.jackson.databind.Module; @@ -8,8 +27,8 @@ import com.metamx.druid.http.log.EmittingRequestLoggerProvider; import com.metamx.druid.http.log.FileRequestLoggerProvider; import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.http.log.RequestLoggerProvider; -import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.initialization.DruidModule; +import io.druid.query.QuerySegmentWalker; import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java index 86bd39e6522..4287fb620a3 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,11 +25,11 @@ import com.metamx.druid.client.CachingClusteredClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java index 228d03a3a9e..a2aef1c7a85 100644 --- a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,9 +22,9 @@ package com.metamx.druid.http; import com.metamx.druid.client.DirectDruidClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 6e11d8c2825..131186be24e 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -32,11 +32,11 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.http.log.RequestLogger; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; +import io.druid.query.QuerySegmentWalker; import org.eclipse.jetty.server.Request; import org.joda.time.DateTime; diff --git a/client/src/main/java/io/druid/query/QueryRunner.java b/client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java similarity index 67% rename from client/src/main/java/io/druid/query/QueryRunner.java rename to client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 4d20739412a..b7bd32d3e99 100644 --- a/client/src/main/java/io/druid/query/QueryRunner.java +++ b/client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,22 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.query; +package com.metamx.druid.jackson; -import com.metamx.common.guava.Sequence; +import com.fasterxml.jackson.core.JsonFactory; /** */ -public interface QueryRunner +public class DefaultObjectMapper extends CommonObjectMapper { - public Sequence run(Query query); + public DefaultObjectMapper() + { + this(null); + } + + public DefaultObjectMapper(JsonFactory factory) + { + super(factory); + registerModule(new QueryRegisteringModule()); + } } diff --git a/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java b/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java new file mode 100644 index 00000000000..73cdee0bae7 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java @@ -0,0 +1,85 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.jackson; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.ExtractionDimensionSpec; +import com.metamx.druid.query.dimension.LegacyDimensionSpec; +import com.metamx.druid.query.extraction.PartialDimExtractionFn; +import com.metamx.druid.query.extraction.RegexDimExtractionFn; +import com.metamx.druid.query.extraction.SearchQuerySpecDimExtractionFn; +import com.metamx.druid.query.extraction.TimeDimExtractionFn; +import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; +import com.metamx.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.Query; +import io.druid.query.spec.DimExtractionFn; +import io.druid.query.spec.DimensionSpec; + +/** + */ +public class QueryRegisteringModule extends SimpleModule +{ + public QueryRegisteringModule() + { + super("QueryRegistering"); + + setMixInAnnotation(Query.class, QueriesMixin.class); + setMixInAnnotation(DimensionSpec.class, DimensionSpecMixin.class); + setMixInAnnotation(DimExtractionFn.class, DimensionSpecMixin.class); + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "timeseries", value = TimeseriesQuery.class), + @JsonSubTypes.Type(name = "search", value = SearchQuery.class), + @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryQuery.class), + @JsonSubTypes.Type(name = "groupBy", value = GroupByQuery.class), + @JsonSubTypes.Type(name = "segmentMetadata", value = SegmentMetadataQuery.class) + }) + public static interface QueriesMixin + { + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class) + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class), + @JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class) + }) + public static interface DimensionSpecMixin + { + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type") + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class), + @JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class), + @JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class), + @JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class) + }) + public static interface DimExtractionFnMixin + { + } +} diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java index b27b361eda0..6bb896083b1 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,7 +21,8 @@ package com.metamx.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.spec.DimExtractionFn; +import io.druid.query.spec.DimensionSpec; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java deleted file mode 100644 index 0773ef6ca06..00000000000 --- a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java +++ /dev/null @@ -1,39 +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.query.dimension; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.query.extraction.DimExtractionFn; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class), - @JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class) -}) -public interface DimensionSpec -{ - public String getDimension(); - public String getOutputName(); - public DimExtractionFn getDimExtractionFn(); - public byte[] getCacheKey(); -} diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java index c746b7b3108..a48504d4ef8 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,7 +21,8 @@ package com.metamx.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.spec.DimExtractionFn; +import io.druid.query.spec.DimensionSpec; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java deleted file mode 100644 index 45b84313542..00000000000 --- a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java +++ /dev/null @@ -1,38 +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.query.extraction; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class), - @JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class), - @JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class), - @JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class) -}) -public interface DimExtractionFn -{ - public byte[] getCacheKey(); - public String apply(String dimValue); -} diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java index 86b1d1ddbd2..e8fc679bd1a 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,6 +21,7 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java index f5135a2d1c5..f8d68b86f09 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,6 +21,7 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 3b1a323ac1c..d8748387080 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java index 71ff8bed15f..ea721fff1e2 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.ibm.icu.text.SimpleDateFormat; +import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.text.ParseException; diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java index b6a43c5af04..cd15c74f64b 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index b705deac2bf..3c0c42cd848 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -30,12 +30,10 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.BaseQuery; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.Queries; import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.group.having.HavingSpec; import com.metamx.druid.query.group.orderby.DefaultLimitSpec; @@ -43,8 +41,9 @@ import com.metamx.druid.query.group.orderby.LimitSpec; import com.metamx.druid.query.group.orderby.NoopLimitSpec; import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import io.druid.granularity.QueryGranularity; -import io.druid.query.Query; +import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -210,7 +209,7 @@ public class GroupByQuery extends BaseQuery @Override public String getType() { - return Query.GROUP_BY; + return "groupBy"; } public Sequence applyLimit(Sequence results) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index 7ff15dc9371..cf4d733d6ac 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -36,7 +36,6 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; -import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; @@ -44,6 +43,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.spec.DimensionSpec; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java index cbe5efa90b0..4dc9ebac057 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -33,8 +33,8 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java index bcefbf82663..3814d8ab93d 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,8 +25,8 @@ import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java index fddb6ce1904..faf9ab760cd 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.group.orderby; import com.google.common.base.Function; @@ -5,8 +24,8 @@ import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index cc375e57433..99e6c29b5c2 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,7 +21,7 @@ package com.metamx.druid.query.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.BaseQuery; +import io.druid.query.BaseQuery; import io.druid.query.Query; import io.druid.query.spec.QuerySegmentSpec; @@ -69,7 +69,7 @@ public class SegmentMetadataQuery extends BaseQuery @Override public String getType() { - return Query.SEGMENT_METADATA; + return "segmentMetadata"; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index 8702156598b..c6b3d329709 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,12 +24,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.metamx.druid.BaseQuery; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import io.druid.granularity.QueryGranularity; -import io.druid.query.Query; +import io.druid.query.BaseQuery; import io.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; @@ -91,7 +90,7 @@ public class SearchQuery extends BaseQuery> @Override public String getType() { - return Query.SEARCH; + return "search"; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java deleted file mode 100644 index 3bf3b37bea6..00000000000 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java +++ /dev/null @@ -1,47 +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.query.segment; - -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.SegmentDescriptor; -import org.joda.time.Interval; - -/** - */ -public interface QuerySegmentWalker -{ - /** - * Gets the Queryable for a given interval, the Queryable returned can be any version(s) or partitionNumber(s) - * such that it represents the interval. - * - * @param intervals the intervals to find a Queryable for - * @return a Queryable object that represents the interval - */ - public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals); - - /** - * Gets the Queryable for a given list of SegmentSpecs. - * exist. - * - * @return the Queryable object with the given SegmentSpecs - */ - public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs); -} diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java index 55973aff8f4..2587467d930 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,6 +21,7 @@ package com.metamx.druid.query.segment; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index ecded61c0e9..a462d10ad2d 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,10 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.BaseQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import io.druid.query.Query; +import io.druid.query.BaseQuery; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; @@ -74,7 +73,7 @@ public class TimeBoundaryQuery extends BaseQuery @Override public String getType() { - return Query.TIME_BOUNDARY; + return "timeBoundary"; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index ef06fd4480b..8be56a4e3d4 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,14 +23,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; -import com.metamx.druid.BaseQuery; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.Queries; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; -import io.druid.query.Query; +import io.druid.query.BaseQuery; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.spec.QuerySegmentSpec; @@ -76,7 +75,7 @@ public class TimeseriesQuery extends BaseQuery> @Override public String getType() { - return Query.TIMESERIES; + return "timeseries"; } @JsonProperty("filter") diff --git a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java index 5667583e25d..fc400082bc5 100644 --- a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java +++ b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.sql; import com.fasterxml.jackson.core.type.TypeReference; @@ -11,7 +30,6 @@ import com.google.common.io.Closeables; import com.metamx.druid.Druids; import com.metamx.druid.input.Row; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; @@ -19,6 +37,7 @@ import com.metamx.druid.sql.antlr4.DruidSQLLexer; import com.metamx.druid.sql.antlr4.DruidSQLParser; import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import org.antlr.v4.runtime.ANTLRInputStream; import org.antlr.v4.runtime.CharStream; import org.antlr.v4.runtime.CommonTokenStream; diff --git a/client/src/main/java/io/druid/query/CacheStrategy.java b/client/src/main/java/io/druid/query/CacheStrategy.java deleted file mode 100644 index c8d5217841f..00000000000 --- a/client/src/main/java/io/druid/query/CacheStrategy.java +++ /dev/null @@ -1,39 +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 io.druid.query; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Function; -import com.metamx.common.guava.Sequence; - -/** -*/ -public interface CacheStrategy> -{ - public byte[] computeCacheKey(QueryType query); - - public TypeReference getCacheObjectClazz(); - - public Function prepareForCache(); - - public Function pullFromCache(); - - public Sequence mergeSequences(Sequence> seqOfSequences); -} diff --git a/client/src/main/java/io/druid/query/Query.java b/client/src/main/java/io/druid/query/Query.java deleted file mode 100644 index 47f38d6e61f..00000000000 --- a/client/src/main/java/io/druid/query/Query.java +++ /dev/null @@ -1,75 +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 io.druid.query; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.query.timeseries.TimeseriesQuery; -import io.druid.query.spec.QuerySegmentSpec; -import org.joda.time.Duration; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Map; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class), - @JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class), - @JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class), - @JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class), - @JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class) -}) -public interface Query -{ - public static final String TIMESERIES = "timeseries"; - public static final String SEARCH = "search"; - public static final String TIME_BOUNDARY = "timeBoundary"; - public static final String GROUP_BY = "groupBy"; - public static final String SEGMENT_METADATA = "segmentMetadata"; - - public String getDataSource(); - - public boolean hasFilters(); - - public String getType(); - - public Sequence run(QuerySegmentWalker walker); - - public Sequence run(QueryRunner runner); - - public List getIntervals(); - - public Duration getDuration(); - - public String getContextValue(String key); - - public String getContextValue(String key, String defaultValue); - - public Query withOverriddenContext(Map contextOverride); - - public Query withQuerySegmentSpec(QuerySegmentSpec spec); -} diff --git a/client/src/main/java/io/druid/query/QueryToolChest.java b/client/src/main/java/io/druid/query/QueryToolChest.java deleted file mode 100644 index 37fbd41f3ee..00000000000 --- a/client/src/main/java/io/druid/query/QueryToolChest.java +++ /dev/null @@ -1,66 +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 io.druid.query; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Function; -import com.metamx.common.guava.Sequence; -import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.query.aggregation.MetricManipulationFn; -import io.druid.segment.LogicalSegment; - -import java.util.List; - -/** - * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing - * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential - * maintenance burden when upgrading versions. - */ -public abstract class QueryToolChest> -{ - public abstract QueryRunner mergeResults(QueryRunner runner); - - /** - * This method doesn't belong here, but it's here for now just to make it work. - * - * @param seqOfSequences - * @return - */ - public abstract Sequence mergeSequences(Sequence> seqOfSequences); - public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); - public abstract TypeReference getResultTypeReference(); - - public CacheStrategy getCacheStrategy(QueryType query) { - return null; - } - - public QueryRunner preMergeQueryDecoration(QueryRunner runner) { - return runner; - } - - public QueryRunner postMergeQueryDecoration(QueryRunner runner) { - return runner; - } - - public List filterSegments(QueryType query, List segments) { - return segments; - } -} diff --git a/client/src/main/java/io/druid/query/SegmentDescriptor.java b/client/src/main/java/io/druid/query/SegmentDescriptor.java deleted file mode 100644 index f66b7f66d6a..00000000000 --- a/client/src/main/java/io/druid/query/SegmentDescriptor.java +++ /dev/null @@ -1,106 +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 io.druid.query; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** -*/ -public class SegmentDescriptor -{ - private final Interval interval; - private final String version; - private final int partitionNumber; - - @JsonCreator - public SegmentDescriptor( - @JsonProperty("itvl") Interval interval, - @JsonProperty("ver") String version, - @JsonProperty("part") int partitionNumber) - { - this.interval = interval; - this.version = version; - this.partitionNumber = partitionNumber; - } - - @JsonProperty("itvl") - public Interval getInterval() - { - return interval; - } - - @JsonProperty("ver") - public String getVersion() - { - return version; - } - - @JsonProperty("part") - public int getPartitionNumber() - { - return partitionNumber; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - SegmentDescriptor that = (SegmentDescriptor) o; - - if (partitionNumber != that.partitionNumber) { - return false; - } - if (interval != null ? !interval.equals(that.interval) : that.interval != null) { - return false; - } - if (version != null ? !version.equals(that.version) : that.version != null) { - return false; - } - - return true; - } - - @Override - public int hashCode() - { - int result = interval != null ? interval.hashCode() : 0; - result = 31 * result + (version != null ? version.hashCode() : 0); - result = 31 * result + partitionNumber; - return result; - } - - @Override - public String toString() - { - return "SegmentDescriptor{" + - "interval=" + interval + - ", version='" + version + '\'' + - ", partitionNumber=" + partitionNumber + - '}'; - } -} diff --git a/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java b/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java deleted file mode 100644 index 0da9c9f63cc..00000000000 --- a/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java +++ /dev/null @@ -1,27 +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 io.druid.query.aggregation; - -/** -*/ -public interface MetricManipulationFn -{ - public Object manipulate(AggregatorFactory factory, Object object); -} diff --git a/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java b/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java deleted file mode 100644 index 505760aa762..00000000000 --- a/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java +++ /dev/null @@ -1,69 +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 io.druid.query.spec; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import com.metamx.common.IAE; -import org.joda.time.Interval; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -/** - */ -public class LegacySegmentSpec extends MultipleIntervalSegmentSpec -{ - private static List convertValue(Object intervals) - { - final List intervalStringList; - if (intervals instanceof String) { - intervalStringList = Arrays.asList((((String) intervals).split(","))); - } else if (intervals instanceof Map) { - intervalStringList = (List) ((Map) intervals).get("intervals"); - } else if (intervals instanceof List) { - intervalStringList = (List) intervals; - } else { - throw new IAE("Unknown type[%s] for intervals[%s]", intervals.getClass(), intervals); - } - - return Lists.transform( - intervalStringList, - new Function() - { - @Override - public Interval apply(Object input) - { - return new Interval(input); - } - } - ); - } - - @JsonCreator - public LegacySegmentSpec( - Object intervals - ) - { - super(convertValue(intervals)); - } -} diff --git a/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java deleted file mode 100644 index 7c9c7703492..00000000000 --- a/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java +++ /dev/null @@ -1,67 +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 io.druid.query.spec; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.utils.JodaUtils; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import org.joda.time.Interval; - -import java.util.Collections; -import java.util.List; - -/** - */ -public class MultipleIntervalSegmentSpec implements QuerySegmentSpec -{ - private final List intervals; - - @JsonCreator - public MultipleIntervalSegmentSpec( - @JsonProperty("intervals") List intervals - ) - { - this.intervals = Collections.unmodifiableList(JodaUtils.condenseIntervals(intervals)); - } - - @Override - @JsonProperty("intervals") - public List getIntervals() - { - return intervals; - } - - @Override - public QueryRunner lookup(Query query, QuerySegmentWalker walker) - { - return walker.getQueryRunnerForIntervals(query, intervals); - } - - @Override - public String toString() - { - return getClass().getSimpleName() + "{" + - "intervals=" + intervals + - '}'; - } -} diff --git a/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java deleted file mode 100644 index 37e8b6c338d..00000000000 --- a/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java +++ /dev/null @@ -1,94 +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 io.druid.query.spec; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.utils.JodaUtils; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.SegmentDescriptor; -import org.joda.time.Interval; - -import java.util.List; - -/** - */ -public class MultipleSpecificSegmentSpec implements QuerySegmentSpec -{ - private final List descriptors; - - private volatile List intervals = null; - - @JsonCreator - public MultipleSpecificSegmentSpec( - @JsonProperty("segments") List descriptors - ) - { - this.descriptors = descriptors; - } - - @JsonProperty("segments") - public List getDescriptors() - { - return descriptors; - } - - @Override - public List getIntervals() - { - if (intervals != null) { - return intervals; - } - - intervals = JodaUtils.condenseIntervals( - Iterables.transform( - descriptors, - new Function() - { - @Override - public Interval apply(SegmentDescriptor input) - { - return input.getInterval(); - } - } - ) - ); - - return intervals; - } - - @Override - public QueryRunner lookup(Query query, QuerySegmentWalker walker) - { - return walker.getQueryRunnerForSegments(query, descriptors); - } - - @Override - public String toString() - { - return "MultipleSpecificSegmentSpec{" + - "descriptors=" + descriptors + - '}'; - } -} diff --git a/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java b/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java deleted file mode 100644 index 126b1d99b58..00000000000 --- a/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java +++ /dev/null @@ -1,43 +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 io.druid.query.spec; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import org.joda.time.Interval; - -import java.util.List; - -/** - */ -@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) -@JsonSubTypes(value={ - @JsonSubTypes.Type(name="intervals", value=MultipleIntervalSegmentSpec.class), - @JsonSubTypes.Type(name="segments", value=MultipleSpecificSegmentSpec.class) -}) -public interface QuerySegmentSpec -{ - public List getIntervals(); - - public QueryRunner lookup(Query query, QuerySegmentWalker walker); -} diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java index 63f3326a92e..7532aeb1424 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -29,7 +29,6 @@ import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java index 061f8122f2e..a894756dd6e 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,6 +20,7 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; +import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java index 2ec3f3db239..e430695836b 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,6 +20,7 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; +import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java index 06e9cde0ffd..55c6bc696cf 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,6 +22,7 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; import com.metamx.druid.query.search.FragmentSearchQuerySpec; import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java index 7f5b6eabe1c..53ce56f981a 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,6 +20,7 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; +import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java similarity index 91% rename from common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java rename to common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java index 1c6e4b18636..85775df1040 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -28,14 +28,14 @@ import com.fasterxml.jackson.datatype.guava.GuavaModule; /** */ -public class DefaultObjectMapper extends ObjectMapper +public class CommonObjectMapper extends ObjectMapper { - public DefaultObjectMapper() + public CommonObjectMapper() { this(null); } - public DefaultObjectMapper(JsonFactory factory) + public CommonObjectMapper(JsonFactory factory) { super(factory); registerModule(new DruidDefaultSerializersModule()); diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java index 694f4d2bf4c..33eb36125f7 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -42,13 +42,13 @@ public class JacksonModule implements Module @Provides @LazySingleton @Json public ObjectMapper jsonMapper() { - return new DefaultObjectMapper(); + return new CommonObjectMapper(); } @Provides @LazySingleton @Smile public ObjectMapper smileMapper() { - ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); + ObjectMapper retVal = new CommonObjectMapper(new SmileFactory()); retVal.getJsonFactory().setCodec(retVal); return retVal; } diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index ad54b418d01..f98389cd457 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.jackson.CommonObjectMapper; import io.druid.granularity.DurationGranularity; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; @@ -437,7 +437,7 @@ public class QueryGranularityTest @Test public void testSerializePeriod() throws Exception { - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); String json = "{ \"type\": \"period\", \"period\": \"P1D\" }"; QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); @@ -462,7 +462,7 @@ public class QueryGranularityTest @Test public void testSerializeDuration() throws Exception { - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); String json = "{ \"type\": \"duration\", \"duration\": \"3600000\" }"; QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); @@ -479,7 +479,7 @@ public class QueryGranularityTest @Test public void testSerializeSimple() throws Exception { - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); Assert.assertEquals( QueryGranularity.ALL, @@ -501,7 +501,7 @@ public class QueryGranularityTest @Test public void testDeserializeSimple() throws Exception { - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"all\"", QueryGranularity.class)); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class)); diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index f221b0e26df..fa5f9f4e8ea 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,8 +22,6 @@ package com.metamx.druid.histogram; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.Histogram; -import com.metamx.druid.jackson.DefaultObjectMapper; - import org.junit.Assert; import org.junit.Test; @@ -63,7 +61,7 @@ public class HistogramTest Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0}; Double[] visualCounts = { 123., 4., 56., 7. }; - ObjectMapper objectMapper = new DefaultObjectMapper(); + ObjectMapper objectMapper = new ObjectMapper(); String json = objectMapper.writeValueAsString(h.asVisual()); Map expectedObj = Maps.newLinkedHashMap(); diff --git a/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java b/common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java similarity index 89% rename from common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java rename to common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java index aff4ad208c6..45ca35c6ac5 100644 --- a/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java +++ b/common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,9 +26,9 @@ import org.junit.Test; /** */ -public class DefaultObjectMapperTest +public class CommonObjectMapperTest { - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); @Test public void testDateTime() throws Exception diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index f95803831c8..aa12747ba9b 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -52,7 +52,7 @@ import com.metamx.druid.index.serde.FloatGenericColumnSupplier; import com.metamx.druid.index.serde.LongGenericColumnPartSerde; import com.metamx.druid.index.serde.LongGenericColumnSupplier; import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.jackson.CommonObjectMapper; import com.metamx.druid.kv.ArrayIndexed; import com.metamx.druid.kv.ByteBufferSerializer; import com.metamx.druid.kv.ConciseCompressedIndexedInts; @@ -110,7 +110,7 @@ public class IndexIO public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); // This should really be provided by DI, should be changed once we switch around to using a DI framework - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static final ObjectMapper mapper = new CommonObjectMapper(); private static volatile IndexIOHandler handler = null; public static final int CURRENT_VERSION_ID = V9_VERSION; @@ -790,7 +790,7 @@ public class IndexIO Map columns = Maps.newHashMap(); - ObjectMapper mapper = new DefaultObjectMapper(); + ObjectMapper mapper = new CommonObjectMapper(); for (String columnName : cols) { columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java index 418661ce9d6..649f0374eb6 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.indexer.data; import com.fasterxml.jackson.databind.ObjectMapper; @@ -6,7 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.InputRow; -import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.jackson.CommonObjectMapper; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; @@ -15,7 +34,7 @@ import java.nio.ByteBuffer; public class InputRowParserSerdeTest { - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private final ObjectMapper jsonMapper = new CommonObjectMapper(); @Test public void testStringInputRowParserSerde() throws Exception diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java deleted file mode 100644 index 244c1d80901..00000000000 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.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.indexer; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.metamx.common.MapUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.jackson.DefaultObjectMapper; -import org.joda.time.DateTime; -import org.joda.time.Interval; - -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - */ -public class HadoopDruidIndexerAzkWrapper -{ - private static final Logger log = new Logger(HadoopDruidIndexerAzkWrapper.class); - private static final String PROPERTY_PREFIX = "druid.indexer."; - - private final String jobName; - private final Properties properties; - - public HadoopDruidIndexerAzkWrapper( - String jobName, - Properties properties - ) - { - this.jobName = jobName; - this.properties = properties; - } - - public void run() throws Exception - { - final DefaultObjectMapper jsonMapper = new DefaultObjectMapper(); - - final List dataInterval; - final Map theMap = Maps.newTreeMap(); - - for (String propertyName : properties.stringPropertyNames()) { - if (propertyName.startsWith(PROPERTY_PREFIX)) { - final String propValue = properties.getProperty(propertyName); - if (propValue.trim().startsWith("{") || propValue.trim().startsWith("[")) { - theMap.put(propertyName.substring(PROPERTY_PREFIX.length()), jsonMapper.readValue(propValue, Object.class)); - } - else { - theMap.put(propertyName.substring(PROPERTY_PREFIX.length()), propValue); - } - } - } - - log.info("Running with properties:"); - for (Map.Entry entry : theMap.entrySet()) { - log.info("%30s => %s", entry.getKey(), entry.getValue()); - } - - dataInterval = Lists.transform( - Lists.newArrayList(MapUtils.getString(theMap, "timeInterval").split(",")), new StringIntervalFunction() - ); - - final HadoopDruidIndexerConfig config = jsonMapper.convertValue(theMap, HadoopDruidIndexerConfig.class); - config.setIntervals(dataInterval); - config.setVersion(new DateTime().toString()); - - new HadoopDruidIndexerJob(config).run(); - } -} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index e41f0aabb50..7a953bc9ccc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -36,10 +36,10 @@ import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.NoopQueryRunner; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java index d7e60e373f3..e59ad71a401 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -28,8 +28,8 @@ import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.worker.config.WorkerConfig; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; +import io.druid.query.QuerySegmentWalker; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java index 89288d5ab83..4681b644a40 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -31,7 +31,6 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.Sink; @@ -39,6 +38,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import org.joda.time.DateTime; diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 138aab556af..56c591de522 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -41,7 +41,6 @@ import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.emitter.EmittingLogger; @@ -50,6 +49,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.QuerySegmentSpec; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java index e12f067e211..5207efc0699 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,12 +20,12 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; -import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.data.Indexed; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; +import io.druid.query.spec.DimExtractionFn; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 526cb8cc3c4..969d087efe3 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -42,9 +42,9 @@ import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.index.brita.Filters; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.spec.DimensionSpec; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index 66129edcffb..147fa7a72ee 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -38,7 +38,6 @@ import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.RegexDimFilter; import com.metamx.druid.query.group.having.EqualToHavingSpec; import com.metamx.druid.query.group.having.GreaterThanHavingSpec; @@ -51,6 +50,7 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.spec.DimensionSpec; import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -731,7 +731,7 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } From cb90ed05b009801862ca0eaac2acbfc7340474fe Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 16:45:03 -0500 Subject: [PATCH 54/92] Revert the previous commit. After going down this path, I realized that extracting things enough to allow Queries to be extended without depending on Druid proper was going to lead down a very nasty path. So, I've decided against that. Extending queries will require a tight dependency on Druid proper. --- .../com/metamx/druid/sql/antlr4/DruidSQL.g4 | 2 +- .../main/java/com/metamx/druid/BaseQuery.java | 137 ++++++++++++++++++ .../metamx/druid/guice/QueryableModule.java | 2 +- .../druid/http/ClientQuerySegmentWalker.java | 2 +- .../http/DirectClientQuerySegmentWalker.java | 2 +- .../com/metamx/druid/http/QueryServlet.java | 2 +- .../druid/jackson/QueryRegisteringModule.java | 85 ----------- .../query/dimension/DefaultDimensionSpec.java | 3 +- .../druid/query/dimension/DimensionSpec.java | 39 +++++ .../dimension/ExtractionDimensionSpec.java | 3 +- .../query/extraction/DimExtractionFn.java | 38 +++++ .../extraction/PartialDimExtractionFn.java | 1 - .../extraction/RegexDimExtractionFn.java | 1 - .../SearchQuerySpecDimExtractionFn.java | 1 - .../query/extraction/TimeDimExtractionFn.java | 1 - .../query/filter/ExtractionDimFilter.java | 2 +- .../druid/query/group/GroupByQuery.java | 7 +- .../group/GroupByQueryQueryToolChest.java | 2 +- .../query/group/orderby/DefaultLimitSpec.java | 2 +- .../druid/query/group/orderby/LimitSpec.java | 2 +- .../query/group/orderby/NoopLimitSpec.java | 2 +- .../query/metadata/SegmentMetadataQuery.java | 4 +- .../druid/query/search/SearchQuery.java | 5 +- .../query/segment/QuerySegmentWalker.java | 47 ++++++ .../query/segment/SpecificSegmentSpec.java | 1 - .../query/timeboundary/TimeBoundaryQuery.java | 5 +- .../query/timeseries/TimeseriesQuery.java | 5 +- .../java/com/metamx/druid/sql/SQLRunner.java | 2 +- .../java/io/druid/query/CacheStrategy.java | 39 +++++ .../src/main/java/io/druid/query/Query.java | 75 ++++++++++ .../druid/query/QueryRunner.java} | 17 +-- .../java/io/druid/query/QueryToolChest.java | 66 +++++++++ .../io/druid/query/SegmentDescriptor.java | 106 ++++++++++++++ .../aggregation/MetricManipulationFn.java | 27 ++++ .../druid/query/spec/LegacySegmentSpec.java | 69 +++++++++ .../spec/MultipleIntervalSegmentSpec.java | 67 +++++++++ .../spec/MultipleSpecificSegmentSpec.java | 94 ++++++++++++ .../io/druid/query/spec/QuerySegmentSpec.java | 43 ++++++ .../PartialDimExtractionFnTest.java | 1 - .../extraction/RegexDimExtractionFnTest.java | 1 - .../SearchQuerySpecDimExtractionFnTest.java | 1 - .../extraction/TimeDimExtractionFnTest.java | 1 - ...ctMapper.java => DefaultObjectMapper.java} | 6 +- .../metamx/druid/jackson/JacksonModule.java | 4 +- .../metamx/druid/QueryGranularityTest.java | 10 +- .../metamx/druid/histogram/HistogramTest.java | 3 +- ...Test.java => DefaultObjectMapperTest.java} | 4 +- .../com/metamx/druid/index/v1/IndexIO.java | 6 +- .../indexer/data/InputRowParserSerdeTest.java | 4 +- .../coordinator/ThreadPoolTaskRunner.java | 2 +- .../indexing/worker/WorkerTaskMonitor.java | 2 +- .../druid/realtime/RealtimeManager.java | 2 +- .../druid/coordination/ServerManager.java | 2 +- .../druid/index/brita/ExtractionFilter.java | 2 +- .../druid/query/group/GroupByQueryEngine.java | 2 +- .../query/group/GroupByQueryRunnerTest.java | 4 +- 56 files changed, 906 insertions(+), 159 deletions(-) create mode 100644 client/src/main/java/com/metamx/druid/BaseQuery.java delete mode 100644 client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java create mode 100644 client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java create mode 100644 client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java create mode 100644 client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java create mode 100644 client/src/main/java/io/druid/query/CacheStrategy.java create mode 100644 client/src/main/java/io/druid/query/Query.java rename client/src/main/java/{com/metamx/druid/jackson/DefaultObjectMapper.java => io/druid/query/QueryRunner.java} (71%) create mode 100644 client/src/main/java/io/druid/query/QueryToolChest.java create mode 100644 client/src/main/java/io/druid/query/SegmentDescriptor.java create mode 100644 client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java create mode 100644 client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java create mode 100644 client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java create mode 100644 client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java create mode 100644 client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java rename common/src/main/java/com/metamx/druid/jackson/{CommonObjectMapper.java => DefaultObjectMapper.java} (93%) rename common/src/test/java/com/metamx/druid/jackson/{CommonObjectMapperTest.java => DefaultObjectMapperTest.java} (93%) diff --git a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 index 825e0728afa..d1ce41a1cc5 100644 --- a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 +++ b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 @@ -12,7 +12,7 @@ import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import io.druid.query.spec.DimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.NotDimFilter; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/client/src/main/java/com/metamx/druid/BaseQuery.java new file mode 100644 index 00000000000..02c75bd48ac --- /dev/null +++ b/client/src/main/java/com/metamx/druid/BaseQuery.java @@ -0,0 +1,137 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.collect.Maps; +import com.metamx.common.guava.Sequence; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; + +/** + */ +public abstract class BaseQuery implements Query +{ + private final String dataSource; + private final Map context; + private final QuerySegmentSpec querySegmentSpec; + + private volatile Duration duration; + + public BaseQuery( + String dataSource, + QuerySegmentSpec querySegmentSpec, + Map context + ) + { + Preconditions.checkNotNull(dataSource, "dataSource can't be null"); + Preconditions.checkNotNull(querySegmentSpec, "querySegmentSpec can't be null"); + + this.dataSource = dataSource.toLowerCase(); + this.context = context; + this.querySegmentSpec = querySegmentSpec; + + } + + @JsonProperty + @Override + public String getDataSource() + { + return dataSource; + } + + @JsonProperty("intervals") + public QuerySegmentSpec getQuerySegmentSpec() + { + return querySegmentSpec; + } + + @Override + public Sequence run(QuerySegmentWalker walker) + { + return run(querySegmentSpec.lookup(this, walker)); + } + + public Sequence run(QueryRunner runner) + { + return runner.run(this); + } + + @Override + public List getIntervals() + { + return querySegmentSpec.getIntervals(); + } + + @Override + public Duration getDuration() + { + if (duration == null) { + Duration totalDuration = new Duration(0); + for (Interval interval : querySegmentSpec.getIntervals()) { + if (interval != null) { + totalDuration = totalDuration.plus(interval.toDuration()); + } + } + duration = totalDuration; + } + + return duration; + } + + @JsonProperty + public Map getContext() + { + return context; + } + + @Override + public String getContextValue(String key) + { + return context == null ? null : context.get(key); + } + + @Override + public String getContextValue(String key, String defaultValue) + { + String retVal = getContextValue(key); + return retVal == null ? defaultValue : retVal; + } + + protected Map computeOverridenContext(Map overrides) + { + Map overridden = Maps.newTreeMap(); + final Map context = getContext(); + if (context != null) { + overridden.putAll(context); + } + overridden.putAll(overrides); + + return overridden; + } +} diff --git a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java index 29a65462d36..d131c600aab 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java +++ b/client/src/main/java/com/metamx/druid/guice/QueryableModule.java @@ -27,8 +27,8 @@ import com.metamx.druid.http.log.EmittingRequestLoggerProvider; import com.metamx.druid.http.log.FileRequestLoggerProvider; import com.metamx.druid.http.log.RequestLogger; import com.metamx.druid.http.log.RequestLoggerProvider; +import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.initialization.DruidModule; -import io.druid.query.QuerySegmentWalker; import java.util.Arrays; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java index 4287fb620a3..39c568fda04 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java @@ -25,11 +25,11 @@ import com.metamx.druid.client.CachingClusteredClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java index a2aef1c7a85..f3c8c8956be 100644 --- a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java +++ b/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java @@ -22,9 +22,9 @@ package com.metamx.druid.http; import com.metamx.druid.client.DirectDruidClient; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.QueryToolChestWarehouse; +import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/client/src/main/java/com/metamx/druid/http/QueryServlet.java index 131186be24e..58fedde82af 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/client/src/main/java/com/metamx/druid/http/QueryServlet.java @@ -32,11 +32,11 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.http.log.RequestLogger; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; -import io.druid.query.QuerySegmentWalker; import org.eclipse.jetty.server.Request; import org.joda.time.DateTime; diff --git a/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java b/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java deleted file mode 100644 index 73cdee0bae7..00000000000 --- a/client/src/main/java/com/metamx/druid/jackson/QueryRegisteringModule.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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.jackson; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.ExtractionDimensionSpec; -import com.metamx.druid.query.dimension.LegacyDimensionSpec; -import com.metamx.druid.query.extraction.PartialDimExtractionFn; -import com.metamx.druid.query.extraction.RegexDimExtractionFn; -import com.metamx.druid.query.extraction.SearchQuerySpecDimExtractionFn; -import com.metamx.druid.query.extraction.TimeDimExtractionFn; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.query.timeseries.TimeseriesQuery; -import io.druid.query.Query; -import io.druid.query.spec.DimExtractionFn; -import io.druid.query.spec.DimensionSpec; - -/** - */ -public class QueryRegisteringModule extends SimpleModule -{ - public QueryRegisteringModule() - { - super("QueryRegistering"); - - setMixInAnnotation(Query.class, QueriesMixin.class); - setMixInAnnotation(DimensionSpec.class, DimensionSpecMixin.class); - setMixInAnnotation(DimExtractionFn.class, DimensionSpecMixin.class); - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") - @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "timeseries", value = TimeseriesQuery.class), - @JsonSubTypes.Type(name = "search", value = SearchQuery.class), - @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryQuery.class), - @JsonSubTypes.Type(name = "groupBy", value = GroupByQuery.class), - @JsonSubTypes.Type(name = "segmentMetadata", value = SegmentMetadataQuery.class) - }) - public static interface QueriesMixin - { - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class) - @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class), - @JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class) - }) - public static interface DimensionSpecMixin - { - } - - @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type") - @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class), - @JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class), - @JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class), - @JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class) - }) - public static interface DimExtractionFnMixin - { - } -} diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java index 6bb896083b1..2f57b60135f 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java @@ -21,8 +21,7 @@ package com.metamx.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.spec.DimExtractionFn; -import io.druid.query.spec.DimensionSpec; +import com.metamx.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java new file mode 100644 index 00000000000..09c2b4316d8 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java @@ -0,0 +1,39 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.dimension; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.metamx.druid.query.extraction.DimExtractionFn; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDimensionSpec.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "default", value = DefaultDimensionSpec.class), + @JsonSubTypes.Type(name = "extraction", value = ExtractionDimensionSpec.class) +}) +public interface DimensionSpec +{ + public String getDimension(); + public String getOutputName(); + public DimExtractionFn getDimExtractionFn(); + public byte[] getCacheKey(); +} diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java index a48504d4ef8..1089df851d5 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java @@ -21,8 +21,7 @@ package com.metamx.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.spec.DimExtractionFn; -import io.druid.query.spec.DimensionSpec; +import com.metamx.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java new file mode 100644 index 00000000000..88326e536bb --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java @@ -0,0 +1,38 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.extraction; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property="type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "time", value = TimeDimExtractionFn.class), + @JsonSubTypes.Type(name = "regex", value = RegexDimExtractionFn.class), + @JsonSubTypes.Type(name = "partial", value = PartialDimExtractionFn.class), + @JsonSubTypes.Type(name = "searchQuery", value = SearchQuerySpecDimExtractionFn.class) +}) +public interface DimExtractionFn +{ + public byte[] getCacheKey(); + public String apply(String dimValue); +} diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java index e8fc679bd1a..4dc830e5aa7 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java index f8d68b86f09..c839486e816 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.util.regex.Matcher; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index d8748387080..0d5f674c011 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.query.search.SearchQuerySpec; -import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java index ea721fff1e2..1f8b834279b 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.ibm.icu.text.SimpleDateFormat; -import io.druid.query.spec.DimExtractionFn; import java.nio.ByteBuffer; import java.text.ParseException; diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java index cd15c74f64b..aa1fb1efd2b 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java @@ -22,7 +22,7 @@ package com.metamx.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.query.spec.DimExtractionFn; +import com.metamx.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index 3c0c42cd848..0ff428fe94b 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -30,10 +30,12 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.druid.BaseQuery; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.Queries; import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.group.having.HavingSpec; import com.metamx.druid.query.group.orderby.DefaultLimitSpec; @@ -41,9 +43,8 @@ import com.metamx.druid.query.group.orderby.LimitSpec; import com.metamx.druid.query.group.orderby.NoopLimitSpec; import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import io.druid.granularity.QueryGranularity; -import io.druid.query.BaseQuery; +import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -209,7 +210,7 @@ public class GroupByQuery extends BaseQuery @Override public String getType() { - return "groupBy"; + return Query.GROUP_BY; } public Sequence applyLimit(Sequence results) diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index cf4d733d6ac..d4e4e00f816 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -36,6 +36,7 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; +import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; @@ -43,7 +44,6 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; -import io.druid.query.spec.DimensionSpec; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java index 4dc9ebac057..fdd3f40c13f 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java @@ -33,8 +33,8 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; +import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java index 3814d8ab93d..1a5283a8716 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java @@ -25,8 +25,8 @@ import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; +import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java index faf9ab760cd..339a5f157b4 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java @@ -24,8 +24,8 @@ import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; +import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java index 99e6c29b5c2..0818f85a488 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java @@ -21,7 +21,7 @@ package com.metamx.druid.query.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.BaseQuery; +import com.metamx.druid.BaseQuery; import io.druid.query.Query; import io.druid.query.spec.QuerySegmentSpec; @@ -69,7 +69,7 @@ public class SegmentMetadataQuery extends BaseQuery @Override public String getType() { - return "segmentMetadata"; + return Query.SEGMENT_METADATA; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java index c6b3d329709..4e042772f04 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java @@ -24,11 +24,12 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.metamx.druid.BaseQuery; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; import io.druid.granularity.QueryGranularity; -import io.druid.query.BaseQuery; +import io.druid.query.Query; import io.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; @@ -90,7 +91,7 @@ public class SearchQuery extends BaseQuery> @Override public String getType() { - return "search"; + return Query.SEARCH; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java new file mode 100644 index 00000000000..541eede6e94 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.segment; + +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; +import org.joda.time.Interval; + +/** + */ +public interface QuerySegmentWalker +{ + /** + * Gets the Queryable for a given interval, the Queryable returned can be any version(s) or partitionNumber(s) + * such that it represents the interval. + * + * @param intervals the intervals to find a Queryable for + * @return a Queryable object that represents the interval + */ + public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals); + + /** + * Gets the Queryable for a given list of SegmentSpecs. + * exist. + * + * @return the Queryable object with the given SegmentSpecs + */ + public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs); +} diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java index 2587467d930..33465daba52 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java +++ b/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java @@ -21,7 +21,6 @@ package com.metamx.druid.query.segment; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java index a462d10ad2d..e32e4617d3d 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java @@ -24,9 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.metamx.druid.BaseQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeBoundaryResultValue; -import io.druid.query.BaseQuery; +import io.druid.query.Query; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; @@ -73,7 +74,7 @@ public class TimeBoundaryQuery extends BaseQuery @Override public String getType() { - return "timeBoundary"; + return Query.TIME_BOUNDARY; } @Override diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index 8be56a4e3d4..ccd15540632 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -23,13 +23,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; +import com.metamx.druid.BaseQuery; import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.Queries; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; -import io.druid.query.BaseQuery; +import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.spec.QuerySegmentSpec; @@ -75,7 +76,7 @@ public class TimeseriesQuery extends BaseQuery> @Override public String getType() { - return "timeseries"; + return Query.TIMESERIES; } @JsonProperty("filter") diff --git a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java index fc400082bc5..c2c01574f0c 100644 --- a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java +++ b/client/src/main/java/com/metamx/druid/sql/SQLRunner.java @@ -30,6 +30,7 @@ import com.google.common.io.Closeables; import com.metamx.druid.Druids; import com.metamx.druid.input.Row; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; @@ -37,7 +38,6 @@ import com.metamx.druid.sql.antlr4.DruidSQLLexer; import com.metamx.druid.sql.antlr4.DruidSQLParser; import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import org.antlr.v4.runtime.ANTLRInputStream; import org.antlr.v4.runtime.CharStream; import org.antlr.v4.runtime.CommonTokenStream; diff --git a/client/src/main/java/io/druid/query/CacheStrategy.java b/client/src/main/java/io/druid/query/CacheStrategy.java new file mode 100644 index 00000000000..f77affcb5bf --- /dev/null +++ b/client/src/main/java/io/druid/query/CacheStrategy.java @@ -0,0 +1,39 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.metamx.common.guava.Sequence; + +/** +*/ +public interface CacheStrategy> +{ + public byte[] computeCacheKey(QueryType query); + + public TypeReference getCacheObjectClazz(); + + public Function prepareForCache(); + + public Function pullFromCache(); + + public Sequence mergeSequences(Sequence> seqOfSequences); +} diff --git a/client/src/main/java/io/druid/query/Query.java b/client/src/main/java/io/druid/query/Query.java new file mode 100644 index 00000000000..342a0b332b3 --- /dev/null +++ b/client/src/main/java/io/druid/query/Query.java @@ -0,0 +1,75 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.metamx.common.guava.Sequence; +import com.metamx.druid.query.group.GroupByQuery; +import com.metamx.druid.query.metadata.SegmentMetadataQuery; +import com.metamx.druid.query.search.SearchQuery; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; +import com.metamx.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.spec.QuerySegmentSpec; +import org.joda.time.Duration; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "queryType") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = Query.TIMESERIES, value = TimeseriesQuery.class), + @JsonSubTypes.Type(name = Query.SEARCH, value = SearchQuery.class), + @JsonSubTypes.Type(name = Query.TIME_BOUNDARY, value = TimeBoundaryQuery.class), + @JsonSubTypes.Type(name = Query.GROUP_BY, value = GroupByQuery.class), + @JsonSubTypes.Type(name = Query.SEGMENT_METADATA, value = SegmentMetadataQuery.class) +}) +public interface Query +{ + public static final String TIMESERIES = "timeseries"; + public static final String SEARCH = "search"; + public static final String TIME_BOUNDARY = "timeBoundary"; + public static final String GROUP_BY = "groupBy"; + public static final String SEGMENT_METADATA = "segmentMetadata"; + + public String getDataSource(); + + public boolean hasFilters(); + + public String getType(); + + public Sequence run(QuerySegmentWalker walker); + + public Sequence run(QueryRunner runner); + + public List getIntervals(); + + public Duration getDuration(); + + public String getContextValue(String key); + + public String getContextValue(String key, String defaultValue); + + public Query withOverriddenContext(Map contextOverride); + + public Query withQuerySegmentSpec(QuerySegmentSpec spec); +} diff --git a/client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/client/src/main/java/io/druid/query/QueryRunner.java similarity index 71% rename from client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java rename to client/src/main/java/io/druid/query/QueryRunner.java index b7bd32d3e99..62c44ad5163 100644 --- a/client/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/client/src/main/java/io/druid/query/QueryRunner.java @@ -17,22 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.jackson; +package io.druid.query; -import com.fasterxml.jackson.core.JsonFactory; +import com.metamx.common.guava.Sequence; /** */ -public class DefaultObjectMapper extends CommonObjectMapper +public interface QueryRunner { - public DefaultObjectMapper() - { - this(null); - } - - public DefaultObjectMapper(JsonFactory factory) - { - super(factory); - registerModule(new QueryRegisteringModule()); - } + public Sequence run(Query query); } diff --git a/client/src/main/java/io/druid/query/QueryToolChest.java b/client/src/main/java/io/druid/query/QueryToolChest.java new file mode 100644 index 00000000000..4ed1e40d79e --- /dev/null +++ b/client/src/main/java/io/druid/query/QueryToolChest.java @@ -0,0 +1,66 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.metamx.common.guava.Sequence; +import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.segment.LogicalSegment; + +import java.util.List; + +/** + * The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing + * evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential + * maintenance burden when upgrading versions. + */ +public abstract class QueryToolChest> +{ + public abstract QueryRunner mergeResults(QueryRunner runner); + + /** + * This method doesn't belong here, but it's here for now just to make it work. + * + * @param seqOfSequences + * @return + */ + public abstract Sequence mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract TypeReference getResultTypeReference(); + + public CacheStrategy getCacheStrategy(QueryType query) { + return null; + } + + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public List filterSegments(QueryType query, List segments) { + return segments; + } +} diff --git a/client/src/main/java/io/druid/query/SegmentDescriptor.java b/client/src/main/java/io/druid/query/SegmentDescriptor.java new file mode 100644 index 00000000000..0146a33bcbd --- /dev/null +++ b/client/src/main/java/io/druid/query/SegmentDescriptor.java @@ -0,0 +1,106 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +/** +*/ +public class SegmentDescriptor +{ + private final Interval interval; + private final String version; + private final int partitionNumber; + + @JsonCreator + public SegmentDescriptor( + @JsonProperty("itvl") Interval interval, + @JsonProperty("ver") String version, + @JsonProperty("part") int partitionNumber) + { + this.interval = interval; + this.version = version; + this.partitionNumber = partitionNumber; + } + + @JsonProperty("itvl") + public Interval getInterval() + { + return interval; + } + + @JsonProperty("ver") + public String getVersion() + { + return version; + } + + @JsonProperty("part") + public int getPartitionNumber() + { + return partitionNumber; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SegmentDescriptor that = (SegmentDescriptor) o; + + if (partitionNumber != that.partitionNumber) { + return false; + } + if (interval != null ? !interval.equals(that.interval) : that.interval != null) { + return false; + } + if (version != null ? !version.equals(that.version) : that.version != null) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = interval != null ? interval.hashCode() : 0; + result = 31 * result + (version != null ? version.hashCode() : 0); + result = 31 * result + partitionNumber; + return result; + } + + @Override + public String toString() + { + return "SegmentDescriptor{" + + "interval=" + interval + + ", version='" + version + '\'' + + ", partitionNumber=" + partitionNumber + + '}'; + } +} diff --git a/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java b/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java new file mode 100644 index 00000000000..0e9bab40e60 --- /dev/null +++ b/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation; + +/** +*/ +public interface MetricManipulationFn +{ + public Object manipulate(AggregatorFactory factory, Object object); +} diff --git a/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java b/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java new file mode 100644 index 00000000000..3e27c464f46 --- /dev/null +++ b/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java @@ -0,0 +1,69 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.spec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.common.IAE; +import org.joda.time.Interval; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + */ +public class LegacySegmentSpec extends MultipleIntervalSegmentSpec +{ + private static List convertValue(Object intervals) + { + final List intervalStringList; + if (intervals instanceof String) { + intervalStringList = Arrays.asList((((String) intervals).split(","))); + } else if (intervals instanceof Map) { + intervalStringList = (List) ((Map) intervals).get("intervals"); + } else if (intervals instanceof List) { + intervalStringList = (List) intervals; + } else { + throw new IAE("Unknown type[%s] for intervals[%s]", intervals.getClass(), intervals); + } + + return Lists.transform( + intervalStringList, + new Function() + { + @Override + public Interval apply(Object input) + { + return new Interval(input); + } + } + ); + } + + @JsonCreator + public LegacySegmentSpec( + Object intervals + ) + { + super(convertValue(intervals)); + } +} diff --git a/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java new file mode 100644 index 00000000000..5044679f9cf --- /dev/null +++ b/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java @@ -0,0 +1,67 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.spec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import com.metamx.druid.utils.JodaUtils; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import org.joda.time.Interval; + +import java.util.Collections; +import java.util.List; + +/** + */ +public class MultipleIntervalSegmentSpec implements QuerySegmentSpec +{ + private final List intervals; + + @JsonCreator + public MultipleIntervalSegmentSpec( + @JsonProperty("intervals") List intervals + ) + { + this.intervals = Collections.unmodifiableList(JodaUtils.condenseIntervals(intervals)); + } + + @Override + @JsonProperty("intervals") + public List getIntervals() + { + return intervals; + } + + @Override + public QueryRunner lookup(Query query, QuerySegmentWalker walker) + { + return walker.getQueryRunnerForIntervals(query, intervals); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "intervals=" + intervals + + '}'; + } +} diff --git a/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java b/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java new file mode 100644 index 00000000000..993f2c01d9e --- /dev/null +++ b/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java @@ -0,0 +1,94 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.spec; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import com.metamx.druid.utils.JodaUtils; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import io.druid.query.SegmentDescriptor; +import org.joda.time.Interval; + +import java.util.List; + +/** + */ +public class MultipleSpecificSegmentSpec implements QuerySegmentSpec +{ + private final List descriptors; + + private volatile List intervals = null; + + @JsonCreator + public MultipleSpecificSegmentSpec( + @JsonProperty("segments") List descriptors + ) + { + this.descriptors = descriptors; + } + + @JsonProperty("segments") + public List getDescriptors() + { + return descriptors; + } + + @Override + public List getIntervals() + { + if (intervals != null) { + return intervals; + } + + intervals = JodaUtils.condenseIntervals( + Iterables.transform( + descriptors, + new Function() + { + @Override + public Interval apply(SegmentDescriptor input) + { + return input.getInterval(); + } + } + ) + ); + + return intervals; + } + + @Override + public QueryRunner lookup(Query query, QuerySegmentWalker walker) + { + return walker.getQueryRunnerForSegments(query, descriptors); + } + + @Override + public String toString() + { + return "MultipleSpecificSegmentSpec{" + + "descriptors=" + descriptors + + '}'; + } +} diff --git a/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java b/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java new file mode 100644 index 00000000000..2bf8c5d7425 --- /dev/null +++ b/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java @@ -0,0 +1,43 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.spec; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.metamx.druid.query.segment.QuerySegmentWalker; +import io.druid.query.Query; +import io.druid.query.QueryRunner; +import org.joda.time.Interval; + +import java.util.List; + +/** + */ +@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = LegacySegmentSpec.class) +@JsonSubTypes(value={ + @JsonSubTypes.Type(name="intervals", value=MultipleIntervalSegmentSpec.class), + @JsonSubTypes.Type(name="segments", value=MultipleSpecificSegmentSpec.class) +}) +public interface QuerySegmentSpec +{ + public List getIntervals(); + + public QueryRunner lookup(Query query, QuerySegmentWalker walker); +} diff --git a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java index a894756dd6e..ed6e369a1bf 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java @@ -20,7 +20,6 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; -import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java index e430695836b..790f0c5adc1 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java @@ -20,7 +20,6 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; -import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java index 55c6bc696cf..ab825199e6b 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java @@ -22,7 +22,6 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; import com.metamx.druid.query.search.FragmentSearchQuerySpec; import com.metamx.druid.query.search.SearchQuerySpec; -import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java b/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java index 53ce56f981a..56d3213c633 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java +++ b/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java @@ -20,7 +20,6 @@ package com.metamx.druid.query.extraction; import com.google.common.collect.Sets; -import io.druid.query.spec.DimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java similarity index 93% rename from common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java rename to common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 85775df1040..f098d60309a 100644 --- a/common/src/main/java/com/metamx/druid/jackson/CommonObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -28,14 +28,14 @@ import com.fasterxml.jackson.datatype.guava.GuavaModule; /** */ -public class CommonObjectMapper extends ObjectMapper +public class DefaultObjectMapper extends ObjectMapper { - public CommonObjectMapper() + public DefaultObjectMapper() { this(null); } - public CommonObjectMapper(JsonFactory factory) + public DefaultObjectMapper(JsonFactory factory) { super(factory); registerModule(new DruidDefaultSerializersModule()); diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java index 33eb36125f7..694f4d2bf4c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -42,13 +42,13 @@ public class JacksonModule implements Module @Provides @LazySingleton @Json public ObjectMapper jsonMapper() { - return new CommonObjectMapper(); + return new DefaultObjectMapper(); } @Provides @LazySingleton @Smile public ObjectMapper smileMapper() { - ObjectMapper retVal = new CommonObjectMapper(new SmileFactory()); + ObjectMapper retVal = new DefaultObjectMapper(new SmileFactory()); retVal.getJsonFactory().setCodec(retVal); return retVal; } diff --git a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java index f98389cd457..fd2eab75cab 100644 --- a/common/src/test/java/com/metamx/druid/QueryGranularityTest.java +++ b/common/src/test/java/com/metamx/druid/QueryGranularityTest.java @@ -22,7 +22,7 @@ package com.metamx.druid; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.metamx.druid.jackson.CommonObjectMapper; +import com.metamx.druid.jackson.DefaultObjectMapper; import io.druid.granularity.DurationGranularity; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; @@ -437,7 +437,7 @@ public class QueryGranularityTest @Test public void testSerializePeriod() throws Exception { - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); String json = "{ \"type\": \"period\", \"period\": \"P1D\" }"; QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); @@ -462,7 +462,7 @@ public class QueryGranularityTest @Test public void testSerializeDuration() throws Exception { - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); String json = "{ \"type\": \"duration\", \"duration\": \"3600000\" }"; QueryGranularity gran = mapper.readValue(json, QueryGranularity.class); @@ -479,7 +479,7 @@ public class QueryGranularityTest @Test public void testSerializeSimple() throws Exception { - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); Assert.assertEquals( QueryGranularity.ALL, @@ -501,7 +501,7 @@ public class QueryGranularityTest @Test public void testDeserializeSimple() throws Exception { - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"all\"", QueryGranularity.class)); Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class)); diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index fa5f9f4e8ea..dd71ef22553 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -22,6 +22,7 @@ package com.metamx.druid.histogram; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.metamx.druid.aggregation.Histogram; +import com.metamx.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -61,7 +62,7 @@ public class HistogramTest Double[] visualBreaks = {-1.0, -0.5, 0.0, 0.5, 1.0}; Double[] visualCounts = { 123., 4., 56., 7. }; - ObjectMapper objectMapper = new ObjectMapper(); + ObjectMapper objectMapper = new DefaultObjectMapper(); String json = objectMapper.writeValueAsString(h.asVisual()); Map expectedObj = Maps.newLinkedHashMap(); diff --git a/common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java similarity index 93% rename from common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java rename to common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java index 45ca35c6ac5..8538b933183 100644 --- a/common/src/test/java/com/metamx/druid/jackson/CommonObjectMapperTest.java +++ b/common/src/test/java/com/metamx/druid/jackson/DefaultObjectMapperTest.java @@ -26,9 +26,9 @@ import org.junit.Test; /** */ -public class CommonObjectMapperTest +public class DefaultObjectMapperTest { - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); @Test public void testDateTime() throws Exception diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index aa12747ba9b..b2977cfbf65 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -52,7 +52,7 @@ import com.metamx.druid.index.serde.FloatGenericColumnSupplier; import com.metamx.druid.index.serde.LongGenericColumnPartSerde; import com.metamx.druid.index.serde.LongGenericColumnSupplier; import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier; -import com.metamx.druid.jackson.CommonObjectMapper; +import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.kv.ArrayIndexed; import com.metamx.druid.kv.ByteBufferSerializer; import com.metamx.druid.kv.ConciseCompressedIndexedInts; @@ -110,7 +110,7 @@ public class IndexIO public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); // This should really be provided by DI, should be changed once we switch around to using a DI framework - private static final ObjectMapper mapper = new CommonObjectMapper(); + private static final ObjectMapper mapper = new DefaultObjectMapper(); private static volatile IndexIOHandler handler = null; public static final int CURRENT_VERSION_ID = V9_VERSION; @@ -790,7 +790,7 @@ public class IndexIO Map columns = Maps.newHashMap(); - ObjectMapper mapper = new CommonObjectMapper(); + ObjectMapper mapper = new DefaultObjectMapper(); for (String columnName : cols) { columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java b/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java index 649f0374eb6..00c08750f77 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java +++ b/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java @@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.InputRow; -import com.metamx.druid.jackson.CommonObjectMapper; +import com.metamx.druid.jackson.DefaultObjectMapper; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; @@ -34,7 +34,7 @@ import java.nio.ByteBuffer; public class InputRowParserSerdeTest { - private final ObjectMapper jsonMapper = new CommonObjectMapper(); + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); @Test public void testStringInputRowParserSerde() throws Exception diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 7a953bc9ccc..5268ca3eeef 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -36,10 +36,10 @@ import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.NoopQueryRunner; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import org.apache.commons.io.FileUtils; import org.joda.time.Interval; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java index e59ad71a401..10c76b3559a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java @@ -28,8 +28,8 @@ import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.indexing.worker.config.WorkerConfig; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; -import io.druid.query.QuerySegmentWalker; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java index 4681b644a40..1e02b2f9bb4 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java @@ -31,6 +31,7 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.Sink; @@ -38,7 +39,6 @@ import com.metamx.emitter.EmittingLogger; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; -import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import org.joda.time.DateTime; diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 56c591de522..234688c647d 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -41,6 +41,7 @@ import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner; +import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.emitter.EmittingLogger; @@ -49,7 +50,6 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; -import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.QuerySegmentSpec; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java index 5207efc0699..85711900dbe 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java @@ -20,12 +20,12 @@ package com.metamx.druid.index.brita; import com.google.common.collect.Lists; +import com.metamx.druid.query.extraction.DimExtractionFn; import io.druid.data.Indexed; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import io.druid.query.spec.DimExtractionFn; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index 969d087efe3..ca493bd6530 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -42,9 +42,9 @@ import com.metamx.druid.guice.annotations.Global; import com.metamx.druid.index.brita.Filters; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; +import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; -import io.druid.query.spec.DimensionSpec; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index 147fa7a72ee..543eccafa35 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -38,6 +38,7 @@ import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.dimension.DefaultDimensionSpec; +import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.RegexDimFilter; import com.metamx.druid.query.group.having.EqualToHavingSpec; import com.metamx.druid.query.group.having.GreaterThanHavingSpec; @@ -50,7 +51,6 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.spec.DimensionSpec; import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -731,7 +731,7 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } From 3c1b0a8a025c27096c50b39cfbc16f73a0f9ae12 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 16:47:17 -0500 Subject: [PATCH 55/92] 1) Make it compile after merging with master --- .../com/metamx/druid/query/group/GroupByQueryRunnerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index 543eccafa35..b31e0a7f629 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -731,7 +731,7 @@ public class GroupByQueryRunnerTest ); TestHelper.assertExpectedObjects(expectedResults, runner.run(query), "normal"); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } From ca2c56d8272e0f3386882a97e5a8c30da8ecf839 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 16:50:09 -0500 Subject: [PATCH 56/92] 1) Remove PostAggregator and put it in druid-api 2) Remove io.druid files from client --- .../com/metamx/druid/sql/antlr4/DruidSQL.g4 | 2 +- .../main/java/com/metamx/druid/Druids.java | 4 +- .../com/metamx/druid/TimeseriesBinaryFn.java | 4 +- .../java/com/metamx/druid/query/Queries.java | 4 +- .../druid/query/group/GroupByQuery.java | 2 +- .../query/group/orderby/DefaultLimitSpec.java | 2 +- .../druid/query/group/orderby/LimitSpec.java | 2 +- .../query/group/orderby/NoopLimitSpec.java | 2 +- .../query/timeseries/TimeseriesQuery.java | 2 +- .../TimeseriesQueryQueryToolChest.java | 4 +- .../metamx/druid/TimeseriesBinaryFnTest.java | 4 +- .../com/metamx/druid/query/QueriesTest.java | 4 +- .../post/ArithmeticPostAggregator.java | 3 +- .../post/ConstantPostAggregator.java | 3 +- .../post/FieldAccessPostAggregator.java | 3 +- .../aggregation/post/PostAggregator.java | 47 ------------------- ...riesModule.java => AggregatorsModule.java} | 17 ++++++- .../druid/jackson/DefaultObjectMapper.java | 2 +- .../post/ArithmeticPostAggregatorTest.java | 3 +- .../druid/index/v1/IncrementalIndex.java | 4 +- .../metamx/druid/TimeseriesResultBuilder.java | 4 +- .../druid/query/group/GroupByQueryEngine.java | 2 +- .../timeseries/TimeseriesQueryEngine.java | 4 +- .../timeseries/TimeseriesQueryRunnerTest.java | 4 +- 24 files changed, 51 insertions(+), 81 deletions(-) delete mode 100644 common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java rename common/src/main/java/com/metamx/druid/jackson/{AggregatorFactoriesModule.java => AggregatorsModule.java} (74%) diff --git a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 index d1ce41a1cc5..c8c6d0effb1 100644 --- a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 +++ b/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 @@ -10,7 +10,7 @@ import com.metamx.druid.aggregation.MinAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import com.metamx.druid.aggregation.post.PostAggregator; +import io.druid.query.aggregation.PostAggregator; import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.druid.query.filter.AndDimFilter; diff --git a/client/src/main/java/com/metamx/druid/Druids.java b/client/src/main/java/com/metamx/druid/Druids.java index 53ad282a036..b6a3aa5db6e 100644 --- a/client/src/main/java/com/metamx/druid/Druids.java +++ b/client/src/main/java/com/metamx/druid/Druids.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,6 @@ package com.metamx.druid; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.query.filter.NoopDimFilter; @@ -37,6 +36,7 @@ import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.result.TimeBoundaryResultValue; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; diff --git a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java b/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java index 345a18a5f28..94377f543d0 100644 --- a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java +++ b/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,12 +20,12 @@ package com.metamx.druid; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.AllGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import java.util.LinkedHashMap; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/client/src/main/java/com/metamx/druid/query/Queries.java index 8cb56b92942..2913f8f6f3d 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/client/src/main/java/com/metamx/druid/query/Queries.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,8 +23,8 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.aggregation.post.PostAggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import javax.annotation.Nullable; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java index 0ff428fe94b..dbfbc9f1bd4 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java @@ -31,7 +31,6 @@ import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.BaseQuery; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.Queries; import com.metamx.druid.query.dimension.DefaultDimensionSpec; @@ -45,6 +44,7 @@ import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java index fdd3f40c13f..050527454c8 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java @@ -31,10 +31,10 @@ import com.google.common.primitives.Longs; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java index 1a5283a8716..88037488d4a 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java index 339a5f157b4..542caf29908 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java +++ b/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java @@ -22,10 +22,10 @@ package com.metamx.druid.query.group.orderby; import com.google.common.base.Function; import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java index ccd15540632..2b962a9e7a0 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; import com.metamx.druid.BaseQuery; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.Queries; import com.metamx.druid.query.filter.DimFilter; import com.metamx.druid.result.Result; @@ -32,6 +31,7 @@ import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.spec.QuerySegmentSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 3eb48759864..1dbf46a8d60 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -30,7 +30,6 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.druid.ResultGranularTimestampComparator; import com.metamx.druid.TimeseriesBinaryFn; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.collect.OrderedMergeSequence; import com.metamx.druid.query.IntervalChunkingQueryRunner; import com.metamx.druid.query.QueryCacheHelper; @@ -46,6 +45,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.aggregation.PostAggregator; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java b/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java index ad72b669bcc..4d4364c7523 100644 --- a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java +++ b/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,11 +26,11 @@ import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/QueriesTest.java b/client/src/test/java/com/metamx/druid/query/QueriesTest.java index a9ee0e18d42..d447d6c475a 100644 --- a/client/src/test/java/com/metamx/druid/query/QueriesTest.java +++ b/client/src/test/java/com/metamx/druid/query/QueriesTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,8 +24,8 @@ import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; import com.metamx.druid.aggregation.post.ConstantPostAggregator; import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import com.metamx.druid.aggregation.post.PostAggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import junit.framework.Assert; import org.junit.Test; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java index 4d768b6420b..5d0c9b7647e 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.IAE; +import io.druid.query.aggregation.PostAggregator; import java.util.Comparator; import java.util.Iterator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java index 21c1e70a50b..b60ac62b6a8 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,6 +22,7 @@ package com.metamx.druid.aggregation.post; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; +import io.druid.query.aggregation.PostAggregator; import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java index a06bf0cf891..777090451d2 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,6 +22,7 @@ package com.metamx.druid.aggregation.post; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; +import io.druid.query.aggregation.PostAggregator; import java.util.Comparator; import java.util.Map; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java b/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java deleted file mode 100644 index df78ad3bf7b..00000000000 --- a/common/src/main/java/com/metamx/druid/aggregation/post/PostAggregator.java +++ /dev/null @@ -1,47 +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.aggregation.post; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; - -import java.util.Comparator; -import java.util.Map; -import java.util.Set; - -/** - * Functionally similar to an Aggregator. See the Aggregator interface for more comments. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "arithmetic", value = ArithmeticPostAggregator.class), - @JsonSubTypes.Type(name = "fieldAccess", value = FieldAccessPostAggregator.class), - @JsonSubTypes.Type(name = "constant", value = ConstantPostAggregator.class) -}) -public interface PostAggregator -{ - public Set getDependentFields(); - - public Comparator getComparator(); - - public Object compute(Map combinedAggregators); - - public String getName(); -} diff --git a/common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java b/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java similarity index 74% rename from common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java rename to common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java index d566642f5ac..978d556534d 100644 --- a/common/src/main/java/com/metamx/druid/jackson/AggregatorFactoriesModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java @@ -29,17 +29,22 @@ import com.metamx.druid.aggregation.JavaScriptAggregatorFactory; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.MaxAggregatorFactory; import com.metamx.druid.aggregation.MinAggregatorFactory; +import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; +import com.metamx.druid.aggregation.post.ConstantPostAggregator; +import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; /** */ -public class AggregatorFactoriesModule extends SimpleModule +public class AggregatorsModule extends SimpleModule { - public AggregatorFactoriesModule() + public AggregatorsModule() { super("AggregatorFactories"); setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); + setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class); } @JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type") @@ -53,4 +58,12 @@ public class AggregatorFactoriesModule extends SimpleModule @JsonSubTypes.Type(name="histogram", value=HistogramAggregatorFactory.class) }) public static interface AggregatorFactoryMixin {} + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "arithmetic", value = ArithmeticPostAggregator.class), + @JsonSubTypes.Type(name = "fieldAccess", value = FieldAccessPostAggregator.class), + @JsonSubTypes.Type(name = "constant", value = ConstantPostAggregator.class) + }) + public static interface PostAggregatorMixin {} } diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index f098d60309a..7f5f37262f3 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -41,7 +41,7 @@ public class DefaultObjectMapper extends ObjectMapper registerModule(new DruidDefaultSerializersModule()); registerModule(new GuavaModule()); registerModule(new QueryGranularityModule()); - registerModule(new AggregatorFactoriesModule()); + registerModule(new AggregatorsModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java b/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java index c8365de19b6..144483392bd 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java +++ b/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,6 +21,7 @@ package com.metamx.druid.aggregation.post; import com.google.common.collect.Lists; import com.metamx.druid.aggregation.CountAggregator; +import io.druid.query.aggregation.PostAggregator; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java index b3b08ee8166..f910eab0483 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -33,7 +33,6 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.index.v1.serde.ComplexMetricExtractor; import com.metamx.druid.index.v1.serde.ComplexMetricSerde; import com.metamx.druid.index.v1.serde.ComplexMetrics; @@ -43,6 +42,7 @@ import com.metamx.druid.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.segment.ComplexMetricSelector; import io.druid.segment.FloatMetricSelector; import io.druid.segment.MetricSelectorFactory; diff --git a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java index 40aad21b5cb..79440aa1ed7 100644 --- a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java +++ b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,10 +19,10 @@ package com.metamx.druid; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.PostAggregator; import org.joda.time.DateTime; import java.util.HashMap; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index ca493bd6530..bf77b1a4ec9 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -35,7 +35,6 @@ import com.metamx.common.guava.FunctionalIterator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.parsers.CloseableIterator; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.guice.annotations.Global; @@ -45,6 +44,7 @@ import com.metamx.druid.input.Row; import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.PostAggregator; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java index 8f64b0321f6..83e4b53f954 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,13 +23,13 @@ import com.google.common.base.Function; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.druid.TimeseriesResultBuilder; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.index.brita.Filters; import com.metamx.druid.query.QueryRunnerHelper; import com.metamx.druid.result.Result; import com.metamx.druid.result.TimeseriesResultValue; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.segment.Cursor; import io.druid.segment.StorageAdapter; diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 612108fdd71..71824e90729 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -27,7 +27,6 @@ import com.metamx.druid.TestHelper; import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.aggregation.MaxAggregatorFactory; import com.metamx.druid.aggregation.MinAggregatorFactory; -import com.metamx.druid.aggregation.post.PostAggregator; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.filter.AndDimFilter; import com.metamx.druid.query.filter.DimFilter; @@ -38,6 +37,7 @@ import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.QueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; From 846b3c9b94b69c275772b148f19a583d9f95505d Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 16:51:05 -0500 Subject: [PATCH 57/92] 1) Spell better --- .../com/metamx/druid/query/group/GroupByQueryRunnerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index b31e0a7f629..d9a6fea2c49 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -701,7 +701,7 @@ public class GroupByQueryRunnerTest } @Test - public void testGroupByWithNonexistantDimension() throws Exception + public void testGroupByWithNonexistentDimension() throws Exception { GroupByQuery.Builder builder = GroupByQuery .builder() From 2e26a4d8b9bf91f6a1175733bdcb37a30852dcfa Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 29 Aug 2013 17:23:16 -0500 Subject: [PATCH 58/92] 1) Add back segment-related interfaces and classes to new druid-segment module --- pom.xml | 3 +- segment/pom.xml | 125 ++++++++++++++++++ .../src/main/java/io/druid/data/Indexed.java | 31 +++++ .../java/io/druid/data/IndexedFloats.java | 32 +++++ .../main/java/io/druid/data/IndexedInts.java | 29 ++++ .../main/java/io/druid/data/IndexedLongs.java | 34 +++++ .../java/io/druid/segment/Capabilities.java | 62 +++++++++ .../java/io/druid/segment/ColumnSelector.java | 28 ++++ .../druid/segment/ComplexMetricSelector.java | 26 ++++ .../main/java/io/druid/segment/Cursor.java | 30 +++++ .../java/io/druid/segment/CursorFactory.java | 29 ++++ .../io/druid/segment/DimensionSelector.java | 82 ++++++++++++ .../segment/DimensionSelectorFactory.java | 25 ++++ .../io/druid/segment/FloatMetricSelector.java | 28 ++++ .../java/io/druid/segment/LogicalSegment.java | 27 ++++ .../druid/segment/MetricSelectorFactory.java | 28 ++++ .../druid/segment/ObjectMetricSelector.java | 24 ++++ .../java/io/druid/segment/QueryableIndex.java | 40 ++++++ .../main/java/io/druid/segment/Segment.java | 32 +++++ .../java/io/druid/segment/StorageAdapter.java | 35 +++++ .../io/druid/segment/column/BitmapIndex.java | 33 +++++ .../java/io/druid/segment/column/Column.java | 35 +++++ .../segment/column/ColumnCapabilities.java | 33 +++++ .../druid/segment/column/ComplexColumn.java | 31 +++++ .../column/DictionaryEncodedColumn.java | 35 +++++ .../druid/segment/column/GenericColumn.java | 42 ++++++ .../druid/segment/column/RunLengthColumn.java | 27 ++++ .../io/druid/segment/column/SpatialIndex.java | 28 ++++ .../io/druid/segment/column/ValueType.java | 30 +++++ server/pom.xml | 4 +- 30 files changed, 1045 insertions(+), 3 deletions(-) create mode 100644 segment/pom.xml create mode 100644 segment/src/main/java/io/druid/data/Indexed.java create mode 100644 segment/src/main/java/io/druid/data/IndexedFloats.java create mode 100644 segment/src/main/java/io/druid/data/IndexedInts.java create mode 100644 segment/src/main/java/io/druid/data/IndexedLongs.java create mode 100644 segment/src/main/java/io/druid/segment/Capabilities.java create mode 100644 segment/src/main/java/io/druid/segment/ColumnSelector.java create mode 100644 segment/src/main/java/io/druid/segment/ComplexMetricSelector.java create mode 100644 segment/src/main/java/io/druid/segment/Cursor.java create mode 100644 segment/src/main/java/io/druid/segment/CursorFactory.java create mode 100644 segment/src/main/java/io/druid/segment/DimensionSelector.java create mode 100644 segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java create mode 100644 segment/src/main/java/io/druid/segment/FloatMetricSelector.java create mode 100644 segment/src/main/java/io/druid/segment/LogicalSegment.java create mode 100644 segment/src/main/java/io/druid/segment/MetricSelectorFactory.java create mode 100644 segment/src/main/java/io/druid/segment/ObjectMetricSelector.java create mode 100644 segment/src/main/java/io/druid/segment/QueryableIndex.java create mode 100644 segment/src/main/java/io/druid/segment/Segment.java create mode 100644 segment/src/main/java/io/druid/segment/StorageAdapter.java create mode 100644 segment/src/main/java/io/druid/segment/column/BitmapIndex.java create mode 100644 segment/src/main/java/io/druid/segment/column/Column.java create mode 100644 segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java create mode 100644 segment/src/main/java/io/druid/segment/column/ComplexColumn.java create mode 100644 segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java create mode 100644 segment/src/main/java/io/druid/segment/column/GenericColumn.java create mode 100644 segment/src/main/java/io/druid/segment/column/RunLengthColumn.java create mode 100644 segment/src/main/java/io/druid/segment/column/SpatialIndex.java create mode 100644 segment/src/main/java/io/druid/segment/column/ValueType.java diff --git a/pom.xml b/pom.xml index 7c6c323b069..2ae494402b5 100644 --- a/pom.xml +++ b/pom.xml @@ -1,7 +1,7 @@ + + + 4.0.0 + com.metamx.druid + druid-segment + druid-segment + A module that is everything required to understands Druid Segments + + + com.metamx + druid + 0.5.50-SNAPSHOT + + + + + com.metamx.druid + druid-indexing-common + ${project.parent.version} + + + + com.metamx + java-util + + + + com.ning + compress-lzf + + + org.skife.config + config-magic + + + it.uniroma3.mat + extendedset + + + com.google.guava + guava + + + com.google.inject + guice + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + javax.inject + javax.inject + + + joda-time + joda-time + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + junit + junit + test + + + org.easymock + easymock + test + + + com.metamx.druid + druid-indexing-common + ${project.parent.version} + test-jar + test + + + + + + + maven-jar-plugin + + + + test-jar + + + + + + + diff --git a/segment/src/main/java/io/druid/data/Indexed.java b/segment/src/main/java/io/druid/data/Indexed.java new file mode 100644 index 00000000000..447fcf4a407 --- /dev/null +++ b/segment/src/main/java/io/druid/data/Indexed.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data; + +/** + * @param + */ +public interface Indexed extends Iterable +{ + Class getClazz(); + int size(); + T get(int index); + int indexOf(T value); +} diff --git a/segment/src/main/java/io/druid/data/IndexedFloats.java b/segment/src/main/java/io/druid/data/IndexedFloats.java new file mode 100644 index 00000000000..78a1a02016e --- /dev/null +++ b/segment/src/main/java/io/druid/data/IndexedFloats.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data; + +import java.io.Closeable; + +/** + * Get a float at an index (array or list lookup abstraction without boxing). + */ +public interface IndexedFloats extends Closeable +{ + public int size(); + public float get(int index); + public void fill(int index, float[] toFill); +} diff --git a/segment/src/main/java/io/druid/data/IndexedInts.java b/segment/src/main/java/io/druid/data/IndexedInts.java new file mode 100644 index 00000000000..85bb8aa8b18 --- /dev/null +++ b/segment/src/main/java/io/druid/data/IndexedInts.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data; + +/** + * Get a int an index (array or list lookup abstraction without boxing). + */ +public interface IndexedInts extends Iterable +{ + int size(); + int get(int index); +} diff --git a/segment/src/main/java/io/druid/data/IndexedLongs.java b/segment/src/main/java/io/druid/data/IndexedLongs.java new file mode 100644 index 00000000000..d42d2d166af --- /dev/null +++ b/segment/src/main/java/io/druid/data/IndexedLongs.java @@ -0,0 +1,34 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data; + +import java.io.Closeable; + +/** + * Get a long at an index (array or list lookup abstraction without boxing). + */ +public interface IndexedLongs extends Closeable +{ + public int size(); + public long get(int index); + public void fill(int index, long[] toFill); + int binarySearch(long key); + int binarySearch(long key, int from, int to); +} diff --git a/segment/src/main/java/io/druid/segment/Capabilities.java b/segment/src/main/java/io/druid/segment/Capabilities.java new file mode 100644 index 00000000000..574f07f4efe --- /dev/null +++ b/segment/src/main/java/io/druid/segment/Capabilities.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;/** + */ +public class Capabilities +{ + private final boolean dimensionValuesSorted; + + public static CapabilitiesBuilder builder() + { + return new CapabilitiesBuilder(); + } + + private Capabilities( + boolean dimensionValuesSorted + ) + { + this.dimensionValuesSorted = dimensionValuesSorted; + } + + public boolean dimensionValuesSorted() + { + return dimensionValuesSorted; + } + + public static class CapabilitiesBuilder + { + private boolean dimensionValuesSorted = false; + + private CapabilitiesBuilder() {} + + public CapabilitiesBuilder dimensionValuesSorted(boolean value) + { + dimensionValuesSorted = value; + return this; + } + + public io.druid.segment.Capabilities build() + { + return new io.druid.segment.Capabilities( + dimensionValuesSorted + ); + } + } +} diff --git a/segment/src/main/java/io/druid/segment/ColumnSelector.java b/segment/src/main/java/io/druid/segment/ColumnSelector.java new file mode 100644 index 00000000000..0710effb9d2 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/ColumnSelector.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import io.druid.segment.column.Column; + +/** + */ +public interface ColumnSelector +{ + public Column getTimeColumn(); + public Column getColumn(String columnName); +} diff --git a/segment/src/main/java/io/druid/segment/ComplexMetricSelector.java b/segment/src/main/java/io/druid/segment/ComplexMetricSelector.java new file mode 100644 index 00000000000..67a733c00ae --- /dev/null +++ b/segment/src/main/java/io/druid/segment/ComplexMetricSelector.java @@ -0,0 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;/** + */ +public interface ComplexMetricSelector +{ + public Class classOfObject(); + public T get(); +} diff --git a/segment/src/main/java/io/druid/segment/Cursor.java b/segment/src/main/java/io/druid/segment/Cursor.java new file mode 100644 index 00000000000..fa50e52b8e8 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/Cursor.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import org.joda.time.DateTime; + +/** + */ +public interface Cursor extends MetricSelectorFactory, DimensionSelectorFactory +{ + public DateTime getTime(); + public void advance(); + public boolean isDone(); + public void reset(); +} diff --git a/segment/src/main/java/io/druid/segment/CursorFactory.java b/segment/src/main/java/io/druid/segment/CursorFactory.java new file mode 100644 index 00000000000..05a72bad543 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/CursorFactory.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import io.druid.granularity.QueryGranularity; +import io.druid.query.filter.Filter; +import org.joda.time.Interval; + +/** + */ +public interface CursorFactory +{ + public Iterable makeCursors(Filter filter, Interval interval, QueryGranularity gran); +} diff --git a/segment/src/main/java/io/druid/segment/DimensionSelector.java b/segment/src/main/java/io/druid/segment/DimensionSelector.java new file mode 100644 index 00000000000..8011af47eaf --- /dev/null +++ b/segment/src/main/java/io/druid/segment/DimensionSelector.java @@ -0,0 +1,82 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import io.druid.data.IndexedInts; + +/** + */ +public interface DimensionSelector +{ + /** + * Gets all values for the row inside of an IntBuffer. I.e. one possible implementation could be + * + * return IntBuffer.wrap(lookupExpansion(get()); + * + * @return all values for the row as an IntBuffer + */ + public IndexedInts getRow(); + + /** + * Value cardinality is the cardinality of the different occurring values. If there were 4 rows: + * + * A,B + * A + * B + * A + * + * Value cardinality would be 2. + * + * @return + */ + public int getValueCardinality(); + + /** + * The Name is the String name of the actual field. It is assumed that storage layers convert names + * into id values which can then be used to get the string value. For example + * + * A,B + * A + * A,B + * B + * + * getRow() would return + * + * getRow(0) => [0 1] + * getRow(1) => [0] + * getRow(2) => [0 1] + * getRow(3) => [1] + * + * and then lookupName would return: + * + * lookupName(0) => A + * lookupName(1) => B + * + * @param id + * @return + */ + public String lookupName(int id); + + /** + * The ID is the int id value of the field. + * + * @param name + * @return + */ + public int lookupId(String name); +} diff --git a/segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java b/segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java new file mode 100644 index 00000000000..b4a1f8ca2fb --- /dev/null +++ b/segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java @@ -0,0 +1,25 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;/** + */ +public interface DimensionSelectorFactory +{ + public DimensionSelector makeDimensionSelector(String dimensionName); +} diff --git a/segment/src/main/java/io/druid/segment/FloatMetricSelector.java b/segment/src/main/java/io/druid/segment/FloatMetricSelector.java new file mode 100644 index 00000000000..300ab25b0dd --- /dev/null +++ b/segment/src/main/java/io/druid/segment/FloatMetricSelector.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;/** + * An object that gets a metric value. Metric values are always floats and there is an assumption that the + * FloatMetricSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls + * to get() (though, that doesn't have to be the case if you always want the same value...). + */ +public interface FloatMetricSelector +{ + public float get(); +} diff --git a/segment/src/main/java/io/druid/segment/LogicalSegment.java b/segment/src/main/java/io/druid/segment/LogicalSegment.java new file mode 100644 index 00000000000..0f29cea60be --- /dev/null +++ b/segment/src/main/java/io/druid/segment/LogicalSegment.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import org.joda.time.Interval; + +public interface LogicalSegment +{ + public Interval getInterval(); +} diff --git a/segment/src/main/java/io/druid/segment/MetricSelectorFactory.java b/segment/src/main/java/io/druid/segment/MetricSelectorFactory.java new file mode 100644 index 00000000000..d8f743eabd9 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/MetricSelectorFactory.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;/** + * Factory class for MetricSelectors + */ +public interface MetricSelectorFactory +{ + public FloatMetricSelector makeFloatMetricSelector(String metricName); + public ComplexMetricSelector makeComplexMetricSelector(String metricName); + public ObjectMetricSelector makeObjectColumnSelector(String columnName); +} diff --git a/segment/src/main/java/io/druid/segment/ObjectMetricSelector.java b/segment/src/main/java/io/druid/segment/ObjectMetricSelector.java new file mode 100644 index 00000000000..63941227d19 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/ObjectMetricSelector.java @@ -0,0 +1,24 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;public interface ObjectMetricSelector +{ + public Class classOfObject(); + public T get(); +} diff --git a/segment/src/main/java/io/druid/segment/QueryableIndex.java b/segment/src/main/java/io/druid/segment/QueryableIndex.java new file mode 100644 index 00000000000..d82960fa570 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/QueryableIndex.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import io.druid.data.Indexed; +import org.joda.time.Interval; + +import java.io.IOException; + +/** + */ +public interface QueryableIndex extends ColumnSelector +{ + public Interval getDataInterval(); + public int getNumRows(); + public Indexed getColumnNames(); + public Indexed getAvailableDimensions(); + + /** + * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. + * @throws java.io.IOException + */ + @Deprecated + public void close() throws IOException; +} diff --git a/segment/src/main/java/io/druid/segment/Segment.java b/segment/src/main/java/io/druid/segment/Segment.java new file mode 100644 index 00000000000..d72fac930a9 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/Segment.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import org.joda.time.Interval; + +import java.io.Closeable; + +/** + */ +public interface Segment extends Closeable +{ + public String getIdentifier(); + public Interval getDataInterval(); + public QueryableIndex asQueryableIndex(); + public StorageAdapter asStorageAdapter(); +} diff --git a/segment/src/main/java/io/druid/segment/StorageAdapter.java b/segment/src/main/java/io/druid/segment/StorageAdapter.java new file mode 100644 index 00000000000..cfd463b9e3b --- /dev/null +++ b/segment/src/main/java/io/druid/segment/StorageAdapter.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment;import io.druid.data.Indexed; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +/** + */ +public interface StorageAdapter extends CursorFactory +{ + public String getSegmentIdentifier(); + public Interval getInterval(); + public Indexed getAvailableDimensions(); + public int getDimensionCardinality(String dimension); + public DateTime getMinTime(); + public DateTime getMaxTime(); + public Capabilities getCapabilities(); +} diff --git a/segment/src/main/java/io/druid/segment/column/BitmapIndex.java b/segment/src/main/java/io/druid/segment/column/BitmapIndex.java new file mode 100644 index 00000000000..573b38982ce --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/BitmapIndex.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** + */ +public interface BitmapIndex +{ + public int getCardinality(); + public String getValue(int index); + public boolean hasNulls(); + public ImmutableConciseSet getConciseSet(String value); + public ImmutableConciseSet getConciseSet(int idx); +} diff --git a/segment/src/main/java/io/druid/segment/column/Column.java b/segment/src/main/java/io/druid/segment/column/Column.java new file mode 100644 index 00000000000..14de694e149 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/Column.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +/** + */ +public interface Column +{ + public ColumnCapabilities getCapabilities(); + + public int getLength(); + public DictionaryEncodedColumn getDictionaryEncoding(); + public RunLengthColumn getRunLengthColumn(); + public GenericColumn getGenericColumn(); + public ComplexColumn getComplexColumn(); + public BitmapIndex getBitmapIndex(); + public SpatialIndex getSpatialIndex(); +} diff --git a/segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java new file mode 100644 index 00000000000..fbb3e87fd31 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +/** + */ +public interface ColumnCapabilities +{ + public ValueType getType(); + + public boolean isDictionaryEncoded(); + public boolean isRunLengthEncoded(); + public boolean hasBitmapIndexes(); + public boolean hasSpatialIndexes(); + public boolean hasMultipleValues(); +} diff --git a/segment/src/main/java/io/druid/segment/column/ComplexColumn.java b/segment/src/main/java/io/druid/segment/column/ComplexColumn.java new file mode 100644 index 00000000000..c4dcad927e4 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/ComplexColumn.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +import java.io.Closeable; + +/** + */ +public interface ComplexColumn extends Closeable +{ + public Class getClazz(); + public String getTypeName(); + public Object getRowValue(int rowNum); +} diff --git a/segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java new file mode 100644 index 00000000000..6bf4c22a106 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +import io.druid.data.IndexedInts; + +/** + */ +public interface DictionaryEncodedColumn +{ + public int length(); + public boolean hasMultipleValues(); + public int getSingleValueRow(int rowNum); + public IndexedInts getMultiValueRow(int rowNum); + public String lookupName(int id); + public int lookupId(String name); + public int getCardinality(); +} diff --git a/segment/src/main/java/io/druid/segment/column/GenericColumn.java b/segment/src/main/java/io/druid/segment/column/GenericColumn.java new file mode 100644 index 00000000000..f66fbd3c45b --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/GenericColumn.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +import io.druid.data.Indexed; +import io.druid.data.IndexedFloats; +import io.druid.data.IndexedLongs; + +import java.io.Closeable; + +/** + */ +public interface GenericColumn extends Closeable +{ + public int length(); + public ValueType getType(); + public boolean hasMultipleValues(); + + public String getStringSingleValueRow(int rowNum); + public Indexed getStringMultiValueRow(int rowNum); + public float getFloatSingleValueRow(int rowNum); + public IndexedFloats getFloatMultiValueRow(int rowNum); + public long getLongSingleValueRow(int rowNum); + public IndexedLongs getLongMultiValueRow(int rowNum); +} diff --git a/segment/src/main/java/io/druid/segment/column/RunLengthColumn.java b/segment/src/main/java/io/druid/segment/column/RunLengthColumn.java new file mode 100644 index 00000000000..e71e41e59b4 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/RunLengthColumn.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +/** + */ +public interface RunLengthColumn +{ + public void thisIsAFictionalInterfaceThatWillHopefullyMeanSomethingSometime(); +} diff --git a/segment/src/main/java/io/druid/segment/column/SpatialIndex.java b/segment/src/main/java/io/druid/segment/column/SpatialIndex.java new file mode 100644 index 00000000000..822e878d659 --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/SpatialIndex.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +import com.metamx.collections.spatial.ImmutableRTree; + +/** + */ +public interface SpatialIndex +{ + public ImmutableRTree getRTree(); +} diff --git a/segment/src/main/java/io/druid/segment/column/ValueType.java b/segment/src/main/java/io/druid/segment/column/ValueType.java new file mode 100644 index 00000000000..a59d6a7e20b --- /dev/null +++ b/segment/src/main/java/io/druid/segment/column/ValueType.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.column; + +/** +*/ +public enum ValueType +{ + FLOAT, + LONG, + STRING, + COMPLEX +} diff --git a/server/pom.xml b/server/pom.xml index 86b3718b7dc..70e2ffc97f8 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -1,7 +1,7 @@ - - - 4.0.0 - com.metamx.druid - druid-client - druid-client - druid-client - - - com.metamx - druid - 0.5.50-SNAPSHOT - - - - - com.metamx.druid - druid-common - ${project.parent.version} - - - com.metamx.druid - druid-indexing-common - ${project.parent.version} - - - - com.metamx - emitter - - - com.metamx - http-client - - - com.metamx - java-util - - - com.metamx - server-metrics - - - - com.davekoelle - alphanum - - - commons-codec - commons-codec - - - commons-httpclient - commons-httpclient - - - org.skife.config - config-magic - - - org.apache.curator - curator-client - - - org.apache.curator - curator-framework - - - org.apache.curator - curator-x-discovery - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-servlet - - - org.eclipse.jetty - jetty-servlets - - - com.google.guava - guava - - - com.google.inject - guice - - - com.google.inject.extensions - guice-servlet - - - com.ibm.icu - icu4j - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - javax.inject - javax.inject - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.sun.jersey.contribs - jersey-guice - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - - - com.google.inject.extensions - guice-servlet - - - log4j - log4j - - - org.jboss.netty - netty - 3.2.4.Final - - - javax.servlet - javax.servlet-api - - - org.slf4j - slf4j-log4j12 - - - com.google.code.simple-spring-memcached - spymemcached - - - org.antlr - antlr4-runtime - - - commons-cli - commons-cli - - - com.metamx - bytebuffer-collections - - - net.jpountz.lz4 - lz4 - - - - - junit - junit - test - - - org.easymock - easymock - test - - - com.google.caliper - caliper - test - - - org.apache.curator - curator-test - test - - - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - org.antlr - antlr4-maven-plugin - - - - antlr4 - - - - - - - diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java b/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java deleted file mode 100644 index a0a320b1879..00000000000 --- a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryViewProvider.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.metamx.druid.client; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.initialization.ZkPathsConfig; -import org.apache.curator.framework.CuratorFramework; - -import javax.validation.constraints.NotNull; - -/** - */ -public class BatchServerInventoryViewProvider implements ServerInventoryViewProvider -{ - @JacksonInject - @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; - - @JacksonInject - @NotNull - private ObjectMapper jsonMapper = null; - - @Override - public ServerInventoryView get() - { - return new BatchServerInventoryView(zkPaths, curator, jsonMapper); - } -} diff --git a/client/src/main/java/com/metamx/druid/client/InventoryView.java b/client/src/main/java/com/metamx/druid/client/InventoryView.java deleted file mode 100644 index bbf7f6e46cc..00000000000 --- a/client/src/main/java/com/metamx/druid/client/InventoryView.java +++ /dev/null @@ -1,9 +0,0 @@ -package com.metamx.druid.client; - -/** - */ -public interface InventoryView -{ - public DruidServer getInventoryValue(String string); - public Iterable getInventory(); -} diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java deleted file mode 100644 index 6eea6e5a5e6..00000000000 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewProvider.java +++ /dev/null @@ -1,16 +0,0 @@ -package com.metamx.druid.client; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleServerInventoryProvider.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "legacy", value = SingleServerInventoryProvider.class), - @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class) -}) -public interface ServerInventoryViewProvider extends Provider -{ -} diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java deleted file mode 100644 index 93fae69a77b..00000000000 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryProvider.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.metamx.druid.client; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.initialization.ZkPathsConfig; -import org.apache.curator.framework.CuratorFramework; - -import javax.validation.constraints.NotNull; - -/** - */ -public class SingleServerInventoryProvider implements ServerInventoryViewProvider -{ - @JacksonInject - @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; - - @JacksonInject - @NotNull - private ObjectMapper jsonMapper = null; - - @Override - public ServerInventoryView get() - { - return new SingleServerInventoryView(zkPaths, curator, jsonMapper); - } -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java b/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java deleted file mode 100644 index cd0e09700b9..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheProvider.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.client.cache; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LocalCacheProvider.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "local", value = LocalCacheProvider.class), - @JsonSubTypes.Type(name = "memcached", value = MemcachedCacheProvider.class) -}) -public interface CacheProvider extends Provider -{ -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java deleted file mode 100644 index 672fa9a2bba..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheConfig.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.metamx.druid.client.cache; - -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.validation.constraints.NotNull; - -public class MemcachedCacheConfig -{ - @JsonProperty - private int expiration = 2592000; // What is this number? - - @JsonProperty - private int timeout = 500; - - @JsonProperty - @NotNull - private String hosts; - - @JsonProperty - private int maxObjectSize = 50 * 1024 * 1024; - - @JsonProperty - private String memcachedPrefix = "druid"; - - public int getExpiration() - { - return expiration; - } - - public int getTimeout() - { - return timeout; - } - - public String getHosts() - { - return hosts; - } - - public int getMaxObjectSize() - { - return maxObjectSize; - } - - public String getMemcachedPrefix() - { - return memcachedPrefix; - } -} diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java b/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java deleted file mode 100644 index a199b6323bd..00000000000 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCacheProvider.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.metamx.druid.client.cache; - -public class MemcachedCacheProvider extends MemcachedCacheConfig implements CacheProvider -{ - @Override - public Cache get() - { - return MemcachedCache.create(this); - } -} diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java deleted file mode 100644 index 4198703ee63..00000000000 --- a/client/src/main/java/com/metamx/druid/client/indexing/IndexingService.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.client.indexing; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@BindingAnnotation -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -public @interface IndexingService -{ -} diff --git a/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java b/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java deleted file mode 100644 index 1b55472e431..00000000000 --- a/client/src/main/java/com/metamx/druid/client/selector/DiscoverySelector.java +++ /dev/null @@ -1,8 +0,0 @@ -package com.metamx.druid.client.selector; - -/** - */ -public interface DiscoverySelector -{ - public T pick(); -} diff --git a/client/src/main/java/com/metamx/druid/client/selector/Server.java b/client/src/main/java/com/metamx/druid/client/selector/Server.java deleted file mode 100644 index a5f6b910e9a..00000000000 --- a/client/src/main/java/com/metamx/druid/client/selector/Server.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.metamx.druid.client.selector; - -/** - */ -public interface Server -{ - public String getScheme(); - public String getHost(); - public int getPort(); -} diff --git a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java deleted file mode 100644 index 7b849575835..00000000000 --- a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerProvider.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.metamx.druid.coordination; - -import com.fasterxml.jackson.annotation.JacksonInject; - -import javax.validation.constraints.NotNull; - -/** - */ -public class BatchDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider -{ - @JacksonInject - @NotNull - private BatchDataSegmentAnnouncer batchAnnouncer = null; - - @Override - public DataSegmentAnnouncer get() - { - return batchAnnouncer; - } -} diff --git a/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java deleted file mode 100644 index 6f43fb15c0b..00000000000 --- a/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncerProvider.java +++ /dev/null @@ -1,16 +0,0 @@ -package com.metamx.druid.coordination; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSegmentAnnouncerProvider.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "legacy", value = LegacyDataSegmentAnnouncerProvider.class), - @JsonSubTypes.Type(name = "batch", value = BatchDataSegmentAnnouncerProvider.class) -}) -public interface DataSegmentAnnouncerProvider extends Provider -{ -} diff --git a/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java b/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java deleted file mode 100644 index 86b1f5fce88..00000000000 --- a/client/src/main/java/com/metamx/druid/coordination/LegacyDataSegmentAnnouncerProvider.java +++ /dev/null @@ -1,30 +0,0 @@ -package com.metamx.druid.coordination; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.guice.annotations.Json; - -import javax.validation.constraints.NotNull; -import java.util.Arrays; - -/** - */ -public class LegacyDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider -{ - @JacksonInject - @NotNull - private SingleDataSegmentAnnouncer singleAnnouncer = null; - - @JacksonInject - @NotNull - private BatchDataSegmentAnnouncer batchAnnouncer = null; - - @Override - public DataSegmentAnnouncer get() - { - return new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( - Arrays.asList(singleAnnouncer, batchAnnouncer) - ); - } -} diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java deleted file mode 100644 index 87c65bc9837..00000000000 --- a/client/src/main/java/com/metamx/druid/curator/discovery/NoopServiceAnnouncer.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.metamx.druid.curator.discovery; - -import com.metamx.druid.initialization.DruidNode; - -/** - * Does nothing. - */ -public class NoopServiceAnnouncer implements ServiceAnnouncer -{ - @Override - public void announce(DruidNode node) - { - - } - - @Override - public void unannounce(DruidNode node) - { - - } -} diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java b/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java deleted file mode 100644 index bdd9361fd5d..00000000000 --- a/client/src/main/java/com/metamx/druid/curator/discovery/ServiceAnnouncer.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.curator.discovery; - -import com.metamx.druid.initialization.DruidNode; - -/** - * Announces our ability to serve a particular function. Multiple users may announce the same service, in which - * case they are treated as interchangeable instances of that service. - */ -public interface ServiceAnnouncer -{ - public void announce(DruidNode node); - - public void unannounce(DruidNode node); -} diff --git a/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java b/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java deleted file mode 100644 index f3a6cd46a05..00000000000 --- a/client/src/main/java/com/metamx/druid/guice/AnnouncerModule.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Provides; -import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; -import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.coordination.DataSegmentAnnouncerProvider; -import com.metamx.druid.coordination.SingleDataSegmentAnnouncer; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import org.apache.curator.framework.CuratorFramework; - -/** - */ -public class AnnouncerModule implements Module -{ - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); - JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); - binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); - binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); - binder.bind(SingleDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); - } - - @Provides - @ManageLifecycle - public Announcer getAnnouncer(CuratorFramework curator) - { - return new Announcer(curator, Execs.singleThreaded("Announcer-%s")); - } -} diff --git a/client/src/main/java/com/metamx/druid/guice/BrokerModule.java b/client/src/main/java/com/metamx/druid/guice/BrokerModule.java deleted file mode 100644 index 9cf33e4f71c..00000000000 --- a/client/src/main/java/com/metamx/druid/guice/BrokerModule.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; -import com.metamx.druid.client.BrokerServerView; -import com.metamx.druid.client.CachingClusteredClient; -import com.metamx.druid.client.TimelineServerView; -import com.metamx.druid.client.cache.Cache; -import com.metamx.druid.client.cache.CacheProvider; -import com.metamx.druid.query.MapQueryToolChestWarehouse; -import com.metamx.druid.query.QueryToolChestWarehouse; - -/** - */ -public class BrokerModule implements Module -{ - @Override - public void configure(Binder binder) - { - binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); - - binder.bind(CachingClusteredClient.class).in(LazySingleton.class); - binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); - - binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); - JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class); - } -} diff --git a/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java b/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java deleted file mode 100644 index 3c9dd142deb..00000000000 --- a/client/src/main/java/com/metamx/druid/guice/ServerViewModule.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.ServerInventoryViewProvider; -import com.metamx.druid.client.ServerView; - -/** - */ -public class ServerViewModule implements Module -{ - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); - binder.bind(InventoryView.class).to(ServerInventoryView.class); - binder.bind(ServerView.class).to(ServerInventoryView.class); - binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); - } -} diff --git a/client/src/main/java/com/metamx/druid/guice/annotations/Client.java b/client/src/main/java/com/metamx/druid/guice/annotations/Client.java deleted file mode 100644 index 6a8eabc28dc..00000000000 --- a/client/src/main/java/com/metamx/druid/guice/annotations/Client.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface Client -{ -} diff --git a/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java b/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java deleted file mode 100644 index f0a75fce898..00000000000 --- a/client/src/main/java/com/metamx/druid/http/ZkPathsModule.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.http; - -import com.google.inject.Binder; -import com.google.inject.Module; - -/** - */ -public class ZkPathsModule implements Module -{ - @Override - public void configure(Binder binder) - { - } -} diff --git a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java deleted file mode 100644 index 9aaca387ab9..00000000000 --- a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLoggerProvider.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.metamx.druid.http.log; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.metamx.emitter.service.ServiceEmitter; - -import javax.validation.constraints.NotNull; - -/** - */ -@JsonTypeName("emitter") -public class EmittingRequestLoggerProvider implements RequestLoggerProvider -{ - @JsonProperty - @NotNull - private String feed = null; - - @JacksonInject - @NotNull - private ServiceEmitter emitter = null; - - @Inject - public void injectMe(Injector injector) - { - System.out.println("YAYAYAYAYAYA!!!"); - } - - @Override - public RequestLogger get() - { - return new EmittingRequestLogger(emitter, feed); - } -} diff --git a/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java deleted file mode 100644 index 9526aa65ffe..00000000000 --- a/client/src/main/java/com/metamx/druid/http/log/FileRequestLoggerProvider.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.metamx.druid.http.log; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.druid.guice.annotations.Json; - -import javax.validation.constraints.NotNull; -import java.io.File; - -/** - */ -@JsonTypeName("file") -public class FileRequestLoggerProvider implements RequestLoggerProvider -{ - @JsonProperty - @NotNull - private File dir = null; - - @JacksonInject - @NotNull - private ScheduledExecutorFactory factory = null; - - - @JacksonInject - @NotNull - @Json - private ObjectMapper jsonMapper = null; - - @Override - public RequestLogger get() - { - return new FileRequestLogger(jsonMapper, factory.create(1, "RequestLogger-%s"), dir); - } -} diff --git a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java deleted file mode 100644 index 062f8ddc82c..00000000000 --- a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLogger.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.http.log; - -import com.metamx.druid.http.RequestLogLine; - -/** - */ -public class NoopRequestLogger implements RequestLogger -{ - @Override - public void log(RequestLogLine requestLogLine) throws Exception - { - // This is a no op! - } -} diff --git a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java deleted file mode 100644 index 0bafb7945a3..00000000000 --- a/client/src/main/java/com/metamx/druid/http/log/NoopRequestLoggerProvider.java +++ /dev/null @@ -1,12 +0,0 @@ -package com.metamx.druid.http.log; - -/** - */ -public class NoopRequestLoggerProvider implements RequestLoggerProvider -{ - @Override - public RequestLogger get() - { - return new NoopRequestLogger(); - } -} diff --git a/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java b/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java deleted file mode 100644 index 0e2ecc15b44..00000000000 --- a/client/src/main/java/com/metamx/druid/http/log/RequestLoggerProvider.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.metamx.druid.http.log; - -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.google.inject.Provider; - -/** - * A Marker interface for things that can provide a RequestLogger. This can be combined with jackson polymorphic serde - * to provide new RequestLogger implementations as plugins. - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopRequestLoggerProvider.class) -public interface RequestLoggerProvider extends Provider -{ -} diff --git a/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java b/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java deleted file mode 100644 index 3b80641f943..00000000000 --- a/client/src/main/java/com/metamx/druid/initialization/BatchDataSegmentAnnouncerConfig.java +++ /dev/null @@ -1,30 +0,0 @@ -package com.metamx.druid.initialization; - -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.validation.constraints.Max; -import javax.validation.constraints.Min; - -/** - */ -public class BatchDataSegmentAnnouncerConfig -{ - @JsonProperty - @Min(1) - private int segmentsPerNode = 50; - - @JsonProperty - @Max(1024 * 1024) - @Min(1024) - private long maxBytesPerNode = 512 * 1024; - - public int getSegmentsPerNode() - { - return segmentsPerNode; - } - - public long getMaxBytesPerNode() - { - return maxBytesPerNode; - } -} diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java b/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java deleted file mode 100644 index 05b11c98e40..00000000000 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerInitializer.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.metamx.druid.initialization; - -import com.google.inject.Injector; -import org.eclipse.jetty.server.Server; - -/** - */ -public interface JettyServerInitializer -{ - public void initialize(Server server, Injector injector); -} diff --git a/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java b/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java deleted file mode 100644 index f6838425939..00000000000 --- a/client/src/main/java/com/metamx/druid/query/MapQueryToolChestWarehouse.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.metamx.druid.query; - -import com.google.inject.Inject; -import io.druid.query.Query; -import io.druid.query.QueryToolChest; - -import java.util.Map; - -/** - */ -public class MapQueryToolChestWarehouse implements QueryToolChestWarehouse -{ - private final Map, QueryToolChest> toolchests; - - @Inject - public MapQueryToolChestWarehouse( - Map, QueryToolChest> toolchests - ) - { - this.toolchests = toolchests; - } - - @Override - @SuppressWarnings("unchecked") - public > QueryToolChest getToolChest(QueryType query) - { - return toolchests.get(query.getClass()); - } -} diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java deleted file mode 100644 index ea7be8f9a2c..00000000000 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryConfig.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.metamx.druid.query.group; - -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - */ -public class GroupByQueryConfig -{ - @JsonProperty - private boolean singleThreaded = false; - - @JsonProperty - private int maxIntermediateRows = 50000; - - @JsonProperty - private int maxResults = 500000; - - public boolean isSingleThreaded() - { - return singleThreaded; - } - - public int getMaxIntermediateRows() - { - return maxIntermediateRows; - } - - public void setMaxIntermediateRows(int maxIntermediateRows) - { - this.maxIntermediateRows = maxIntermediateRows; - } - - public int getMaxResults() - { - return maxResults; - } -} diff --git a/client/src/main/java/com/metamx/druid/query/group/having/AlwaysHavingSpec.java b/client/src/main/java/com/metamx/druid/query/group/having/AlwaysHavingSpec.java deleted file mode 100644 index 3fa27b99ace..00000000000 --- a/client/src/main/java/com/metamx/druid/query/group/having/AlwaysHavingSpec.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.query.group.having; - -import com.metamx.druid.input.Row; - -/** - */ -public class AlwaysHavingSpec implements HavingSpec -{ - @Override - public boolean eval(Row row) - { - return true; - } -} diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/OrderedPriorityQueueItems.java b/client/src/main/java/com/metamx/druid/query/group/orderby/OrderedPriorityQueueItems.java deleted file mode 100644 index 5b5a3f48cb6..00000000000 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/OrderedPriorityQueueItems.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.metamx.druid.query.group.orderby; - -import com.google.common.collect.MinMaxPriorityQueue; - -import java.util.Iterator; - -/** - * Utility class that supports iterating a priority queue in sorted order. - */ -class OrderedPriorityQueueItems implements Iterable -{ - private MinMaxPriorityQueue rows; - - public OrderedPriorityQueueItems(MinMaxPriorityQueue rows) - { - this.rows = rows; - } - - @Override - public Iterator iterator() - { - return new Iterator() { - - @Override - public boolean hasNext() - { - return !rows.isEmpty(); - } - - @Override - public T next() - { - return rows.poll(); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException("Can't remove any item from an intermediary heap for orderBy/limit"); - } - }; - } -} diff --git a/segment/src/main/java/io/druid/segment/LogicalSegment.java b/common/src/main/java/com/metamx/druid/LogicalSegment.java similarity index 97% rename from segment/src/main/java/io/druid/segment/LogicalSegment.java rename to common/src/main/java/com/metamx/druid/LogicalSegment.java index 0f29cea60be..70f89ec0b6d 100644 --- a/segment/src/main/java/io/druid/segment/LogicalSegment.java +++ b/common/src/main/java/com/metamx/druid/LogicalSegment.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment; +package com.metamx.druid; import org.joda.time.Interval; diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index 80e0e415bf1..defb0c128f1 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,6 @@ package com.metamx.druid; import com.metamx.druid.partition.PartitionHolder; -import io.druid.segment.LogicalSegment; import org.joda.time.Interval; /** diff --git a/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java b/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java deleted file mode 100644 index 624bd030175..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/DbConnectorModule.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Provides; -import com.metamx.druid.db.DbConnector; -import com.metamx.druid.db.DbConnectorConfig; -import com.metamx.druid.db.DbTablesConfig; -import org.skife.jdbi.v2.IDBI; - -/** - */ -public class DbConnectorModule implements Module -{ - @Override - public void configure(Binder binder) - { - JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class); - JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class); - - binder.bind(DbConnector.class); - } - - @Provides @LazySingleton - public IDBI getDbi(final DbConnector dbConnector) - { - return dbConnector.getDBI(); - } -} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java b/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java deleted file mode 100644 index 1010e56188c..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/DruidGuiceExtensions.java +++ /dev/null @@ -1,16 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.Binder; -import com.google.inject.Module; - -/** - */ -public class DruidGuiceExtensions implements Module -{ - @Override - public void configure(Binder binder) - { - binder.requireExplicitBindings(); - binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); - } -} diff --git a/common/src/main/java/com/metamx/druid/guice/LazySingleton.java b/common/src/main/java/com/metamx/druid/guice/LazySingleton.java deleted file mode 100644 index 005e9795de4..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/LazySingleton.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.ScopeAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.Target; - -import static java.lang.annotation.RetentionPolicy.RUNTIME; - -/** - */ -@Target({ ElementType.TYPE, ElementType.METHOD }) -@Retention(RUNTIME) -@ScopeAnnotation -public @interface LazySingleton -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java deleted file mode 100644 index 24e269ae00b..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/ManageLifecycle.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.ScopeAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.Target; - -import static java.lang.annotation.RetentionPolicy.RUNTIME; - -/** - * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} - * - * This Scope gets defined by {@link com.metamx.druid.guice.LifecycleModule} - */ -@Target({ ElementType.TYPE, ElementType.METHOD }) -@Retention(RUNTIME) -@ScopeAnnotation -public @interface ManageLifecycle -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java b/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java deleted file mode 100644 index ae34f21dc9e..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/ManageLifecycleLast.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.inject.ScopeAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.Target; - -import static java.lang.annotation.RetentionPolicy.RUNTIME; - -/** - * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST - * - * This Scope gets defined by {@link com.metamx.druid.guice.LifecycleModule} - */ -@Target({ ElementType.TYPE, ElementType.METHOD }) -@Retention(RUNTIME) -@ScopeAnnotation -public @interface ManageLifecycleLast -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java b/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java deleted file mode 100644 index e7dcc875949..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/SupplierProvider.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.metamx.druid.guice; - -import com.google.common.base.Supplier; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.Provider; - -/** - */ -public class SupplierProvider implements Provider -{ - private final Key> supplierKey; - - private Provider> supplierProvider; - - public SupplierProvider( - Key> supplierKey - ) - { - this.supplierKey = supplierKey; - } - - @Inject - public void configure(Injector injector) - { - this.supplierProvider = injector.getProvider(supplierKey); - } - - @Override - public T get() - { - return supplierProvider.get().get(); - } -} diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Global.java b/common/src/main/java/com/metamx/druid/guice/annotations/Global.java deleted file mode 100644 index 497e1d17c6c..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/annotations/Global.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@BindingAnnotation -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -public @interface Global -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Json.java b/common/src/main/java/com/metamx/druid/guice/annotations/Json.java deleted file mode 100644 index c20a3f5f2f0..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/annotations/Json.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface Json -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Self.java b/common/src/main/java/com/metamx/druid/guice/annotations/Self.java deleted file mode 100644 index f3e9073a7ca..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/annotations/Self.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface Self -{ -} diff --git a/common/src/main/java/com/metamx/druid/guice/annotations/Smile.java b/common/src/main/java/com/metamx/druid/guice/annotations/Smile.java deleted file mode 100644 index c3bee835d6c..00000000000 --- a/common/src/main/java/com/metamx/druid/guice/annotations/Smile.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -@BindingAnnotation -public @interface Smile -{ -} diff --git a/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java b/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java index 978d556534d..506dbf5130c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/AggregatorsModule.java @@ -22,18 +22,18 @@ package com.metamx.druid.jackson; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.module.SimpleModule; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.HistogramAggregatorFactory; -import com.metamx.druid.aggregation.JavaScriptAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.aggregation.MaxAggregatorFactory; -import com.metamx.druid.aggregation.MinAggregatorFactory; -import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; -import com.metamx.druid.aggregation.post.ConstantPostAggregator; -import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.HistogramAggregatorFactory; +import io.druid.query.aggregation.aggregation.JavaScriptAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.aggregation.post.FieldAccessPostAggregator; /** */ diff --git a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java index 694f4d2bf4c..73ee342b92c 100644 --- a/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java +++ b/common/src/main/java/com/metamx/druid/jackson/JacksonModule.java @@ -25,9 +25,9 @@ import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.guice.annotations.Json; -import com.metamx.druid.guice.annotations.Smile; +import io.druid.guice.guice.LazySingleton; +import io.druid.guice.guice.annotations.Json; +import io.druid.guice.guice.annotations.Smile; /** */ diff --git a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java b/common/src/main/java/io/druid/guice/guice/ConfigProvider.java similarity index 66% rename from common/src/main/java/com/metamx/druid/guice/ConfigProvider.java rename to common/src/main/java/io/druid/guice/guice/ConfigProvider.java index 40559a1e798..6822b48f334 100644 --- a/common/src/main/java/com/metamx/druid/guice/ConfigProvider.java +++ b/common/src/main/java/io/druid/guice/guice/ConfigProvider.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.common.base.Preconditions; import com.google.inject.Binder; diff --git a/common/src/main/java/io/druid/guice/guice/DbConnectorModule.java b/common/src/main/java/io/druid/guice/guice/DbConnectorModule.java new file mode 100644 index 00000000000..1daf09c4f73 --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/DbConnectorModule.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.db.DbConnector; +import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.db.DbTablesConfig; +import org.skife.jdbi.v2.IDBI; + +/** + */ +public class DbConnectorModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class); + JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class); + + binder.bind(DbConnector.class); + } + + @Provides @LazySingleton + public IDBI getDbi(final DbConnector dbConnector) + { + return dbConnector.getDBI(); + } +} diff --git a/common/src/main/java/io/druid/guice/guice/DruidGuiceExtensions.java b/common/src/main/java/io/druid/guice/guice/DruidGuiceExtensions.java new file mode 100644 index 00000000000..0588495701a --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/DruidGuiceExtensions.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; + +/** + */ +public class DruidGuiceExtensions implements Module +{ + @Override + public void configure(Binder binder) + { + binder.requireExplicitBindings(); + binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); + } +} diff --git a/common/src/main/java/com/metamx/druid/guice/DruidScopes.java b/common/src/main/java/io/druid/guice/guice/DruidScopes.java similarity index 62% rename from common/src/main/java/com/metamx/druid/guice/DruidScopes.java rename to common/src/main/java/io/druid/guice/guice/DruidScopes.java index 151325e587b..117f01f8cb7 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidScopes.java +++ b/common/src/main/java/io/druid/guice/guice/DruidScopes.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.inject.Inject; import com.google.inject.Key; diff --git a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java b/common/src/main/java/io/druid/guice/guice/DruidSecondaryModule.java similarity index 74% rename from common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java rename to common/src/main/java/io/druid/guice/guice/DruidSecondaryModule.java index 2793b9481a8..943e9d09ed1 100644 --- a/common/src/main/java/com/metamx/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/io/druid/guice/guice/DruidSecondaryModule.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; @@ -10,8 +29,8 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.Provides; -import com.metamx.druid.guice.annotations.Json; -import com.metamx.druid.guice.annotations.Smile; +import io.druid.guice.guice.annotations.Json; +import io.druid.guice.guice.annotations.Smile; import org.skife.config.ConfigurationObjectFactory; import javax.validation.Validator; diff --git a/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java b/common/src/main/java/io/druid/guice/guice/JacksonConfigManagerModule.java similarity index 60% rename from common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java rename to common/src/main/java/io/druid/guice/guice/JacksonConfigManagerModule.java index 8bb33dafabf..4edb08b40e9 100644 --- a/common/src/main/java/com/metamx/druid/guice/JacksonConfigManagerModule.java +++ b/common/src/main/java/io/druid/guice/guice/JacksonConfigManagerModule.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.common.base.Supplier; import com.google.inject.Binder; diff --git a/common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java b/common/src/main/java/io/druid/guice/guice/JacksonConfigProvider.java similarity index 98% rename from common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java rename to common/src/main/java/io/druid/guice/guice/JacksonConfigProvider.java index 0d4bb53f01d..b48ff8aee50 100644 --- a/common/src/main/java/com/metamx/druid/guice/JacksonConfigProvider.java +++ b/common/src/main/java/io/druid/guice/guice/JacksonConfigProvider.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Supplier; diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java b/common/src/main/java/io/druid/guice/guice/JsonConfigProvider.java similarity index 99% rename from common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java rename to common/src/main/java/io/druid/guice/guice/JsonConfigProvider.java index 3066286ea7f..2315279bdc9 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigProvider.java +++ b/common/src/main/java/io/druid/guice/guice/JsonConfigProvider.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; diff --git a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java b/common/src/main/java/io/druid/guice/guice/JsonConfigurator.java similarity index 86% rename from common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java rename to common/src/main/java/io/druid/guice/guice/JsonConfigurator.java index a06636dad35..573f6d2d0d6 100644 --- a/common/src/main/java/com/metamx/druid/guice/JsonConfigurator.java +++ b/common/src/main/java/io/druid/guice/guice/JsonConfigurator.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/io/druid/guice/guice/LazySingleton.java b/common/src/main/java/io/druid/guice/guice/LazySingleton.java new file mode 100644 index 00000000000..2dfa5802125 --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/LazySingleton.java @@ -0,0 +1,37 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface LazySingleton +{ +} diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java b/common/src/main/java/io/druid/guice/guice/LifecycleModule.java similarity index 99% rename from common/src/main/java/com/metamx/druid/guice/LifecycleModule.java rename to common/src/main/java/io/druid/guice/guice/LifecycleModule.java index 5ec992a96b1..b1a13dc973c 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleModule.java +++ b/common/src/main/java/io/druid/guice/guice/LifecycleModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.common.base.Preconditions; import com.google.inject.Binder; diff --git a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java b/common/src/main/java/io/druid/guice/guice/LifecycleScope.java similarity index 66% rename from common/src/main/java/com/metamx/druid/guice/LifecycleScope.java rename to common/src/main/java/io/druid/guice/guice/LifecycleScope.java index a4b076ebb67..320803c1dad 100644 --- a/common/src/main/java/com/metamx/druid/guice/LifecycleScope.java +++ b/common/src/main/java/io/druid/guice/guice/LifecycleScope.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.common.collect.Lists; import com.google.inject.Key; diff --git a/common/src/main/java/com/metamx/druid/guice/ListProvider.java b/common/src/main/java/io/druid/guice/guice/ListProvider.java similarity index 98% rename from common/src/main/java/com/metamx/druid/guice/ListProvider.java rename to common/src/main/java/io/druid/guice/guice/ListProvider.java index 86d7a03c24a..b3f5c5f01d5 100644 --- a/common/src/main/java/com/metamx/druid/guice/ListProvider.java +++ b/common/src/main/java/io/druid/guice/guice/ListProvider.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.common.collect.Lists; import com.google.inject.Inject; diff --git a/common/src/main/java/io/druid/guice/guice/ManageLifecycle.java b/common/src/main/java/io/druid/guice/guice/ManageLifecycle.java new file mode 100644 index 00000000000..106ff9f1afb --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/ManageLifecycle.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} + * + * This Scope gets defined by {@link io.druid.guice.guice.LifecycleModule} + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface ManageLifecycle +{ +} diff --git a/common/src/main/java/io/druid/guice/guice/ManageLifecycleLast.java b/common/src/main/java/io/druid/guice/guice/ManageLifecycleLast.java new file mode 100644 index 00000000000..d420ccb8717 --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/ManageLifecycleLast.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.ScopeAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +/** + * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST + * + * This Scope gets defined by {@link io.druid.guice.guice.LifecycleModule} + */ +@Target({ ElementType.TYPE, ElementType.METHOD }) +@Retention(RUNTIME) +@ScopeAnnotation +public @interface ManageLifecycleLast +{ +} diff --git a/common/src/main/java/com/metamx/druid/guice/PolyBind.java b/common/src/main/java/io/druid/guice/guice/PolyBind.java similarity index 99% rename from common/src/main/java/com/metamx/druid/guice/PolyBind.java rename to common/src/main/java/io/druid/guice/guice/PolyBind.java index 195f2923306..ba7df6e49d0 100644 --- a/common/src/main/java/com/metamx/druid/guice/PolyBind.java +++ b/common/src/main/java/io/druid/guice/guice/PolyBind.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Inject; diff --git a/common/src/main/java/io/druid/guice/guice/SupplierProvider.java b/common/src/main/java/io/druid/guice/guice/SupplierProvider.java new file mode 100644 index 00000000000..b9a0e236511 --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/SupplierProvider.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.common.base.Supplier; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; + +/** + */ +public class SupplierProvider implements Provider +{ + private final Key> supplierKey; + + private Provider> supplierProvider; + + public SupplierProvider( + Key> supplierKey + ) + { + this.supplierKey = supplierKey; + } + + @Inject + public void configure(Injector injector) + { + this.supplierProvider = injector.getProvider(supplierKey); + } + + @Override + public T get() + { + return supplierProvider.get().get(); + } +} diff --git a/common/src/main/java/io/druid/guice/guice/annotations/Global.java b/common/src/main/java/io/druid/guice/guice/annotations/Global.java new file mode 100644 index 00000000000..cca8afefc3c --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/annotations/Global.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Global +{ +} diff --git a/common/src/main/java/io/druid/guice/guice/annotations/Json.java b/common/src/main/java/io/druid/guice/guice/annotations/Json.java new file mode 100644 index 00000000000..a52008fa5eb --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/annotations/Json.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Json +{ +} diff --git a/common/src/main/java/io/druid/guice/guice/annotations/Self.java b/common/src/main/java/io/druid/guice/guice/annotations/Self.java new file mode 100644 index 00000000000..fe5e2df0dce --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/annotations/Self.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Self +{ +} diff --git a/common/src/main/java/io/druid/guice/guice/annotations/Smile.java b/common/src/main/java/io/druid/guice/guice/annotations/Smile.java new file mode 100644 index 00000000000..0e5537ef04a --- /dev/null +++ b/common/src/main/java/io/druid/guice/guice/annotations/Smile.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Smile +{ +} diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregator.java similarity index 94% rename from common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregator.java index 6c307fecb64..65091226992 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregatorFactory.java similarity index 96% rename from common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregatorFactory.java index dcc389084b6..c5d67db8012 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/CountAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/CountBufferAggregator.java similarity index 93% rename from common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/CountBufferAggregator.java index 9cc0e41c42c..3a9966660b7 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/CountBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/CountBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregator.java index 318c1d8fbf2..7cc8d9181ab 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.collect.Ordering; import com.google.common.primitives.Doubles; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorFactory.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorFactory.java index e130f9d90a4..4dbda6ad359 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumBufferAggregator.java similarity index 94% rename from common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumBufferAggregator.java index 72436c51b6e..2d1689b0ba0 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/DoubleSumBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/DoubleSumBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java b/common/src/main/java/io/druid/query/aggregation/aggregation/Histogram.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/Histogram.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/Histogram.java index 5c2c0f5f069..40880b97230 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/Histogram.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/Histogram.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregator.java index cf2f24e2c30..bbb73a4fc96 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.primitives.Longs; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregatorFactory.java similarity index 98% rename from common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregatorFactory.java index f4d4f00dafc..7901228df72 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramBufferAggregator.java similarity index 96% rename from common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/HistogramBufferAggregator.java index e312f187593..8404c6e054f 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; diff --git a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramVisual.java similarity index 96% rename from common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/HistogramVisual.java index 8c696638aa9..e81d42686fa 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/HistogramVisual.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/HistogramVisual.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregator.java index fb6de96192a..e97e1da5d02 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.collect.Lists; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorFactory.java similarity index 98% rename from common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorFactory.java index ae2375069b5..e2b72b40cf0 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptBufferAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptBufferAggregator.java index cf55c32ce07..880b12d6cbb 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/JavaScriptBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/JavaScriptBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.collect.Lists; import io.druid.query.aggregation.BufferAggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregator.java index 3defff46019..1e1c4932d35 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.primitives.Longs; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregatorFactory.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregatorFactory.java index 7ff27a47fdd..e653e45ebba 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumBufferAggregator.java similarity index 94% rename from common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/LongSumBufferAggregator.java index f0cbca7098b..854a3affe42 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/LongSumBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/LongSumBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregator.java index 337615730d8..273af8980e0 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.Aggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregatorFactory.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregatorFactory.java index e9e85ae45cc..5ccac95330b 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxBufferAggregator.java similarity index 94% rename from common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MaxBufferAggregator.java index b5a07cd288c..c51767c6002 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MaxBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MaxBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregator.java similarity index 95% rename from common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregator.java index 586556915c8..3d2f8c2bc92 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.Aggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregatorFactory.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregatorFactory.java index ad31b46142e..00ab840bd7e 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MinAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/MinBufferAggregator.java similarity index 94% rename from common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/MinBufferAggregator.java index 7eab06f06c7..b77f9d5f8f5 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/MinBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/MinBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/NoopAggregator.java similarity index 93% rename from common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/NoopAggregator.java index f5907f100eb..8f778ff7c0e 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/NoopAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/NoopAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/NoopBufferAggregator.java similarity index 92% rename from common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/NoopBufferAggregator.java index 51ceafe04c8..931863dc26a 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/NoopBufferAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/NoopBufferAggregator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.BufferAggregator; diff --git a/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java b/common/src/main/java/io/druid/query/aggregation/aggregation/ToLowerCaseAggregatorFactory.java similarity index 96% rename from common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/ToLowerCaseAggregatorFactory.java index a48b33c4f77..06cd02deb53 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/ToLowerCaseAggregatorFactory.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/ToLowerCaseAggregatorFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregator.java similarity index 98% rename from common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregator.java index 5d0c9b7647e..8f449e23a85 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregator.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregator.java index b60ac62b6a8..85cb3bf67f0 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/ConstantPostAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java b/common/src/main/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregator.java similarity index 97% rename from common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java rename to common/src/main/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregator.java index 777090451d2..c33b63959cd 100644 --- a/common/src/main/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregator.java +++ b/common/src/main/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java index dd71ef22553..6d20d255708 100644 --- a/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java +++ b/common/src/test/java/com/metamx/druid/histogram/HistogramTest.java @@ -21,8 +21,8 @@ package com.metamx.druid.histogram; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; -import com.metamx.druid.aggregation.Histogram; import com.metamx.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.aggregation.Histogram; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java b/common/src/test/java/io/druid/guice/guice/LifecycleScopeTest.java similarity index 86% rename from common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java rename to common/src/test/java/io/druid/guice/guice/LifecycleScopeTest.java index 9948dfe1470..999d9c1ef8b 100644 --- a/common/src/test/java/com/metamx/druid/guice/LifecycleScopeTest.java +++ b/common/src/test/java/io/druid/guice/guice/LifecycleScopeTest.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Guice; diff --git a/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java b/common/src/test/java/io/druid/guice/guice/PolyBindTest.java similarity index 99% rename from common/src/test/java/com/metamx/druid/guice/PolyBindTest.java rename to common/src/test/java/io/druid/guice/guice/PolyBindTest.java index a5f24e220fe..002ef99fedf 100644 --- a/common/src/test/java/com/metamx/druid/guice/PolyBindTest.java +++ b/common/src/test/java/io/druid/guice/guice/PolyBindTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.common.collect.Iterables; import com.google.inject.Binder; diff --git a/common/src/test/java/com/metamx/druid/aggregation/CountAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/CountAggregatorTest.java similarity index 95% rename from common/src/test/java/com/metamx/druid/aggregation/CountAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/CountAggregatorTest.java index 8a780146504..1de3f4166e9 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/CountAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/CountAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/DoubleSumAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorTest.java similarity index 96% rename from common/src/test/java/com/metamx/druid/aggregation/DoubleSumAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorTest.java index d1a87ac9c2d..9d686904dc4 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/DoubleSumAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/DoubleSumAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/HistogramAggregatorTest.java similarity index 98% rename from common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/HistogramAggregatorTest.java index 93079506b81..83672731c00 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/HistogramAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/HistogramAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.collect.Lists; import io.druid.query.aggregation.Aggregator; diff --git a/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorTest.java similarity index 98% rename from common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorTest.java index bf287cce344..3b526ee9381 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/JavaScriptAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/JavaScriptAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/common/src/test/java/com/metamx/druid/aggregation/LongSumAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/LongSumAggregatorTest.java similarity index 95% rename from common/src/test/java/com/metamx/druid/aggregation/LongSumAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/LongSumAggregatorTest.java index 8db1a4a4755..4caf2c89a2d 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/LongSumAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/LongSumAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/MaxAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/MaxAggregatorTest.java similarity index 93% rename from common/src/test/java/com/metamx/druid/aggregation/MaxAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/MaxAggregatorTest.java index ed2a55efbb2..f30b03dc54d 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/MaxAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/MaxAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java b/common/src/test/java/io/druid/query/aggregation/aggregation/MetricSelectorUtils.java similarity index 92% rename from common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/MetricSelectorUtils.java index 3cb856fdd2d..e42e13df020 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/MetricSelectorUtils.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/MetricSelectorUtils.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.segment.FloatMetricSelector; import io.druid.segment.ObjectMetricSelector; diff --git a/common/src/test/java/com/metamx/druid/aggregation/MinAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/MinAggregatorTest.java similarity index 93% rename from common/src/test/java/com/metamx/druid/aggregation/MinAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/MinAggregatorTest.java index 9544595ec82..998de6ac084 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/MinAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/MinAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java b/common/src/test/java/io/druid/query/aggregation/aggregation/TestFloatMetricSelector.java similarity index 92% rename from common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/TestFloatMetricSelector.java index 759a33a9902..e3fe63a0565 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/TestFloatMetricSelector.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/TestFloatMetricSelector.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation; +package io.druid.query.aggregation.aggregation; import io.druid.segment.FloatMetricSelector; diff --git a/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregatorTest.java similarity index 96% rename from common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregatorTest.java index 144483392bd..33c8cef519b 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/post/ArithmeticPostAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/post/ArithmeticPostAggregatorTest.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.CountAggregator; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.aggregation.CountAggregator; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/post/ConstantPostAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregatorTest.java similarity index 94% rename from common/src/test/java/com/metamx/druid/aggregation/post/ConstantPostAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregatorTest.java index 0de6e2c62e7..fb235581b50 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/post/ConstantPostAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/post/ConstantPostAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; import org.junit.Assert; import org.junit.Test; diff --git a/common/src/test/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregatorTest.java b/common/src/test/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregatorTest.java similarity index 90% rename from common/src/test/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregatorTest.java rename to common/src/test/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregatorTest.java index e413504b1ad..d2de2aef34f 100644 --- a/common/src/test/java/com/metamx/druid/aggregation/post/FieldAccessPostAggregatorTest.java +++ b/common/src/test/java/io/druid/query/aggregation/aggregation/post/FieldAccessPostAggregatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.aggregation.post; +package io.druid.query.aggregation.aggregation.post; -import com.metamx.druid.aggregation.CountAggregator; +import io.druid.query.aggregation.aggregation.CountAggregator; import org.junit.Assert; import org.junit.Test; diff --git a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java index 66b8de4450c..d008a4d0997 100644 --- a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,10 +25,10 @@ import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Iterators; import com.google.common.io.Closeables; -import com.metamx.druid.indexer.data.StringInputRowParser; import com.metamx.druid.input.InputRow; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; +import io.druid.data.input.StringInputRowParser; import java.io.BufferedReader; import java.io.File; diff --git a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java index d2ebf350968..046589fde04 100644 --- a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java +++ b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 druid.examples.guice; import com.fasterxml.jackson.databind.jsontype.NamedType; @@ -6,16 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.TypeLiteral; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.ServerView; -import com.metamx.druid.coordination.DataSegmentAnnouncer; -import com.metamx.druid.guice.FireDepartmentsProvider; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.guice.NoopSegmentPublisherProvider; -import com.metamx.druid.guice.RealtimeManagerConfig; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.realtime.FireDepartment; import com.metamx.druid.realtime.RealtimeManager; @@ -24,7 +33,17 @@ import druid.examples.flights.FlightsFirehoseFactory; import druid.examples.rand.RandomFirehoseFactory; import druid.examples.twitter.TwitterSpritzerFirehoseFactory; import druid.examples.web.WebFirehoseFactory; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.client.InventoryView; +import io.druid.client.ServerView; +import io.druid.guice.guice.FireDepartmentsProvider; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.guice.guice.NoopSegmentPublisherProvider; +import io.druid.guice.guice.RealtimeManagerConfig; import io.druid.initialization.DruidModule; +import io.druid.server.coordination.DataSegmentAnnouncer; import java.io.File; import java.io.IOException; diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml deleted file mode 100644 index 6bd76a53385..00000000000 --- a/indexing-common/pom.xml +++ /dev/null @@ -1,110 +0,0 @@ - - - - - 4.0.0 - com.metamx.druid - druid-indexing-common - druid-indexing-common - Druid Indexing Common - - - com.metamx - druid - 0.5.50-SNAPSHOT - - - - - com.metamx.druid - druid-common - ${project.parent.version} - - - - com.metamx - java-util - - - com.metamx - bytebuffer-collections - - - - com.ning - compress-lzf - - - it.uniroma3.mat - extendedset - - - com.fasterxml.jackson.core - jackson-core - - - net.java.dev.jets3t - jets3t - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - - - com.google.guava - guava - - - commons-io - commons-io - - - com.google.protobuf - protobuf-java - - - - - - junit - junit - test - - - - - - - maven-jar-plugin - - - - test-jar - - - - - - - diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/Index.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/Index.java deleted file mode 100644 index fcec6135b86..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/Index.java +++ /dev/null @@ -1,166 +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.index.v1; - -import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.common.logger.Logger; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.joda.time.Interval; - -import java.util.HashMap; -import java.util.Map; - -/** - * In-memory representation of a segment - */ -public class Index -{ - private static final Logger log = new Logger(Index.class); - private static final ImmutableConciseSet emptySet = new ImmutableConciseSet(); - - final Map dimToPositionMap = new HashMap(); - final Map metricToPositionMap = new HashMap(); - - final String[] dimensions; - final String[] metrics; - final Interval dataInterval; - final long[] timeOffsets; - final Map metricVals; - final Map> dimIdLookup; - final Map reverseDimLookup; - final Map indexes; - final Map spatialIndexes; - final Map dimensionValues; - - /* - * If we name the various occurrences of String and int then the types are more informative: - * - * dimToPositionMap : Dim -> DimColumn - * dimensions : DimColumn -> Dim - * metricToPositionMap : Met -> MetColumn - * metrics : MetColumn -> Met - * - * dataInterval : Interval - * - * timeOffsets : Milli[] - * metricVals : float[] (size() == timeOffsets.size() * metrics.size()) - * - * dimIdLookup : Dim -> Value -> ValueID - * reverseDimLookup : Dim -> ValueID -> Value - * indexes : Dim -> ValueID -> Row[] - * - * dimensionValues : Dim -> - * getRowValues : Row -> ExpansionID - * getDimensionExpansions : ExpansionID -> ValueID[] - * getDimValues : Row -> ValueID[] - */ - - public Index( - String[] dimensions, - String[] metrics, - Interval dataInterval, - long[] timeOffsets, - Map metricVals, - Map> dimIdLookup, - Map reverseDimLookup, - Map indexes, - Map spatialIndexes, - Map dimensionValues - ) - { - this.dimensions = dimensions; - this.metrics = metrics; - this.dataInterval = dataInterval; - this.timeOffsets = timeOffsets; - this.metricVals = metricVals; - this.dimIdLookup = dimIdLookup; - this.reverseDimLookup = reverseDimLookup; - this.indexes = indexes; - this.spatialIndexes = spatialIndexes; - this.dimensionValues = dimensionValues; - - for (int i = 0; i < dimensions.length; i++) { - dimToPositionMap.put(dimensions[i].toLowerCase(), i); - } - for (int i = 0; i < metrics.length; i++) { - metricToPositionMap.put(metrics[i].toLowerCase(), i); - } - } - - public ImmutableConciseSet getInvertedIndex(String dimension, String value) - { - final Map lookup = dimIdLookup.get(dimension); - if (lookup == null) { - return emptySet; - } - - final Integer integer = lookup.get(value); - if (integer == null) { - return emptySet; - } - - try { - return indexes.get(dimension)[integer]; - } - catch (NullPointerException e) { - log.warn( - e, - "NPE on dimension[%s], value[%s], with index over interval[%s]", - dimension, - value, - dataInterval - ); - return emptySet; - } - } - - public ImmutableConciseSet getInvertedIndex(String dimension, int valueIndex) - { - try { - return indexes.get(dimension)[valueIndex]; - } - catch (NullPointerException e) { - log.warn( - e, - "NPE on dimension[%s], valueIndex[%d], with index over interval[%s]", - dimension, - valueIndex, - dataInterval - ); - return emptySet; - } - } - - public ImmutableRTree getSpatialIndex(String dimension) - { - try { - return spatialIndexes.get(dimension); - } - catch (NullPointerException e) { - log.warn( - e, - "NPE on dimension[%s] over interval[%s]", - dimension, - dataInterval - ); - return new ImmutableRTree(); - } - } -} diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java deleted file mode 100644 index 83a32dedfbe..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java +++ /dev/null @@ -1,296 +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.index.v1; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.primitives.Ints; -import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.common.io.smoosh.SmooshedFileMapper; -import com.metamx.common.logger.Logger; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedList; -import com.metamx.druid.kv.VSizeIndexed; -import com.metamx.druid.kv.VSizeIndexedInts; -import io.druid.data.Indexed; -import io.druid.data.IndexedLongs; -import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import org.joda.time.Interval; - -import java.io.IOException; -import java.nio.ByteOrder; -import java.nio.LongBuffer; -import java.util.Arrays; -import java.util.Map; - -/** - */ -public class MMappedIndex -{ - private static final Logger log = new Logger(MMappedIndex.class); - private static final ImmutableConciseSet emptySet = new ImmutableConciseSet(); - - final GenericIndexed availableDimensions; - final GenericIndexed availableMetrics; - final Interval dataInterval; - final CompressedLongsIndexedSupplier timestamps; - final Map metrics; - final Map> dimValueLookups; - final Map dimColumns; - final Map> invertedIndexes; - final Map spatialIndexes; - final SmooshedFileMapper fileMapper; - - private final Map metricIndexes = Maps.newHashMap(); - - public MMappedIndex( - GenericIndexed availableDimensions, - GenericIndexed availableMetrics, - Interval dataInterval, - CompressedLongsIndexedSupplier timestamps, - Map metrics, - Map> dimValueLookups, - Map dimColumns, - Map> invertedIndexes, - Map spatialIndexes, - SmooshedFileMapper fileMapper - ) - { - this.availableDimensions = availableDimensions; - this.availableMetrics = availableMetrics; - this.dataInterval = dataInterval; - this.timestamps = timestamps; - this.metrics = metrics; - this.dimValueLookups = dimValueLookups; - this.dimColumns = dimColumns; - this.invertedIndexes = invertedIndexes; - this.spatialIndexes = spatialIndexes; - this.fileMapper = fileMapper; - - for (int i = 0; i < availableMetrics.size(); i++) { - metricIndexes.put(availableMetrics.get(i), i); - } - } - - public CompressedLongsIndexedSupplier getTimestamps() - { - return timestamps; - } - - public GenericIndexed getAvailableDimensions() - { - return availableDimensions; - } - - public GenericIndexed getAvailableMetrics() - { - return availableMetrics; - } - - public Map getMetrics() - { - return metrics; - } - - public Integer getMetricIndex(String metricName) - { - return metricIndexes.get(metricName); - } - - public Interval getDataInterval() - { - return dataInterval; - } - - public IndexedLongs getReadOnlyTimestamps() - { - return timestamps.get(); - } - - public MetricHolder getMetricHolder(String metric) - { - final MetricHolder retVal = metrics.get(metric); - - if (retVal == null) { - return null; - } - - return retVal; - } - - public GenericIndexed getDimValueLookup(String dimension) - { - return dimValueLookups.get(dimension); - } - - public VSizeIndexed getDimColumn(String dimension) - { - return dimColumns.get(dimension); - } - - public Map> getInvertedIndexes() - { - return invertedIndexes; - } - - public Map getSpatialIndexes() - { - return spatialIndexes; - } - - public ImmutableConciseSet getInvertedIndex(String dimension, String value) - { - final GenericIndexed lookup = dimValueLookups.get(dimension); - if (lookup == null) { - return emptySet; - } - - int indexOf = lookup.indexOf(value); - if (indexOf < 0) { - return emptySet; - } - - ImmutableConciseSet retVal = invertedIndexes.get(dimension).get(indexOf); - return (retVal == null) ? emptySet : retVal; - } - - public SmooshedFileMapper getFileMapper() - { - return fileMapper; - } - - public void close() throws IOException - { - if (fileMapper != null) { - fileMapper.close(); - } - } - - public static MMappedIndex fromIndex(Index index) - { - log.info("Converting timestamps"); - CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(index.timeOffsets), ByteOrder.nativeOrder() - ); - - log.info("Converting dimValueLookups"); - Map> dimValueLookups = Maps.newHashMap(); - for (Map.Entry entry : index.reverseDimLookup.entrySet()) { - String[] theValues = Arrays.copyOf(entry.getValue(), entry.getValue().length); - Arrays.sort(theValues); - - dimValueLookups.put(entry.getKey(), GenericIndexed.fromArray(theValues, GenericIndexed.stringStrategy)); - } - - Map dimColumns = Maps.newHashMap(); - Map> invertedIndexes = Maps.newLinkedHashMap(); - Map spatialIndexes = Maps.newLinkedHashMap(); - - for (String dimension : Arrays.asList(index.dimensions)) { - final String[] dimVals = index.reverseDimLookup.get(dimension); - final DimensionColumn dimColumn = index.dimensionValues.get(dimension); - - log.info( - "Converting dim[%s] with valueCardinality[%,d] and expansionCardinality[%,d]", - dimension, - dimColumn.getDimensionExpansions().length, - dimVals.length - ); - - final Indexed lookup = dimValueLookups.get(dimension); - final int[] expansionConversion = new int[dimVals.length]; - final int[] reverseExpansionConversion = new int[dimVals.length]; - for (int i = 0; i < expansionConversion.length; i++) { - expansionConversion[i] = lookup.indexOf(dimVals[i]); - reverseExpansionConversion[expansionConversion[i]] = i; - } - - int[][] originalDimExpansions = dimColumn.getDimensionExpansions(); - final int[][] dimensionExpansions = new int[originalDimExpansions.length][]; - for (int i = 0; i < originalDimExpansions.length; i++) { - int[] originalDimExpansion = originalDimExpansions[i]; - int[] mappedValues = new int[originalDimExpansion.length]; - - for (int j = 0; j < originalDimExpansion.length; j++) { - mappedValues[j] = expansionConversion[originalDimExpansion[j]]; - } - Arrays.sort(mappedValues); - - dimensionExpansions[i] = mappedValues; - } - - final int[] originalRows = dimColumn.getDimensionRowValues(); - final ImmutableConciseSet[] origInvertedIndexes = index.indexes.get(dimension); - - dimColumns.put( - dimension, - VSizeIndexed.fromIterable( - Iterables.transform( - Ints.asList(originalRows), - new Function() - { - @Override - public VSizeIndexedInts apply(Integer input) - { - return VSizeIndexedInts.fromArray(dimensionExpansions[input], dimVals.length - 1); - } - } - ) - ) - ); - - invertedIndexes.put( - dimension, - GenericIndexed.fromIterable( - Iterables.transform( - new IndexedList(lookup), - new Function() - { - @Override - public ImmutableConciseSet apply(String input) - { - return origInvertedIndexes[reverseExpansionConversion[lookup.indexOf(input)]]; - } - } - ), - ConciseCompressedIndexedInts.objectStrategy - ) - ); - - spatialIndexes.put(dimension, index.getSpatialIndex(dimension)); - } - - log.info("Making MMappedIndex"); - return new MMappedIndex( - GenericIndexed.fromArray(index.dimensions, GenericIndexed.stringStrategy), - GenericIndexed.fromArray(index.metrics, GenericIndexed.stringStrategy), - index.dataInterval, - timestamps, - index.metricVals, - dimValueLookups, - dimColumns, - invertedIndexes, - spatialIndexes, - null - ); - } -} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java deleted file mode 100644 index d0764e72a26..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ByteBufferInputRowParser.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.metamx.druid.indexer.data; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; - -import java.nio.ByteBuffer; - -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), - @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class) -}) -public interface ByteBufferInputRowParser extends InputRowParser { -} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java deleted file mode 100644 index 231cbd44102..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.metamx.druid.indexer.data; - -import com.metamx.common.exception.FormattedException; -import com.metamx.druid.input.InputRow; - -public interface InputRowParser -{ - public InputRow parse(T input) throws FormattedException; - public void addDimensionExclusion(String dimension); -} diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java b/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java deleted file mode 100644 index 93f49451941..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java +++ /dev/null @@ -1,30 +0,0 @@ -package com.metamx.druid.kv; - -import com.google.common.primitives.Ints; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; - -/** - */ -public class ByteBufferSerializer -{ - public static T read(ByteBuffer buffer, ObjectStrategy strategy) - { - int size = buffer.getInt(); - ByteBuffer bufferToUse = buffer.asReadOnlyBuffer(); - bufferToUse.limit(bufferToUse.position() + size); - buffer.position(bufferToUse.limit()); - - return strategy.fromByteBuffer(bufferToUse, size); - } - - public static void writeToChannel(T obj, ObjectStrategy strategy, WritableByteChannel channel) - throws IOException - { - byte[] toWrite = strategy.toBytes(obj); - channel.write(ByteBuffer.allocate(Ints.BYTES).putInt(0, toWrite.length)); - channel.write(ByteBuffer.wrap(toWrite)); - } -} diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java deleted file mode 100644 index 57bd0d957bb..00000000000 --- a/indexing-common/src/main/java/com/metamx/druid/kv/SingleIndexedInts.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.metamx.druid.kv; - -import com.google.common.collect.Iterators; -import io.druid.data.IndexedInts; - -import java.util.Iterator; - -/** -*/ -public class SingleIndexedInts implements IndexedInts -{ - private final int value; - - public SingleIndexedInts(int value) { - this.value = value; - } - - @Override - public int size() - { - return 1; - } - - @Override - public int get(int index) - { - return value; - } - - @Override - public Iterator iterator() - { - return Iterators.singletonIterator(value); - } -} diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index be2d75deff9..1571eacff50 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -1,7 +1,7 @@ @@ -99,13 +111,6 @@ easymock test - - com.metamx.druid - druid-indexing-common - ${project.parent.version} - test-jar - test - diff --git a/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java b/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java new file mode 100644 index 00000000000..d033013bc6d --- /dev/null +++ b/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data.input; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +import java.nio.ByteBuffer; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "protobuf", value = ProtoBufInputRowParser.class), + @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class) +}) +public interface ByteBufferInputRowParser extends InputRowParser { +} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java b/processing/src/main/java/io/druid/data/input/CSVDataSpec.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java rename to processing/src/main/java/io/druid/data/input/CSVDataSpec.java index 60303703e18..c0cedfe2095 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java +++ b/processing/src/main/java/io/druid/data/input/CSVDataSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -25,7 +25,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.parsers.CSVParser; import com.metamx.common.parsers.Parser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/processing/src/main/java/io/druid/data/input/DataSpec.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java rename to processing/src/main/java/io/druid/data/input/DataSpec.java index 099d0c8d535..b45faec2a9a 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java +++ b/processing/src/main/java/io/druid/data/input/DataSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.parsers.Parser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java b/processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java rename to processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java index c9aa2d253ce..f9e819f07ec 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java +++ b/processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -25,7 +25,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.parsers.DelimitedParser; import com.metamx.common.parsers.Parser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import java.util.List; diff --git a/processing/src/main/java/io/druid/data/input/InputRowParser.java b/processing/src/main/java/io/druid/data/input/InputRowParser.java new file mode 100644 index 00000000000..d4159437cee --- /dev/null +++ b/processing/src/main/java/io/druid/data/input/InputRowParser.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data.input; + +import com.metamx.common.exception.FormattedException; +import com.metamx.druid.input.InputRow; + +public interface InputRowParser +{ + public InputRow parse(T input) throws FormattedException; + public void addDimensionExclusion(String dimension); +} diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java b/processing/src/main/java/io/druid/data/input/JSONDataSpec.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java rename to processing/src/main/java/io/druid/data/input/JSONDataSpec.java index 0f07161b4bf..9badb3e2527 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java +++ b/processing/src/main/java/io/druid/data/input/JSONDataSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.metamx.common.parsers.JSONParser; import com.metamx.common.parsers.Parser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java b/processing/src/main/java/io/druid/data/input/MapInputRowParser.java similarity index 77% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java rename to processing/src/main/java/io/druid/data/input/MapInputRowParser.java index b39b70471fa..a152f3acd59 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/MapInputRowParser.java @@ -1,4 +1,23 @@ -package com.metamx.druid.indexer.data; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java similarity index 71% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java rename to processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 3130a86a1b4..ec5d33362a5 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -1,4 +1,23 @@ -package com.metamx.druid.indexer.data; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -9,6 +28,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import com.metamx.common.exception.FormattedException; +import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; import java.io.InputStream; @@ -22,6 +42,7 @@ import static com.google.protobuf.Descriptors.FileDescriptor; public class ProtoBufInputRowParser implements ByteBufferInputRowParser { + private static final Logger log = new Logger(ProtoBufInputRowParser.class); private final MapInputRowParser inputRowCreator; private final Descriptor descriptor; @@ -42,8 +63,8 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser @Override public InputRow parse(ByteBuffer input) throws FormattedException { - // TODO there should be a ProtoBufBasedInputRow that does not need an intermediate map but accesses - // the DynamicMessage directly + // We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses + // the DynamicMessage directly... Map theMap = buildStringKeyMap(input); return inputRowCreator.parse(theMap); @@ -64,9 +85,8 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser if (theMap.containsKey(name)) { continue; - // TODO - // throw new RuntimeException("dupicate key " + name + " in " + - // message); + // Perhaps throw an exception here? + // throw new RuntimeException("dupicate key " + name + " in " + message); } Object value = entry.getValue(); if(value instanceof Descriptors.EnumValueDescriptor) { @@ -79,8 +99,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser } catch (InvalidProtocolBufferException e) { - // TODO - e.printStackTrace(); + log.warn(e, "Problem with protobuf something"); } return theMap; } diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/processing/src/main/java/io/druid/data/input/StringInputRowParser.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java rename to processing/src/main/java/io/druid/data/input/StringInputRowParser.java index 3d23d5d1dde..28b9bff649f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/StringInputRowParser.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java b/processing/src/main/java/io/druid/data/input/TimestampSpec.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java rename to processing/src/main/java/io/druid/data/input/TimestampSpec.java index ab216a2b415..4052cb63537 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java +++ b/processing/src/main/java/io/druid/data/input/TimestampSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.metamx.common.parsers.ParserUtils; - import org.joda.time.DateTime; import java.util.Map; diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java b/processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java rename to processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java index c4eda7ffc55..cba57080af6 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java +++ b/processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.parsers.Parser; import com.metamx.common.parsers.ToLowerCaseParser; -import com.metamx.druid.index.v1.SpatialDimensionSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import java.util.List; diff --git a/processing/src/main/java/io/druid/granularity/AllGranularity.java b/processing/src/main/java/io/druid/granularity/AllGranularity.java new file mode 100644 index 00000000000..eebe4c43ab1 --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/AllGranularity.java @@ -0,0 +1,74 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +import com.google.common.collect.ImmutableList; + +public final class AllGranularity extends BaseQueryGranularity +{ + @Override + public long next(long offset) + { + return Long.MAX_VALUE; + } + + @Override + public long truncate(long offset) + { + return Long.MIN_VALUE; + } + + @Override + public byte[] cacheKey() + { + return new byte[]{0x7f}; + } + + @Override + public Iterable iterable(long start, long end) + { + return ImmutableList.of(start); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return 0; + } + + @Override + public String toString() + { + return "AllGranularity"; + } +} diff --git a/processing/src/main/java/io/druid/granularity/BaseQueryGranularity.java b/processing/src/main/java/io/druid/granularity/BaseQueryGranularity.java new file mode 100644 index 00000000000..b9755a4ed03 --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/BaseQueryGranularity.java @@ -0,0 +1,83 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +public abstract class BaseQueryGranularity extends QueryGranularity +{ + public abstract long next(long offset); + + public abstract long truncate(long offset); + + public abstract byte[] cacheKey(); + + public DateTime toDateTime(long offset) + { + return new DateTime(offset, DateTimeZone.UTC); + } + + public Iterable iterable(final long start, final long end) + { + return new Iterable() + { + @Override + public Iterator iterator() + { + return new Iterator() + { + long curr = truncate(start); + long next = BaseQueryGranularity.this.next(curr); + + @Override + public boolean hasNext() + { + return curr < end; + } + + @Override + public Long next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long retVal = curr; + + curr = next; + next = BaseQueryGranularity.this.next(curr); + + return retVal; + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + }; + } +} diff --git a/processing/src/main/java/io/druid/granularity/DurationGranularity.java b/processing/src/main/java/io/druid/granularity/DurationGranularity.java new file mode 100644 index 00000000000..a4e8393eabd --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/DurationGranularity.java @@ -0,0 +1,128 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Longs; +import org.joda.time.DateTime; + +import java.nio.ByteBuffer; + +public class DurationGranularity extends BaseQueryGranularity +{ + private final long length; + private final long origin; + + @JsonCreator + public DurationGranularity( + @JsonProperty("duration") long duration, + @JsonProperty("origin") DateTime origin + ) + { + this(duration, origin == null ? 0 : origin.getMillis()); + } + + public DurationGranularity(long millis, long origin) + { + this.length = millis; + this.origin = origin % length; + } + + @JsonProperty("duration") + public long getDuration() + { + return length; + } + + @JsonProperty("origin") + public DateTime getOrigin() + { + return new DateTime(origin); + } + + public long getOriginMillis() + { + return origin; + } + + @Override + public long next(long t) + { + return t + getDurationMillis(); + } + + @Override + public long truncate(long t) + { + final long duration = getDurationMillis(); + return t - t % duration + origin; + } + + @Override + public byte[] cacheKey() + { + return ByteBuffer.allocate(2 * Longs.BYTES).putLong(length).putLong(origin).array(); + } + + public long getDurationMillis() + { + return length; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DurationGranularity that = (DurationGranularity) o; + + if (length != that.length) { + return false; + } + if (origin != that.origin) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = (int) (length ^ (length >>> 32)); + result = 31 * result + (int) (origin ^ (origin >>> 32)); + return result; + } + + @Override + public String toString() + { + return "DurationGranularity{" + + "length=" + length + + ", origin=" + origin + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/granularity/NoneGranularity.java b/processing/src/main/java/io/druid/granularity/NoneGranularity.java new file mode 100644 index 00000000000..3cf9398733a --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/NoneGranularity.java @@ -0,0 +1,66 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +public final class NoneGranularity extends BaseQueryGranularity +{ + @Override + public long next(long offset) + { + return offset + 1; + } + + @Override + public long truncate(long offset) + { + return offset; + } + + @Override + public byte[] cacheKey() + { + return new byte[]{0x0}; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return 0; + } + + @Override + public String toString() + { + return "NoneGranularity"; + } +} diff --git a/processing/src/main/java/io/druid/granularity/PeriodGranularity.java b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java new file mode 100644 index 00000000000..cc8faf66d41 --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/PeriodGranularity.java @@ -0,0 +1,375 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Chronology; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Period; +import org.joda.time.chrono.ISOChronology; + +public class PeriodGranularity extends BaseQueryGranularity +{ + private final Period period; + private final Chronology chronology; + private final long origin; + private final boolean hasOrigin; + + private final boolean isCompound; + + @JsonCreator + public PeriodGranularity( + @JsonProperty("period") Period period, + @JsonProperty("origin") DateTime origin, + @JsonProperty("timeZone") DateTimeZone tz + ) + { + this.period = period; + this.chronology = tz == null ? ISOChronology.getInstanceUTC() : ISOChronology.getInstance(tz); + if(origin == null) + { + // default to origin in given time zone when aligning multi-period granularities + this.origin = new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(chronology.getZone()).getMillis(); + this.hasOrigin = false; + } + else + { + this.origin = origin.getMillis(); + this.hasOrigin = true; + } + this.isCompound = isCompoundPeriod(period); + } + + @JsonProperty("period") + public Period getPeriod() + { + return period; + } + + @JsonProperty("timeZone") + public DateTimeZone getTimeZone() + { + return chronology.getZone(); + } + + @JsonProperty("origin") + public DateTime getOrigin() + { + return hasOrigin ? new DateTime(origin) : null; + } + + @Override + public DateTime toDateTime(long t) + { + return new DateTime(t, chronology.getZone()); + } + + @Override + public long next(long t) + { + return chronology.add(period, t, 1); + } + + @Override + public long truncate(long t) + { + if(isCompound) + { + try { + return truncateMillisPeriod(t); + } catch(UnsupportedOperationException e) { + return truncateCompoundPeriod(t); + } + } + + final int years = period.getYears(); + if(years > 0) + { + if(years > 1 || hasOrigin) + { + int y = chronology.years().getDifference(t, origin); + y -= y % years; + long tt = chronology.years().add(origin, y); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.years().add(tt, -years); + else t = tt; + return t; + } + else + { + return chronology.year().roundFloor(t); + } + } + + final int months = period.getMonths(); + if(months > 0) + { + if(months > 1 || hasOrigin) + { + int m = chronology.months().getDifference(t, origin); + m -= m % months; + long tt = chronology.months().add(origin, m); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.months().add(tt, -months); + else t = tt; + return t; + } + else + { + return chronology.monthOfYear().roundFloor(t); + } + } + + final int weeks = period.getWeeks(); + if(weeks > 0) + { + if(weeks > 1 || hasOrigin) + { + // align on multiples from origin + int w = chronology.weeks().getDifference(t, origin); + w -= w % weeks; + long tt = chronology.weeks().add(origin, w); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.weeks().add(tt, -weeks); + else t = tt; + return t; + } + else + { + t = chronology.dayOfWeek().roundFloor(t); + // default to Monday as beginning of the week + return chronology.dayOfWeek().set(t, 1); + } + } + + final int days = period.getDays(); + if(days > 0) + { + if(days > 1 || hasOrigin) + { + // align on multiples from origin + int d = chronology.days().getDifference(t, origin); + d -= d % days; + long tt = chronology.days().add(origin, d); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.days().add(tt, -days); + else t = tt; + return t; + } + else + { + t = chronology.hourOfDay().roundFloor(t); + return chronology.hourOfDay().set(t, 0); + } + } + + final int hours = period.getHours(); + if(hours > 0) + { + if(hours > 1 || hasOrigin) + { + // align on multiples from origin + long h = chronology.hours().getDifferenceAsLong(t, origin); + h -= h % hours; + long tt = chronology.hours().add(origin, h); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.hours().add(tt, -hours); + else t = tt; + return t; + } + else + { + t = chronology.minuteOfHour().roundFloor(t); + return chronology.minuteOfHour().set(t, 0); + } + } + + final int minutes = period.getMinutes(); + if(minutes > 0) + { + // align on multiples from origin + if(minutes > 1 || hasOrigin) + { + long m = chronology.minutes().getDifferenceAsLong(t, origin); + m -= m % minutes; + long tt = chronology.minutes().add(origin, m); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.minutes().add(tt, -minutes); + else t = tt; + return t; + } + else + { + t = chronology.secondOfMinute().roundFloor(t); + return chronology.secondOfMinute().set(t, 0); + } + } + + final int seconds = period.getSeconds(); + if(seconds > 0) + { + // align on multiples from origin + if(seconds > 1 || hasOrigin) + { + long s = chronology.seconds().getDifferenceAsLong(t, origin); + s -= s % seconds; + long tt = chronology.seconds().add(origin, s); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.seconds().add(tt, -seconds); + else t = tt; + return t; + } + else + { + return chronology.millisOfSecond().set(t, 0); + } + } + + final int millis = period.getMillis(); + if(millis > 0) + { + if(millis > 1) + { + long ms = chronology.millis().getDifferenceAsLong(t, origin); + ms -= ms % millis; + long tt = chronology.millis().add(origin, ms); + // always round down to the previous period (for timestamps prior to origin) + if(t < tt) t = chronology.millis().add(tt, -millis); + else t = tt; + return t; + } + else { + return t; + } + } + + return t; + } + + private static boolean isCompoundPeriod(Period period) + { + int[] values = period.getValues(); + boolean single = false; + for(int v : values) + { + if(v > 0) + { + if(single) return true; + single = true; + } + } + return false; + } + + private long truncateCompoundPeriod(long t) + { + long current; + if(t >= origin) + { + long next = origin; + do { + current = next; + next = chronology.add(period, current, 1); + } while(t >= next); + } + else + { + current = origin; + do { + current = chronology.add(period, current, -1); + } while(t < current); + } + return current; + } + + private long truncateMillisPeriod(long t) + { + // toStandardDuration assumes days are always 24h, and hours are always 60 minutes, + // which may not always be the case, e.g if there are daylight saving changes. + if(chronology.days().isPrecise() && chronology.hours().isPrecise()) { + final long millis = period.toStandardDuration().getMillis(); + t -= t % millis + origin % millis; + return t; + } + else + { + throw new UnsupportedOperationException( + "Period cannot be converted to milliseconds as some fields mays vary in length with chronology " + + chronology.toString() + ); + } + } + + @Override + public byte[] cacheKey() + { + return (period.toString() + ":" + chronology.getZone().toString()).getBytes(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + PeriodGranularity that = (PeriodGranularity) o; + + if (hasOrigin != that.hasOrigin) { + return false; + } + if (origin != that.origin) { + return false; + } + if (!chronology.equals(that.chronology)) { + return false; + } + if (!period.equals(that.period)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = period.hashCode(); + result = 31 * result + chronology.hashCode(); + result = 31 * result + (int) (origin ^ (origin >>> 32)); + result = 31 * result + (hasOrigin ? 1 : 0); + return result; + } + + @Override + public String toString() + { + return "PeriodGranularity{" + + "period=" + period + + ", timeZone=" + chronology .getZone() + + ", origin=" + (hasOrigin ? origin : "null") + + '}'; + } +} diff --git a/processing/src/main/java/io/druid/granularity/QueryGranularity.java b/processing/src/main/java/io/druid/granularity/QueryGranularity.java new file mode 100644 index 00000000000..16b86ac397d --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/QueryGranularity.java @@ -0,0 +1,91 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.granularity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.metamx.common.IAE; +import org.joda.time.DateTime; +import org.joda.time.ReadableDuration; + +public abstract class QueryGranularity +{ + public abstract long next(long offset); + + public abstract long truncate(long offset); + + public abstract byte[] cacheKey(); + + public abstract DateTime toDateTime(long offset); + + public abstract Iterable iterable(final long start, final long end); + + public static final QueryGranularity ALL = new AllGranularity(); + public static final QueryGranularity NONE = new NoneGranularity(); + + public static final QueryGranularity MINUTE = fromString("MINUTE"); + public static final QueryGranularity HOUR = fromString("HOUR"); + public static final QueryGranularity DAY = fromString("DAY"); + public static final QueryGranularity SECOND = fromString("SECOND"); + + @JsonCreator + public static QueryGranularity fromString(String str) + { + String name = str.toUpperCase(); + if(name.equals("ALL")) + { + return QueryGranularity.ALL; + } + else if(name.equals("NONE")) + { + return QueryGranularity.NONE; + } + return new DurationGranularity(convertValue(str), 0); + } + + private static enum MillisIn + { + SECOND ( 1000), + MINUTE ( 60 * 1000), + FIFTEEN_MINUTE (15 * 60 * 1000), + THIRTY_MINUTE (30 * 60 * 1000), + HOUR ( 3600 * 1000), + DAY (24 * 3600 * 1000); + + private final long millis; + MillisIn(final long millis) { this.millis = millis; } + } + + private static long convertValue(Object o) + { + if(o instanceof String) + { + return MillisIn.valueOf(((String) o).toUpperCase()).millis; + } + else if(o instanceof ReadableDuration) + { + return ((ReadableDuration)o).getMillis(); + } + else if(o instanceof Number) + { + return ((Number)o).longValue(); + } + throw new IAE("Cannot convert [%s] to QueryGranularity", o.getClass()); + } +} diff --git a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java b/processing/src/main/java/io/druid/guice/QueryToolChestModule.java similarity index 50% rename from client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java rename to processing/src/main/java/io/druid/guice/QueryToolChestModule.java index ffdf639ea7a..8cbc9801367 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryToolChestModule.java +++ b/processing/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -1,24 +1,45 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.group.GroupByQueryConfig; -import com.metamx.druid.query.group.GroupByQueryQueryToolChest; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQueryQueryToolChest; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQueryConfig; -import com.metamx.druid.query.search.SearchQueryQueryToolChest; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; -import com.metamx.druid.query.timeseries.TimeseriesQuery; -import com.metamx.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.LazySingleton; import io.druid.query.Query; import io.druid.query.QueryToolChest; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/BaseQuery.java b/processing/src/main/java/io/druid/query/BaseQuery.java similarity index 95% rename from client/src/main/java/com/metamx/druid/BaseQuery.java rename to processing/src/main/java/io/druid/query/BaseQuery.java index 02c75bd48ac..b1e1a587123 100644 --- a/client/src/main/java/com/metamx/druid/BaseQuery.java +++ b/processing/src/main/java/io/druid/query/BaseQuery.java @@ -17,15 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import io.druid.query.Query; -import io.druid.query.QueryRunner; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Duration; import org.joda.time.Interval; @@ -55,7 +52,6 @@ public abstract class BaseQuery implements Query this.dataSource = dataSource.toLowerCase(); this.context = context; this.querySegmentSpec = querySegmentSpec; - } @JsonProperty diff --git a/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java similarity index 92% rename from client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java rename to processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 43b2f7be7c4..79c6a4e03a9 100644 --- a/client/src/main/java/com/metamx/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.collect.Lists; import com.metamx.common.guava.Accumulator; @@ -26,10 +26,6 @@ import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.Yielders; import com.metamx.common.guava.YieldingAccumulator; -import com.metamx.druid.result.BySegmentResultValueClass; -import com.metamx.druid.result.Result; -import io.druid.query.Query; -import io.druid.query.QueryRunner; import org.joda.time.DateTime; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentResultValue.java b/processing/src/main/java/io/druid/query/BySegmentResultValue.java similarity index 92% rename from client/src/main/java/com/metamx/druid/result/BySegmentResultValue.java rename to processing/src/main/java/io/druid/query/BySegmentResultValue.java index df92619fc4d..3d2dc0802f3 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentResultValue.java +++ b/processing/src/main/java/io/druid/query/BySegmentResultValue.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java similarity index 95% rename from client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java rename to processing/src/main/java/io/druid/query/BySegmentResultValueClass.java index 9bc3ae81e2c..c26bfb35706 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentResultValueClass.java +++ b/processing/src/main/java/io/druid/query/BySegmentResultValueClass.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java similarity index 90% rename from client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java rename to processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java index 004481a4a90..1711fb316f9 100644 --- a/client/src/main/java/com/metamx/druid/query/BySegmentSkippingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentSkippingQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.metamx.common.guava.Sequence; -import io.druid.query.Query; -import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java similarity index 100% rename from client/src/main/java/io/druid/query/CacheStrategy.java rename to processing/src/main/java/io/druid/query/CacheStrategy.java diff --git a/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java rename to processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index f29642e5cce..8710b627ece 100644 --- a/client/src/main/java/com/metamx/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.google.common.base.Predicates; @@ -30,8 +30,6 @@ import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import io.druid.query.Query; -import io.druid.query.QueryRunner; import java.util.Arrays; import java.util.Iterator; diff --git a/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java rename to processing/src/main/java/io/druid/query/ConcatQueryRunner.java index 95bc6dceea0..9735d0f5a94 100644 --- a/client/src/main/java/com/metamx/druid/query/ConcatQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ConcatQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import io.druid.query.Query; -import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/DelegatingExecutorService.java b/processing/src/main/java/io/druid/query/DelegatingExecutorService.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/DelegatingExecutorService.java rename to processing/src/main/java/io/druid/query/DelegatingExecutorService.java index f9d377694e8..5a6be3f854c 100644 --- a/client/src/main/java/com/metamx/druid/query/DelegatingExecutorService.java +++ b/processing/src/main/java/io/druid/query/DelegatingExecutorService.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import java.util.Collection; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/Druids.java b/processing/src/main/java/io/druid/query/Druids.java similarity index 95% rename from client/src/main/java/com/metamx/druid/Druids.java rename to processing/src/main/java/io/druid/query/Druids.java index b6a3aa5db6e..99d5acdc6f6 100644 --- a/client/src/main/java/com/metamx/druid/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -17,28 +17,27 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query; import com.google.common.collect.Lists; -import com.metamx.druid.query.filter.AndDimFilter; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.filter.NoopDimFilter; -import com.metamx.druid.query.filter.NotDimFilter; -import com.metamx.druid.query.filter.OrDimFilter; -import com.metamx.druid.query.filter.SelectorDimFilter; -import com.metamx.druid.query.search.InsensitiveContainsSearchQuerySpec; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.search.SearchQuerySpec; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.query.timeseries.TimeseriesQuery; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; -import com.metamx.druid.result.TimeBoundaryResultValue; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.NoopDimFilter; +import io.druid.query.filter.NotDimFilter; +import io.druid.query.filter.OrDimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQuerySpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import io.druid.query.timeseries.TimeseriesQuery; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -295,7 +294,7 @@ public class Druids * .build(); * * - * @see com.metamx.druid.query.timeseries.TimeseriesQuery + * @see io.druid.query.timeseries.TimeseriesQuery */ public static class TimeseriesQueryBuilder { @@ -489,7 +488,7 @@ public class Druids * .build(); * * - * @see com.metamx.druid.query.search.SearchQuery + * @see io.druid.query.search.search.SearchQuery */ public static class SearchQueryBuilder { @@ -673,7 +672,7 @@ public class Druids * .build(); * * - * @see com.metamx.druid.query.timeboundary.TimeBoundaryQuery + * @see io.druid.query.timeboundary.TimeBoundaryQuery */ public static class TimeBoundaryQueryBuilder { @@ -754,7 +753,7 @@ public class Druids * .build(); * * - * @see com.metamx.druid.result.Result + * @see Result */ public static class ResultBuilder { diff --git a/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java similarity index 92% rename from client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java rename to processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java index cd0e2a3108b..77823a60730 100644 --- a/client/src/main/java/com/metamx/druid/query/FinalizeResultsQueryRunner.java +++ b/processing/src/main/java/io/druid/query/FinalizeResultsQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,18 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.result.BySegmentResultValueClass; -import com.metamx.druid.result.Result; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; diff --git a/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java rename to processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java index bbb12ce7e01..53e889306f5 100644 --- a/client/src/main/java/com/metamx/druid/query/IntervalChunkingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/IntervalChunkingQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.google.common.collect.Lists; @@ -25,8 +25,6 @@ import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.PeriodGranularity; -import io.druid.query.Query; -import io.druid.query.QueryRunner; import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/processing/src/main/java/io/druid/query/MapQueryToolChestWarehouse.java b/processing/src/main/java/io/druid/query/MapQueryToolChestWarehouse.java new file mode 100644 index 00000000000..c39af79654d --- /dev/null +++ b/processing/src/main/java/io/druid/query/MapQueryToolChestWarehouse.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import com.google.inject.Inject; + +import java.util.Map; + +/** + */ +public class MapQueryToolChestWarehouse implements QueryToolChestWarehouse +{ + private final Map, QueryToolChest> toolchests; + + @Inject + public MapQueryToolChestWarehouse( + Map, QueryToolChest> toolchests + ) + { + this.toolchests = toolchests; + } + + @Override + @SuppressWarnings("unchecked") + public > QueryToolChest getToolChest(QueryType query) + { + return toolchests.get(query.getClass()); + } +} diff --git a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java b/processing/src/main/java/io/druid/query/MetricValueExtractor.java similarity index 89% rename from client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java rename to processing/src/main/java/io/druid/query/MetricValueExtractor.java index f330a12400d..a364200f9d9 100644 --- a/client/src/main/java/com/metamx/druid/result/MetricValueExtractor.java +++ b/processing/src/main/java/io/druid/query/MetricValueExtractor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -import java.lang.Double;import java.lang.Float;import java.lang.Long;import java.lang.Number;import java.lang.Object;import java.lang.Override;import java.lang.String;import java.util.Map; +import java.util.Map; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java b/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java rename to processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java index 65f6f2833e9..4893ebd8adf 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricsEmittingExecutorService.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingExecutorService.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; diff --git a/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java rename to processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 9b346099a53..9da3101003c 100644 --- a/client/src/main/java/com/metamx/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.metamx.common.guava.Accumulator; @@ -26,8 +26,6 @@ import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import io.druid.query.Query; -import io.druid.query.QueryRunner; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java b/processing/src/main/java/io/druid/query/NoopQueryRunner.java similarity index 87% rename from client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java rename to processing/src/main/java/io/druid/query/NoopQueryRunner.java index a143a43641d..355ee4f7a20 100644 --- a/client/src/main/java/com/metamx/druid/query/NoopQueryRunner.java +++ b/processing/src/main/java/io/druid/query/NoopQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import io.druid.query.Query; -import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/PrioritizedCallable.java b/processing/src/main/java/io/druid/query/PrioritizedCallable.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/PrioritizedCallable.java rename to processing/src/main/java/io/druid/query/PrioritizedCallable.java index 3771b8a03c2..e664a8d682b 100644 --- a/client/src/main/java/com/metamx/druid/query/PrioritizedCallable.java +++ b/processing/src/main/java/io/druid/query/PrioritizedCallable.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import java.util.concurrent.Callable; diff --git a/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java b/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java rename to processing/src/main/java/io/druid/query/PrioritizedExecutorService.java index b3892477a3d..f390fe1b7a4 100644 --- a/client/src/main/java/com/metamx/druid/query/PrioritizedExecutorService.java +++ b/processing/src/main/java/io/druid/query/PrioritizedExecutorService.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ThreadFactoryBuilder; diff --git a/client/src/main/java/com/metamx/druid/query/Queries.java b/processing/src/main/java/io/druid/query/Queries.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/Queries.java rename to processing/src/main/java/io/druid/query/Queries.java index 2913f8f6f3d..3fd1f802189 100644 --- a/client/src/main/java/com/metamx/druid/query/Queries.java +++ b/processing/src/main/java/io/druid/query/Queries.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.base.Function; import com.google.common.base.Preconditions; diff --git a/client/src/main/java/io/druid/query/Query.java b/processing/src/main/java/io/druid/query/Query.java similarity index 88% rename from client/src/main/java/io/druid/query/Query.java rename to processing/src/main/java/io/druid/query/Query.java index 342a0b332b3..daf1410308e 100644 --- a/client/src/main/java/io/druid/query/Query.java +++ b/processing/src/main/java/io/druid/query/Query.java @@ -22,13 +22,12 @@ package io.druid.query; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.search.search.SearchQuery; import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeseries.TimeseriesQuery; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java b/processing/src/main/java/io/druid/query/QueryCacheHelper.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java rename to processing/src/main/java/io/druid/query/QueryCacheHelper.java index 16b3a0836c1..d61a5778d03 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryCacheHelper.java +++ b/processing/src/main/java/io/druid/query/QueryCacheHelper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.collect.Lists; import io.druid.query.aggregation.AggregatorFactory; diff --git a/client/src/main/java/io/druid/query/QueryRunner.java b/processing/src/main/java/io/druid/query/QueryRunner.java similarity index 100% rename from client/src/main/java/io/druid/query/QueryRunner.java rename to processing/src/main/java/io/druid/query/QueryRunner.java diff --git a/processing/src/main/java/io/druid/query/QueryRunnerFactory.java b/processing/src/main/java/io/druid/query/QueryRunnerFactory.java new file mode 100644 index 00000000000..3cbe11d6241 --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryRunnerFactory.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query; + +import io.druid.segment.Segment; + +import java.util.concurrent.ExecutorService; + +/** + */ +public interface QueryRunnerFactory> +{ + public QueryRunner createRunner(Segment segment); + public QueryRunner mergeRunners(ExecutorService queryExecutor, Iterable> queryRunners); + public QueryToolChest getToolchest(); +} diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java b/processing/src/main/java/io/druid/query/QuerySegmentWalker.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java rename to processing/src/main/java/io/druid/query/QuerySegmentWalker.java index 541eede6e94..8e8c6b8c0fe 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentWalker.java +++ b/processing/src/main/java/io/druid/query/QuerySegmentWalker.java @@ -17,11 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query; -import io.druid.query.Query; -import io.druid.query.QueryRunner; -import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; /** diff --git a/client/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java similarity index 98% rename from client/src/main/java/io/druid/query/QueryToolChest.java rename to processing/src/main/java/io/druid/query/QueryToolChest.java index 4ed1e40d79e..3d4726bc434 100644 --- a/client/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -22,9 +22,9 @@ package io.druid.query; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; +import com.metamx.druid.LogicalSegment; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.aggregation.MetricManipulationFn; -import io.druid.segment.LogicalSegment; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java b/processing/src/main/java/io/druid/query/QueryToolChestWarehouse.java similarity index 86% rename from client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java rename to processing/src/main/java/io/druid/query/QueryToolChestWarehouse.java index a7a61b2aede..c501eec5885 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChestWarehouse.java +++ b/processing/src/main/java/io/druid/query/QueryToolChestWarehouse.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; - -import io.druid.query.Query; -import io.druid.query.QueryToolChest; +package io.druid.query; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/ReflectionLoaderThingy.java b/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/ReflectionLoaderThingy.java rename to processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java index cd9eba24f25..75636554217 100644 --- a/client/src/main/java/com/metamx/druid/query/ReflectionLoaderThingy.java +++ b/processing/src/main/java/io/druid/query/ReflectionLoaderThingy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; diff --git a/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java b/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java similarity index 89% rename from client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java rename to processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java index 75db6b8fb2f..37ad0e979c6 100644 --- a/client/src/main/java/com/metamx/druid/query/ReflectionQueryToolChestWarehouse.java +++ b/processing/src/main/java/io/druid/query/ReflectionQueryToolChestWarehouse.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; - -import io.druid.query.Query; -import io.druid.query.QueryToolChest; +package io.druid.query; /** */ diff --git a/client/src/main/java/com/metamx/druid/result/Result.java b/processing/src/main/java/io/druid/query/Result.java similarity index 96% rename from client/src/main/java/com/metamx/druid/result/Result.java rename to processing/src/main/java/io/druid/query/Result.java index e807ce1ddfa..bb299d51c78 100644 --- a/client/src/main/java/com/metamx/druid/result/Result.java +++ b/processing/src/main/java/io/druid/query/Result.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java b/processing/src/main/java/io/druid/query/ResultGranularTimestampComparator.java similarity index 92% rename from client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java rename to processing/src/main/java/io/druid/query/ResultGranularTimestampComparator.java index a66ae16bf6f..a7d5c509346 100644 --- a/client/src/main/java/com/metamx/druid/ResultGranularTimestampComparator.java +++ b/processing/src/main/java/io/druid/query/ResultGranularTimestampComparator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query; import com.google.common.primitives.Longs; -import com.metamx.druid.result.Result; import io.druid.granularity.QueryGranularity; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java rename to processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java index 5bcb3edc278..f1ba9185fe8 100644 --- a/client/src/main/java/com/metamx/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.druid.guava.CombiningSequence; -import io.druid.query.Query; -import io.druid.query.QueryRunner; /** */ diff --git a/client/src/main/java/io/druid/query/SegmentDescriptor.java b/processing/src/main/java/io/druid/query/SegmentDescriptor.java similarity index 98% rename from client/src/main/java/io/druid/query/SegmentDescriptor.java rename to processing/src/main/java/io/druid/query/SegmentDescriptor.java index 0146a33bcbd..6805cd3c8bd 100644 --- a/client/src/main/java/io/druid/query/SegmentDescriptor.java +++ b/processing/src/main/java/io/druid/query/SegmentDescriptor.java @@ -35,7 +35,8 @@ public class SegmentDescriptor public SegmentDescriptor( @JsonProperty("itvl") Interval interval, @JsonProperty("ver") String version, - @JsonProperty("part") int partitionNumber) + @JsonProperty("part") int partitionNumber + ) { this.interval = interval; this.version = version; diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java new file mode 100644 index 00000000000..4ca3d64c302 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation; + +/** + * An Aggregator is an object that can aggregate metrics. Its aggregation-related methods (namely, aggregate() and get()) + * do not take any arguments as the assumption is that the Aggregator was given something in its constructor that + * it can use to get at the next bit of data. + * + * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls + * to aggregate(). This is currently (as of this documentation) implemented through the use of Offset and + * FloatMetricSelector objects. The Aggregator has a handle on a FloatMetricSelector object which has a handle on an Offset. + * QueryableIndex has both the Aggregators and the Offset object and iterates through the Offset calling the aggregate() + * method on the Aggregators for each applicable row. + * + * This interface is old and going away. It is being replaced by BufferAggregator + */ +public interface Aggregator { + void aggregate(); + void reset(); + Object get(); + float getFloat(); + String getName(); + void close(); +} diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java new file mode 100644 index 00000000000..18168b81986 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -0,0 +1,100 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation; + +import io.druid.segment.MetricSelectorFactory; + +import java.util.Comparator; +import java.util.List; + +/** + * Processing related interface + * + * An AggregatorFactory is an object that knows how to generate an Aggregator using a MetricSelectorFactory. + * + * This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects + * without making any assumptions about how they are pulling values out of the base data. That is, the data is + * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how + * the data is actually stored and accessed. + */ +public interface AggregatorFactory +{ + public Aggregator factorize(MetricSelectorFactory metricFactory); + public BufferAggregator factorizeBuffered(MetricSelectorFactory metricFactory); + public Comparator getComparator(); + + /** + * A method that knows how to combine the outputs of the getIntermediate() method from the Aggregators + * produced via factorize(). Note, even though this is called combine, this method's contract *does* + * allow for mutation of the input objects. Thus, any use of lhs or rhs after calling this method is + * highly discouraged. + * + * @param lhs The left hand side of the combine + * @param rhs The right hand side of the combine + * @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs + */ + public Object combine(Object lhs, Object rhs); + + /** + * Returns an AggregatorFactory that can be used to combine the output of aggregators from this factory. This + * generally amounts to simply creating a new factory that is the same as the current except with its input + * column renamed to the same as the output column. + * + * @return a new Factory that can be used for operations on top of data output from the current factory. + */ + public AggregatorFactory getCombiningFactory(); + + /** + * A method that knows how to "deserialize" the object from whatever form it might have been put into + * in order to transfer via JSON. + * + * @param object the object to deserialize + * @return the deserialized object + */ + public Object deserialize(Object object); + + /** + * "Finalizes" the computation of an object. Primarily useful for complex types that have a different mergeable + * intermediate format than their final resultant output. + * + * @param object the object to be finalized + * @return the finalized value that should be returned for the initial query + */ + public Object finalizeComputation(Object object); + + public String getName(); + public List requiredFields(); + public byte[] getCacheKey(); + public String getTypeName(); + + /** + * Returns the maximum size that this aggregator will require in bytes for intermediate storage of results. + * + * @return the maximum number of bytes that an aggregator of this type will require for intermediate result storage. + */ + public int getMaxIntermediateSize(); + + /** + * Returns the starting value for a corresponding aggregator. For example, 0 for sums, - Infinity for max, an empty mogrifier + * + * @return the starting value for a corresponding aggregator. + */ + public Object getAggregatorStartValue(); +} diff --git a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java new file mode 100644 index 00000000000..da581ae3064 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -0,0 +1,96 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation; + +import java.nio.ByteBuffer; + +/** + * A BufferAggregator is an object that can aggregate metrics into a ByteBuffer. Its aggregation-related methods + * (namely, aggregate(...) and get(...)) only take the ByteBuffer and position because it is assumed that the Aggregator + * was given something (one or more MetricSelector(s)) in its constructor that it can use to get at the next bit of data. + * + * Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls + * to aggregate(...). + */ +public interface BufferAggregator +{ + /** + * Initializes the buffer location + * + * Implementations of this method must initialize the byte buffer at the given position + * + * Implementations must not change the position, limit or mark of the given buffer + * + * This method must not exceed the number of bytes returned by {@link AggregatorFactory#getMaxIntermediateSize()} + * in the corresponding {@link AggregatorFactory} + * + * @param buf byte buffer to initialize + * @param position offset within the byte buffer for initialization + */ + void init(ByteBuffer buf, int position); + + /** + * Aggregates metric values into the given aggregate byte representation + * + * Implementations of this method must read in the aggregate value from the buffer at the given position, + * aggregate the next element of data and write the updated aggregate value back into the buffer. + * + * Implementations must not change the position, limit or mark of the given buffer + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the current aggregate value is stored + */ + void aggregate(ByteBuffer buf, int position); + + /** + * Returns the intermediate object representation of the given aggregate. + * + * Converts the given byte buffer representation into an intermediate aggregate Object + * + * Implementations must not change the position, limit or mark of the given buffer + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * @return the Object representation of the aggregate + */ + Object get(ByteBuffer buf, int position); + + /** + * Returns the float representation of the given aggregate byte array + * + * Converts the given byte buffer representation into the intermediate aggregate value. + * + * Implementations must not change the position, limit or mark of the given buffer + * + * Implementations are only required to support this method if they are aggregations which + * have an {@link AggregatorFactory#getTypeName()} of "float". + * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * @return the float representation of the aggregate + */ + float getFloat(ByteBuffer buf, int position); + + /** + * Release any resources used by the aggregator + */ + void close(); +} diff --git a/client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java b/processing/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java similarity index 100% rename from client/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java rename to processing/src/main/java/io/druid/query/aggregation/MetricManipulationFn.java diff --git a/processing/src/main/java/io/druid/query/aggregation/PostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/PostAggregator.java new file mode 100644 index 00000000000..37d0ce3e03b --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/PostAggregator.java @@ -0,0 +1,38 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +/** + * Functionally similar to an Aggregator. See the Aggregator interface for more comments. + */ +public interface PostAggregator +{ + public Set getDependentFields(); + + public Comparator getComparator(); + + public Object compute(Map combinedAggregators); + + public String getName(); +} diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java rename to processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java index 2f57b60135f..3a9137b008e 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DefaultDimensionSpec.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.dimension; +package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java rename to processing/src/main/java/io/druid/query/dimension/DimensionSpec.java index 09c2b4316d8..3c552e20763 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/DimensionSpec.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.dimension; +package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.DimExtractionFn; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java rename to processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java index 1089df851d5..82fba73d0a1 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/ExtractionDimensionSpec.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.dimension; +package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java b/processing/src/main/java/io/druid/query/dimension/LegacyDimensionSpec.java similarity index 94% rename from client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java rename to processing/src/main/java/io/druid/query/dimension/LegacyDimensionSpec.java index 352142c1166..6e1088ed788 100644 --- a/client/src/main/java/com/metamx/druid/query/dimension/LegacyDimensionSpec.java +++ b/processing/src/main/java/io/druid/query/dimension/LegacyDimensionSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.dimension; +package io.druid.query.dimension; import com.fasterxml.jackson.annotation.JsonCreator; import com.metamx.common.IAE; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java rename to processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java index 88326e536bb..65b5c2a5d7b 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/DimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/DimExtractionFn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java rename to processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java index 4dc830e5aa7..5ae4e2bf019 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/PartialDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/PartialDimExtractionFn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java rename to processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java index c839486e816..c25d0ba6c7a 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/RegexDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/RegexDimExtractionFn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java rename to processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java index 0d5f674c011..02c13eca630 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/SearchQuerySpecDimExtractionFn.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.search.search.SearchQuerySpec; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java rename to processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java index 1f8b834279b..57b7f6617d8 100644 --- a/client/src/main/java/com/metamx/druid/query/extraction/TimeDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/TimeDimExtractionFn.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java b/processing/src/main/java/io/druid/query/filter/AndDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java rename to processing/src/main/java/io/druid/query/filter/AndDimFilter.java index 6a1bb9109aa..00b15c42e71 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/AndDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/AndDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java b/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java new file mode 100644 index 00000000000..c0a2abca40b --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/BitmapIndexSelector.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.filter; + +import com.metamx.collections.spatial.ImmutableRTree; +import io.druid.segment.data.Indexed; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** + */ +public interface BitmapIndexSelector +{ + public Indexed getDimensionValues(String dimension); + public int getNumRows(); + public ImmutableConciseSet getConciseInvertedIndex(String dimension, String value); + public ImmutableConciseSet getConciseInvertedIndex(String dimension, int idx); + public ImmutableRTree getSpatialIndex(String dimension); +} diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java b/processing/src/main/java/io/druid/query/filter/DimFilter.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/filter/DimFilter.java rename to processing/src/main/java/io/druid/query/filter/DimFilter.java index 26f9bf8519c..7e303f0f253 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilterCacheHelper.java b/processing/src/main/java/io/druid/query/filter/DimFilterCacheHelper.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/filter/DimFilterCacheHelper.java rename to processing/src/main/java/io/druid/query/filter/DimFilterCacheHelper.java index 02d20ea528c..5fa3a7386b6 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilterCacheHelper.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilterCacheHelper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import java.nio.ByteBuffer; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/filter/DimFilters.java b/processing/src/main/java/io/druid/query/filter/DimFilters.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/DimFilters.java rename to processing/src/main/java/io/druid/query/filter/DimFilters.java index 0b74f443c3e..5147565d7ca 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/DimFilters.java +++ b/processing/src/main/java/io/druid/query/filter/DimFilters.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.google.common.base.Function; import com.google.common.collect.Lists; diff --git a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java b/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java rename to processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java index aa1fb1efd2b..8ce26f71f99 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/ExtractionDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/ExtractionDimFilter.java @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.metamx.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.DimExtractionFn; import java.nio.ByteBuffer; diff --git a/processing/src/main/java/io/druid/query/filter/Filter.java b/processing/src/main/java/io/druid/query/filter/Filter.java new file mode 100644 index 00000000000..73d39074c75 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/Filter.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.filter; + +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; + +/** + */ +public interface Filter +{ + public ImmutableConciseSet goConcise(BitmapIndexSelector selector); + public ValueMatcher makeMatcher(ValueMatcherFactory factory); +} diff --git a/client/src/main/java/com/metamx/druid/query/filter/JavaScriptDimFilter.java b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java similarity index 60% rename from client/src/main/java/com/metamx/druid/query/filter/JavaScriptDimFilter.java rename to processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java index 00db76b4f05..b9cef360238 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/JavaScriptDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/JavaScriptDimFilter.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.filter; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/NoopDimFilter.java b/processing/src/main/java/io/druid/query/filter/NoopDimFilter.java similarity index 92% rename from client/src/main/java/com/metamx/druid/query/filter/NoopDimFilter.java rename to processing/src/main/java/io/druid/query/filter/NoopDimFilter.java index 6ceb7f565cd..1d518c07171 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/NoopDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/NoopDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java b/processing/src/main/java/io/druid/query/filter/NotDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java rename to processing/src/main/java/io/druid/query/filter/NotDimFilter.java index 9f690f1322d..b2037b49ba9 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/NotDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/NotDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java b/processing/src/main/java/io/druid/query/filter/OrDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java rename to processing/src/main/java/io/druid/query/filter/OrDimFilter.java index 4fdde0f46bb..bb4cb769a33 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/OrDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/OrDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java b/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java rename to processing/src/main/java/io/druid/query/filter/RegexDimFilter.java index edbd1b7fc5d..0644250f819 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/RegexDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/RegexDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java b/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java rename to processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java index edb7ce791e0..76c5ecd0148 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SearchQueryDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SearchQueryDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; -import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.search.search.SearchQuerySpec; import java.nio.ByteBuffer; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java rename to processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java index dd2f39dfe13..b8d1580a169 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SelectorDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SelectorDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/filter/SpatialDimFilter.java b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/filter/SpatialDimFilter.java rename to processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java index 4a50668b2e5..6899d306602 100644 --- a/client/src/main/java/com/metamx/druid/query/filter/SpatialDimFilter.java +++ b/processing/src/main/java/io/druid/query/filter/SpatialDimFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.filter; +package io.druid.query.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/io/druid/query/filter/ValueMatcher.java b/processing/src/main/java/io/druid/query/filter/ValueMatcher.java new file mode 100644 index 00000000000..aac95f44633 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/ValueMatcher.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.filter; + +/** + */ +public interface ValueMatcher +{ + public boolean matches(); +} diff --git a/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java b/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java new file mode 100644 index 00000000000..45834974639 --- /dev/null +++ b/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.filter; + +import com.google.common.base.Predicate; +import com.metamx.collections.spatial.search.Bound; + +/** + */ +public interface ValueMatcherFactory +{ + public ValueMatcher makeValueMatcher(String dimension, String value); + public ValueMatcher makeValueMatcher(String dimension, Predicate value); + public ValueMatcher makeValueMatcher(String dimension, Bound bound); +} diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java rename to processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index dbfbc9f1bd4..bd75455bb02 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.group; +package io.druid.query.groupby; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -30,21 +30,20 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.BaseQuery; import com.metamx.druid.input.Row; -import com.metamx.druid.query.Queries; -import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.group.having.HavingSpec; -import com.metamx.druid.query.group.orderby.DefaultLimitSpec; -import com.metamx.druid.query.group.orderby.LimitSpec; -import com.metamx.druid.query.group.orderby.NoopLimitSpec; -import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import io.druid.granularity.QueryGranularity; -import io.druid.query.Query; +import io.druid.query.BaseQuery; +import io.druid.query.Queries; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DimFilter; +import io.druid.query.groupby.having.HavingSpec; +import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.groupby.orderby.LimitSpec; +import io.druid.query.groupby.orderby.NoopLimitSpec; +import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; @@ -210,7 +209,7 @@ public class GroupByQuery extends BaseQuery @Override public String getType() { - return Query.GROUP_BY; + return GROUP_BY; } public Sequence applyLimit(Sequence results) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java new file mode 100644 index 00000000000..98062a4f4c9 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class GroupByQueryConfig +{ + @JsonProperty + private boolean singleThreaded = false; + + @JsonProperty + private int maxIntermediateRows = 50000; + + @JsonProperty + private int maxResults = 500000; + + public boolean isSingleThreaded() + { + return singleThreaded; + } + + public int getMaxIntermediateRows() + { + return maxIntermediateRows; + } + + public void setMaxIntermediateRows(int maxIntermediateRows) + { + this.maxIntermediateRows = maxIntermediateRows; + } + + public int getMaxResults() + { + return maxResults; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java rename to processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index d4e4e00f816..030bd20e7fd 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.group; +package io.druid.query.groupby; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; @@ -32,11 +32,9 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.input.Rows; -import com.metamx.druid.query.dimension.DimensionSpec; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; @@ -44,6 +42,8 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.dimension.DimensionSpec; +import io.druid.segment.incremental.IncrementalIndex; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java new file mode 100644 index 00000000000..f4e45b8bb1d --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; + +import com.metamx.druid.input.Row; + +/** + */ +public class AlwaysHavingSpec implements HavingSpec +{ + @Override + public boolean eval(Row row) + { + return true; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/group/having/AndHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java similarity index 64% rename from client/src/main/java/com/metamx/druid/query/group/having/AndHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java index f474bc67c95..0cd712a976f 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/AndHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/EqualToHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java similarity index 72% rename from client/src/main/java/com/metamx/druid/query/group/having/EqualToHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java index f7e5060e4e9..65eb9444df9 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/EqualToHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/GreaterThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java similarity index 70% rename from client/src/main/java/com/metamx/druid/query/group/having/GreaterThanHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java index a64f5ee5050..41f7864063c 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/GreaterThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/HavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java similarity index 64% rename from client/src/main/java/com/metamx/druid/query/group/having/HavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java index b40ae662534..4d1dca3ad0f 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/HavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/LessThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java similarity index 70% rename from client/src/main/java/com/metamx/druid/query/group/having/LessThanHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java index 86ff4dfaf9c..d54e1975b59 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/LessThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.Row; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/NotHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java similarity index 60% rename from client/src/main/java/com/metamx/druid/query/group/having/NotHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java index 9324268fbda..67397a568a4 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/NotHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/group/having/OrHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java similarity index 63% rename from client/src/main/java/com/metamx/druid/query/group/having/OrHavingSpec.java rename to processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java index 76c8920182f..92a48cd7a55 100644 --- a/client/src/main/java/com/metamx/druid/query/group/having/OrHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java similarity index 98% rename from client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java rename to processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index 050527454c8..0b3933be110 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.group.orderby; +package io.druid.query.groupby.orderby; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -32,9 +32,9 @@ import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java rename to processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java index 88037488d4a..72a4417a051 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/LimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.group.orderby; +package io.druid.query.groupby.orderby; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java rename to processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java index 542caf29908..0e2b6f3ed78 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.group.orderby; +package io.druid.query.groupby.orderby; import com.google.common.base.Function; import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/OrderByColumnSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java similarity index 82% rename from client/src/main/java/com/metamx/druid/query/group/orderby/OrderByColumnSpec.java rename to processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java index 6a02ec962f8..147f1911816 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/OrderByColumnSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.orderby; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.orderby; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/OrderedPriorityQueueItems.java b/processing/src/main/java/io/druid/query/groupby/orderby/OrderedPriorityQueueItems.java new file mode 100644 index 00000000000..c2ae523345a --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/orderby/OrderedPriorityQueueItems.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.orderby; + +import com.google.common.collect.MinMaxPriorityQueue; + +import java.util.Iterator; + +/** + * Utility class that supports iterating a priority queue in sorted order. + */ +class OrderedPriorityQueueItems implements Iterable +{ + private MinMaxPriorityQueue rows; + + public OrderedPriorityQueueItems(MinMaxPriorityQueue rows) + { + this.rows = rows; + } + + @Override + public Iterator iterator() + { + return new Iterator() { + + @Override + public boolean hasNext() + { + return !rows.isEmpty(); + } + + @Override + public T next() + { + return rows.poll(); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException("Can't remove any item from an intermediary heap for orderBy/limit"); + } + }; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/group/orderby/TopNSorter.java b/processing/src/main/java/io/druid/query/groupby/orderby/TopNSorter.java similarity index 56% rename from client/src/main/java/com/metamx/druid/query/group/orderby/TopNSorter.java rename to processing/src/main/java/io/druid/query/groupby/orderby/TopNSorter.java index 7160ba62074..d612e5784be 100644 --- a/client/src/main/java/com/metamx/druid/query/group/orderby/TopNSorter.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/TopNSorter.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.orderby; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.orderby; import com.google.common.collect.ImmutableList; import com.google.common.collect.MinMaxPriorityQueue; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java rename to processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 7d09f7e9d5c..40e4aeac0b5 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; @@ -32,14 +32,17 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.ResultMergeQueryRunner; import com.metamx.druid.utils.JodaUtils; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.CacheStrategy; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.metadata.metadata.ColumnAnalysis; +import io.druid.query.metadata.metadata.SegmentAnalysis; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java b/processing/src/main/java/io/druid/query/metadata/metadata/AllColumnIncluderator.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java rename to processing/src/main/java/io/druid/query/metadata/metadata/AllColumnIncluderator.java index cd96b5d718a..905025db7ca 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/AllColumnIncluderator.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/AllColumnIncluderator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java similarity index 86% rename from client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java rename to processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java index 8c353964a30..c41bd629ca4 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnAnalysis.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnAnalysis.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.segment.column.ValueType; /** */ @@ -31,7 +30,7 @@ public class ColumnAnalysis public static ColumnAnalysis error(String reason) { - return new ColumnAnalysis(ValueType.STRING, -1, null, ERROR_PREFIX + reason); + return new ColumnAnalysis("STRING", -1, null, ERROR_PREFIX + reason); } private final String type; @@ -41,21 +40,11 @@ public class ColumnAnalysis @JsonCreator public ColumnAnalysis( - @JsonProperty("type") ValueType type, + @JsonProperty("type") String type, @JsonProperty("size") long size, @JsonProperty("cardinality") Integer cardinality, @JsonProperty("errorMessage") String errorMessage ) - { - this(type.name(), size, cardinality, errorMessage); - } - - private ColumnAnalysis( - String type, - long size, - Integer cardinality, - String errorMessage - ) { this.type = type; this.size = size; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnIncluderator.java similarity index 94% rename from client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java rename to processing/src/main/java/io/druid/query/metadata/metadata/ColumnIncluderator.java index 230c21a2498..5b5b09940a3 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ColumnIncluderator.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/ColumnIncluderator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java b/processing/src/main/java/io/druid/query/metadata/metadata/ListColumnIncluderator.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java rename to processing/src/main/java/io/druid/query/metadata/metadata/ListColumnIncluderator.java index d6f4b99d74b..89b5cf8dd6b 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/ListColumnIncluderator.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/ListColumnIncluderator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java b/processing/src/main/java/io/druid/query/metadata/metadata/NoneColumnIncluderator.java similarity index 91% rename from client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java rename to processing/src/main/java/io/druid/query/metadata/metadata/NoneColumnIncluderator.java index d1d66d26778..afe36427be4 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/NoneColumnIncluderator.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/NoneColumnIncluderator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; /** */ diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentAnalysis.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java rename to processing/src/main/java/io/druid/query/metadata/metadata/SegmentAnalysis.java index 359e1020273..077fe3c99c6 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentAnalysis.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentAnalysis.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java rename to processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java index 0818f85a488..37361815968 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQuery.java +++ b/processing/src/main/java/io/druid/query/metadata/metadata/SegmentMetadataQuery.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.metadata; +package io.druid.query.metadata.metadata; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.BaseQuery; +import io.druid.query.BaseQuery; import io.druid.query.Query; import io.druid.query.spec.QuerySegmentSpec; diff --git a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java b/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java similarity index 91% rename from client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java rename to processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java index 64d2d6a7ec8..0ecbdf509c2 100644 --- a/client/src/main/java/com/metamx/druid/result/BySegmentSearchResultValue.java +++ b/processing/src/main/java/io/druid/query/search/BySegmentSearchResultValue.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query.search; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; -import com.metamx.druid.query.search.SearchHit; +import io.druid.query.BySegmentResultValue; +import io.druid.query.Result; +import io.druid.query.search.search.SearchHit; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/SearchBinaryFn.java b/processing/src/main/java/io/druid/query/search/SearchBinaryFn.java similarity index 89% rename from client/src/main/java/com/metamx/druid/SearchBinaryFn.java rename to processing/src/main/java/io/druid/query/search/SearchBinaryFn.java index b459ead8cd3..99093911739 100644 --- a/client/src/main/java/com/metamx/druid/SearchBinaryFn.java +++ b/processing/src/main/java/io/druid/query/search/SearchBinaryFn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,17 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query.search; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.SearchSortSpec; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; import io.druid.granularity.AllGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchSortSpec; import java.util.TreeSet; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java rename to processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index efcdfe7e465..5fe1c591d6c 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; @@ -35,21 +35,21 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.ResultGranularTimestampComparator; -import com.metamx.druid.SearchBinaryFn; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.IntervalChunkingQueryRunner; -import com.metamx.druid.query.ResultMergeQueryRunner; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.result.BySegmentSearchResultValue; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.query.CacheStrategy; +import io.druid.query.IntervalChunkingQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.ResultGranularTimestampComparator; +import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.filter.DimFilter; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQueryConfig; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; diff --git a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java b/processing/src/main/java/io/druid/query/search/SearchResultValue.java similarity index 93% rename from client/src/main/java/com/metamx/druid/result/SearchResultValue.java rename to processing/src/main/java/io/druid/query/search/SearchResultValue.java index 1403d111945..1ffbf510c96 100644 --- a/client/src/main/java/com/metamx/druid/result/SearchResultValue.java +++ b/processing/src/main/java/io/druid/query/search/SearchResultValue.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query.search; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -import com.metamx.druid.query.search.SearchHit; +import io.druid.query.search.search.SearchHit; import java.util.Iterator; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/processing/src/main/java/io/druid/query/search/search/FragmentSearchQuerySpec.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java rename to processing/src/main/java/io/druid/query/search/search/FragmentSearchQuerySpec.java index 3abf19d5124..6f5092d6d85 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/processing/src/main/java/io/druid/query/search/search/FragmentSearchQuerySpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/processing/src/main/java/io/druid/query/search/search/InsensitiveContainsSearchQuerySpec.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java rename to processing/src/main/java/io/druid/query/search/search/InsensitiveContainsSearchQuerySpec.java index 44b0ee07197..acc23576fea 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/processing/src/main/java/io/druid/query/search/search/InsensitiveContainsSearchQuerySpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java b/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java rename to processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java index 99ac489cadd..588aca7493a 100644 --- a/client/src/main/java/com/metamx/druid/query/search/LexicographicSearchSortSpec.java +++ b/processing/src/main/java/io/druid/query/search/search/LexicographicSearchSortSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java b/processing/src/main/java/io/druid/query/search/search/SearchHit.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/search/SearchHit.java rename to processing/src/main/java/io/druid/query/search/search/SearchHit.java index 1b2acad6f16..d9deba2b639 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchHit.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchHit.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/search/SearchQuery.java rename to processing/src/main/java/io/druid/query/search/search/SearchQuery.java index 4e042772f04..0ca276b53a6 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuery.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java @@ -17,19 +17,19 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; -import com.metamx.druid.BaseQuery; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.BaseQuery; import io.druid.query.Query; +import io.druid.query.Result; +import io.druid.query.filter.DimFilter; +import io.druid.query.search.SearchResultValue; import io.druid.query.spec.QuerySegmentSpec; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java b/processing/src/main/java/io/druid/query/search/search/SearchQueryConfig.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java rename to processing/src/main/java/io/druid/query/search/search/SearchQueryConfig.java index f9b06988262..611fecc2c11 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryConfig.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQueryConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java b/processing/src/main/java/io/druid/query/search/search/SearchQuerySpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java rename to processing/src/main/java/io/druid/query/search/search/SearchQuerySpec.java index a54aae35ea8..1a2b67cd4b5 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQuerySpec.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuerySpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java b/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java rename to processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java index 7510cb582e1..e2e721a4acf 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchSortSpec.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchSortSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/query/search/StrlenSearchSortSpec.java b/processing/src/main/java/io/druid/query/search/search/StrlenSearchSortSpec.java similarity index 93% rename from client/src/main/java/com/metamx/druid/query/search/StrlenSearchSortSpec.java rename to processing/src/main/java/io/druid/query/search/search/StrlenSearchSortSpec.java index e269c241f2d..517463ab9fe 100644 --- a/client/src/main/java/com/metamx/druid/query/search/StrlenSearchSortSpec.java +++ b/processing/src/main/java/io/druid/query/search/search/StrlenSearchSortSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search.search; import java.util.Comparator; diff --git a/client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java b/processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java similarity index 100% rename from client/src/main/java/io/druid/query/spec/LegacySegmentSpec.java rename to processing/src/main/java/io/druid/query/spec/LegacySegmentSpec.java diff --git a/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java similarity index 97% rename from client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java rename to processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java index 5044679f9cf..c8b1884b691 100644 --- a/client/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/MultipleIntervalSegmentSpec.java @@ -21,10 +21,10 @@ package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.utils.JodaUtils; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import org.joda.time.Interval; import java.util.Collections; diff --git a/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java similarity index 97% rename from client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java rename to processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java index 993f2c01d9e..dbd536340b5 100644 --- a/client/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/MultipleSpecificSegmentSpec.java @@ -23,10 +23,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.druid.utils.JodaUtils; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java similarity index 96% rename from client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java rename to processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java index 2bf8c5d7425..2a5014d76f2 100644 --- a/client/src/main/java/io/druid/query/spec/QuerySegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpec.java @@ -21,9 +21,9 @@ package io.druid.query.spec; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import org.joda.time.Interval; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java similarity index 84% rename from client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java rename to processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java index 7c61493f056..bfb3fe77a3c 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/QuerySegmentSpecs.java +++ b/processing/src/main/java/io/druid/query/spec/QuerySegmentSpecs.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; -import io.druid.query.spec.LegacySegmentSpec; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java similarity index 97% rename from client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java rename to processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java index d074f45a948..e9bcd73139d 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentQueryRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import com.google.common.base.Throwables; import com.metamx.common.guava.Accumulator; @@ -26,7 +26,6 @@ import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import io.druid.query.Query; import io.druid.query.QueryRunner; -import io.druid.query.spec.QuerySegmentSpec; import java.io.IOException; import java.util.concurrent.Callable; diff --git a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java similarity index 94% rename from client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java rename to processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java index 33465daba52..be48d55e651 100644 --- a/client/src/main/java/com/metamx/druid/query/segment/SpecificSegmentSpec.java +++ b/processing/src/main/java/io/druid/query/spec/SpecificSegmentSpec.java @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.segment; +package io.druid.query.spec; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.SegmentDescriptor; -import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import java.util.Arrays; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java similarity index 96% rename from client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java rename to processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index e32e4617d3d..957390acf15 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -17,17 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.timeboundary; +package io.druid.query.timeboundary; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.BaseQuery; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.query.BaseQuery; import io.druid.query.Query; +import io.druid.query.Result; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.DateTime; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java similarity index 95% rename from client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java rename to processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index e3e4924c418..a84f238e02d 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.timeboundary; +package io.druid.query.timeboundary; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; @@ -29,17 +29,16 @@ import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.druid.LogicalSegment; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.BySegmentSkippingQueryRunner; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeBoundaryResultValue; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.query.BySegmentSkippingQueryRunner; import io.druid.query.CacheStrategy; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulationFn; -import io.druid.segment.LogicalSegment; import org.joda.time.DateTime; import javax.annotation.Nullable; diff --git a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java similarity index 94% rename from client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java rename to processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java index 455119dcd28..1210e95af2c 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeBoundaryResultValue.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query.timeboundary; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; import com.metamx.common.IAE; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; - import org.joda.time.DateTime; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java similarity index 96% rename from client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java rename to processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java index 94377f543d0..f8530ce334f 100644 --- a/client/src/main/java/com/metamx/druid/TimeseriesBinaryFn.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesBinaryFn.java @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query.timeseries; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.AllGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java similarity index 94% rename from client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java rename to processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java index 2b962a9e7a0..ab5b649a896 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQuery.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQuery.java @@ -17,21 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.timeseries; +package io.druid.query.timeseries; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.ImmutableList; -import com.metamx.druid.BaseQuery; -import com.metamx.druid.query.Queries; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.BaseQuery; +import io.druid.query.Queries; import io.druid.query.Query; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.filter.DimFilter; import io.druid.query.spec.QuerySegmentSpec; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java similarity index 94% rename from client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java rename to processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 1dbf46a8d60..48c2cef25bf 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.timeseries; +package io.druid.query.timeseries; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; @@ -28,24 +28,22 @@ import com.google.common.collect.Ordering; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.ResultGranularTimestampComparator; -import com.metamx.druid.TimeseriesBinaryFn; import com.metamx.druid.collect.OrderedMergeSequence; -import com.metamx.druid.query.IntervalChunkingQueryRunner; -import com.metamx.druid.query.QueryCacheHelper; -import com.metamx.druid.query.ResultMergeQueryRunner; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.granularity.QueryGranularity; import io.druid.query.CacheStrategy; +import io.druid.query.IntervalChunkingQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryCacheHelper; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.ResultGranularTimestampComparator; +import io.druid.query.ResultMergeQueryRunner; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.filter.DimFilter; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; @@ -64,12 +62,11 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest> TYPE_REFERENCE = new TypeReference>() - { - }; - private static final TypeReference OBJECT_TYPE_REFERENCE = new TypeReference() - { - }; + private static final TypeReference OBJECT_TYPE_REFERENCE = + new TypeReference(){}; + + private static final TypeReference> TYPE_REFERENCE = + new TypeReference>() {}; @Override public QueryRunner> mergeResults(QueryRunner> queryRunner) diff --git a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesResultValue.java similarity index 89% rename from client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java rename to processing/src/main/java/io/druid/query/timeseries/TimeseriesResultValue.java index add4bfe6b62..511494c04de 100644 --- a/client/src/main/java/com/metamx/druid/result/TimeseriesResultValue.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesResultValue.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.result; +package io.druid.query.timeseries; import com.fasterxml.jackson.annotation.JsonCreator; +import io.druid.query.MetricValueExtractor; import java.util.Map; diff --git a/segment/src/main/java/io/druid/segment/Capabilities.java b/processing/src/main/java/io/druid/segment/Capabilities.java similarity index 98% rename from segment/src/main/java/io/druid/segment/Capabilities.java rename to processing/src/main/java/io/druid/segment/Capabilities.java index 574f07f4efe..ca2764d1c9a 100644 --- a/segment/src/main/java/io/druid/segment/Capabilities.java +++ b/processing/src/main/java/io/druid/segment/Capabilities.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;/** +package io.druid.segment; + +/** */ public class Capabilities { diff --git a/segment/src/main/java/io/druid/segment/ColumnSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelector.java similarity index 100% rename from segment/src/main/java/io/druid/segment/ColumnSelector.java rename to processing/src/main/java/io/druid/segment/ColumnSelector.java diff --git a/server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java similarity index 96% rename from server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java rename to processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 6489655a59a..fc124621cb8 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -17,17 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.io.Closeables; import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.druid.kv.IndexedIterable; -import io.druid.data.Indexed; import io.druid.query.filter.BitmapIndexSelector; -import io.druid.segment.ColumnSelector; import io.druid.segment.column.Column; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedIterable; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.Iterator; diff --git a/segment/src/main/java/io/druid/segment/ComplexMetricSelector.java b/processing/src/main/java/io/druid/segment/ComplexMetricSelector.java similarity index 96% rename from segment/src/main/java/io/druid/segment/ComplexMetricSelector.java rename to processing/src/main/java/io/druid/segment/ComplexMetricSelector.java index 67a733c00ae..0403562c1c9 100644 --- a/segment/src/main/java/io/druid/segment/ComplexMetricSelector.java +++ b/processing/src/main/java/io/druid/segment/ComplexMetricSelector.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;/** +package io.druid.segment; + +/** */ public interface ComplexMetricSelector { diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java b/processing/src/main/java/io/druid/segment/CompressedPools.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java rename to processing/src/main/java/io/druid/segment/CompressedPools.java index a3061b67030..658a09d263a 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java +++ b/processing/src/main/java/io/druid/segment/CompressedPools.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.base.Supplier; import com.metamx.common.logger.Logger; diff --git a/server/src/main/java/com/metamx/druid/index/v1/ConciseOffset.java b/processing/src/main/java/io/druid/segment/ConciseOffset.java similarity index 93% rename from server/src/main/java/com/metamx/druid/index/v1/ConciseOffset.java rename to processing/src/main/java/io/druid/segment/ConciseOffset.java index b08449e687c..2fd9120f22a 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/ConciseOffset.java +++ b/processing/src/main/java/io/druid/segment/ConciseOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; -import com.metamx.druid.index.v1.processing.Offset; +import io.druid.segment.data.Offset; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.IntSet; diff --git a/segment/src/main/java/io/druid/segment/Cursor.java b/processing/src/main/java/io/druid/segment/Cursor.java similarity index 100% rename from segment/src/main/java/io/druid/segment/Cursor.java rename to processing/src/main/java/io/druid/segment/Cursor.java diff --git a/segment/src/main/java/io/druid/segment/CursorFactory.java b/processing/src/main/java/io/druid/segment/CursorFactory.java similarity index 100% rename from segment/src/main/java/io/druid/segment/CursorFactory.java rename to processing/src/main/java/io/druid/segment/CursorFactory.java diff --git a/segment/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java similarity index 96% rename from segment/src/main/java/io/druid/segment/DimensionSelector.java rename to processing/src/main/java/io/druid/segment/DimensionSelector.java index 8011af47eaf..3922d345049 100644 --- a/segment/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;import io.druid.data.IndexedInts; +package io.druid.segment;import io.druid.segment.data.IndexedInts; /** */ diff --git a/segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java b/processing/src/main/java/io/druid/segment/DimensionSelectorFactory.java similarity index 100% rename from segment/src/main/java/io/druid/segment/DimensionSelectorFactory.java rename to processing/src/main/java/io/druid/segment/DimensionSelectorFactory.java diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java rename to processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index 20ec5a4d30d..4caa6b95fe2 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.io.Files; -import com.metamx.druid.kv.IOPeon; +import io.druid.segment.data.CompressedFloatsSupplierSerializer; +import io.druid.segment.data.IOPeon; import java.io.File; import java.io.IOException; diff --git a/segment/src/main/java/io/druid/segment/FloatMetricSelector.java b/processing/src/main/java/io/druid/segment/FloatMetricSelector.java similarity index 97% rename from segment/src/main/java/io/druid/segment/FloatMetricSelector.java rename to processing/src/main/java/io/druid/segment/FloatMetricSelector.java index 300ab25b0dd..f71d918a226 100644 --- a/segment/src/main/java/io/druid/segment/FloatMetricSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricSelector.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;/** +package io.druid.segment; + +/** * An object that gets a metric value. Metric values are always floats and there is an assumption that the * FloatMetricSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls * to get() (though, that doesn't have to be the case if you always want the same value...). diff --git a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java similarity index 83% rename from server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java rename to processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java index 2675d864f60..9e04e2af836 100644 --- a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/IncrementalIndexSegment.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; -import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; -import io.druid.segment.QueryableIndex; -import io.druid.segment.Segment; -import io.druid.segment.StorageAdapter; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java b/processing/src/main/java/io/druid/segment/IndexGranularity.java similarity index 99% rename from server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java rename to processing/src/main/java/io/druid/segment/IndexGranularity.java index 2900d479611..29b3564b4fc 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexGranularity.java +++ b/processing/src/main/java/io/druid/segment/IndexGranularity.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.fasterxml.jackson.annotation.JsonCreator; import io.druid.granularity.QueryGranularity; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java similarity index 88% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java rename to processing/src/main/java/io/druid/segment/IndexIO.java index b2977cfbf65..38d6367a360 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; @@ -39,32 +39,32 @@ import com.metamx.common.io.smoosh.Smoosh; import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.SimpleQueryableIndex; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.column.ColumnDescriptor; -import com.metamx.druid.index.serde.BitmapIndexColumnPartSupplier; -import com.metamx.druid.index.serde.ComplexColumnPartSerde; -import com.metamx.druid.index.serde.ComplexColumnPartSupplier; -import com.metamx.druid.index.serde.DictionaryEncodedColumnPartSerde; -import com.metamx.druid.index.serde.DictionaryEncodedColumnSupplier; -import com.metamx.druid.index.serde.FloatGenericColumnPartSerde; -import com.metamx.druid.index.serde.FloatGenericColumnSupplier; -import com.metamx.druid.index.serde.LongGenericColumnPartSerde; -import com.metamx.druid.index.serde.LongGenericColumnSupplier; -import com.metamx.druid.index.serde.SpatialIndexColumnPartSupplier; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.kv.ArrayIndexed; -import com.metamx.druid.kv.ByteBufferSerializer; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedIterable; -import com.metamx.druid.kv.IndexedRTree; -import com.metamx.druid.kv.VSizeIndexed; -import com.metamx.druid.kv.VSizeIndexedInts; import com.metamx.druid.utils.SerializerUtils; -import io.druid.segment.QueryableIndex; +import com.metamx.emitter.EmittingLogger; import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; +import io.druid.segment.data.ArrayIndexed; +import io.druid.segment.data.ByteBufferSerializer; +import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.VSizeIndexed; +import io.druid.segment.data.VSizeIndexedInts; +import io.druid.segment.serde.BitmapIndexColumnPartSupplier; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; +import io.druid.segment.serde.DictionaryEncodedColumnSupplier; +import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.serde.FloatGenericColumnSupplier; +import io.druid.segment.serde.LongGenericColumnPartSerde; +import io.druid.segment.serde.LongGenericColumnSupplier; +import io.druid.segment.serde.SpatialIndexColumnPartSupplier; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; @@ -105,7 +105,7 @@ public class IndexIO .put(9, new V9IndexLoader()) .build(); - private static final Logger log = new Logger(IndexIO.class); + private static final EmittingLogger log = new EmittingLogger(IndexIO.class); private static final SerializerUtils serializerUtils = new SerializerUtils(); public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); @@ -115,18 +115,6 @@ public class IndexIO private static volatile IndexIOHandler handler = null; public static final int CURRENT_VERSION_ID = V9_VERSION; - public static Index readIndex(File inDir) throws IOException - { - init(); - return handler.readIndex(inDir); - } - - public static boolean canBeMapped(final File inDir) throws IOException - { - init(); - return handler.canBeMapped(inDir); - } - @Deprecated public static MMappedIndex mapDir(final File inDir) throws IOException { @@ -148,11 +136,6 @@ public class IndexIO } } - public static void storeLatest(Index index, File file) throws IOException - { - handler.storeLatest(index, file); - } - public static boolean hasHandler() { return (IndexIO.handler != null); @@ -210,16 +193,10 @@ public class IndexIO case 1: case 2: case 3: - final String mappableDirName = "mappable"; - if (toConvert.getName().equals(mappableDirName)) { - throw new ISE("Infinite recursion at play! OMFG quit it, please, it hurts!"); - } - - File mappable = new File(toConvert, mappableDirName); - final Index index = readIndex(toConvert); - storeLatest(index, mappable); - - return convertSegment(mappable, converted); + log.makeAlert("Attempt to load segment of version <= 3.") + .addData("version", version) + .emit(); + return false; case 4: case 5: case 6: @@ -241,51 +218,13 @@ public class IndexIO public static interface IndexIOHandler { - /** - * This only exists for some legacy compatibility reasons, Metamarkets is working on getting rid of it in - * future versions - * - * @param inDir - * - * @return - */ - public Index readIndex(File inDir) throws IOException; - - /** - * This should really always return true, but it exists for legacy compatibility reasons, Metamarkets - * is working on getting rid of it in future versions - * - * @return - */ - public boolean canBeMapped(File inDir) throws IOException; - public MMappedIndex mapDir(File inDir) throws IOException; - - /** - * This only exists for some legacy compatibility reasons, Metamarkets is working on getting rid of it in - * future versions. Normal persisting of indexes is done via IndexMerger. - * - * @param file - */ - public void storeLatest(Index index, File file) throws IOException; } public static class DefaultIndexIOHandler implements IndexIOHandler { private static final Logger log = new Logger(DefaultIndexIOHandler.class); - @Override - public Index readIndex(File inDir) - { - throw new UnsupportedOperationException("Shouldn't ever happen in a cluster that is not owned by MMX."); - } - - @Override - public boolean canBeMapped(File inDir) - { - return true; - } - @Override public MMappedIndex mapDir(File inDir) throws IOException { @@ -384,12 +323,6 @@ public class IndexIO return retVal; } - @Override - public void storeLatest(Index index, File file) - { - throw new UnsupportedOperationException("Shouldn't ever happen in a cluster that is not owned by MMX."); - } - public static void convertV8toV9(File v8Dir, File v9Dir) throws IOException { log.info("Converting v8[%s] to v9[%s]", v8Dir, v9Dir); diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java rename to processing/src/main/java/io/druid/segment/IndexMerger.java index b03fbbbbcaf..4f63cf33ee5 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.base.Function; import com.google.common.base.Objects; @@ -44,26 +44,29 @@ import com.metamx.common.guava.nary.BinaryFn; import com.metamx.common.io.smoosh.Smoosh; import com.metamx.common.logger.Logger; import com.metamx.druid.CombiningIterable; -import com.metamx.druid.aggregation.ToLowerCaseAggregatorFactory; import com.metamx.druid.guava.FileOutputSupplier; import com.metamx.druid.guava.GuavaUtils; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.ByteBufferWriter; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IOPeon; -import com.metamx.druid.kv.IndexedIterable; -import com.metamx.druid.kv.IndexedRTree; -import com.metamx.druid.kv.TmpFileIOPeon; -import com.metamx.druid.kv.VSizeIndexedWriter; import com.metamx.druid.utils.JodaUtils; import com.metamx.druid.utils.SerializerUtils; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.QueryableIndex; +import io.druid.query.aggregation.aggregation.ToLowerCaseAggregatorFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.CompressedLongsSupplierSerializer; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.TmpFileIOPeon; +import io.druid.segment.data.VSizeIndexedWriter; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; +import io.druid.segment.serde.ComplexMetricColumnSerializer; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.apache.commons.io.FileUtils; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java b/processing/src/main/java/io/druid/segment/IndexableAdapter.java similarity index 88% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java rename to processing/src/main/java/io/druid/segment/IndexableAdapter.java index 2b0f51b1325..a9c6f07f7a9 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/IndexableAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; /** diff --git a/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java b/processing/src/main/java/io/druid/segment/IndexedIntsOffset.java similarity index 89% rename from server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java rename to processing/src/main/java/io/druid/segment/IndexedIntsOffset.java index e55c581567b..1a4b9e8868d 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IndexedIntsOffset.java +++ b/processing/src/main/java/io/druid/segment/IndexedIntsOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; -import com.metamx.druid.index.v1.processing.Offset; -import io.druid.data.IndexedInts; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.Offset; /** */ diff --git a/processing/src/main/java/io/druid/segment/MMappedIndex.java b/processing/src/main/java/io/druid/segment/MMappedIndex.java new file mode 100644 index 00000000000..5f422ac1a84 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/MMappedIndex.java @@ -0,0 +1,178 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import com.google.common.collect.Maps; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.common.io.smoosh.SmooshedFileMapper; +import com.metamx.common.logger.Logger; +import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.IndexedLongs; +import io.druid.segment.data.VSizeIndexed; +import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.Map; + +/** + */ +public class MMappedIndex +{ + private static final Logger log = new Logger(MMappedIndex.class); + private static final ImmutableConciseSet emptySet = new ImmutableConciseSet(); + + final GenericIndexed availableDimensions; + final GenericIndexed availableMetrics; + final Interval dataInterval; + final CompressedLongsIndexedSupplier timestamps; + final Map metrics; + final Map> dimValueLookups; + final Map dimColumns; + final Map> invertedIndexes; + final Map spatialIndexes; + final SmooshedFileMapper fileMapper; + + private final Map metricIndexes = Maps.newHashMap(); + + public MMappedIndex( + GenericIndexed availableDimensions, + GenericIndexed availableMetrics, + Interval dataInterval, + CompressedLongsIndexedSupplier timestamps, + Map metrics, + Map> dimValueLookups, + Map dimColumns, + Map> invertedIndexes, + Map spatialIndexes, + SmooshedFileMapper fileMapper + ) + { + this.availableDimensions = availableDimensions; + this.availableMetrics = availableMetrics; + this.dataInterval = dataInterval; + this.timestamps = timestamps; + this.metrics = metrics; + this.dimValueLookups = dimValueLookups; + this.dimColumns = dimColumns; + this.invertedIndexes = invertedIndexes; + this.spatialIndexes = spatialIndexes; + this.fileMapper = fileMapper; + + for (int i = 0; i < availableMetrics.size(); i++) { + metricIndexes.put(availableMetrics.get(i), i); + } + } + + public CompressedLongsIndexedSupplier getTimestamps() + { + return timestamps; + } + + public GenericIndexed getAvailableDimensions() + { + return availableDimensions; + } + + public GenericIndexed getAvailableMetrics() + { + return availableMetrics; + } + + public Map getMetrics() + { + return metrics; + } + + public Integer getMetricIndex(String metricName) + { + return metricIndexes.get(metricName); + } + + public Interval getDataInterval() + { + return dataInterval; + } + + public IndexedLongs getReadOnlyTimestamps() + { + return timestamps.get(); + } + + public MetricHolder getMetricHolder(String metric) + { + final MetricHolder retVal = metrics.get(metric); + + if (retVal == null) { + return null; + } + + return retVal; + } + + public GenericIndexed getDimValueLookup(String dimension) + { + return dimValueLookups.get(dimension); + } + + public VSizeIndexed getDimColumn(String dimension) + { + return dimColumns.get(dimension); + } + + public Map> getInvertedIndexes() + { + return invertedIndexes; + } + + public Map getSpatialIndexes() + { + return spatialIndexes; + } + + public ImmutableConciseSet getInvertedIndex(String dimension, String value) + { + final GenericIndexed lookup = dimValueLookups.get(dimension); + if (lookup == null) { + return emptySet; + } + + int indexOf = lookup.indexOf(value); + if (indexOf < 0) { + return emptySet; + } + + ImmutableConciseSet retVal = invertedIndexes.get(dimension).get(indexOf); + return (retVal == null) ? emptySet : retVal; + } + + public SmooshedFileMapper getFileMapper() + { + return fileMapper; + } + + public void close() throws IOException + { + if (fileMapper != null) { + fileMapper.close(); + } + } +} diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java rename to processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java index fffc26fee0f..0db39fb9261 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.collect.Maps; import com.google.common.io.Closeables; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; -import io.druid.data.IndexedInts; -import io.druid.data.IndexedLongs; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedLongs; import org.joda.time.Interval; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/MetricColumnSerializer.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java rename to processing/src/main/java/io/druid/segment/MetricColumnSerializer.java index 2452f820b25..e55b8e38bc6 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/MetricColumnSerializer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java rename to processing/src/main/java/io/druid/segment/MetricHolder.java index 5196af561ce..11a25cc0c7e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; @@ -25,14 +25,16 @@ import com.google.common.io.InputSupplier; import com.google.common.io.OutputSupplier; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.ObjectStrategy; import com.metamx.druid.utils.SerializerUtils; -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; +import io.druid.segment.data.CompressedFloatsSupplierSerializer; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; import java.io.IOException; import java.io.InputStream; diff --git a/segment/src/main/java/io/druid/segment/MetricSelectorFactory.java b/processing/src/main/java/io/druid/segment/MetricSelectorFactory.java similarity index 97% rename from segment/src/main/java/io/druid/segment/MetricSelectorFactory.java rename to processing/src/main/java/io/druid/segment/MetricSelectorFactory.java index d8f743eabd9..68472e134ab 100644 --- a/segment/src/main/java/io/druid/segment/MetricSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/MetricSelectorFactory.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;/** +package io.druid.segment; + +/** * Factory class for MetricSelectors */ public interface MetricSelectorFactory diff --git a/segment/src/main/java/io/druid/segment/ObjectMetricSelector.java b/processing/src/main/java/io/druid/segment/ObjectMetricSelector.java similarity index 92% rename from segment/src/main/java/io/druid/segment/ObjectMetricSelector.java rename to processing/src/main/java/io/druid/segment/ObjectMetricSelector.java index 63941227d19..52b93f086ea 100644 --- a/segment/src/main/java/io/druid/segment/ObjectMetricSelector.java +++ b/processing/src/main/java/io/druid/segment/ObjectMetricSelector.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;public interface ObjectMetricSelector +package io.druid.segment; + +public interface ObjectMetricSelector { public Class classOfObject(); public T get(); diff --git a/segment/src/main/java/io/druid/segment/QueryableIndex.java b/processing/src/main/java/io/druid/segment/QueryableIndex.java similarity index 95% rename from segment/src/main/java/io/druid/segment/QueryableIndex.java rename to processing/src/main/java/io/druid/segment/QueryableIndex.java index d82960fa570..7dc1dcbe5d4 100644 --- a/segment/src/main/java/io/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndex.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;import io.druid.data.Indexed; +package io.druid.segment;import io.druid.segment.data.Indexed; import org.joda.time.Interval; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java rename to processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 2a346774315..efe09b51f55 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -25,20 +25,19 @@ import com.google.common.collect.Sets; import com.google.common.io.Closeables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.kv.ArrayBasedIndexedInts; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.EmptyIndexedInts; -import com.metamx.druid.kv.IndexedIterable; -import com.metamx.druid.kv.ListIndexed; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; -import io.druid.segment.QueryableIndex; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.ValueType; +import io.druid.segment.data.ArrayBasedIndexedInts; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.ListIndexed; import org.joda.time.Interval; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java similarity index 86% rename from server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java rename to processing/src/main/java/io/druid/segment/QueryableIndexSegment.java index 1f3945c2965..c2a7ee18aa6 100644 --- a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexSegment.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; -import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; -import io.druid.segment.QueryableIndex; -import io.druid.segment.Segment; -import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java similarity index 98% rename from server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java rename to processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 817ac8a9dec..cb06b7ab49f 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.base.Function; import com.google.common.base.Functions; @@ -27,27 +27,18 @@ import com.google.common.io.Closeables; import com.metamx.common.collect.MoreIterators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterator; -import com.metamx.druid.index.v1.processing.Offset; -import com.metamx.druid.kv.SingleIndexedInts; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; import io.druid.query.filter.Filter; -import io.druid.segment.Capabilities; -import io.druid.segment.ColumnSelector; -import io.druid.segment.ComplexMetricSelector; -import io.druid.segment.Cursor; -import io.druid.segment.DimensionSelector; -import io.druid.segment.FloatMetricSelector; -import io.druid.segment.ObjectMetricSelector; -import io.druid.segment.QueryableIndex; -import io.druid.segment.StorageAdapter; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.ValueType; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.Offset; +import io.druid.segment.data.SingleIndexedInts; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java rename to processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java index ed4e732182b..3ed246e4edf 100644 --- a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java +++ b/processing/src/main/java/io/druid/segment/ReferenceCountingSegment.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; import com.metamx.emitter.EmittingLogger; -import io.druid.segment.QueryableIndex; -import io.druid.segment.Segment; -import io.druid.segment.StorageAdapter; import org.joda.time.Interval; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java rename to processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java index ca7066ececb..2f51e4ddc10 100644 --- a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java +++ b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; import com.metamx.common.guava.ResourceClosingYielder; import com.metamx.common.guava.Sequence; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java rename to processing/src/main/java/io/druid/segment/Rowboat.java index b4615a4ef93..f6367ffcd61 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java +++ b/processing/src/main/java/io/druid/segment/Rowboat.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.collect.Maps; import com.google.common.collect.Sets; diff --git a/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java similarity index 93% rename from server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java rename to processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java index 3fa346f341b..7cfa6c9c217 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/RowboatFilteringIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/RowboatFilteringIndexAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; import org.joda.time.Interval; /** diff --git a/segment/src/main/java/io/druid/segment/Segment.java b/processing/src/main/java/io/druid/segment/Segment.java similarity index 94% rename from segment/src/main/java/io/druid/segment/Segment.java rename to processing/src/main/java/io/druid/segment/Segment.java index d72fac930a9..78c34541ebc 100644 --- a/segment/src/main/java/io/druid/segment/Segment.java +++ b/processing/src/main/java/io/druid/segment/Segment.java @@ -17,7 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;import org.joda.time.Interval; +package io.druid.segment; + +import org.joda.time.Interval; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java rename to processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java index 98ef7afd409..2e3bc85958f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java +++ b/processing/src/main/java/io/druid/segment/SimpleQueryableIndex.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index; +package io.druid.segment; import com.metamx.common.io.smoosh.SmooshedFileMapper; -import io.druid.data.Indexed; -import io.druid.segment.QueryableIndex; import io.druid.segment.column.Column; +import io.druid.segment.data.Indexed; import org.joda.time.Interval; import java.io.IOException; diff --git a/segment/src/main/java/io/druid/segment/StorageAdapter.java b/processing/src/main/java/io/druid/segment/StorageAdapter.java similarity index 95% rename from segment/src/main/java/io/druid/segment/StorageAdapter.java rename to processing/src/main/java/io/druid/segment/StorageAdapter.java index cfd463b9e3b..ef6ed7c6bd6 100644 --- a/segment/src/main/java/io/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/StorageAdapter.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment;import io.druid.data.Indexed; +package io.druid.segment;import io.druid.segment.data.Indexed; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java b/processing/src/main/java/io/druid/segment/column/AbstractColumn.java similarity index 77% rename from indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java rename to processing/src/main/java/io/druid/segment/column/AbstractColumn.java index 6929bc8c428..b43f5119c3e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java +++ b/processing/src/main/java/io/druid/segment/column/AbstractColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,16 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; - -import io.druid.segment.column.BitmapIndex; -import io.druid.segment.column.Column; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ComplexColumn; -import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.RunLengthColumn; -import io.druid.segment.column.SpatialIndex; +package io.druid.segment.column; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/BitmapIndex.java b/processing/src/main/java/io/druid/segment/column/BitmapIndex.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/BitmapIndex.java rename to processing/src/main/java/io/druid/segment/column/BitmapIndex.java diff --git a/segment/src/main/java/io/druid/segment/column/Column.java b/processing/src/main/java/io/druid/segment/column/Column.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/Column.java rename to processing/src/main/java/io/druid/segment/column/Column.java diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java b/processing/src/main/java/io/druid/segment/column/ColumnBuilder.java similarity index 87% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java rename to processing/src/main/java/io/druid/segment/column/ColumnBuilder.java index 3dc3e865cbc..ba8c329b3ac 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnBuilder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,18 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import io.druid.segment.column.BitmapIndex; -import io.druid.segment.column.Column; -import io.druid.segment.column.ComplexColumn; -import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.RunLengthColumn; -import io.druid.segment.column.SpatialIndex; -import io.druid.segment.column.ValueType; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/ColumnCapabilities.java rename to processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java rename to processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java index fce82246ce2..f7b09d698c1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java rename to processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java index 8324a930488..e352d1d161e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,16 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; -import com.metamx.druid.index.serde.ColumnPartSerde; -import io.druid.segment.column.Column; -import io.druid.segment.column.ValueType; +import io.druid.segment.serde.ColumnPartSerde; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/segment/src/main/java/io/druid/segment/column/ComplexColumn.java b/processing/src/main/java/io/druid/segment/column/ComplexColumn.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/ComplexColumn.java rename to processing/src/main/java/io/druid/segment/column/ComplexColumn.java diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java b/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java similarity index 84% rename from indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java rename to processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java index caccdd20787..98f80288151 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import io.druid.data.Indexed; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ComplexColumn; -import io.druid.segment.column.ValueType; +import io.druid.segment.data.Indexed; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java similarity index 96% rename from segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java rename to processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java index 6bf4c22a106..4925df0f62d 100644 --- a/segment/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -19,7 +19,7 @@ package io.druid.segment.column; -import io.druid.data.IndexedInts; +import io.druid.segment.data.IndexedInts; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java b/processing/src/main/java/io/druid/segment/column/FloatColumn.java similarity index 82% rename from indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java rename to processing/src/main/java/io/druid/segment/column/FloatColumn.java index fa2f759eea8..ee72160274f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/GenericColumn.java b/processing/src/main/java/io/druid/segment/column/GenericColumn.java similarity index 91% rename from segment/src/main/java/io/druid/segment/column/GenericColumn.java rename to processing/src/main/java/io/druid/segment/column/GenericColumn.java index f66fbd3c45b..8afa8624599 100644 --- a/segment/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -19,9 +19,9 @@ package io.druid.segment.column; -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; -import io.druid.data.IndexedLongs; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedLongs; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java similarity index 89% rename from indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java rename to processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java index 241ca019a48..a48f2f2659c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedComplexColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import io.druid.data.Indexed; -import io.druid.segment.column.ComplexColumn; +import io.druid.segment.data.Indexed; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java similarity index 88% rename from indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java rename to processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java index 20e13d5b652..8aa260e4e53 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; -import io.druid.data.IndexedLongs; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedLongs; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java similarity index 88% rename from indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java rename to processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java index ee9767c98ad..368e7b11ebe 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import io.druid.data.Indexed; -import io.druid.data.IndexedFloats; -import io.druid.data.IndexedLongs; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedLongs; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java b/processing/src/main/java/io/druid/segment/column/LongColumn.java similarity index 82% rename from indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java rename to processing/src/main/java/io/druid/segment/column/LongColumn.java index 6bbfb87cfe4..7923c0c303c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedLongsIndexedSupplier; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/RunLengthColumn.java b/processing/src/main/java/io/druid/segment/column/RunLengthColumn.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/RunLengthColumn.java rename to processing/src/main/java/io/druid/segment/column/RunLengthColumn.java diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java similarity index 86% rename from indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java rename to processing/src/main/java/io/druid/segment/column/SimpleColumn.java index 72a6dce582d..597069a3b93 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,18 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; import com.google.common.base.Supplier; import com.google.common.io.Closeables; -import io.druid.segment.column.BitmapIndex; -import io.druid.segment.column.Column; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ComplexColumn; -import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.RunLengthColumn; -import io.druid.segment.column.SpatialIndex; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java similarity index 86% rename from indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java rename to processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 71133d618c4..5e3bb3d0b9f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.column; +package io.druid.segment.column; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.VSizeIndexed; -import com.metamx.druid.kv.VSizeIndexedInts; -import io.druid.data.IndexedInts; -import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.VSizeIndexed; +import io.druid.segment.data.VSizeIndexedInts; /** */ diff --git a/segment/src/main/java/io/druid/segment/column/SpatialIndex.java b/processing/src/main/java/io/druid/segment/column/SpatialIndex.java similarity index 99% rename from segment/src/main/java/io/druid/segment/column/SpatialIndex.java rename to processing/src/main/java/io/druid/segment/column/SpatialIndex.java index 822e878d659..115a4a341c2 100644 --- a/segment/src/main/java/io/druid/segment/column/SpatialIndex.java +++ b/processing/src/main/java/io/druid/segment/column/SpatialIndex.java @@ -16,6 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ + package io.druid.segment.column; import com.metamx.collections.spatial.ImmutableRTree; diff --git a/segment/src/main/java/io/druid/segment/column/ValueType.java b/processing/src/main/java/io/druid/segment/column/ValueType.java similarity index 100% rename from segment/src/main/java/io/druid/segment/column/ValueType.java rename to processing/src/main/java/io/druid/segment/column/ValueType.java diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java rename to processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java index db3a1a7dd37..6616ec859ed 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.IndexedInts; +package io.druid.segment.data; import java.util.Iterator; diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/ArrayBasedOffset.java b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java similarity index 94% rename from server/src/main/java/com/metamx/druid/index/v1/processing/ArrayBasedOffset.java rename to processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java index 42ac85ed869..d1f1fd42829 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/ArrayBasedOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayBasedOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java b/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java rename to processing/src/main/java/io/druid/segment/data/ArrayIndexed.java index d008e53a2a3..eda5a70e824 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/ArrayIndexed.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.Indexed; +package io.druid.segment.data; import java.util.Arrays; import java.util.Iterator; diff --git a/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java b/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java new file mode 100644 index 00000000000..0145c339f1d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.data; + +import com.google.common.primitives.Ints; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + */ +public class ByteBufferSerializer +{ + public static T read(ByteBuffer buffer, ObjectStrategy strategy) + { + int size = buffer.getInt(); + ByteBuffer bufferToUse = buffer.asReadOnlyBuffer(); + bufferToUse.limit(bufferToUse.position() + size); + buffer.position(bufferToUse.limit()); + + return strategy.fromByteBuffer(bufferToUse, size); + } + + public static void writeToChannel(T obj, ObjectStrategy strategy, WritableByteChannel channel) + throws IOException + { + byte[] toWrite = strategy.toBytes(obj); + channel.write(ByteBuffer.allocate(Ints.BYTES).putInt(0, toWrite.length)); + channel.write(ByteBuffer.wrap(toWrite)); + } +} diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java similarity index 74% rename from indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java rename to processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java index 0b384b3a852..91897402f80 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java @@ -1,4 +1,23 @@ -package com.metamx.druid.kv; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.data; import com.google.common.base.Function; import com.google.common.base.Preconditions; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java rename to processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java index 6aa2355dc15..7a9906cd364 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.collect.Ordering; import com.google.common.primitives.Floats; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java rename to processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java index 5f561620222..b21418e4048 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; @@ -27,8 +27,6 @@ import com.google.common.primitives.Ints; import com.metamx.common.IAE; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexed; -import io.druid.data.IndexedFloats; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java rename to processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java index 441ea2d0519..16b816e1df4 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; @@ -25,8 +25,6 @@ import com.google.common.io.OutputSupplier; import com.google.common.primitives.Ints; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IOPeon; import java.io.IOException; import java.io.OutputStream; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java rename to processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java index 0fed227d5a3..823c86a0516 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.collect.Ordering; import com.google.common.primitives.Longs; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java rename to processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index 6ec7fdc2ac1..153d10d29f9 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; @@ -27,8 +27,6 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexed; -import io.druid.data.IndexedLongs; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java rename to processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java index 3358e209db0..742332a7413 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; @@ -26,8 +26,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IOPeon; import java.io.IOException; import java.io.OutputStream; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java rename to processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index 7733ec7fb2f..fa69034fb43 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.metamx.druid.collect.ResourceHolder; -import com.metamx.druid.kv.ObjectStrategy; import com.ning.compress.lzf.ChunkEncoder; import com.ning.compress.lzf.LZFChunk; import com.ning.compress.lzf.LZFDecoder; +import io.druid.segment.CompressedPools; import java.io.IOException; import java.nio.Buffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java rename to processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java index 02b78854a5d..e81e9389a32 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.collect.Ordering; -import io.druid.data.IndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import it.uniroma3.mat.extendedset.intset.IntSet; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java b/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java rename to processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java index dcdf68dbdd0..fa0b09cc8e5 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/EmptyIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.collect.ImmutableList; -import io.druid.data.IndexedInts; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java similarity index 98% rename from indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java rename to processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 52e087d741e..1ae8ec8ea01 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.base.Charsets; import com.google.common.collect.Ordering; import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.metamx.common.IAE; -import io.druid.data.Indexed; import java.io.ByteArrayOutputStream; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java similarity index 98% rename from indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java rename to processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index dd970ee403e..bd6258b2a52 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.base.Preconditions; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IOPeon.java b/processing/src/main/java/io/druid/segment/data/IOPeon.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/kv/IOPeon.java rename to processing/src/main/java/io/druid/segment/data/IOPeon.java index c57be9d1bb9..baac06ca3f8 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IOPeon.java +++ b/processing/src/main/java/io/druid/segment/data/IOPeon.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import java.io.IOException; import java.io.InputStream; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java rename to processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java index 4557c5bda6d..15420d7c900 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -25,8 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Closeables; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexed; -import io.druid.data.IndexedFloats; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java rename to processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java index 811de5118df..de62d7b2de0 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -25,8 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Closeables; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidResourceHolder; -import com.metamx.druid.kv.GenericIndexed; -import io.druid.data.IndexedLongs; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/segment/src/main/java/io/druid/data/Indexed.java b/processing/src/main/java/io/druid/segment/data/Indexed.java similarity index 96% rename from segment/src/main/java/io/druid/data/Indexed.java rename to processing/src/main/java/io/druid/segment/data/Indexed.java index 447fcf4a407..f8def0481b7 100644 --- a/segment/src/main/java/io/druid/data/Indexed.java +++ b/processing/src/main/java/io/druid/segment/data/Indexed.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.data; +package io.druid.segment.data; /** * @param diff --git a/segment/src/main/java/io/druid/data/IndexedFloats.java b/processing/src/main/java/io/druid/segment/data/IndexedFloats.java similarity index 97% rename from segment/src/main/java/io/druid/data/IndexedFloats.java rename to processing/src/main/java/io/druid/segment/data/IndexedFloats.java index 78a1a02016e..cc01524f86f 100644 --- a/segment/src/main/java/io/druid/data/IndexedFloats.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedFloats.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.data; +package io.druid.segment.data; import java.io.Closeable; diff --git a/segment/src/main/java/io/druid/data/IndexedInts.java b/processing/src/main/java/io/druid/segment/data/IndexedInts.java similarity index 96% rename from segment/src/main/java/io/druid/data/IndexedInts.java rename to processing/src/main/java/io/druid/segment/data/IndexedInts.java index 85bb8aa8b18..de0f213fb4b 100644 --- a/segment/src/main/java/io/druid/data/IndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedInts.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.data; +package io.druid.segment.data; /** * Get a int an index (array or list lookup abstraction without boxing). diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java b/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java rename to processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java index 4b72d0dc7f2..6b5a40b2818 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedIntsIterator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; -import io.druid.data.IndexedInts; import java.util.Iterator; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java b/processing/src/main/java/io/druid/segment/data/IndexedIterable.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java rename to processing/src/main/java/io/druid/segment/data/IndexedIterable.java index 07b349f528c..f8d331e0deb 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedIterable.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.Indexed; +package io.druid.segment.data; import java.util.Iterator; import java.util.NoSuchElementException; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java b/processing/src/main/java/io/druid/segment/data/IndexedList.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java rename to processing/src/main/java/io/druid/segment/data/IndexedList.java index 544d64f7d68..8839231231a 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedList.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.Indexed; +package io.druid.segment.data; import java.util.AbstractList; diff --git a/segment/src/main/java/io/druid/data/IndexedLongs.java b/processing/src/main/java/io/druid/segment/data/IndexedLongs.java similarity index 97% rename from segment/src/main/java/io/druid/data/IndexedLongs.java rename to processing/src/main/java/io/druid/segment/data/IndexedLongs.java index d42d2d166af..eabd9be5a82 100644 --- a/segment/src/main/java/io/druid/data/IndexedLongs.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedLongs.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.data; +package io.druid.segment.data; import java.io.Closeable; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java similarity index 68% rename from indexing-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java rename to processing/src/main/java/io/druid/segment/data/IndexedRTree.java index 8c8e3c46d50..b5cb07388fd 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java @@ -1,4 +1,23 @@ -package com.metamx.druid.kv; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.data; import com.google.common.collect.Ordering; import com.metamx.collections.spatial.ImmutableRTree; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java b/processing/src/main/java/io/druid/segment/data/Indexedids.java similarity index 90% rename from indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java rename to processing/src/main/java/io/druid/segment/data/Indexedids.java index 4a87a0eb42c..9ced3b72c4e 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java +++ b/processing/src/main/java/io/druid/segment/data/Indexedids.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.IndexedInts; +package io.druid.segment.data; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java b/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java similarity index 96% rename from indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java rename to processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java index d59b642b56d..ebe7b01ae80 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.metamx.druid.IntList; -import io.druid.data.IndexedInts; import java.nio.ByteBuffer; import java.nio.IntBuffer; diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/IntersectingOffset.java b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/v1/processing/IntersectingOffset.java rename to processing/src/main/java/io/druid/segment/data/IntersectingOffset.java index 25cacff2093..0f5a7f7b885 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/IntersectingOffset.java +++ b/processing/src/main/java/io/druid/segment/data/IntersectingOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java b/processing/src/main/java/io/druid/segment/data/ListIndexed.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java rename to processing/src/main/java/io/druid/segment/data/ListIndexed.java index c5b52f54b0d..b58a9ed516c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/ListIndexed.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; - -import io.druid.data.Indexed; +package io.druid.segment.data; import java.util.Iterator; import java.util.List; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java rename to processing/src/main/java/io/druid/segment/data/ObjectStrategy.java index 57c7a9426a7..7e2753523ba 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import java.nio.ByteBuffer; import java.util.Comparator; diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/Offset.java b/processing/src/main/java/io/druid/segment/data/Offset.java similarity index 92% rename from server/src/main/java/com/metamx/druid/index/v1/processing/Offset.java rename to processing/src/main/java/io/druid/segment/data/Offset.java index 17d0a2ebace..db178f04f30 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/Offset.java +++ b/processing/src/main/java/io/druid/segment/data/Offset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** * The "mutable" version of a ReadableOffset. Introduces "increment()" and "withinBounds()" methods, which are diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/ReadableOffset.java b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java similarity index 92% rename from server/src/main/java/com/metamx/druid/index/v1/processing/ReadableOffset.java rename to processing/src/main/java/io/druid/segment/data/ReadableOffset.java index aea02d16fb6..f55a10c4342 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/ReadableOffset.java +++ b/processing/src/main/java/io/druid/segment/data/ReadableOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** * A ReadableOffset is an object that provides an integer offset, ostensibly as an index into an array. diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java b/processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java similarity index 57% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java rename to processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java index f2975600d23..796f7ef655d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java +++ b/processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,36 +17,37 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; + +import com.google.common.collect.Iterators; + +import java.util.Iterator; /** - */ -public class DimensionColumn +*/ +public class SingleIndexedInts implements IndexedInts { - final int[][] dimensionLookups; - final int[] dimensionValues; + private final int value; - public DimensionColumn( - int[][] dimensionLookups, - int[] dimensionValues - ) - { - this.dimensionLookups = dimensionLookups; - this.dimensionValues = dimensionValues; + public SingleIndexedInts(int value) { + this.value = value; } - public int[] getDimValues(int rowId) + @Override + public int size() { - return dimensionLookups[dimensionValues[rowId]]; + return 1; } - public int[][] getDimensionExpansions() + @Override + public int get(int index) { - return dimensionLookups; + return value; } - public int[] getDimensionRowValues() + @Override + public Iterator iterator() { - return dimensionValues; + return Iterators.singletonIterator(value); } } diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/StartLimitedOffset.java b/processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java similarity index 94% rename from server/src/main/java/com/metamx/druid/index/v1/processing/StartLimitedOffset.java rename to processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java index 0bb84122334..dd4d34585c5 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/StartLimitedOffset.java +++ b/processing/src/main/java/io/druid/segment/data/StartLimitedOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java b/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java rename to processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java index 9af7e0b0337..a60ed66838c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java +++ b/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.collect.Maps; diff --git a/server/src/main/java/com/metamx/druid/index/v1/processing/UnioningOffset.java b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java similarity index 97% rename from server/src/main/java/com/metamx/druid/index/v1/processing/UnioningOffset.java rename to processing/src/main/java/io/druid/segment/data/UnioningOffset.java index 263aa665ff9..a12256519c6 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/processing/UnioningOffset.java +++ b/processing/src/main/java/io/druid/segment/data/UnioningOffset.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.processing; +package io.druid.segment.data; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java rename to processing/src/main/java/io/druid/segment/data/VSizeIndexed.java index 8a9f3620549..8c1f481b2d1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,18 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.primitives.Ints; import com.metamx.common.IAE; import com.metamx.common.ISE; -import io.druid.data.Indexed; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.Arrays; import java.util.Iterator; /** @@ -37,11 +35,6 @@ public class VSizeIndexed implements Indexed { private static final byte version = 0x1; - public static VSizeIndexed fromArray(VSizeIndexedInts[] objects) - { - return fromIterable(Arrays.asList(objects)); - } - public static VSizeIndexed fromIterable(Iterable objectsIterable) { Iterator objects = objectsIterable.iterator(); diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java rename to processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index 02d7e2a5d75..a2c106ff309 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.primitives.Ints; import com.metamx.common.IAE; -import io.druid.data.IndexedInts; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java similarity index 98% rename from indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java rename to processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java index c50ea15ad53..87fcc127155 100644 --- a/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.base.Preconditions; diff --git a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java b/processing/src/main/java/io/druid/segment/filter/AndFilter.java similarity index 96% rename from server/src/main/java/com/metamx/druid/index/brita/AndFilter.java rename to processing/src/main/java/io/druid/segment/filter/AndFilter.java index d5acb100f05..70b374459db 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/AndFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/AndFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.collect.Lists; import io.druid.query.filter.BitmapIndexSelector; diff --git a/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java b/processing/src/main/java/io/druid/segment/filter/BooleanValueMatcher.java similarity index 92% rename from server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java rename to processing/src/main/java/io/druid/segment/filter/BooleanValueMatcher.java index fe99271d0a6..c60a72befff 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/BooleanValueMatcher.java +++ b/processing/src/main/java/io/druid/segment/filter/BooleanValueMatcher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import io.druid.query.filter.ValueMatcher; diff --git a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java rename to processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java index 87e40e2a715..b3f06b949a3 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/DimensionPredicateFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/DimensionPredicateFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.metamx.common.guava.FunctionalIterable; -import io.druid.data.Indexed; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; +import io.druid.segment.data.Indexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java b/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java similarity index 94% rename from server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java rename to processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java index 85711900dbe..62a92ca193a 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/ExtractionFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/ExtractionFilter.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.collect.Lists; -import com.metamx.druid.query.extraction.DimExtractionFn; -import io.druid.data.Indexed; +import io.druid.query.extraction.DimExtractionFn; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; +import io.druid.segment.data.Indexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java similarity index 84% rename from server/src/main/java/com/metamx/druid/index/brita/Filters.java rename to processing/src/main/java/io/druid/segment/filter/Filters.java index 1c3f0785301..21c4722d1fd 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.metamx.druid.query.filter.AndDimFilter; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.filter.ExtractionDimFilter; -import com.metamx.druid.query.filter.JavaScriptDimFilter; -import com.metamx.druid.query.filter.NotDimFilter; -import com.metamx.druid.query.filter.OrDimFilter; -import com.metamx.druid.query.filter.RegexDimFilter; -import com.metamx.druid.query.filter.SearchQueryDimFilter; -import com.metamx.druid.query.filter.SelectorDimFilter; -import com.metamx.druid.query.filter.SpatialDimFilter; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.ExtractionDimFilter; import io.druid.query.filter.Filter; +import io.druid.query.filter.JavaScriptDimFilter; +import io.druid.query.filter.NotDimFilter; +import io.druid.query.filter.OrDimFilter; +import io.druid.query.filter.RegexDimFilter; +import io.druid.query.filter.SearchQueryDimFilter; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.query.filter.SpatialDimFilter; import javax.annotation.Nullable; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java similarity index 81% rename from server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java rename to processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java index 1288b35bfa9..743a07421cb 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/JavaScriptFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java @@ -1,4 +1,23 @@ -package com.metamx.druid.index.brita; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.filter; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; diff --git a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java b/processing/src/main/java/io/druid/segment/filter/NotFilter.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/brita/NotFilter.java rename to processing/src/main/java/io/druid/segment/filter/NotFilter.java index 39f5f9d5f4e..d517613f978 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/NotFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/NotFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; diff --git a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java b/processing/src/main/java/io/druid/segment/filter/OrFilter.java similarity index 96% rename from server/src/main/java/com/metamx/druid/index/brita/OrFilter.java rename to processing/src/main/java/io/druid/segment/filter/OrFilter.java index 31a861258b4..8476066a933 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/OrFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/OrFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.collect.Lists; import io.druid.query.filter.BitmapIndexSelector; diff --git a/server/src/main/java/com/metamx/druid/index/brita/RegexFilter.java b/processing/src/main/java/io/druid/segment/filter/RegexFilter.java similarity index 91% rename from server/src/main/java/com/metamx/druid/index/brita/RegexFilter.java rename to processing/src/main/java/io/druid/segment/filter/RegexFilter.java index b8880f4289e..05b5e70fad6 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/RegexFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/RegexFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.google.common.base.Predicate; -import javax.annotation.Nullable; import java.util.regex.Pattern; /** diff --git a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java b/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java similarity index 91% rename from server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java rename to processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java index a94a518dc61..0f80683187e 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SearchQueryFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SearchQueryFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Predicate; -import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.search.search.SearchQuerySpec; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java b/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java similarity index 94% rename from server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java rename to processing/src/main/java/io/druid/segment/filter/SelectorFilter.java index 13b0cdd6c07..f57d040a1c0 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SelectorFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SelectorFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.Filter; diff --git a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java similarity index 95% rename from server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java rename to processing/src/main/java/io/druid/segment/filter/SpatialFilter.java index bfe74c0e329..538b133be57 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/SpatialFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.brita; +package io.druid.segment.filter; import com.metamx.collections.spatial.search.Bound; import io.druid.query.filter.BitmapIndexSelector; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java similarity index 98% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java rename to processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index f910eab0483..ffe02cd8716 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -33,9 +33,6 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.v1.serde.ComplexMetricExtractor; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; @@ -47,6 +44,9 @@ import io.druid.segment.ComplexMetricSelector; import io.druid.segment.FloatMetricSelector; import io.druid.segment.MetricSelectorFactory; import io.druid.segment.ObjectMetricSelector; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java similarity index 95% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java rename to processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index 9968484f386..4ebea50e24d 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,18 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.collect.Maps; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import com.metamx.druid.kv.EmptyIndexedInts; -import com.metamx.druid.kv.IndexedIterable; -import com.metamx.druid.kv.ListIndexed; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; import io.druid.query.aggregation.Aggregator; +import io.druid.segment.IndexableAdapter; +import io.druid.segment.Rowboat; +import io.druid.segment.data.EmptyIndexedInts; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.ListIndexed; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.IntSet; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java similarity index 77% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java rename to processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java index 54d94ac74a2..5bd0250c90f 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java @@ -1,4 +1,23 @@ -package com.metamx.druid.index.v1; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.collect.Iterables; diff --git a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java similarity index 98% rename from server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java rename to processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 80a734fe8ea..8e2935a7d4b 100644 --- a/server/src/main/java/com/metamx/druid/index/v1/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.base.Predicate; @@ -27,12 +27,6 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.Bound; import com.metamx.common.guava.FunctionalIterator; -import com.metamx.druid.index.brita.BooleanValueMatcher; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.ListIndexed; -import io.druid.data.Indexed; -import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.Aggregator; import io.druid.query.filter.Filter; @@ -45,6 +39,12 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatMetricSelector; import io.druid.segment.ObjectMetricSelector; import io.druid.segment.StorageAdapter; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.ListIndexed; +import io.druid.segment.filter.BooleanValueMatcher; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java similarity index 98% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java rename to processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java index 22c0c5c7edf..0eca207f8fe 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.base.Joiner; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java similarity index 51% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java rename to processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java index 311626e5bc0..6ef1e788c6c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java @@ -1,4 +1,23 @@ -package com.metamx.druid.index.v1; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.incremental; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java rename to processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java index f5ada9c15fb..d36e6a684e1 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; -import com.metamx.druid.kv.GenericIndexed; import io.druid.segment.column.BitmapIndex; +import io.druid.segment.data.GenericIndexed; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; /** diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java rename to processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java index 556fab5ea4d..c22685ddc65 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.index.column.ColumnBuilder; +import io.druid.segment.column.ColumnBuilder; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java similarity index 87% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java rename to processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index 03e13cfe371..080978e77ab 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; -import com.metamx.druid.kv.GenericIndexed; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSupplier.java similarity index 87% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java rename to processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSupplier.java index 092a3820a7b..7251e557ab4 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.IndexedComplexColumn; -import com.metamx.druid.kv.GenericIndexed; import io.druid.segment.column.ComplexColumn; +import io.druid.segment.column.IndexedComplexColumn; +import io.druid.segment.data.GenericIndexed; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java similarity index 87% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java rename to processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java index 55032f54da0..dd5e5939bea 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricColumnSerializer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.serde; import com.google.common.io.Files; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.kv.GenericIndexedWriter; -import com.metamx.druid.kv.IOPeon; +import io.druid.segment.IndexIO; +import io.druid.segment.MetricColumnSerializer; +import io.druid.segment.MetricHolder; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; import java.io.File; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricExtractor.java similarity index 91% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java rename to processing/src/main/java/io/druid/segment/serde/ComplexMetricExtractor.java index b2c5545db73..03b54bb9f8c 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricExtractor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.serde; +package io.druid.segment.serde; import com.metamx.druid.input.InputRow; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java rename to processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java index eb39ece5a70..2ac3b309f62 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.serde; +package io.druid.segment.serde; import com.google.common.base.Function; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.serde.ColumnPartSerde; -import com.metamx.druid.kv.ObjectStrategy; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.ObjectStrategy; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetrics.java similarity index 93% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java rename to processing/src/main/java/io/druid/segment/serde/ComplexMetrics.java index 33093f8f825..ae7cee1679a 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetrics.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.serde; +package io.druid.segment.serde; import com.google.common.collect.Maps; import com.metamx.common.ISE; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java rename to processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index b8fa919a35f..391bc62ffdb 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.primitives.Ints; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.kv.ByteBufferSerializer; -import com.metamx.druid.kv.ConciseCompressedIndexedInts; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.IndexedRTree; -import com.metamx.druid.kv.VSizeIndexed; -import com.metamx.druid.kv.VSizeIndexedInts; +import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ValueType; +import io.druid.segment.data.ByteBufferSerializer; +import io.druid.segment.data.ConciseCompressedIndexedInts; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.VSizeIndexed; +import io.druid.segment.data.VSizeIndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.io.IOException; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java similarity index 84% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java rename to processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java index c1a3cc0f077..d2a5ee82a85 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.SimpleDictionaryEncodedColumn; -import com.metamx.druid.kv.GenericIndexed; -import com.metamx.druid.kv.VSizeIndexed; -import com.metamx.druid.kv.VSizeIndexedInts; import io.druid.segment.column.DictionaryEncodedColumn; +import io.druid.segment.column.SimpleDictionaryEncodedColumn; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.VSizeIndexed; +import io.druid.segment.data.VSizeIndexedInts; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java rename to processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index b18a06c21d1..0a0951af793 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; +import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java similarity index 86% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java rename to processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java index 3cfad0184f1..41d7c17fcdb 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.IndexedFloatsGenericColumn; -import com.metamx.druid.index.v1.CompressedFloatsIndexedSupplier; import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedFloatsGenericColumn; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; import java.nio.ByteOrder; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java similarity index 92% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java rename to processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index 32e0703dacc..92e25b17e79 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.index.column.ColumnBuilder; -import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; +import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedLongsIndexedSupplier; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java similarity index 85% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java rename to processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java index 3c1401a18a6..edc8483efb3 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; -import com.metamx.druid.index.column.IndexedLongsGenericColumn; -import com.metamx.druid.index.v1.CompressedLongsIndexedSupplier; import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedLongsGenericColumn; +import io.druid.segment.data.CompressedLongsIndexedSupplier; /** */ diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java b/processing/src/main/java/io/druid/segment/serde/Registererer.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java rename to processing/src/main/java/io/druid/segment/serde/Registererer.java index 8a18b7033cb..6c1fdcceb91 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java +++ b/processing/src/main/java/io/druid/segment/serde/Registererer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1.serde; +package io.druid.segment.serde; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/client/src/main/java/com/metamx/druid/RegisteringNode.java b/processing/src/main/java/io/druid/segment/serde/Registererers.java similarity index 89% rename from client/src/main/java/com/metamx/druid/RegisteringNode.java rename to processing/src/main/java/io/druid/segment/serde/Registererers.java index a0abbf2ecad..e56e42605f2 100644 --- a/client/src/main/java/com/metamx/druid/RegisteringNode.java +++ b/processing/src/main/java/io/druid/segment/serde/Registererers.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.segment.serde; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.index.v1.serde.Registererer; /** */ -public class RegisteringNode +public class Registererers { public static void registerHandlers(Iterable registererers, Iterable mappers) { diff --git a/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java b/processing/src/main/java/io/druid/segment/serde/SpatialIndexColumnPartSupplier.java similarity index 94% rename from indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java rename to processing/src/main/java/io/druid/segment/serde/SpatialIndexColumnPartSupplier.java index ced6fcf65fa..48014bc8bb6 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/SpatialIndexColumnPartSupplier.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -16,7 +16,7 @@ * along with this program; if not, write to the Free Software * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.serde; +package io.druid.segment.serde; import com.google.common.base.Supplier; import com.metamx.collections.spatial.ImmutableRTree; diff --git a/client/src/test/java/com/metamx/druid/CombiningIterableTest.java b/processing/src/test/java/io/druid/collections/CombiningIterableTest.java similarity index 88% rename from client/src/test/java/com/metamx/druid/CombiningIterableTest.java rename to processing/src/test/java/io/druid/collections/CombiningIterableTest.java index 94650467656..da839f1c249 100644 --- a/client/src/test/java/com/metamx/druid/CombiningIterableTest.java +++ b/processing/src/test/java/io/druid/collections/CombiningIterableTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.collections; import com.metamx.common.guava.nary.BinaryFn; -import com.metamx.druid.result.Result; +import com.metamx.druid.CombiningIterable; +import io.druid.query.Result; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -69,7 +70,10 @@ public class CombiningIterableTest return arg1; } - return new Result(arg1.getTimestamp(), ((Long) arg1.getValue()).longValue() + ((Long) arg2.getValue()).longValue()); + return new Result( + arg1.getTimestamp(), + ((Long) arg1.getValue()).longValue() + ((Long) arg2.getValue()).longValue() + ); } } ); diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java b/processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java similarity index 97% rename from indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java rename to processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java index 00c08750f77..264b9bcd4e7 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/InputRowParserSerdeTest.java +++ b/processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.indexer.data; +package io.druid.data.input; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.InputRow; import com.metamx.druid.jackson.DefaultObjectMapper; +import io.druid.segment.incremental.SpatialDimensionSchema; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java similarity index 69% rename from indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java rename to processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index fd402c56b41..92c337a0832 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -1,4 +1,23 @@ -package com.metamx.druid.indexer.data; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.data.input; import com.metamx.druid.input.InputRow; import org.joda.time.DateTime; @@ -9,7 +28,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; -import static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE; import static org.junit.Assert.assertEquals; public class ProtoBufInputRowParserTest { @@ -42,7 +60,7 @@ public class ProtoBufInputRowParserTest { DateTime dateTime = new DateTime(2012, 07, 12, 9, 30); ProtoTestEventWrapper.ProtoTestEvent event = ProtoTestEventWrapper.ProtoTestEvent.newBuilder() .setDescription("description") - .setEventType(CATEGORY_ONE) + .setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE) .setId(4711L) .setIsValid(true) .setSomeOtherId(4712) @@ -64,7 +82,7 @@ public class ProtoBufInputRowParserTest { assertDimensionEquals(row, "isValid", "true"); assertDimensionEquals(row, "someOtherId", "4712"); assertDimensionEquals(row, "description", "description"); - assertDimensionEquals(row, "eventType", CATEGORY_ONE.name()); + assertDimensionEquals(row, "eventType", ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ONE.name()); assertEquals(47.11F, row.getFloatMetric("someFloatColumn"), 0.0); diff --git a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java similarity index 86% rename from indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java rename to processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java index 9c86246398a..965859fecfc 100644 --- a/indexing-common/src/test/java/com/metamx/druid/indexer/data/ProtoTestEventWrapper.java +++ b/processing/src/test/java/io/druid/data/input/ProtoTestEventWrapper.java @@ -1,7 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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. + */ + // Generated by the protocol buffer compiler. DO NOT EDIT! // source: ProtoTest.proto -package com.metamx.druid.indexer.data; +package io.druid.data.input; + +import com.google.protobuf.AbstractMessage; public final class ProtoTestEventWrapper { private ProtoTestEventWrapper() {} @@ -13,7 +34,7 @@ public final class ProtoTestEventWrapper { // required .prototest.ProtoTestEvent.EventCategory eventType = 1; boolean hasEventType(); - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); + ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType(); // required uint64 id = 2; boolean hasId(); @@ -67,12 +88,12 @@ public final class ProtoTestEventWrapper { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; } public enum EventCategory @@ -120,7 +141,7 @@ public final class ProtoTestEventWrapper { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); + return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor().getEnumTypes().get(0); } private static final EventCategory[] VALUES = { @@ -150,11 +171,11 @@ public final class ProtoTestEventWrapper { private int bitField0_; // required .prototest.ProtoTestEvent.EventCategory eventType = 1; public static final int EVENTTYPE_FIELD_NUMBER = 1; - private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; + private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_; public boolean hasEventType() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { + public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { return eventType_; } @@ -283,7 +304,7 @@ public final class ProtoTestEventWrapper { } private void initFields() { - eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; id_ = 0L; timestamp_ = ""; someOtherId_ = 0; @@ -401,41 +422,41 @@ public final class ProtoTestEventWrapper { return super.writeReplace(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) + public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -444,7 +465,7 @@ public final class ProtoTestEventWrapper { return null; } } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -455,12 +476,12 @@ public final class ProtoTestEventWrapper { return null; } } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent parseFrom( + public static ProtoTestEventWrapper.ProtoTestEvent parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -470,7 +491,7 @@ public final class ProtoTestEventWrapper { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent prototype) { + public static Builder newBuilder(ProtoTestEventWrapper.ProtoTestEvent prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -483,18 +504,18 @@ public final class ProtoTestEventWrapper { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEventOrBuilder { + implements ProtoTestEventWrapper.ProtoTestEventOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; + return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; + return ProtoTestEventWrapper.internal_static_prototest_ProtoTestEvent_fieldAccessorTable; } - // Construct using com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() + // Construct using io.druid.data.input.ProtoTestEventWrapper.ProtoTestEvent.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -513,7 +534,7 @@ public final class ProtoTestEventWrapper { public Builder clear() { super.clear(); - eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; bitField0_ = (bitField0_ & ~0x00000001); id_ = 0L; bitField0_ = (bitField0_ & ~0x00000002); @@ -540,33 +561,34 @@ public final class ProtoTestEventWrapper { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDescriptor(); + return ProtoTestEventWrapper.ProtoTestEvent.getDescriptor(); } - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { - return com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); + public ProtoTestEventWrapper.ProtoTestEvent getDefaultInstanceForType() { + return ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance(); } - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent build() { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + public ProtoTestEventWrapper.ProtoTestEvent build() { + ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); if (!result.isInitialized()) { - throw newUninitializedMessageException(result); + throw AbstractMessage.Builder.newUninitializedMessageException(result); } return result; } - private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildParsed() + private ProtoTestEventWrapper.ProtoTestEvent buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); + ProtoTestEventWrapper.ProtoTestEvent result = buildPartial(); if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); + throw AbstractMessage.Builder.newUninitializedMessageException( + result + ).asInvalidProtocolBufferException(); } return result; } - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent buildPartial() { - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent result = new com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent(this); + public ProtoTestEventWrapper.ProtoTestEvent buildPartial() { + ProtoTestEventWrapper.ProtoTestEvent result = new ProtoTestEventWrapper.ProtoTestEvent(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { @@ -611,16 +633,16 @@ public final class ProtoTestEventWrapper { } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent) { - return mergeFrom((com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent)other); + if (other instanceof ProtoTestEventWrapper.ProtoTestEvent) { + return mergeFrom((ProtoTestEventWrapper.ProtoTestEvent)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent other) { - if (other == com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this; + public Builder mergeFrom(ProtoTestEventWrapper.ProtoTestEvent other) { + if (other == ProtoTestEventWrapper.ProtoTestEvent.getDefaultInstance()) return this; if (other.hasEventType()) { setEventType(other.getEventType()); } @@ -693,7 +715,7 @@ public final class ProtoTestEventWrapper { } case 8: { int rawValue = input.readEnum(); - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue); + ProtoTestEventWrapper.ProtoTestEvent.EventCategory value = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { @@ -749,14 +771,14 @@ public final class ProtoTestEventWrapper { private int bitField0_; // required .prototest.ProtoTestEvent.EventCategory eventType = 1; - private com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + private ProtoTestEventWrapper.ProtoTestEvent.EventCategory eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; public boolean hasEventType() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { + public ProtoTestEventWrapper.ProtoTestEvent.EventCategory getEventType() { return eventType_; } - public Builder setEventType(com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) { + public Builder setEventType(ProtoTestEventWrapper.ProtoTestEvent.EventCategory value) { if (value == null) { throw new NullPointerException(); } @@ -767,7 +789,7 @@ public final class ProtoTestEventWrapper { } public Builder clearEventType() { bitField0_ = (bitField0_ & ~0x00000001); - eventType_ = com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; + eventType_ = ProtoTestEventWrapper.ProtoTestEvent.EventCategory.CATEGORY_ZERO; onChanged(); return this; } @@ -1017,8 +1039,8 @@ public final class ProtoTestEventWrapper { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_prototest_ProtoTestEvent_descriptor, new java.lang.String[] { "EventType", "Id", "Timestamp", "SomeOtherId", "IsValid", "Description", "SomeFloatColumn", "SomeIntColumn", "SomeLongColumn", }, - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.class, - com.metamx.druid.indexer.data.ProtoTestEventWrapper.ProtoTestEvent.Builder.class); + ProtoTestEventWrapper.ProtoTestEvent.class, + ProtoTestEventWrapper.ProtoTestEvent.Builder.class); return null; } }; diff --git a/client/src/test/java/com/metamx/druid/query/PrioritizedExecutorServiceTest.java b/processing/src/test/java/io/druid/query/PrioritizedExecutorServiceTest.java similarity index 97% rename from client/src/test/java/com/metamx/druid/query/PrioritizedExecutorServiceTest.java rename to processing/src/test/java/io/druid/query/PrioritizedExecutorServiceTest.java index b6ce2a9ee1c..0209fafa5ec 100644 --- a/client/src/test/java/com/metamx/druid/query/PrioritizedExecutorServiceTest.java +++ b/processing/src/test/java/io/druid/query/PrioritizedExecutorServiceTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; import com.google.common.collect.ImmutableList; import com.metamx.common.concurrent.ExecutorServiceConfig; diff --git a/client/src/test/java/com/metamx/druid/query/QueriesTest.java b/processing/src/test/java/io/druid/query/QueriesTest.java similarity index 94% rename from client/src/test/java/com/metamx/druid/query/QueriesTest.java rename to processing/src/test/java/io/druid/query/QueriesTest.java index d447d6c475a..4b0f195bf9c 100644 --- a/client/src/test/java/com/metamx/druid/query/QueriesTest.java +++ b/processing/src/test/java/io/druid/query/QueriesTest.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; -import com.metamx.druid.aggregation.post.ConstantPostAggregator; -import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.aggregation.post.FieldAccessPostAggregator; import junit.framework.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java b/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java similarity index 96% rename from client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java rename to processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java index ad43fd1a31a..02665b7491d 100644 --- a/client/src/test/java/com/metamx/druid/ResultGranularTimestampComparatorTest.java +++ b/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query; -import com.metamx.druid.result.Result; import io.druid.granularity.QueryGranularity; import org.joda.time.DateTime; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java similarity index 92% rename from client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java rename to processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java index ed6e369a1bf..8c9cb6b093a 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/PartialDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/PartialDimExtractionFnTest.java @@ -17,9 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction.extraction; import com.google.common.collect.Sets; +import io.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.PartialDimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java similarity index 95% rename from client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java rename to processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java index 790f0c5adc1..43f90a1ac01 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/RegexDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/RegexDimExtractionFnTest.java @@ -17,9 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction.extraction; import com.google.common.collect.Sets; +import io.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.RegexDimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/SearchQuerySpecDimExtractionFnTest.java similarity index 87% rename from client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java rename to processing/src/test/java/io/druid/query/extraction/extraction/SearchQuerySpecDimExtractionFnTest.java index ab825199e6b..30104f523a9 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/SearchQuerySpecDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/SearchQuerySpecDimExtractionFnTest.java @@ -17,11 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction.extraction; import com.google.common.collect.Sets; -import com.metamx.druid.query.search.FragmentSearchQuerySpec; -import com.metamx.druid.query.search.SearchQuerySpec; +import io.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.SearchQuerySpecDimExtractionFn; +import io.druid.query.search.search.FragmentSearchQuerySpec; +import io.druid.query.search.search.SearchQuerySpec; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java b/processing/src/test/java/io/druid/query/extraction/extraction/TimeDimExtractionFnTest.java similarity index 93% rename from client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java rename to processing/src/test/java/io/druid/query/extraction/extraction/TimeDimExtractionFnTest.java index 56d3213c633..1ac839a7e4c 100644 --- a/client/src/test/java/com/metamx/druid/query/extraction/TimeDimExtractionFnTest.java +++ b/processing/src/test/java/io/druid/query/extraction/extraction/TimeDimExtractionFnTest.java @@ -17,9 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.extraction; +package io.druid.query.extraction.extraction; import com.google.common.collect.Sets; +import io.druid.query.extraction.DimExtractionFn; +import io.druid.query.extraction.TimeDimExtractionFn; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/group/having/HavingSpecTest.java b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java similarity index 88% rename from client/src/test/java/com/metamx/druid/query/group/having/HavingSpecTest.java rename to processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java index 28c91878e57..f62e9185ff3 100644 --- a/client/src/test/java/com/metamx/druid/query/group/having/HavingSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.having; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; diff --git a/client/src/test/java/com/metamx/druid/query/group/orderby/TopNSorterTest.java b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSorterTest.java similarity index 73% rename from client/src/test/java/com/metamx/druid/query/group/orderby/TopNSorterTest.java rename to processing/src/test/java/io/druid/query/groupby/orderby/TopNSorterTest.java index 872155e5049..7e2eeb051bf 100644 --- a/client/src/test/java/com/metamx/druid/query/group/orderby/TopNSorterTest.java +++ b/processing/src/test/java/io/druid/query/groupby/orderby/TopNSorterTest.java @@ -1,4 +1,23 @@ -package com.metamx.druid.query.group.orderby; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.orderby; import com.google.common.base.Splitter; import com.google.common.collect.Lists; diff --git a/client/src/test/java/com/metamx/druid/query/search/LexicographicSearchSortSpecTest.java b/processing/src/test/java/io/druid/query/search/LexicographicSearchSortSpecTest.java similarity index 84% rename from client/src/test/java/com/metamx/druid/query/search/LexicographicSearchSortSpecTest.java rename to processing/src/test/java/io/druid/query/search/LexicographicSearchSortSpecTest.java index 46c8a7537ff..0ec86b46343 100644 --- a/client/src/test/java/com/metamx/druid/query/search/LexicographicSearchSortSpecTest.java +++ b/processing/src/test/java/io/druid/query/search/LexicographicSearchSortSpecTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,8 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search; +import io.druid.query.search.search.LexicographicSearchSortSpec; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchSortSpec; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java b/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java similarity index 96% rename from client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java rename to processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java index 3f9aaa4cbaf..9f277bb267b 100644 --- a/client/src/test/java/com/metamx/druid/SearchBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,15 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query.search; import com.google.common.collect.ImmutableList; -import com.metamx.druid.query.search.LexicographicSearchSortSpec; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.query.search.StrlenSearchSortSpec; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; +import io.druid.query.search.search.LexicographicSearchSortSpec; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.StrlenSearchSortSpec; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/search/StrlenSearchSortSpecTest.java b/processing/src/test/java/io/druid/query/search/StrlenSearchSortSpecTest.java similarity index 84% rename from client/src/test/java/com/metamx/druid/query/search/StrlenSearchSortSpecTest.java rename to processing/src/test/java/io/druid/query/search/StrlenSearchSortSpecTest.java index 36eaba1330d..dcdf9074b8a 100644 --- a/client/src/test/java/com/metamx/druid/query/search/StrlenSearchSortSpecTest.java +++ b/processing/src/test/java/io/druid/query/search/StrlenSearchSortSpecTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,8 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.search; +package io.druid.query.search; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchSortSpec; +import io.druid.query.search.search.StrlenSearchSortSpec; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java b/processing/src/test/java/io/druid/query/spec/QuerySegmentSpecTest.java similarity index 93% rename from client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java rename to processing/src/test/java/io/druid/query/spec/QuerySegmentSpecTest.java index 7e6c8e06894..277ecd8e45b 100644 --- a/client/src/test/java/com/metamx/druid/query/QuerySegmentSpecTest.java +++ b/processing/src/test/java/io/druid/query/spec/QuerySegmentSpecTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,17 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query; +package io.druid.query.spec; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.metamx.druid.jackson.DefaultObjectMapper; import io.druid.query.SegmentDescriptor; -import io.druid.query.spec.LegacySegmentSpec; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.spec.MultipleSpecificSegmentSpec; -import io.druid.query.spec.QuerySegmentSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java similarity index 95% rename from client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java rename to processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java index 2a80bab2309..ccf7459fbb8 100644 --- a/client/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.query.timeboundary; +package io.druid.query.timeboundary; -import io.druid.segment.LogicalSegment; +import com.metamx.druid.LogicalSegment; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; diff --git a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java similarity index 93% rename from client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java rename to processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java index 4d4364c7523..cdc797a7e5a 100644 --- a/client/src/test/java/com/metamx/druid/TimeseriesBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java @@ -17,20 +17,19 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid; +package io.druid.query.timeseries; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; -import com.metamx.druid.aggregation.post.ConstantPostAggregator; -import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.aggregation.post.FieldAccessPostAggregator; import junit.framework.Assert; import org.joda.time.DateTime; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java similarity index 98% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java rename to processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java index cf49cb1c61f..d9cf4c5b6ab 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.Closeables; import com.google.common.primitives.Floats; -import io.druid.data.IndexedFloats; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java similarity index 93% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java rename to processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java index f59a8958d35..c19adad7786 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.OutputSupplier; import com.metamx.druid.collect.ResourceHolder; -import com.metamx.druid.kv.GenericIndexedWriter; -import io.druid.data.IndexedFloats; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java similarity index 98% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java rename to processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java index 36e847f78bd..873d3b3d423 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.Closeables; import com.google.common.primitives.Longs; -import io.druid.data.IndexedLongs; import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java similarity index 93% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java rename to processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java index d2988459c4b..23ed9053be4 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,12 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.io.OutputSupplier; import com.metamx.druid.collect.ResourceHolder; -import com.metamx.druid.kv.GenericIndexedWriter; -import io.druid.data.IndexedLongs; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java b/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java similarity index 97% rename from indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java rename to processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java index 9b126feaf65..61d64eb1d9a 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java +++ b/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.collect.Maps; -import io.druid.data.Indexed; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java b/processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java similarity index 94% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java rename to processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java index 6229d222e83..132c13a23b4 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java +++ b/processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.collect.Maps; -import com.metamx.druid.kv.IOPeon; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java similarity index 95% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java rename to processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java index 1b3d64bb384..18c2b8d75d3 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java +++ b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; -import io.druid.data.IndexedFloats; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java similarity index 95% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java rename to processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java index 103253809f1..1c8199ccc29 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java +++ b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; -import io.druid.data.IndexedLongs; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java similarity index 95% rename from indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java rename to processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java index 410c4c6b751..e70d3c2a880 100644 --- a/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.index.v1; +package io.druid.segment.data; import com.google.common.collect.ImmutableMap; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.incremental.IncrementalIndex; import junit.framework.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java b/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java similarity index 94% rename from indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java rename to processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java index a130ea15d3e..629cf72b722 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java +++ b/processing/src/test/java/io/druid/segment/data/IndexedIntsTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; -import io.druid.data.IndexedInts; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java similarity index 96% rename from indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java rename to processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java index d953958116b..543cd956b42 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import org.junit.Assert; import org.junit.Test; diff --git a/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java similarity index 95% rename from indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java rename to processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java index 631ec03869f..4d2f1a95710 100644 --- a/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.kv; +package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -27,7 +27,6 @@ import org.junit.Test; import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.util.Arrays; import java.util.List; diff --git a/indexing-common/src/test/resources/ProtoTest.proto b/processing/src/test/resources/ProtoTest.proto similarity index 100% rename from indexing-common/src/test/resources/ProtoTest.proto rename to processing/src/test/resources/ProtoTest.proto diff --git a/indexing-common/src/test/resources/druid.sample.tsv b/processing/src/test/resources/druid.sample.tsv similarity index 100% rename from indexing-common/src/test/resources/druid.sample.tsv rename to processing/src/test/resources/druid.sample.tsv diff --git a/indexing-common/src/test/resources/druid.sample.tsv.bottom b/processing/src/test/resources/druid.sample.tsv.bottom similarity index 100% rename from indexing-common/src/test/resources/druid.sample.tsv.bottom rename to processing/src/test/resources/druid.sample.tsv.bottom diff --git a/indexing-common/src/test/resources/druid.sample.tsv.top b/processing/src/test/resources/druid.sample.tsv.top similarity index 100% rename from indexing-common/src/test/resources/druid.sample.tsv.top rename to processing/src/test/resources/druid.sample.tsv.top diff --git a/indexing-common/src/test/resources/prototest.desc b/processing/src/test/resources/prototest.desc similarity index 100% rename from indexing-common/src/test/resources/prototest.desc rename to processing/src/test/resources/prototest.desc diff --git a/realtime/pom.xml b/realtime/pom.xml index d97e2da4d9c..950ae5bf83c 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -1,7 +1,7 @@ - + 4.0.0 com.metamx.druid druid-server @@ -34,12 +35,7 @@ com.metamx.druid - druid-client - ${project.parent.version} - - - com.metamx.druid - druid-segment + druid-processing ${project.parent.version} @@ -199,7 +195,30 @@ astyanax 1.0.1 - + + org.antlr + antlr4-runtime + + + com.google.code.simple-spring-memcached + spymemcached + + + net.jpountz.lz4 + lz4 + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + org.eclipse.jetty + jetty-servlets + @@ -214,14 +233,7 @@ com.metamx.druid - druid-indexing-common - ${project.parent.version} - test-jar - test - - - com.metamx.druid - druid-client + druid-processing ${project.parent.version} test-jar test @@ -231,6 +243,11 @@ curator-test test + + com.google.caliper + caliper + test + @@ -247,6 +264,17 @@ + + org.antlr + antlr4-maven-plugin + + + + antlr4 + + + + diff --git a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 b/server/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 similarity index 92% rename from client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 rename to server/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 index c8c6d0effb1..316bcce2af7 100644 --- a/client/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 +++ b/server/src/main/antlr4/com/metamx/druid/sql/antlr4/DruidSQL.g4 @@ -3,25 +3,25 @@ grammar DruidSQL; @header { import com.google.common.base.Joiner; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.MaxAggregatorFactory; -import com.metamx.druid.aggregation.MinAggregatorFactory; -import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; -import com.metamx.druid.aggregation.post.ConstantPostAggregator; -import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import io.druid.query.aggregation.PostAggregator; -import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.query.filter.AndDimFilter; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.filter.NotDimFilter; -import com.metamx.druid.query.filter.OrDimFilter; -import com.metamx.druid.query.filter.RegexDimFilter; -import com.metamx.druid.query.filter.SelectorDimFilter; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.aggregation.post.FieldAccessPostAggregator; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.NotDimFilter; +import io.druid.query.filter.OrDimFilter; +import io.druid.query.filter.RegexDimFilter; +import io.druid.query.filter.SelectorDimFilter; import org.antlr.v4.runtime.NoViableAltException; import org.antlr.v4.runtime.Parser; import org.antlr.v4.runtime.ParserRuleContext; diff --git a/server/src/main/java/com/metamx/druid/SearchResultBuilder.java b/server/src/main/java/com/metamx/druid/SearchResultBuilder.java index 21614d2e036..dfd37885f61 100644 --- a/server/src/main/java/com/metamx/druid/SearchResultBuilder.java +++ b/server/src/main/java/com/metamx/druid/SearchResultBuilder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,9 +20,9 @@ package com.metamx.druid; import com.google.common.collect.Lists; -import com.metamx.druid.query.search.SearchHit; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; +import io.druid.query.Result; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchHit; import org.joda.time.DateTime; /** diff --git a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java index 79440aa1ed7..f67c1d3973c 100644 --- a/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java +++ b/server/src/main/java/com/metamx/druid/TimeseriesResultBuilder.java @@ -19,10 +19,10 @@ package com.metamx.druid; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.Result; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.timeseries.TimeseriesResultValue; import org.joda.time.DateTime; import java.util.HashMap; diff --git a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java index 6f686832cb6..15620c31db3 100644 --- a/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java +++ b/server/src/main/java/com/metamx/druid/client/indexing/IndexingServiceSelector.java @@ -1,11 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.selector.DiscoverySelector; -import com.metamx.druid.client.selector.Server; +import io.druid.client.indexing.IndexingService; +import io.druid.client.selector.DiscoverySelector; +import io.druid.client.selector.Server; import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; diff --git a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeHandler.java b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeHandler.java index bc88027bbb8..640abfba82b 100644 --- a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeHandler.java +++ b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeHandler.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,7 @@ package com.metamx.druid.coordination; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java index 596d49428f6..2649dd0492a 100644 --- a/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/com/metamx/druid/coordination/DataSegmentChangeRequest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,7 +21,6 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.client.DataSegment; /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java index 05d4121c039..b115db25ffa 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestDrop.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonUnwrapped; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java index 64bb64fc3ed..4582689560f 100644 --- a/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/com/metamx/druid/coordination/SegmentChangeRequestLoad.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonUnwrapped; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; /** */ diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 234688c647d..4ea95e757f8 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -27,32 +27,32 @@ import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; -import com.metamx.druid.guice.annotations.Processing; -import com.metamx.druid.index.ReferenceCountingSegment; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; -import com.metamx.druid.query.BySegmentQueryRunner; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.MetricsEmittingQueryRunner; -import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.ReferenceCountingSegmentQueryRunner; -import com.metamx.druid.query.segment.QuerySegmentWalker; -import com.metamx.druid.query.segment.SpecificSegmentQueryRunner; -import com.metamx.druid.query.segment.SpecificSegmentSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.DataSegment; +import io.druid.guice.guice.annotations.Processing; +import io.druid.query.BySegmentQueryRunner; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.MetricsEmittingQueryRunner; +import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.SegmentDescriptor; import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.spec.SpecificSegmentQueryRunner; +import io.druid.query.spec.SpecificSegmentSpec; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index fef63f11547..941a7e72aed 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,11 +26,13 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index 334d629e2ad..03242062563 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -32,10 +32,10 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.guice.annotations.Json; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.guice.guice.annotations.Json; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index 06ef0dbccf3..5eeeaf320a6 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -32,11 +32,11 @@ import com.metamx.common.lifecycle.LifecycleStop; 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.DruidDataSource; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.druid.partition.PartitionChunk; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.guice.guice.ManageLifecycle; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java b/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java deleted file mode 100644 index f3279d2af90..00000000000 --- a/server/src/main/java/com/metamx/druid/guice/annotations/Processing.java +++ /dev/null @@ -1,17 +0,0 @@ -package com.metamx.druid.guice.annotations; - -import com.google.inject.BindingAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - */ -@BindingAnnotation -@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) -@Retention(RetentionPolicy.RUNTIME) -public @interface Processing -{ -} diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java index 85483eaa505..e2360155fdc 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentKiller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,7 @@ package com.metamx.druid.loading; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; /** */ diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java index 306d7f449af..4325561e943 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,7 @@ package com.metamx.druid.loading; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java index 5369480d6b9..4d877477a4f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,7 @@ package com.metamx.druid.loading; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.io.File; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java index c71832d9f63..643d2762a01 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.google.common.base.Joiner; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import org.joda.time.format.ISODateTimeFormat; /** diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index cced8f28add..91ed6ec0a3f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.metamx.common.MapUtils; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import io.druid.segment.Segment; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java index 0e25a2de792..f79cc2a6213 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPuller.java @@ -1,9 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.common.io.Closeables; import com.google.inject.Inject; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.utils.CompressionUtils; +import io.druid.client.DataSegment; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java index e68a93f5282..e97e1327755 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,9 +26,9 @@ import com.google.common.io.Closeables; import com.google.common.io.OutputSupplier; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.utils.CompressionUtils; +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java index 8cdb8e0a7a2..8378d932b1a 100644 --- a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,8 +22,8 @@ package com.metamx.druid.loading; import com.google.common.io.Files; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.utils.CompressionUtils; +import io.druid.client.DataSegment; import java.io.File; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java index ed4ab5472c9..0d191c48f17 100644 --- a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,9 +25,9 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.utils.CompressionUtils; +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; import java.io.File; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 8b530ea061f..3480a96d47a 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.v1.IndexIO; +import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; import org.apache.commons.io.FileUtils; @@ -37,25 +37,6 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { - if (! IndexIO.canBeMapped(parentDir)) { - File canBeMappedDir = new File(parentDir, "forTheMapping"); - if (canBeMappedDir.exists()) { - FileUtils.deleteDirectory(canBeMappedDir); - } - canBeMappedDir.mkdirs(); - - IndexIO.storeLatest(IndexIO.readIndex(parentDir), canBeMappedDir); - if (! IndexIO.canBeMapped(canBeMappedDir)) { - throw new SegmentLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); - } - for (File file : canBeMappedDir.listFiles()) { - if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new SegmentLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); - } - } - FileUtils.deleteDirectory(canBeMappedDir); - } - return IndexIO.loadIndex(parentDir); } catch (IOException e) { diff --git a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java index 421676a108d..fe44c8895f0 100644 --- a/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/OmniSegmentLoader.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.common.base.Preconditions; @@ -9,9 +28,9 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndexSegment; +import io.druid.client.DataSegment; import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import org.apache.commons.io.FileUtils; diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java index 63b815a7e85..01c3c50b9d4 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentKiller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ 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 io.druid.client.DataSegment; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java index c6a82ffa3ff..eb3d5200112 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -27,9 +27,9 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; import com.metamx.druid.utils.CompressionUtils; +import io.druid.client.DataSegment; +import io.druid.storage.s3.S3Utils; import org.apache.commons.io.FileUtils; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; 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 a5e76bf7283..78d1ad2b515 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,11 +26,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.inject.Inject; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.utils.CompressionUtils; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; +import io.druid.storage.s3.S3Utils; import org.jets3t.service.ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 3a303ab9076..0402a964b3c 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,7 @@ package com.metamx.druid.loading; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import io.druid.segment.Segment; /** diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 31be1e8ef57..36955406461 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -27,9 +27,9 @@ import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.QueryableIndexSegment; +import io.druid.client.DataSegment; import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import org.apache.commons.io.FileUtils; diff --git a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java index a11c9e2ec2f..74bffc1d757 100644 --- a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPuller.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,7 +23,6 @@ import com.google.common.io.Files; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.loading.DataSegmentPuller; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.utils.CompressionUtils; @@ -32,6 +31,7 @@ import com.netflix.astyanax.connectionpool.exceptions.ConnectionException; import com.netflix.astyanax.model.ColumnList; import com.netflix.astyanax.recipes.storage.ChunkedStorage; import com.netflix.astyanax.recipes.storage.ObjectMetadata; +import io.druid.client.DataSegment; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java index 63483075db9..dfe925d97f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/cassandra/CassandraDataSegmentPusher.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,13 +24,13 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DataSegmentPusherUtil; import com.metamx.druid.utils.CompressionUtils; import com.netflix.astyanax.MutationBatch; import com.netflix.astyanax.recipes.storage.ChunkedStorage; +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; import java.io.File; import java.io.FileInputStream; diff --git a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java index 7d62968e4b9..25375ed8ca6 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerSegmentHolder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,8 +19,8 @@ package com.metamx.druid.master; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; /** */ diff --git a/server/src/main/java/com/metamx/druid/master/BalancerStrategy.java b/server/src/main/java/com/metamx/druid/master/BalancerStrategy.java index 84944114b49..715f7f271be 100644 --- a/server/src/main/java/com/metamx/druid/master/BalancerStrategy.java +++ b/server/src/main/java/com/metamx/druid/master/BalancerStrategy.java @@ -1,25 +1,25 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java b/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java index 8c31261d5bb..460fe1b3184 100644 --- a/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java +++ b/server/src/main/java/com/metamx/druid/master/CostBalancerStrategy.java @@ -1,27 +1,27 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.common.Pair; -import com.metamx.druid.client.DataSegment; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/DruidCluster.java b/server/src/main/java/com/metamx/druid/master/DruidCluster.java index d7ef284ede0..e96750f4315 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidCluster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidCluster.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.master; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Ordering; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DruidServer; import java.util.Map; 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 aaf94eb3ed5..8a60ba00760 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -37,21 +37,21 @@ import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.ServerInventoryView; -import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.concurrent.Execs; import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.client.DruidServer; +import io.druid.client.ServerInventoryView; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.segment.IndexIO; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; import org.apache.curator.framework.recipes.leader.LeaderLatchListener; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java index bb547bae8d8..ae4604c8274 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterBalancer.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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; @@ -23,9 +23,9 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.common.guava.Comparators; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; import org.joda.time.DateTime; import java.util.Comparator; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterCleanup.java b/server/src/main/java/com/metamx/druid/master/DruidMasterCleanup.java index 6ba31a77249..72d011afbe8 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterCleanup.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterCleanup.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,9 +25,9 @@ import com.metamx.common.guava.Comparators; 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.DruidDataSource; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.client.DruidServer; import java.util.Map; import java.util.Set; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java index 1b226933a52..5b2a3a5486d 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,12 +22,12 @@ package com.metamx.druid.master; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; import com.metamx.druid.collect.CountingMap; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.client.DruidServer; import java.util.Map; import java.util.Set; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java index 04a3ce55da6..cff13183cfd 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuleRunner.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,10 +19,10 @@ package com.metamx.druid.master; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.master.rules.Rule; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; import org.joda.time.DateTime; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java index 0dfe9afca4f..15a7d605b00 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterRuntimeParams.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,10 +22,10 @@ package com.metamx.druid.master; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.guava.Comparators; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; import org.joda.time.DateTime; import java.util.Collection; diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentInfoLoader.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentInfoLoader.java index 2158328c1d8..835e0979a7b 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentInfoLoader.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentInfoLoader.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,7 @@ package com.metamx.druid.master; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.Set; 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 d178cf1a8ad..90fe682a8c0 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -34,10 +34,10 @@ import com.metamx.common.guava.FunctionalIterable; 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.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java b/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java index 2b5c96cb187..b681f749b2a 100644 --- a/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java +++ b/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,12 +25,12 @@ import com.google.common.collect.Collections2; import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.coordination.DataSegmentChangeRequest; import com.metamx.druid.coordination.SegmentChangeRequestDrop; import com.metamx.druid.coordination.SegmentChangeRequestLoad; import com.metamx.druid.coordination.SegmentChangeRequestNoop; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorWatcher; import org.apache.curator.utils.ZKPaths; diff --git a/server/src/main/java/com/metamx/druid/master/RandomBalancerStrategy.java b/server/src/main/java/com/metamx/druid/master/RandomBalancerStrategy.java index d953b69b3db..b2ab0af8739 100644 --- a/server/src/main/java/com/metamx/druid/master/RandomBalancerStrategy.java +++ b/server/src/main/java/com/metamx/druid/master/RandomBalancerStrategy.java @@ -1,25 +1,25 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.List; import java.util.Random; diff --git a/server/src/main/java/com/metamx/druid/master/ReservoirSegmentSampler.java b/server/src/main/java/com/metamx/druid/master/ReservoirSegmentSampler.java index 4db994b821a..0019c9e9f9d 100644 --- a/server/src/main/java/com/metamx/druid/master/ReservoirSegmentSampler.java +++ b/server/src/main/java/com/metamx/druid/master/ReservoirSegmentSampler.java @@ -1,25 +1,25 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.List; import java.util.Random; diff --git a/server/src/main/java/com/metamx/druid/master/SegmentReplicantLookup.java b/server/src/main/java/com/metamx/druid/master/SegmentReplicantLookup.java index 560787247f5..de8ce5e310f 100644 --- a/server/src/main/java/com/metamx/druid/master/SegmentReplicantLookup.java +++ b/server/src/main/java/com/metamx/druid/master/SegmentReplicantLookup.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,8 +23,8 @@ import com.google.common.collect.HashBasedTable; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Table; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/master/ServerHolder.java b/server/src/main/java/com/metamx/druid/master/ServerHolder.java index 5fba424f429..e362b0fe8e3 100644 --- a/server/src/main/java/com/metamx/druid/master/ServerHolder.java +++ b/server/src/main/java/com/metamx/druid/master/ServerHolder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,8 +20,8 @@ package com.metamx.druid.master; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; /** */ diff --git a/server/src/main/java/com/metamx/druid/master/rules/DropRule.java b/server/src/main/java/com/metamx/druid/master/rules/DropRule.java index 5806031d7c6..218204462e7 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/DropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/DropRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,10 +19,10 @@ package com.metamx.druid.master.rules; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; +import io.druid.client.DataSegment; /** * DropRules indicate when segments should be completely removed from the cluster. diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java index 1363c88b9da..367d9953e51 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalDropRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,8 +21,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.client.DataSegment; - +import io.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java index 0cfe603e2b1..07b7009cf74 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/IntervalLoadRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,8 +22,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; - +import io.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java index b133af1799e..0216e6a638b 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/LoadRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,7 +21,6 @@ package com.metamx.druid.master.rules; import com.google.common.collect.Lists; import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.BalancerStrategy; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; @@ -30,6 +29,7 @@ import com.metamx.druid.master.MasterStats; import com.metamx.druid.master.ReplicationThrottler; import com.metamx.druid.master.ServerHolder; import com.metamx.emitter.EmittingLogger; +import io.druid.client.DataSegment; import org.joda.time.DateTime; import java.util.ArrayList; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java index 5fa6affcf9e..61d926fdbe9 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodDropRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,8 +21,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.client.DataSegment; - +import io.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java index 632b6acbc67..9ffc81fbaf5 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/PeriodLoadRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,8 +22,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; - +import io.druid.client.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/master/rules/Rule.java b/server/src/main/java/com/metamx/druid/master/rules/Rule.java index 82e710cfbfa..55b9adb9be2 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/Rule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/Rule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,10 +21,10 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterRuntimeParams; import com.metamx.druid.master.MasterStats; +import io.druid.client.DataSegment; import org.joda.time.DateTime; /** diff --git a/server/src/main/java/com/metamx/druid/master/rules/SizeDropRule.java b/server/src/main/java/com/metamx/druid/master/rules/SizeDropRule.java index 0bd9f94cd00..3fd474bb395 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/SizeDropRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/SizeDropRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Range; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import org.joda.time.DateTime; /** diff --git a/server/src/main/java/com/metamx/druid/master/rules/SizeLoadRule.java b/server/src/main/java/com/metamx/druid/master/rules/SizeLoadRule.java index 421432ec6b9..e864eca3114 100644 --- a/server/src/main/java/com/metamx/druid/master/rules/SizeLoadRule.java +++ b/server/src/main/java/com/metamx/druid/master/rules/SizeLoadRule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid.master.rules; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Range; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import org.joda.time.DateTime; /** diff --git a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java index bb0063872ac..99f567e46ff 100644 --- a/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java +++ b/server/src/main/java/com/metamx/druid/metrics/MetricsModule.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metrics; import com.google.common.base.Preconditions; @@ -12,13 +31,13 @@ import com.google.inject.Provides; import com.google.inject.name.Names; import com.metamx.common.logger.Logger; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.JsonConfigurator; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.JsonConfigurator; +import io.druid.guice.guice.LazySingleton; +import io.druid.guice.guice.ManageLifecycle; import java.util.List; import java.util.Properties; diff --git a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java index 72e24f40a32..20f2c913860 100644 --- a/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java +++ b/server/src/main/java/com/metamx/druid/metrics/ServerMonitor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,11 +20,11 @@ package com.metamx.druid.metrics; import com.google.inject.Inject; -import com.metamx.druid.client.DruidServerConfig; import com.metamx.druid.coordination.ServerManager; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; import com.metamx.metrics.AbstractMonitor; +import io.druid.client.DruidServerConfig; import java.util.Map; diff --git a/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java b/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java index 513fbd31756..688e9339262 100644 --- a/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java +++ b/server/src/main/java/com/metamx/druid/query/QueryRunnerHelper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,8 +23,8 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; -import com.metamx.druid.result.Result; import io.druid.granularity.QueryGranularity; +import io.druid.query.Result; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.filter.Filter; diff --git a/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java b/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java index 6d6b68acd07..438fbe5f1aa 100644 --- a/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/server/src/main/java/com/metamx/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -1,12 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query; import com.google.common.io.Closeables; import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.index.ReferenceCountingSegment; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.segment.ReferenceCountingSegment; import java.io.Closeable; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java index bf77b1a4ec9..feec009ccdd 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryEngine.java @@ -37,17 +37,19 @@ import com.metamx.common.guava.Sequences; import com.metamx.common.parsers.CloseableIterator; import com.metamx.druid.collect.ResourceHolder; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.guice.annotations.Global; -import com.metamx.druid.index.brita.Filters; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; -import com.metamx.druid.query.dimension.DimensionSpec; +import io.druid.guice.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.StorageAdapter; +import io.druid.segment.filter.Filters; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java index 23de48bb9bc..f525c2b6631 100644 --- a/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/group/GroupByQueryRunnerFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -30,12 +30,15 @@ import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.input.Row; -import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.query.ConcatQueryRunner; +import io.druid.query.ChainedExecutionQueryRunner; +import io.druid.query.ConcatQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java index 8839a985278..e84eebf22c1 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentAnalyzer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,14 +25,15 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import com.google.common.primitives.Longs; import com.metamx.common.logger.Logger; -import com.metamx.druid.index.v1.serde.ComplexMetricSerde; -import com.metamx.druid.index.v1.serde.ComplexMetrics; +import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.segment.QueryableIndex; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.ValueType; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; import java.util.Map; @@ -105,7 +106,7 @@ public class SegmentAnalyzer return ColumnAnalysis.error("multi_value"); } - return new ColumnAnalysis(capabilities.getType(), column.getLength() * numBytes, null, null); + return new ColumnAnalysis(capabilities.getType().name(), column.getLength() * numBytes, null, null); } public ColumnAnalysis analyzeStringColumn(Column column) @@ -125,7 +126,7 @@ public class SegmentAnalyzer } } - return new ColumnAnalysis(capabilities.getType(), size, cardinality, null); + return new ColumnAnalysis(capabilities.getType().name(), size, cardinality, null); } return ColumnAnalysis.error("string_no_bitmap"); @@ -153,6 +154,6 @@ public class SegmentAnalyzer size += inputSizeFn.apply(complexColumn.getRowValue(i)); } - return new ColumnAnalysis(capabilities.getType(), size, null, null); + return new ColumnAnalysis(capabilities.getType().name(), size, null, null); } } diff --git a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index dc6f6b7c89b..5b87c9bf3ee 100644 --- a/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,11 +25,16 @@ import com.google.common.collect.Maps; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.query.ConcatQueryRunner; +import io.druid.query.ConcatQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; +import io.druid.query.metadata.metadata.ColumnAnalysis; +import io.druid.query.metadata.metadata.ColumnIncluderator; +import io.druid.query.metadata.metadata.SegmentAnalysis; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; import io.druid.segment.QueryableIndex; import io.druid.segment.Segment; diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java index 2e25720122f..ae0e17e7628 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunner.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.search; import com.google.common.base.Strings; @@ -9,16 +28,17 @@ import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.index.brita.Filters; -import com.metamx.druid.index.v1.ColumnSelectorBitmapIndexSelector; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; import com.metamx.emitter.EmittingLogger; -import io.druid.data.IndexedInts; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.Result; import io.druid.query.filter.Filter; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQuerySpec; +import io.druid.segment.ColumnSelectorBitmapIndexSelector; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.QueryableIndex; @@ -26,6 +46,8 @@ import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.filter.Filters; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java index 9df93768f2c..c9e43bbf7fd 100644 --- a/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/search/SearchQueryRunnerFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,12 +20,14 @@ package com.metamx.druid.query.search; import com.google.inject.Inject; -import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; +import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchQuery; import io.druid.segment.Segment; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 9742380e4fd..a9a8df88ec1 100644 --- a/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,13 +22,15 @@ package com.metamx.druid.query.timeboundary; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; +import io.druid.query.timeboundary.TimeBoundaryResultValue; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java index 83e4b53f954..e80a687a538 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryEngine.java @@ -23,15 +23,16 @@ import com.google.common.base.Function; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.druid.TimeseriesResultBuilder; -import com.metamx.druid.index.brita.Filters; import com.metamx.druid.query.QueryRunnerHelper; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.Result; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.Cursor; import io.druid.segment.StorageAdapter; +import io.druid.segment.filter.Filters; import java.util.Iterator; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 7ddc4947625..082dba2a2c3 100644 --- a/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/server/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,13 +21,15 @@ package com.metamx.druid.query.timeseries; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; -import com.metamx.druid.query.ChainedExecutionQueryRunner; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; +import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.Result; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; diff --git a/server/src/main/java/com/metamx/druid/utils/DruidSetup.java b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java index ef5185ac703..8d8360b9c67 100644 --- a/server/src/main/java/com/metamx/druid/utils/DruidSetup.java +++ b/server/src/main/java/com/metamx/druid/utils/DruidSetup.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,7 +24,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.Sets; import com.google.common.io.Closeables; import com.metamx.common.config.Config; -import com.metamx.druid.initialization.ZkPathsConfig; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryOneTime; diff --git a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java b/server/src/main/java/io/druid/client/BatchServerInventoryView.java similarity index 94% rename from client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java rename to server/src/main/java/io/druid/client/BatchServerInventoryView.java index f5410536926..77253faebb6 100644 --- a/client/src/main/java/com/metamx/druid/client/BatchServerInventoryView.java +++ b/server/src/main/java/io/druid/client/BatchServerInventoryView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,9 +25,9 @@ import com.google.common.collect.MapMaker; import com.google.common.collect.Sets; import com.google.inject.Inject; import com.metamx.common.ISE; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import java.util.Set; diff --git a/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java new file mode 100644 index 00000000000..1bd0163dca8 --- /dev/null +++ b/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.server.initialization.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class BatchServerInventoryViewProvider implements ServerInventoryViewProvider +{ + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public ServerInventoryView get() + { + return new BatchServerInventoryView(zkPaths, curator, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/BrokerServerView.java rename to server/src/main/java/io/druid/client/BrokerServerView.java index 1e4ed76f23b..cee1168bd33 100644 --- a/client/src/main/java/com/metamx/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; @@ -25,14 +25,14 @@ import com.google.common.collect.Ordering; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.druid.VersionedIntervalTimeline; -import com.metamx.druid.client.selector.QueryableDruidServer; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.guice.annotations.Client; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.druid.query.QueryToolChestWarehouse; import com.metamx.http.client.HttpClient; +import io.druid.client.selector.QueryableDruidServer; +import io.druid.client.selector.ServerSelector; +import io.druid.guice.guice.annotations.Client; import io.druid.query.QueryRunner; +import io.druid.query.QueryToolChestWarehouse; import java.util.Iterator; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java rename to server/src/main/java/io/druid/client/CachingClusteredClient.java index 6c985cf1ab6..a921ccaee02 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -40,18 +40,18 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; -import com.metamx.druid.client.cache.Cache; -import com.metamx.druid.client.selector.QueryableDruidServer; -import com.metamx.druid.client.selector.ServerSelector; import com.metamx.druid.partition.PartitionChunk; -import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.result.BySegmentResultValueClass; -import com.metamx.druid.result.Result; import com.metamx.emitter.EmittingLogger; +import io.druid.client.cache.Cache; +import io.druid.client.selector.QueryableDruidServer; +import io.druid.client.selector.ServerSelector; +import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.Result; import io.druid.query.SegmentDescriptor; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; diff --git a/client/src/main/java/com/metamx/druid/client/DataSegment.java b/server/src/main/java/io/druid/client/DataSegment.java similarity index 98% rename from client/src/main/java/com/metamx/druid/client/DataSegment.java rename to server/src/main/java/io/druid/client/DataSegment.java index 3d048e430e4..f6549d8de58 100644 --- a/client/src/main/java/com/metamx/druid/client/DataSegment.java +++ b/server/src/main/java/io/druid/client/DataSegment.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -31,9 +31,9 @@ import com.google.common.collect.Iterables; import com.metamx.common.Granularity; import com.metamx.druid.jackson.CommaListJoinDeserializer; import com.metamx.druid.jackson.CommaListJoinSerializer; -import com.metamx.druid.shard.NoneShardSpec; -import com.metamx.druid.shard.ShardSpec; import io.druid.query.SegmentDescriptor; +import io.druid.server.shard.shard.NoneShardSpec; +import io.druid.server.shard.shard.ShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/DirectDruidClient.java rename to server/src/main/java/io/druid/client/DirectDruidClient.java index 5f82a8bf5be..80844ccbeda 100644 --- a/client/src/main/java/com/metamx/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; @@ -35,16 +35,16 @@ import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.result.BySegmentResultValueClass; -import com.metamx.druid.result.Result; import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.query.BySegmentResultValueClass; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; +import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import org.jboss.netty.handler.codec.http.HttpChunk; @@ -199,14 +199,16 @@ public class DirectDruidClient implements QueryRunner if (!isBySegment) { retVal = Sequences.map( retVal, - toolChest.makeMetricManipulatorFn(query, new MetricManipulationFn() + toolChest.makeMetricManipulatorFn( + query, new MetricManipulationFn() { @Override public Object manipulate(AggregatorFactory factory, Object object) { return factory.deserialize(object); } - }) + } + ) ); } diff --git a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java b/server/src/main/java/io/druid/client/DruidDataSource.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/DruidDataSource.java rename to server/src/main/java/io/druid/client/DruidDataSource.java index ebe566052f8..330fef388a9 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidDataSource.java +++ b/server/src/main/java/io/druid/client/DruidDataSource.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Maps; diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/server/src/main/java/io/druid/client/DruidServer.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/DruidServer.java rename to server/src/main/java/io/druid/client/DruidServer.java index 937119b3d46..99e98b14b29 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/server/src/main/java/io/druid/client/DruidServer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.metamx.common.logger.Logger; -import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.initialization.DruidNode; +import io.druid.server.DruidNode; +import io.druid.server.coordination.DruidServerMetadata; import java.util.Collections; import java.util.Map; diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/server/src/main/java/io/druid/client/DruidServerConfig.java similarity index 93% rename from client/src/main/java/com/metamx/druid/client/DruidServerConfig.java rename to server/src/main/java/io/druid/client/DruidServerConfig.java index 6a57a38b20f..14cfaee290a 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/server/src/main/java/io/druid/client/DruidServerConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/client/InventoryView.java b/server/src/main/java/io/druid/client/InventoryView.java new file mode 100644 index 00000000000..d955ea21c42 --- /dev/null +++ b/server/src/main/java/io/druid/client/InventoryView.java @@ -0,0 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client; + +/** + */ +public interface InventoryView +{ + public DruidServer getInventoryValue(String string); + public Iterable getInventory(); +} diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java b/server/src/main/java/io/druid/client/ServerInventoryView.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/ServerInventoryView.java rename to server/src/main/java/io/druid/client/ServerInventoryView.java index 32cade52c1d..ebc3c8b8ab0 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryView.java +++ b/server/src/main/java/io/druid/client/ServerInventoryView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; @@ -28,10 +28,10 @@ import com.google.common.collect.MapMaker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.curator.inventory.CuratorInventoryManager; -import com.metamx.druid.curator.inventory.CuratorInventoryManagerStrategy; -import com.metamx.druid.curator.inventory.InventoryManagerConfig; import com.metamx.emitter.EmittingLogger; +import io.druid.curator.inventory.CuratorInventoryManager; +import io.druid.curator.inventory.CuratorInventoryManagerStrategy; +import io.druid.curator.inventory.InventoryManagerConfig; import org.apache.curator.framework.CuratorFramework; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java b/server/src/main/java/io/druid/client/ServerInventoryViewConfig.java similarity index 93% rename from client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java rename to server/src/main/java/io/druid/client/ServerInventoryViewConfig.java index 6130a96a66c..0e96cec5e34 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryViewConfig.java +++ b/server/src/main/java/io/druid/client/ServerInventoryViewConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import org.skife.config.Config; import org.skife.config.Default; diff --git a/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java new file mode 100644 index 00000000000..fa48fba4661 --- /dev/null +++ b/server/src/main/java/io/druid/client/ServerInventoryViewProvider.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = SingleServerInventoryProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "legacy", value = SingleServerInventoryProvider.class), + @JsonSubTypes.Type(name = "batch", value = BatchServerInventoryViewProvider.class) +}) +public interface ServerInventoryViewProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/client/ServerView.java b/server/src/main/java/io/druid/client/ServerView.java similarity index 98% rename from client/src/main/java/com/metamx/druid/client/ServerView.java rename to server/src/main/java/io/druid/client/ServerView.java index 61afd6d1222..843070f1b0f 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerView.java +++ b/server/src/main/java/io/druid/client/ServerView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import java.util.concurrent.Executor; diff --git a/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java b/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java new file mode 100644 index 00000000000..74ec48c9bd6 --- /dev/null +++ b/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java @@ -0,0 +1,50 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.server.initialization.initialization.ZkPathsConfig; +import org.apache.curator.framework.CuratorFramework; + +import javax.validation.constraints.NotNull; + +/** + */ +public class SingleServerInventoryProvider implements ServerInventoryViewProvider +{ + @JacksonInject + @NotNull + private ZkPathsConfig zkPaths = null; + + @JacksonInject + @NotNull + private CuratorFramework curator = null; + + @JacksonInject + @NotNull + private ObjectMapper jsonMapper = null; + + @Override + public ServerInventoryView get() + { + return new SingleServerInventoryView(zkPaths, curator, jsonMapper); + } +} diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/server/src/main/java/io/druid/client/SingleServerInventoryView.java similarity index 91% rename from client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java rename to server/src/main/java/io/druid/client/SingleServerInventoryView.java index 1c04800887f..5f95c52f1c1 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/server/src/main/java/io/druid/client/SingleServerInventoryView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.emitter.EmittingLogger; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; /** diff --git a/client/src/main/java/com/metamx/druid/client/TimelineServerView.java b/server/src/main/java/io/druid/client/TimelineServerView.java similarity index 88% rename from client/src/main/java/com/metamx/druid/client/TimelineServerView.java rename to server/src/main/java/io/druid/client/TimelineServerView.java index adbf863a6b5..29b3164a4f2 100644 --- a/client/src/main/java/com/metamx/druid/client/TimelineServerView.java +++ b/server/src/main/java/io/druid/client/TimelineServerView.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.metamx.druid.VersionedIntervalTimeline; -import com.metamx.druid.client.selector.ServerSelector; +import io.druid.client.selector.ServerSelector; import io.druid.query.QueryRunner; /** diff --git a/client/src/main/java/com/metamx/druid/client/cache/ByteCountingLRUMap.java b/server/src/main/java/io/druid/client/cache/ByteCountingLRUMap.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/ByteCountingLRUMap.java rename to server/src/main/java/io/druid/client/cache/ByteCountingLRUMap.java index 59a60f060e5..b351dfc6837 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/ByteCountingLRUMap.java +++ b/server/src/main/java/io/druid/client/cache/ByteCountingLRUMap.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.metamx.common.logger.Logger; diff --git a/client/src/main/java/com/metamx/druid/client/cache/Cache.java b/server/src/main/java/io/druid/client/cache/Cache.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/Cache.java rename to server/src/main/java/io/druid/client/cache/Cache.java index 6e9463deb56..6ed875edb0b 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/Cache.java +++ b/server/src/main/java/io/druid/client/cache/Cache.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java b/server/src/main/java/io/druid/client/cache/CacheMonitor.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java rename to server/src/main/java/io/druid/client/cache/CacheMonitor.java index ce9ac5e27d2..baa4920e60f 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheMonitor.java +++ b/server/src/main/java/io/druid/client/cache/CacheMonitor.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; diff --git a/server/src/main/java/io/druid/client/cache/CacheProvider.java b/server/src/main/java/io/druid/client/cache/CacheProvider.java new file mode 100644 index 00000000000..87597dba1c7 --- /dev/null +++ b/server/src/main/java/io/druid/client/cache/CacheProvider.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.cache; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LocalCacheProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "local", value = LocalCacheProvider.class), + @JsonSubTypes.Type(name = "memcached", value = MemcachedCacheProvider.class) +}) +public interface CacheProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheSerde.java b/server/src/main/java/io/druid/client/cache/CacheSerde.java similarity index 91% rename from client/src/main/java/com/metamx/druid/client/cache/CacheSerde.java rename to server/src/main/java/io/druid/client/cache/CacheSerde.java index e7e60fcc096..16dbce6ad6d 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheSerde.java +++ b/server/src/main/java/io/druid/client/cache/CacheSerde.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; /** */ diff --git a/client/src/main/java/com/metamx/druid/client/cache/CacheStats.java b/server/src/main/java/io/druid/client/cache/CacheStats.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/CacheStats.java rename to server/src/main/java/io/druid/client/cache/CacheStats.java index 33f0a145082..ea917649154 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/CacheStats.java +++ b/server/src/main/java/io/druid/client/cache/CacheStats.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; /** */ diff --git a/client/src/main/java/com/metamx/druid/client/cache/LZ4Transcoder.java b/server/src/main/java/io/druid/client/cache/LZ4Transcoder.java similarity index 96% rename from client/src/main/java/com/metamx/druid/client/cache/LZ4Transcoder.java rename to server/src/main/java/io/druid/client/cache/LZ4Transcoder.java index 4728430b4e7..7427c2b3db5 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/LZ4Transcoder.java +++ b/server/src/main/java/io/druid/client/cache/LZ4Transcoder.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.primitives.Ints; import net.jpountz.lz4.LZ4Compressor; diff --git a/client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java b/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java similarity index 93% rename from client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java rename to server/src/main/java/io/druid/client/cache/LocalCacheProvider.java index 1457ec928e6..4f78457b5b9 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/LocalCacheProvider.java +++ b/server/src/main/java/io/druid/client/cache/LocalCacheProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java b/server/src/main/java/io/druid/client/cache/MapCache.java similarity index 97% rename from client/src/main/java/com/metamx/druid/client/cache/MapCache.java rename to server/src/main/java/io/druid/client/cache/MapCache.java index 0480776cfde..743ca26463d 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MapCache.java +++ b/server/src/main/java/io/druid/client/cache/MapCache.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; diff --git a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java b/server/src/main/java/io/druid/client/cache/MemcachedCache.java similarity index 99% rename from client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java rename to server/src/main/java/io/druid/client/cache/MemcachedCache.java index befac85326e..155c75e3f86 100644 --- a/client/src/main/java/com/metamx/druid/client/cache/MemcachedCache.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCache.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.base.Function; import com.google.common.base.Preconditions; diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCacheConfig.java b/server/src/main/java/io/druid/client/cache/MemcachedCacheConfig.java new file mode 100644 index 00000000000..2cc06cf3637 --- /dev/null +++ b/server/src/main/java/io/druid/client/cache/MemcachedCacheConfig.java @@ -0,0 +1,68 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.cache; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.NotNull; + +public class MemcachedCacheConfig +{ + @JsonProperty + private int expiration = 2592000; // What is this number? + + @JsonProperty + private int timeout = 500; + + @JsonProperty + @NotNull + private String hosts; + + @JsonProperty + private int maxObjectSize = 50 * 1024 * 1024; + + @JsonProperty + private String memcachedPrefix = "druid"; + + public int getExpiration() + { + return expiration; + } + + public int getTimeout() + { + return timeout; + } + + public String getHosts() + { + return hosts; + } + + public int getMaxObjectSize() + { + return maxObjectSize; + } + + public String getMemcachedPrefix() + { + return memcachedPrefix; + } +} diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCacheProvider.java b/server/src/main/java/io/druid/client/cache/MemcachedCacheProvider.java new file mode 100644 index 00000000000..afdab26dd09 --- /dev/null +++ b/server/src/main/java/io/druid/client/cache/MemcachedCacheProvider.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.cache; + +public class MemcachedCacheProvider extends MemcachedCacheConfig implements CacheProvider +{ + @Override + public Cache get() + { + return MemcachedCache.create(this); + } +} diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java b/server/src/main/java/io/druid/client/indexing/ClientAppendQuery.java similarity index 92% rename from client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java rename to server/src/main/java/io/druid/client/indexing/ClientAppendQuery.java index 6e9a2eb4163..45641518166 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/ClientAppendQuery.java +++ b/server/src/main/java/io/druid/client/indexing/ClientAppendQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.indexing; +package io.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java b/server/src/main/java/io/druid/client/indexing/ClientConversionQuery.java similarity index 51% rename from client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java rename to server/src/main/java/io/druid/client/indexing/ClientConversionQuery.java index c0b96bb80b4..4f67c0e7bc6 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/ClientConversionQuery.java +++ b/server/src/main/java/io/druid/client/indexing/ClientConversionQuery.java @@ -1,7 +1,26 @@ -package com.metamx.druid.client.indexing; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import org.joda.time.Interval; /** diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/io/druid/client/indexing/ClientKillQuery.java similarity index 94% rename from client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java rename to server/src/main/java/io/druid/client/indexing/ClientKillQuery.java index 3ae8dffb225..6f280cef7fa 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/io/druid/client/indexing/ClientKillQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.indexing; +package io.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java b/server/src/main/java/io/druid/client/indexing/ClientMergeQuery.java similarity index 93% rename from client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java rename to server/src/main/java/io/druid/client/indexing/ClientMergeQuery.java index 493e9588f5a..cb5a89599e7 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/ClientMergeQuery.java +++ b/server/src/main/java/io/druid/client/indexing/ClientMergeQuery.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.indexing; +package io.druid.client.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import io.druid.query.aggregation.AggregatorFactory; import java.util.List; diff --git a/server/src/main/java/io/druid/client/indexing/IndexingService.java b/server/src/main/java/io/druid/client/indexing/IndexingService.java new file mode 100644 index 00000000000..e79bab7fbee --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/IndexingService.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.indexing; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface IndexingService +{ +} diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java similarity index 92% rename from client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java rename to server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index e3de082f7fc..2d3a622edae 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,19 +17,19 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.indexing; +package io.druid.client.indexing; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.selector.DiscoverySelector; -import com.metamx.druid.client.selector.Server; -import com.metamx.druid.guice.annotations.Global; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; +import io.druid.client.DataSegment; +import io.druid.client.selector.DiscoverySelector; +import io.druid.client.selector.Server; +import io.druid.guice.guice.annotations.Global; import org.joda.time.Interval; import java.io.InputStream; diff --git a/server/src/main/java/io/druid/client/selector/DiscoverySelector.java b/server/src/main/java/io/druid/client/selector/DiscoverySelector.java new file mode 100644 index 00000000000..d878d7ed921 --- /dev/null +++ b/server/src/main/java/io/druid/client/selector/DiscoverySelector.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.selector; + +/** + */ +public interface DiscoverySelector +{ + public T pick(); +} diff --git a/client/src/main/java/com/metamx/druid/client/selector/QueryableDruidServer.java b/server/src/main/java/io/druid/client/selector/QueryableDruidServer.java similarity index 86% rename from client/src/main/java/com/metamx/druid/client/selector/QueryableDruidServer.java rename to server/src/main/java/io/druid/client/selector/QueryableDruidServer.java index 2528facb8d8..8f4f7f48e82 100644 --- a/client/src/main/java/com/metamx/druid/client/selector/QueryableDruidServer.java +++ b/server/src/main/java/io/druid/client/selector/QueryableDruidServer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.selector; +package io.druid.client.selector; -import com.metamx.druid.client.DirectDruidClient; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DirectDruidClient; +import io.druid.client.DruidServer; /** */ diff --git a/server/src/main/java/io/druid/client/selector/Server.java b/server/src/main/java/io/druid/client/selector/Server.java new file mode 100644 index 00000000000..664d621136e --- /dev/null +++ b/server/src/main/java/io/druid/client/selector/Server.java @@ -0,0 +1,29 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.selector; + +/** + */ +public interface Server +{ + public String getScheme(); + public String getHost(); + public int getPort(); +} diff --git a/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java b/server/src/main/java/io/druid/client/selector/ServerSelector.java similarity index 94% rename from client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java rename to server/src/main/java/io/druid/client/selector/ServerSelector.java index d4876e5680f..53ec1727267 100644 --- a/client/src/main/java/com/metamx/druid/client/selector/ServerSelector.java +++ b/server/src/main/java/io/druid/client/selector/ServerSelector.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.selector; +package io.druid.client.selector; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.Collections; import java.util.Comparator; diff --git a/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java b/server/src/main/java/io/druid/curator/CuratorConfig.java similarity index 93% rename from client/src/main/java/com/metamx/druid/curator/CuratorConfig.java rename to server/src/main/java/io/druid/curator/CuratorConfig.java index 2f396e02824..7ac0247b342 100644 --- a/client/src/main/java/com/metamx/druid/curator/CuratorConfig.java +++ b/server/src/main/java/io/druid/curator/CuratorConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator; +package io.druid.curator; import org.skife.config.Config; import org.skife.config.Default; diff --git a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java b/server/src/main/java/io/druid/curator/CuratorModule.java similarity index 63% rename from client/src/main/java/com/metamx/druid/curator/CuratorModule.java rename to server/src/main/java/io/druid/curator/CuratorModule.java index ee7e59e0e65..89fef96830d 100644 --- a/client/src/main/java/com/metamx/druid/curator/CuratorModule.java +++ b/server/src/main/java/io/druid/curator/CuratorModule.java @@ -1,12 +1,31 @@ -package com.metamx.druid.curator; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.ConfigProvider; -import com.metamx.druid.guice.LazySingleton; +import io.druid.guice.guice.ConfigProvider; +import io.druid.guice.guice.LazySingleton; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.BoundedExponentialBackoffRetry; diff --git a/client/src/main/java/com/metamx/druid/curator/PotentiallyGzippedCompressionProvider.java b/server/src/main/java/io/druid/curator/PotentiallyGzippedCompressionProvider.java similarity index 93% rename from client/src/main/java/com/metamx/druid/curator/PotentiallyGzippedCompressionProvider.java rename to server/src/main/java/io/druid/curator/PotentiallyGzippedCompressionProvider.java index abddd9574fc..c2fda7139be 100644 --- a/client/src/main/java/com/metamx/druid/curator/PotentiallyGzippedCompressionProvider.java +++ b/server/src/main/java/io/druid/curator/PotentiallyGzippedCompressionProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,13 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator; +package io.druid.curator; import org.apache.curator.framework.api.CompressionProvider; import org.apache.curator.framework.imps.GzipCompressionProvider; import java.io.IOException; -import java.util.zip.ZipException; /** */ diff --git a/client/src/main/java/com/metamx/druid/curator/ShutdownNowIgnoringExecutorService.java b/server/src/main/java/io/druid/curator/ShutdownNowIgnoringExecutorService.java similarity index 74% rename from client/src/main/java/com/metamx/druid/curator/ShutdownNowIgnoringExecutorService.java rename to server/src/main/java/io/druid/curator/ShutdownNowIgnoringExecutorService.java index b4a8128f6d9..7ac77fcc5ea 100644 --- a/client/src/main/java/com/metamx/druid/curator/ShutdownNowIgnoringExecutorService.java +++ b/server/src/main/java/io/druid/curator/ShutdownNowIgnoringExecutorService.java @@ -1,4 +1,23 @@ -package com.metamx.druid.curator; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator; import com.google.common.collect.ImmutableList; diff --git a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java b/server/src/main/java/io/druid/curator/announcement/Announcer.java similarity index 97% rename from client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java rename to server/src/main/java/io/druid/curator/announcement/Announcer.java index abb96b76f68..8b017c41a24 100644 --- a/client/src/main/java/com/metamx/druid/curator/announcement/Announcer.java +++ b/server/src/main/java/io/druid/curator/announcement/Announcer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.announcement; +package io.druid.curator.announcement; import com.google.common.base.Throwables; import com.google.common.collect.Lists; @@ -30,9 +30,9 @@ import com.metamx.common.Pair; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.ShutdownNowIgnoringExecutorService; -import com.metamx.druid.curator.cache.PathChildrenCacheFactory; -import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; +import io.druid.curator.ShutdownNowIgnoringExecutorService; +import io.druid.curator.cache.PathChildrenCacheFactory; +import io.druid.curator.cache.SimplePathChildrenCacheFactory; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/client/src/main/java/com/metamx/druid/curator/cache/PathChildrenCacheFactory.java b/server/src/main/java/io/druid/curator/cache/PathChildrenCacheFactory.java similarity index 92% rename from client/src/main/java/com/metamx/druid/curator/cache/PathChildrenCacheFactory.java rename to server/src/main/java/io/druid/curator/cache/PathChildrenCacheFactory.java index 37335d8aa88..d722ba0ba5d 100644 --- a/client/src/main/java/com/metamx/druid/curator/cache/PathChildrenCacheFactory.java +++ b/server/src/main/java/io/druid/curator/cache/PathChildrenCacheFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.cache; +package io.druid.curator.cache; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java b/server/src/main/java/io/druid/curator/cache/SimplePathChildrenCacheFactory.java similarity index 96% rename from client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java rename to server/src/main/java/io/druid/curator/cache/SimplePathChildrenCacheFactory.java index fb7a3044ee4..ef0d3c50b9b 100644 --- a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java +++ b/server/src/main/java/io/druid/curator/cache/SimplePathChildrenCacheFactory.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.cache; +package io.druid.curator.cache; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java b/server/src/main/java/io/druid/curator/discovery/CuratorServiceAnnouncer.java similarity index 75% rename from client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java rename to server/src/main/java/io/druid/curator/discovery/CuratorServiceAnnouncer.java index dc277d927c6..633b987a88a 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/CuratorServiceAnnouncer.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorServiceAnnouncer.java @@ -1,10 +1,29 @@ -package com.metamx.druid.curator.discovery; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator.discovery; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.inject.Inject; -import com.metamx.druid.initialization.DruidNode; import com.metamx.emitter.EmittingLogger; +import io.druid.server.DruidNode; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; diff --git a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java similarity index 86% rename from client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java rename to server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index 9b6bc20af6b..595ae977572 100644 --- a/client/src/main/java/com/metamx/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -1,4 +1,23 @@ -package com.metamx.druid.curator.discovery; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator.discovery; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -14,10 +33,10 @@ import com.google.inject.TypeLiteral; import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.initialization.CuratorDiscoveryConfig; -import com.metamx.druid.initialization.DruidNode; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.LazySingleton; +import io.druid.server.DruidNode; +import io.druid.server.initialization.initialization.CuratorDiscoveryConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; diff --git a/server/src/main/java/io/druid/curator/discovery/NoopServiceAnnouncer.java b/server/src/main/java/io/druid/curator/discovery/NoopServiceAnnouncer.java new file mode 100644 index 00000000000..672b900d5bf --- /dev/null +++ b/server/src/main/java/io/druid/curator/discovery/NoopServiceAnnouncer.java @@ -0,0 +1,40 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator.discovery; + +import io.druid.server.DruidNode; + +/** + * Does nothing. + */ +public class NoopServiceAnnouncer implements ServiceAnnouncer +{ + @Override + public void announce(DruidNode node) + { + + } + + @Override + public void unannounce(DruidNode node) + { + + } +} diff --git a/server/src/main/java/io/druid/curator/discovery/ServiceAnnouncer.java b/server/src/main/java/io/druid/curator/discovery/ServiceAnnouncer.java new file mode 100644 index 00000000000..46da8f5b0ff --- /dev/null +++ b/server/src/main/java/io/druid/curator/discovery/ServiceAnnouncer.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator.discovery; + +import io.druid.server.DruidNode; + +/** + * Announces our ability to serve a particular function. Multiple users may announce the same service, in which + * case they are treated as interchangeable instances of that service. + */ +public interface ServiceAnnouncer +{ + public void announce(DruidNode node); + + public void unannounce(DruidNode node); +} diff --git a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java similarity index 97% rename from client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java rename to server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java index 7c35ad12618..dcff14024e3 100644 --- a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManager.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.inventory; +package io.druid.curator.inventory; import com.google.common.base.Function; import com.google.common.collect.Iterables; @@ -26,9 +26,9 @@ import com.google.common.collect.MapMaker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.ShutdownNowIgnoringExecutorService; -import com.metamx.druid.curator.cache.PathChildrenCacheFactory; -import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; +import io.druid.curator.ShutdownNowIgnoringExecutorService; +import io.druid.curator.cache.PathChildrenCacheFactory; +import io.druid.curator.cache.SimplePathChildrenCacheFactory; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java similarity index 94% rename from client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java rename to server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java index 8cab619e16f..785e35534df 100644 --- a/client/src/main/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerStrategy.java +++ b/server/src/main/java/io/druid/curator/inventory/CuratorInventoryManagerStrategy.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.inventory; +package io.druid.curator.inventory; /** */ diff --git a/client/src/main/java/com/metamx/druid/curator/inventory/InventoryManagerConfig.java b/server/src/main/java/io/druid/curator/inventory/InventoryManagerConfig.java similarity index 95% rename from client/src/main/java/com/metamx/druid/curator/inventory/InventoryManagerConfig.java rename to server/src/main/java/io/druid/curator/inventory/InventoryManagerConfig.java index 418c402ff83..78158b713be 100644 --- a/client/src/main/java/com/metamx/druid/curator/inventory/InventoryManagerConfig.java +++ b/server/src/main/java/io/druid/curator/inventory/InventoryManagerConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.inventory; +package io.druid.curator.inventory; /** */ diff --git a/server/src/main/java/com/metamx/druid/guice/AWSModule.java b/server/src/main/java/io/druid/guice/guice/AWSModule.java similarity index 96% rename from server/src/main/java/com/metamx/druid/guice/AWSModule.java rename to server/src/main/java/io/druid/guice/guice/AWSModule.java index 63e005475be..62030a7c15b 100644 --- a/server/src/main/java/com/metamx/druid/guice/AWSModule.java +++ b/server/src/main/java/io/druid/guice/guice/AWSModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.BasicAWSCredentials; diff --git a/server/src/main/java/io/druid/guice/guice/AnnouncerModule.java b/server/src/main/java/io/druid/guice/guice/AnnouncerModule.java new file mode 100644 index 00000000000..fc5c4159d09 --- /dev/null +++ b/server/src/main/java/io/druid/guice/guice/AnnouncerModule.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.metamx.druid.concurrent.Execs; +import io.druid.curator.announcement.Announcer; +import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DataSegmentAnnouncerProvider; +import io.druid.server.coordination.SingleDataSegmentAnnouncer; +import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; +import org.apache.curator.framework.CuratorFramework; + +/** + */ +public class AnnouncerModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.announcer", BatchDataSegmentAnnouncerConfig.class); + JsonConfigProvider.bind(binder, "druid.announcer", DataSegmentAnnouncerProvider.class); + binder.bind(DataSegmentAnnouncer.class).toProvider(DataSegmentAnnouncerProvider.class); + binder.bind(BatchDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + binder.bind(SingleDataSegmentAnnouncer.class).in(ManageLifecycleLast.class); + } + + @Provides + @ManageLifecycle + public Announcer getAnnouncer(CuratorFramework curator) + { + return new Announcer(curator, Execs.singleThreaded("Announcer-%s")); + } +} diff --git a/server/src/main/java/io/druid/guice/guice/BrokerModule.java b/server/src/main/java/io/druid/guice/guice/BrokerModule.java new file mode 100644 index 00000000000..acb0bd2325f --- /dev/null +++ b/server/src/main/java/io/druid/guice/guice/BrokerModule.java @@ -0,0 +1,47 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.client.BrokerServerView; +import io.druid.client.CachingClusteredClient; +import io.druid.client.TimelineServerView; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheProvider; +import io.druid.query.MapQueryToolChestWarehouse; +import io.druid.query.QueryToolChestWarehouse; + +/** + */ +public class BrokerModule implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(QueryToolChestWarehouse.class).to(MapQueryToolChestWarehouse.class); + + binder.bind(CachingClusteredClient.class).in(LazySingleton.class); + binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); + + binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); + JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java b/server/src/main/java/io/druid/guice/guice/CassandraDataSegmentPusherProvider.java similarity index 94% rename from server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java rename to server/src/main/java/io/druid/guice/guice/CassandraDataSegmentPusherProvider.java index 6d28aff0f87..97a0fde64c1 100644 --- a/server/src/main/java/com/metamx/druid/guice/CassandraDataSegmentPusherProvider.java +++ b/server/src/main/java/io/druid/guice/guice/CassandraDataSegmentPusherProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java b/server/src/main/java/io/druid/guice/guice/CoordinatorModule.java similarity index 64% rename from server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java rename to server/src/main/java/io/druid/guice/guice/CoordinatorModule.java index 8ae217714ca..fb800c68bcd 100644 --- a/server/src/main/java/com/metamx/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/io/druid/guice/guice/CoordinatorModule.java @@ -1,25 +1,44 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.druid.client.ServerInventoryViewConfig; -import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManagerConfig; import com.metamx.druid.db.DatabaseRuleManagerProvider; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DatabaseSegmentManagerProvider; -import com.metamx.druid.http.MasterRedirectInfo; -import com.metamx.druid.http.RedirectFilter; -import com.metamx.druid.http.RedirectInfo; -import com.metamx.druid.http.RedirectServlet; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.LoadQueueTaskMaster; +import io.druid.client.ServerInventoryViewConfig; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.server.http.MasterRedirectInfo; +import io.druid.server.http.RedirectFilter; +import io.druid.server.http.RedirectInfo; +import io.druid.server.http.RedirectServlet; import org.apache.curator.framework.CuratorFramework; /** diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java b/server/src/main/java/io/druid/guice/guice/DataSegmentPullerModule.java similarity index 97% rename from server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java rename to server/src/main/java/io/druid/guice/guice/DataSegmentPullerModule.java index 25a21ff3af2..ec167922f94 100644 --- a/server/src/main/java/com/metamx/druid/guice/DataSegmentPullerModule.java +++ b/server/src/main/java/io/druid/guice/guice/DataSegmentPullerModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java b/server/src/main/java/io/druid/guice/guice/DataSegmentPusherModule.java similarity index 54% rename from server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java rename to server/src/main/java/io/druid/guice/guice/DataSegmentPusherModule.java index bb95cb45273..42de20929bd 100644 --- a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherModule.java +++ b/server/src/main/java/io/druid/guice/guice/DataSegmentPusherModule.java @@ -1,10 +1,28 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.HdfsDataSegmentPusherConfig; -import com.metamx.druid.loading.LocalDataSegmentPusherConfig; import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.cassandra.CassandraDataSegmentConfig; import org.apache.hadoop.conf.Configuration; diff --git a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java b/server/src/main/java/io/druid/guice/guice/DataSegmentPusherProvider.java similarity index 94% rename from server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java rename to server/src/main/java/io/druid/guice/guice/DataSegmentPusherProvider.java index f69c413663e..12e7b396b02 100644 --- a/server/src/main/java/com/metamx/druid/guice/DataSegmentPusherProvider.java +++ b/server/src/main/java/io/druid/guice/guice/DataSegmentPusherProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/guice/DruidProcessingModule.java similarity index 95% rename from server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java rename to server/src/main/java/io/druid/guice/guice/DruidProcessingModule.java index 27a8d069829..ae859ef58d1 100644 --- a/server/src/main/java/com/metamx/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/guice/DruidProcessingModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; @@ -30,11 +30,11 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.guice.annotations.Global; -import com.metamx.druid.guice.annotations.Processing; -import com.metamx.druid.query.MetricsEmittingExecutorService; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.guice.guice.annotations.Global; +import io.druid.guice.guice.annotations.Processing; +import io.druid.query.MetricsEmittingExecutorService; import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; diff --git a/server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java b/server/src/main/java/io/druid/guice/guice/HdfsDataSegmentPusherProvider.java similarity index 95% rename from server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java rename to server/src/main/java/io/druid/guice/guice/HdfsDataSegmentPusherProvider.java index 837b41ce1e1..c0fca1d2fbf 100644 --- a/server/src/main/java/com/metamx/druid/guice/HdfsDataSegmentPusherProvider.java +++ b/server/src/main/java/io/druid/guice/guice/HdfsDataSegmentPusherProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java b/server/src/main/java/io/druid/guice/guice/HistoricalModule.java similarity index 94% rename from server/src/main/java/com/metamx/druid/guice/HistoricalModule.java rename to server/src/main/java/io/druid/guice/guice/HistoricalModule.java index 56526a5f286..1ba3a239dc3 100644 --- a/server/src/main/java/com/metamx/druid/guice/HistoricalModule.java +++ b/server/src/main/java/io/druid/guice/guice/HistoricalModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; diff --git a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java b/server/src/main/java/io/druid/guice/guice/HttpClientModule.java similarity index 83% rename from server/src/main/java/com/metamx/druid/guice/HttpClientModule.java rename to server/src/main/java/io/druid/guice/guice/HttpClientModule.java index 28e4dce5f15..14c224a44cd 100644 --- a/server/src/main/java/com/metamx/druid/guice/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/guice/HttpClientModule.java @@ -1,4 +1,23 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; @@ -11,10 +30,10 @@ import com.google.inject.Module; import com.google.inject.Provider; import com.google.inject.TypeLiteral; import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.guice.annotations.Global; import com.metamx.http.client.HttpClient; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; +import io.druid.guice.guice.annotations.Global; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java b/server/src/main/java/io/druid/guice/guice/IndexingServiceDiscoveryModule.java similarity index 92% rename from server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java rename to server/src/main/java/io/druid/guice/guice/IndexingServiceDiscoveryModule.java index e39c43dd878..a59d0839999 100644 --- a/server/src/main/java/com/metamx/druid/guice/IndexingServiceDiscoveryModule.java +++ b/server/src/main/java/io/druid/guice/guice/IndexingServiceDiscoveryModule.java @@ -17,17 +17,17 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.TypeLiteral; -import com.metamx.druid.client.indexing.IndexingService; import com.metamx.druid.client.indexing.IndexingServiceSelector; import com.metamx.druid.client.indexing.IndexingServiceSelectorConfig; -import com.metamx.druid.client.selector.DiscoverySelector; -import com.metamx.druid.client.selector.Server; +import io.druid.client.indexing.IndexingService; +import io.druid.client.selector.DiscoverySelector; +import io.druid.client.selector.Server; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; diff --git a/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java b/server/src/main/java/io/druid/guice/guice/LocalDataSegmentPusherProvider.java similarity index 94% rename from server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java rename to server/src/main/java/io/druid/guice/guice/LocalDataSegmentPusherProvider.java index 7a51b7264e1..84350024083 100644 --- a/server/src/main/java/com/metamx/druid/guice/LocalDataSegmentPusherProvider.java +++ b/server/src/main/java/io/druid/guice/guice/LocalDataSegmentPusherProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/io/druid/guice/guice/QueryRunnerFactoryModule.java similarity index 57% rename from server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java rename to server/src/main/java/io/druid/guice/guice/QueryRunnerFactoryModule.java index 6abcb9a9414..78a2c76f850 100644 --- a/server/src/main/java/com/metamx/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/io/druid/guice/guice/QueryRunnerFactoryModule.java @@ -1,22 +1,42 @@ -package com.metamx.druid.guice; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; -import com.metamx.druid.query.group.GroupByQuery; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryRunnerFactory; -import com.metamx.druid.query.metadata.SegmentMetadataQuery; import com.metamx.druid.query.metadata.SegmentMetadataQueryRunnerFactory; -import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.query.search.SearchQueryRunnerFactory; -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 io.druid.initialization.Binders; +import io.druid.guice.QueryToolChestModule; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeseries.TimeseriesQuery; import java.util.Map; @@ -39,7 +59,9 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule super.configure(binder); - final MapBinder, QueryRunnerFactory> queryFactoryBinder = Binders.queryFactoryBinder(binder); + final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( + binder, new TypeLiteral>(){}, TypeLiteral.get(QueryRunnerFactory.class) + ); for (Map.Entry, Class> entry : mappings.entrySet()) { queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue()); diff --git a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java b/server/src/main/java/io/druid/guice/guice/QueryableModule.java similarity index 84% rename from client/src/main/java/com/metamx/druid/guice/QueryableModule.java rename to server/src/main/java/io/druid/guice/guice/QueryableModule.java index d131c600aab..b0a83701a4d 100644 --- a/client/src/main/java/com/metamx/druid/guice/QueryableModule.java +++ b/server/src/main/java/io/druid/guice/guice/QueryableModule.java @@ -17,18 +17,18 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; -import com.metamx.druid.http.QueryServlet; -import com.metamx.druid.http.log.EmittingRequestLoggerProvider; -import com.metamx.druid.http.log.FileRequestLoggerProvider; -import com.metamx.druid.http.log.RequestLogger; -import com.metamx.druid.http.log.RequestLoggerProvider; -import com.metamx.druid.query.segment.QuerySegmentWalker; import io.druid.initialization.DruidModule; +import io.druid.query.QuerySegmentWalker; +import io.druid.server.QueryServlet; +import io.druid.server.log.EmittingRequestLoggerProvider; +import io.druid.server.log.FileRequestLoggerProvider; +import io.druid.server.log.RequestLogger; +import io.druid.server.log.RequestLoggerProvider; import java.util.Arrays; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java b/server/src/main/java/io/druid/guice/guice/S3DataSegmentPusherProvider.java similarity index 95% rename from server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java rename to server/src/main/java/io/druid/guice/guice/S3DataSegmentPusherProvider.java index b86fecfdf7a..19b03ef3855 100644 --- a/server/src/main/java/com/metamx/druid/guice/S3DataSegmentPusherProvider.java +++ b/server/src/main/java/io/druid/guice/guice/S3DataSegmentPusherProvider.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/server/src/main/java/com/metamx/druid/guice/ServerModule.java b/server/src/main/java/io/druid/guice/guice/ServerModule.java similarity index 86% rename from server/src/main/java/com/metamx/druid/guice/ServerModule.java rename to server/src/main/java/io/druid/guice/guice/ServerModule.java index 400f4ad040b..3b217f1269e 100644 --- a/server/src/main/java/com/metamx/druid/guice/ServerModule.java +++ b/server/src/main/java/io/druid/guice/guice/ServerModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; @@ -25,9 +25,9 @@ import com.google.inject.Provides; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.initialization.DruidNode; -import com.metamx.druid.initialization.ZkPathsConfig; +import io.druid.guice.guice.annotations.Self; +import io.druid.server.DruidNode; +import io.druid.server.initialization.initialization.ZkPathsConfig; /** */ diff --git a/server/src/main/java/io/druid/guice/guice/ServerViewModule.java b/server/src/main/java/io/druid/guice/guice/ServerViewModule.java new file mode 100644 index 00000000000..f6a439d2782 --- /dev/null +++ b/server/src/main/java/io/druid/guice/guice/ServerViewModule.java @@ -0,0 +1,41 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.client.InventoryView; +import io.druid.client.ServerInventoryView; +import io.druid.client.ServerInventoryViewProvider; +import io.druid.client.ServerView; + +/** + */ +public class ServerViewModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.announcer", ServerInventoryViewProvider.class); + binder.bind(InventoryView.class).to(ServerInventoryView.class); + binder.bind(ServerView.class).to(ServerInventoryView.class); + binder.bind(ServerInventoryView.class).toProvider(ServerInventoryViewProvider.class).in(ManageLifecycle.class); + } +} diff --git a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java b/server/src/main/java/io/druid/guice/guice/StorageNodeModule.java similarity index 88% rename from server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java rename to server/src/main/java/io/druid/guice/guice/StorageNodeModule.java index 33b7688afe7..b5e17a9c673 100644 --- a/server/src/main/java/com/metamx/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/io/druid/guice/guice/StorageNodeModule.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,20 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.guice; +package io.druid.guice.guice; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; -import com.metamx.druid.client.DruidServerConfig; -import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.initialization.DruidNode; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.query.DefaultQueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.client.DruidServerConfig; +import io.druid.guice.guice.annotations.Self; +import io.druid.server.DruidNode; +import io.druid.server.coordination.DruidServerMetadata; /** */ diff --git a/server/src/main/java/io/druid/guice/guice/annotations/Client.java b/server/src/main/java/io/druid/guice/guice/annotations/Client.java new file mode 100644 index 00000000000..34617592e44 --- /dev/null +++ b/server/src/main/java/io/druid/guice/guice/annotations/Client.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Client +{ +} diff --git a/server/src/main/java/io/druid/guice/guice/annotations/Processing.java b/server/src/main/java/io/druid/guice/guice/annotations/Processing.java new file mode 100644 index 00000000000..33117be1083 --- /dev/null +++ b/server/src/main/java/io/druid/guice/guice/annotations/Processing.java @@ -0,0 +1,36 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.guice.annotations; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + */ +@BindingAnnotation +@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface Processing +{ +} diff --git a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java b/server/src/main/java/io/druid/server/ClientInfoResource.java similarity index 94% rename from client/src/main/java/com/metamx/druid/http/ClientInfoResource.java rename to server/src/main/java/io/druid/server/ClientInfoResource.java index 8bcb8865e7c..6dfcf673a3e 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientInfoResource.java +++ b/server/src/main/java/io/druid/server/ClientInfoResource.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,17 +17,17 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.InventoryView; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.client.DruidServer; +import io.druid.client.InventoryView; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java similarity index 87% rename from client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java rename to server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 39c568fda04..76d9d1aa838 100644 --- a/client/src/main/java/com/metamx/druid/http/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -17,20 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.google.common.base.Function; import com.google.inject.Inject; -import com.metamx.druid.client.CachingClusteredClient; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.MetricsEmittingQueryRunner; -import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.CachingClusteredClient; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.MetricsEmittingQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; +import io.druid.query.QueryToolChestWarehouse; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; @@ -40,20 +40,20 @@ import javax.annotation.Nullable; */ public class ClientQuerySegmentWalker implements QuerySegmentWalker { - private final QueryToolChestWarehouse warehouse; private final ServiceEmitter emitter; private final CachingClusteredClient baseClient; + private final QueryToolChestWarehouse warehouse; @Inject public ClientQuerySegmentWalker( - QueryToolChestWarehouse warehouse, ServiceEmitter emitter, - CachingClusteredClient baseClient + CachingClusteredClient baseClient, + QueryToolChestWarehouse warehouse ) { - this.warehouse = warehouse; this.emitter = emitter; this.baseClient = baseClient; + this.warehouse = warehouse; } @Override diff --git a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/DirectClientQuerySegmentWalker.java similarity index 87% rename from client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java rename to server/src/main/java/io/druid/server/DirectClientQuerySegmentWalker.java index f3c8c8956be..224f03691ea 100644 --- a/client/src/main/java/com/metamx/druid/http/DirectClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/DirectClientQuerySegmentWalker.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; -import com.metamx.druid.client.DirectDruidClient; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.QueryToolChestWarehouse; -import com.metamx.druid.query.segment.QuerySegmentWalker; +import io.druid.client.DirectDruidClient; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.QuerySegmentWalker; +import io.druid.query.QueryToolChestWarehouse; import io.druid.query.SegmentDescriptor; import org.joda.time.Interval; diff --git a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java b/server/src/main/java/io/druid/server/DruidNode.java similarity index 98% rename from client/src/main/java/com/metamx/druid/initialization/DruidNode.java rename to server/src/main/java/io/druid/server/DruidNode.java index 43f78ac0682..69fece2c685 100644 --- a/client/src/main/java/com/metamx/druid/initialization/DruidNode.java +++ b/server/src/main/java/io/druid/server/DruidNode.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/http/GuiceServletConfig.java b/server/src/main/java/io/druid/server/GuiceServletConfig.java similarity index 93% rename from client/src/main/java/com/metamx/druid/http/GuiceServletConfig.java rename to server/src/main/java/io/druid/server/GuiceServletConfig.java index fcfac0a66f8..b7a52a4eb6b 100644 --- a/client/src/main/java/com/metamx/druid/http/GuiceServletConfig.java +++ b/server/src/main/java/io/druid/server/GuiceServletConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.google.inject.Injector; import com.google.inject.servlet.GuiceServletContextListener; diff --git a/client/src/main/java/com/metamx/druid/http/QueryServlet.java b/server/src/main/java/io/druid/server/QueryServlet.java similarity index 95% rename from client/src/main/java/com/metamx/druid/http/QueryServlet.java rename to server/src/main/java/io/druid/server/QueryServlet.java index 58fedde82af..d3e2d7e7522 100644 --- a/client/src/main/java/com/metamx/druid/http/QueryServlet.java +++ b/server/src/main/java/io/druid/server/QueryServlet.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; @@ -29,14 +29,14 @@ import com.google.inject.Inject; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.annotations.Json; -import com.metamx.druid.guice.annotations.Smile; -import com.metamx.druid.http.log.RequestLogger; -import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.guice.guice.annotations.Json; +import io.druid.guice.guice.annotations.Smile; import io.druid.query.Query; +import io.druid.query.QuerySegmentWalker; +import io.druid.server.log.RequestLogger; import org.eclipse.jetty.server.Request; import org.joda.time.DateTime; diff --git a/client/src/main/java/com/metamx/druid/http/RequestLogLine.java b/server/src/main/java/io/druid/server/RequestLogLine.java similarity index 96% rename from client/src/main/java/com/metamx/druid/http/RequestLogLine.java rename to server/src/main/java/io/druid/server/RequestLogLine.java index faf403690ac..a52485ae279 100644 --- a/client/src/main/java/com/metamx/druid/http/RequestLogLine.java +++ b/server/src/main/java/io/druid/server/RequestLogLine.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/client/src/main/java/com/metamx/druid/http/StatusResource.java b/server/src/main/java/io/druid/server/StatusResource.java similarity index 96% rename from client/src/main/java/com/metamx/druid/http/StatusResource.java rename to server/src/main/java/io/druid/server/StatusResource.java index ebe2f0e855a..a2b30268d4d 100644 --- a/client/src/main/java/com/metamx/druid/http/StatusResource.java +++ b/server/src/main/java/io/druid/server/StatusResource.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/ZkPathsModule.java b/server/src/main/java/io/druid/server/ZkPathsModule.java new file mode 100644 index 00000000000..34631221353 --- /dev/null +++ b/server/src/main/java/io/druid/server/ZkPathsModule.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server; + +import com.google.inject.Binder; +import com.google.inject.Module; + +/** + */ +public class ZkPathsModule implements Module +{ + @Override + public void configure(Binder binder) + { + } +} diff --git a/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java similarity index 92% rename from client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java index 1bb0b8114ee..f4bd5016430 100644 --- a/client/src/main/java/com/metamx/druid/coordination/AbstractDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,8 +25,8 @@ import com.google.common.base.Throwables; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkPathsConfig; +import io.druid.curator.announcement.Announcer; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; /** diff --git a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java similarity index 96% rename from client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java index 5e454038c3a..efe693f39e0 100644 --- a/client/src/main/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -27,10 +27,10 @@ import com.google.common.collect.Sets; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; +import io.druid.client.DataSegment; +import io.druid.curator.announcement.Announcer; +import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; diff --git a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..e51de3a245c --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncerProvider.java @@ -0,0 +1,39 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; + +import javax.validation.constraints.NotNull; + +/** + */ +public class BatchDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider +{ + @JacksonInject + @NotNull + private BatchDataSegmentAnnouncer batchAnnouncer = null; + + @Override + public DataSegmentAnnouncer get() + { + return batchAnnouncer; + } +} diff --git a/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java similarity index 89% rename from client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java index 71eaaa37276..b9872ead48d 100644 --- a/client/src/main/java/com/metamx/druid/coordination/DataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.io.IOException; diff --git a/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..eeed92b045a --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentAnnouncerProvider.java @@ -0,0 +1,35 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSegmentAnnouncerProvider.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "legacy", value = LegacyDataSegmentAnnouncerProvider.class), + @JsonSubTypes.Type(name = "batch", value = BatchDataSegmentAnnouncerProvider.class) +}) +public interface DataSegmentAnnouncerProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java b/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java similarity index 95% rename from client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java rename to server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java index ce38bd6b0a0..ecb7f33b2ea 100644 --- a/client/src/main/java/com/metamx/druid/coordination/DruidServerMetadata.java +++ b/server/src/main/java/io/druid/server/coordination/DruidServerMetadata.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/coordination/LegacyDataSegmentAnnouncerProvider.java b/server/src/main/java/io/druid/server/coordination/LegacyDataSegmentAnnouncerProvider.java new file mode 100644 index 00000000000..f5a5dc40545 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordination/LegacyDataSegmentAnnouncerProvider.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; + +import javax.validation.constraints.NotNull; +import java.util.Arrays; + +/** + */ +public class LegacyDataSegmentAnnouncerProvider implements DataSegmentAnnouncerProvider +{ + @JacksonInject + @NotNull + private SingleDataSegmentAnnouncer singleAnnouncer = null; + + @JacksonInject + @NotNull + private BatchDataSegmentAnnouncer batchAnnouncer = null; + + @Override + public DataSegmentAnnouncer get() + { + return new MultipleDataSegmentAnnouncerDataSegmentAnnouncer( + Arrays.asList(singleAnnouncer, batchAnnouncer) + ); + } +} diff --git a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java similarity index 94% rename from client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java index 8bdc14e7d95..7595eb5095d 100644 --- a/client/src/main/java/com/metamx/druid/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/MultipleDataSegmentAnnouncerDataSegmentAnnouncer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.io.IOException; diff --git a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java similarity index 91% rename from client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java rename to server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java index 20e617a8e32..225ae22b20c 100644 --- a/client/src/main/java/com/metamx/druid/coordination/SingleDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.ZkPathsConfig; +import io.druid.client.DataSegment; +import io.druid.curator.announcement.Announcer; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; import java.io.IOException; diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/io/druid/server/http/InfoResource.java similarity index 98% rename from server/src/main/java/com/metamx/druid/http/InfoResource.java rename to server/src/main/java/io/druid/server/http/InfoResource.java index 03017425bd8..2eaed9527e5 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/io/druid/server/http/InfoResource.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.google.common.base.Function; import com.google.common.collect.Collections2; @@ -27,15 +27,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.inject.Inject; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidDataSource; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.InventoryView; -import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.rules.Rule; +import io.druid.client.DataSegment; +import io.druid.client.DruidDataSource; +import io.druid.client.DruidServer; +import io.druid.client.InventoryView; +import io.druid.client.indexing.IndexingServiceClient; import org.joda.time.Interval; import javax.annotation.Nullable; diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/io/druid/server/http/MasterMain.java similarity index 100% rename from server/src/main/java/com/metamx/druid/http/MasterMain.java rename to server/src/main/java/io/druid/server/http/MasterMain.java diff --git a/server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java b/server/src/main/java/io/druid/server/http/MasterRedirectInfo.java similarity index 52% rename from server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java rename to server/src/main/java/io/druid/server/http/MasterRedirectInfo.java index f3c263682df..b239112abae 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterRedirectInfo.java +++ b/server/src/main/java/io/druid/server/http/MasterRedirectInfo.java @@ -1,4 +1,23 @@ -package com.metamx.druid.http; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.http; import com.google.common.base.Throwables; import com.google.inject.Inject; diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/io/druid/server/http/MasterResource.java similarity index 97% rename from server/src/main/java/com/metamx/druid/http/MasterResource.java rename to server/src/main/java/io/druid/server/http/MasterResource.java index 5934fdd5f6e..385a3394220 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/io/druid/server/http/MasterResource.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.google.inject.Inject; import com.metamx.druid.master.DruidMaster; diff --git a/server/src/main/java/com/metamx/druid/http/MasterSegmentSettingsResource.java b/server/src/main/java/io/druid/server/http/MasterSegmentSettingsResource.java similarity index 61% rename from server/src/main/java/com/metamx/druid/http/MasterSegmentSettingsResource.java rename to server/src/main/java/io/druid/server/http/MasterSegmentSettingsResource.java index ea14c587f78..0205dfaaff4 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterSegmentSettingsResource.java +++ b/server/src/main/java/io/druid/server/http/MasterSegmentSettingsResource.java @@ -1,22 +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.http; + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.http; import com.metamx.druid.config.JacksonConfigManager; import com.metamx.druid.master.MasterSegmentSettings; diff --git a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java b/server/src/main/java/io/druid/server/http/RedirectFilter.java similarity index 96% rename from server/src/main/java/com/metamx/druid/http/RedirectFilter.java rename to server/src/main/java/io/druid/server/http/RedirectFilter.java index 1535e90ff71..671a2cb207e 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectFilter.java +++ b/server/src/main/java/io/druid/server/http/RedirectFilter.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.google.inject.Inject; import com.metamx.common.logger.Logger; diff --git a/server/src/main/java/com/metamx/druid/http/RedirectInfo.java b/server/src/main/java/io/druid/server/http/RedirectInfo.java similarity index 91% rename from server/src/main/java/com/metamx/druid/http/RedirectInfo.java rename to server/src/main/java/io/druid/server/http/RedirectInfo.java index d14839710fc..3efea33a146 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectInfo.java +++ b/server/src/main/java/io/druid/server/http/RedirectInfo.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import java.net.URL; diff --git a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java b/server/src/main/java/io/druid/server/http/RedirectServlet.java similarity index 96% rename from server/src/main/java/com/metamx/druid/http/RedirectServlet.java rename to server/src/main/java/io/druid/server/http/RedirectServlet.java index 7b4ad2aec53..2226fed3d83 100644 --- a/server/src/main/java/com/metamx/druid/http/RedirectServlet.java +++ b/server/src/main/java/io/druid/server/http/RedirectServlet.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.google.inject.Inject; import com.metamx.common.logger.Logger; diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java b/server/src/main/java/io/druid/server/http/SegmentToDrop.java similarity index 94% rename from server/src/main/java/com/metamx/druid/http/SegmentToDrop.java rename to server/src/main/java/io/druid/server/http/SegmentToDrop.java index 44e9d4a0d33..e3fa092e610 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToDrop.java +++ b/server/src/main/java/io/druid/server/http/SegmentToDrop.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java b/server/src/main/java/io/druid/server/http/SegmentToMove.java similarity index 94% rename from server/src/main/java/com/metamx/druid/http/SegmentToMove.java rename to server/src/main/java/io/druid/server/http/SegmentToMove.java index 85a6ab7227f..159afa0cda2 100644 --- a/server/src/main/java/com/metamx/druid/http/SegmentToMove.java +++ b/server/src/main/java/io/druid/server/http/SegmentToMove.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http; +package io.druid.server.http; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java b/server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java new file mode 100644 index 00000000000..079cb92e5bc --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.initialization.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; + +/** + */ +public class BatchDataSegmentAnnouncerConfig +{ + @JsonProperty + @Min(1) + private int segmentsPerNode = 50; + + @JsonProperty + @Max(1024 * 1024) + @Min(1024) + private long maxBytesPerNode = 512 * 1024; + + public int getSegmentsPerNode() + { + return segmentsPerNode; + } + + public long getMaxBytesPerNode() + { + return maxBytesPerNode; + } +} diff --git a/client/src/main/java/com/metamx/druid/initialization/ConfigModule.java b/server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java similarity index 91% rename from client/src/main/java/com/metamx/druid/initialization/ConfigModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java index 5d1245c23c2..0f696fb5261 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ConfigModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.config.Config; -import com.metamx.druid.guice.JsonConfigurator; -import com.metamx.druid.guice.LazySingleton; +import io.druid.guice.guice.JsonConfigurator; +import io.druid.guice.guice.LazySingleton; import org.skife.config.ConfigurationObjectFactory; import javax.validation.Validation; diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java b/server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java similarity index 90% rename from client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java rename to server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java index 4fc6c28360e..55ead420ff7 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorDiscoveryConfig.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java b/server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java similarity index 94% rename from client/src/main/java/com/metamx/druid/initialization/EmitterModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java index 889b440bf80..5573b4653cc 100644 --- a/client/src/main/java/com/metamx/druid/initialization/EmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.google.common.base.Supplier; import com.google.common.collect.Lists; @@ -33,12 +33,13 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.guice.ManageLifecycle; -import com.metamx.druid.guice.annotations.Self; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.guice.guice.LazySingleton; +import io.druid.guice.guice.ManageLifecycle; +import io.druid.guice.guice.annotations.Self; +import io.druid.server.DruidNode; import java.lang.annotation.Annotation; import java.util.List; diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java b/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java similarity index 95% rename from client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java rename to server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java index 2a9467b3961..97c36874afd 100644 --- a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterConfig.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java b/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java similarity index 92% rename from client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java index 7c2c9f2ca69..cf6478a1397 100644 --- a/client/src/main/java/com/metamx/druid/initialization/HttpEmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.google.common.base.Supplier; import com.google.inject.Binder; @@ -26,13 +26,13 @@ import com.google.inject.Provides; import com.google.inject.name.Named; import com.google.inject.util.Providers; import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.core.HttpPostEmitter; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.LazySingleton; +import io.druid.guice.guice.ManageLifecycle; import javax.annotation.Nullable; import javax.net.ssl.SSLContext; diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java similarity index 93% rename from client/src/main/java/com/metamx/druid/initialization/Initialization.java rename to server/src/main/java/io/druid/server/initialization/initialization/Initialization.java index a310c9c8184..dc3ea672d2a 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -28,11 +28,11 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.metamx.common.ISE; -import com.metamx.druid.guice.DruidGuiceExtensions; -import com.metamx.druid.guice.DruidSecondaryModule; -import com.metamx.druid.guice.annotations.Json; -import com.metamx.druid.guice.annotations.Smile; import com.metamx.druid.jackson.JacksonModule; +import io.druid.guice.guice.DruidGuiceExtensions; +import io.druid.guice.guice.DruidSecondaryModule; +import io.druid.guice.guice.annotations.Json; +import io.druid.guice.guice.annotations.Smile; import io.druid.initialization.DruidModule; import javax.annotation.Nullable; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java b/server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java new file mode 100644 index 00000000000..b16e73001e8 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.initialization.initialization; + +import com.google.inject.Injector; +import org.eclipse.jetty.server.Server; + +/** + */ +public interface JettyServerInitializer +{ + public void initialize(Server server, Injector injector); +} diff --git a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java b/server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java similarity index 82% rename from client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java index cf7da29b23e..c31aa7f5489 100644 --- a/client/src/main/java/com/metamx/druid/initialization/JettyServerModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java @@ -1,4 +1,23 @@ -package com.metamx.druid.initialization; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.initialization.initialization; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; @@ -18,14 +37,15 @@ import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.LazySingleton; -import com.metamx.druid.guice.annotations.Self; import com.sun.jersey.api.core.DefaultResourceConfig; import com.sun.jersey.api.core.ResourceConfig; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.spi.container.servlet.WebConfig; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.LazySingleton; +import io.druid.guice.guice.annotations.Self; +import io.druid.server.DruidNode; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.nio.SelectChannelConnector; diff --git a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java b/server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java similarity index 91% rename from client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java index 5e9b7bd71a7..ea1fa615384 100644 --- a/client/src/main/java/com/metamx/druid/initialization/LogEmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; @@ -25,11 +25,11 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.name.Named; -import com.metamx.druid.guice.JsonConfigProvider; -import com.metamx.druid.guice.ManageLifecycle; import com.metamx.emitter.core.Emitter; import com.metamx.emitter.core.LoggingEmitter; import com.metamx.emitter.core.LoggingEmitterConfig; +import io.druid.guice.guice.JsonConfigProvider; +import io.druid.guice.guice.ManageLifecycle; /** */ diff --git a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java b/server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java similarity index 97% rename from client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java rename to server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java index b9ae29c2d90..a82708ee0b9 100644 --- a/client/src/main/java/com/metamx/druid/initialization/PropertiesModule.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.google.common.base.Throwables; import com.google.common.io.Closeables; diff --git a/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java b/server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java similarity index 92% rename from client/src/main/java/com/metamx/druid/initialization/ServerConfig.java rename to server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java index 5610e476437..8894161fc33 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java similarity index 95% rename from client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java rename to server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java index 6f484c1bc63..4f0c7aeea9e 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.initialization; +package io.druid.server.initialization.initialization; import org.apache.curator.utils.ZKPaths; import org.skife.config.Config; diff --git a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java similarity index 96% rename from client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java rename to server/src/main/java/io/druid/server/log/EmittingRequestLogger.java index c551542374f..70df84450a6 100644 --- a/client/src/main/java/com/metamx/druid/http/log/EmittingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2013 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http.log; +package io.druid.server.log; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.http.RequestLogLine; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.query.Query; +import io.druid.server.RequestLogLine; import org.joda.time.DateTime; import java.util.Map; diff --git a/server/src/main/java/io/druid/server/log/EmittingRequestLoggerProvider.java b/server/src/main/java/io/druid/server/log/EmittingRequestLoggerProvider.java new file mode 100644 index 00000000000..5509b7c38be --- /dev/null +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLoggerProvider.java @@ -0,0 +1,55 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.log; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.metamx.emitter.service.ServiceEmitter; + +import javax.validation.constraints.NotNull; + +/** + */ +@JsonTypeName("emitter") +public class EmittingRequestLoggerProvider implements RequestLoggerProvider +{ + @JsonProperty + @NotNull + private String feed = null; + + @JacksonInject + @NotNull + private ServiceEmitter emitter = null; + + @Inject + public void injectMe(Injector injector) + { + System.out.println("YAYAYAYAYAYA!!!"); + } + + @Override + public RequestLogger get() + { + return new EmittingRequestLogger(emitter, feed); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java b/server/src/main/java/io/druid/server/log/FileRequestLogger.java similarity index 96% rename from client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java rename to server/src/main/java/io/druid/server/log/FileRequestLogger.java index 0891edbd713..ba450ce9c5f 100644 --- a/client/src/main/java/com/metamx/druid/http/log/FileRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/FileRequestLogger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http.log; +package io.druid.server.log; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; @@ -25,7 +25,7 @@ import com.google.common.io.Closeables; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.http.RequestLogLine; +import io.druid.server.RequestLogLine; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.MutableDateTime; diff --git a/server/src/main/java/io/druid/server/log/FileRequestLoggerProvider.java b/server/src/main/java/io/druid/server/log/FileRequestLoggerProvider.java new file mode 100644 index 00000000000..c1b67d509ad --- /dev/null +++ b/server/src/main/java/io/druid/server/log/FileRequestLoggerProvider.java @@ -0,0 +1,56 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.log; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.common.concurrent.ScheduledExecutorFactory; +import io.druid.guice.guice.annotations.Json; + +import javax.validation.constraints.NotNull; +import java.io.File; + +/** + */ +@JsonTypeName("file") +public class FileRequestLoggerProvider implements RequestLoggerProvider +{ + @JsonProperty + @NotNull + private File dir = null; + + @JacksonInject + @NotNull + private ScheduledExecutorFactory factory = null; + + + @JacksonInject + @NotNull + @Json + private ObjectMapper jsonMapper = null; + + @Override + public RequestLogger get() + { + return new FileRequestLogger(jsonMapper, factory.create(1, "RequestLogger-%s"), dir); + } +} diff --git a/server/src/main/java/io/druid/server/log/NoopRequestLogger.java b/server/src/main/java/io/druid/server/log/NoopRequestLogger.java new file mode 100644 index 00000000000..1088a13032a --- /dev/null +++ b/server/src/main/java/io/druid/server/log/NoopRequestLogger.java @@ -0,0 +1,33 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.log; + +import io.druid.server.RequestLogLine; + +/** + */ +public class NoopRequestLogger implements RequestLogger +{ + @Override + public void log(RequestLogLine requestLogLine) throws Exception + { + // This is a no op! + } +} diff --git a/server/src/main/java/io/druid/server/log/NoopRequestLoggerProvider.java b/server/src/main/java/io/druid/server/log/NoopRequestLoggerProvider.java new file mode 100644 index 00000000000..86d41241ab2 --- /dev/null +++ b/server/src/main/java/io/druid/server/log/NoopRequestLoggerProvider.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.log; + +/** + */ +public class NoopRequestLoggerProvider implements RequestLoggerProvider +{ + @Override + public RequestLogger get() + { + return new NoopRequestLogger(); + } +} diff --git a/client/src/main/java/com/metamx/druid/http/log/RequestLogger.java b/server/src/main/java/io/druid/server/log/RequestLogger.java similarity index 87% rename from client/src/main/java/com/metamx/druid/http/log/RequestLogger.java rename to server/src/main/java/io/druid/server/log/RequestLogger.java index ff3b0e84401..2d4c938252b 100644 --- a/client/src/main/java/com/metamx/druid/http/log/RequestLogger.java +++ b/server/src/main/java/io/druid/server/log/RequestLogger.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.http.log; +package io.druid.server.log; -import com.metamx.druid.http.RequestLogLine; +import io.druid.server.RequestLogLine; /** */ diff --git a/server/src/main/java/io/druid/server/log/RequestLoggerProvider.java b/server/src/main/java/io/druid/server/log/RequestLoggerProvider.java new file mode 100644 index 00000000000..236c2452034 --- /dev/null +++ b/server/src/main/java/io/druid/server/log/RequestLoggerProvider.java @@ -0,0 +1,32 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.log; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +/** + * A Marker interface for things that can provide a RequestLogger. This can be combined with jackson polymorphic serde + * to provide new RequestLogger implementations as plugins. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopRequestLoggerProvider.class) +public interface RequestLoggerProvider extends Provider +{ +} diff --git a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java b/server/src/main/java/io/druid/server/shard/shard/LinearShardSpec.java similarity index 54% rename from client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java rename to server/src/main/java/io/druid/server/shard/shard/LinearShardSpec.java index 4eb9c963672..5de078c369a 100644 --- a/client/src/main/java/com/metamx/druid/shard/LinearShardSpec.java +++ b/server/src/main/java/io/druid/server/shard/shard/LinearShardSpec.java @@ -1,4 +1,23 @@ -package com.metamx.druid.shard; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.shard.shard; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java b/server/src/main/java/io/druid/server/shard/shard/NoneShardSpec.java similarity index 94% rename from client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java rename to server/src/main/java/io/druid/server/shard/shard/NoneShardSpec.java index 940f2f5c600..f0dea959b1a 100644 --- a/client/src/main/java/com/metamx/druid/shard/NoneShardSpec.java +++ b/server/src/main/java/io/druid/server/shard/shard/NoneShardSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.metamx.druid.input.InputRow; import com.metamx.druid.partition.PartitionChunk; diff --git a/client/src/main/java/com/metamx/druid/shard/NumberedShardSpec.java b/server/src/main/java/io/druid/server/shard/shard/NumberedShardSpec.java similarity index 96% rename from client/src/main/java/com/metamx/druid/shard/NumberedShardSpec.java rename to server/src/main/java/io/druid/server/shard/shard/NumberedShardSpec.java index 325446cb25b..999afaf1f91 100644 --- a/client/src/main/java/com/metamx/druid/shard/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/server/shard/shard/NumberedShardSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java b/server/src/main/java/io/druid/server/shard/shard/ShardSpec.java similarity index 95% rename from client/src/main/java/com/metamx/druid/shard/ShardSpec.java rename to server/src/main/java/io/druid/server/shard/shard/ShardSpec.java index b5b778283b5..ce3655b8287 100644 --- a/client/src/main/java/com/metamx/druid/shard/ShardSpec.java +++ b/server/src/main/java/io/druid/server/shard/shard/ShardSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; diff --git a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/server/shard/shard/SingleDimensionShardSpec.java similarity index 97% rename from client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java rename to server/src/main/java/io/druid/server/shard/shard/SingleDimensionShardSpec.java index 98ff7f2eae1..dfe7b15ab43 100644 --- a/client/src/main/java/com/metamx/druid/shard/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/server/shard/shard/SingleDimensionShardSpec.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.input.InputRow; diff --git a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java b/server/src/main/java/io/druid/server/sql/SQLRunner.java similarity index 97% rename from client/src/main/java/com/metamx/druid/sql/SQLRunner.java rename to server/src/main/java/io/druid/server/sql/SQLRunner.java index c2c01574f0c..dcd399ae1af 100644 --- a/client/src/main/java/com/metamx/druid/sql/SQLRunner.java +++ b/server/src/main/java/io/druid/server/sql/SQLRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.sql; +package io.druid.server.sql; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -27,17 +27,17 @@ import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; -import com.metamx.druid.Druids; import com.metamx.druid.input.Row; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.query.group.GroupByQuery; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import com.metamx.druid.sql.antlr4.DruidSQLLexer; import com.metamx.druid.sql.antlr4.DruidSQLParser; +import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.timeseries.TimeseriesResultValue; import org.antlr.v4.runtime.ANTLRInputStream; import org.antlr.v4.runtime.CharStream; import org.antlr.v4.runtime.CommonTokenStream; diff --git a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java b/server/src/main/java/io/druid/storage/s3/S3Utils.java similarity index 97% rename from indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java rename to server/src/main/java/io/druid/storage/s3/S3Utils.java index 84ce35df947..044b3fa76c9 100644 --- a/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java +++ b/server/src/main/java/io/druid/storage/s3/S3Utils.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.common.s3; +package io.druid.storage.s3; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; diff --git a/server/src/test/java/com/metamx/druid/TestHelper.java b/server/src/test/java/com/metamx/druid/TestHelper.java index 8a15a73f724..7ad40f59ed7 100644 --- a/server/src/test/java/com/metamx/druid/TestHelper.java +++ b/server/src/test/java/com/metamx/druid/TestHelper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,7 +22,7 @@ package com.metamx.druid; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.result.Result; +import io.druid.query.Result; import org.junit.Assert; import java.util.Iterator; diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java index 92026a533e6..59e0fd0b117 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestDropTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,11 +22,10 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.shard.NoneShardSpec; - +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java index 62688974180..246fd66301a 100644 --- a/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/SegmentChangeRequestLoadTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,11 +22,10 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.shard.NoneShardSpec; - +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index d375ea370b0..9635109bdd9 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -35,31 +35,31 @@ import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; import com.metamx.common.guava.YieldingSequenceBase; -import com.metamx.druid.Druids; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.ReferenceCountingSegment; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; -import com.metamx.druid.query.ConcatQueryRunner; -import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; -import com.metamx.druid.query.search.SearchQuery; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; -import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.client.DataSegment; import io.druid.granularity.QueryGranularity; +import io.druid.query.ConcatQueryRunner; +import io.druid.query.Druids; +import io.druid.query.NoopQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchQuery; +import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; diff --git a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java index f81b7cd91c7..ce269821d40 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ZkCoordinatorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,18 +24,21 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.curator.CuratorTestBase; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.CacheTestSegmentLoader; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.NoopQueryRunnerFactoryConglomerate; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.curator.CuratorTestBase; +import io.druid.curator.announcement.Announcer; +import io.druid.segment.IndexIO; +import io.druid.server.coordination.DataSegmentAnnouncer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.SingleDataSegmentAnnouncer; +import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; diff --git a/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java index dee1229e388..ff687a84348 100644 --- a/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java +++ b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,6 +21,7 @@ package com.metamx.druid.index; import com.google.common.base.Throwables; import io.druid.segment.QueryableIndex; +import io.druid.segment.ReferenceCountingSegment; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; import junit.framework.Assert; diff --git a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java index d83321bb93c..d2a9b74cfad 100644 --- a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java +++ b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterBonusTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,29 +24,29 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; -import com.metamx.druid.Druids; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.index.v1.IncrementalIndexSchema; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.MapBasedInputRow; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.filter.SpatialDimFilter; -import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.SpatialDimFilter; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java index dea44e0c44f..91915dff882 100644 --- a/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java +++ b/server/src/test/java/com/metamx/druid/index/brita/SpatialFilterTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,29 +24,29 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; -import com.metamx.druid.Druids; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.v1.IncrementalIndex; -import com.metamx.druid.index.v1.IncrementalIndexSchema; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.IndexMerger; -import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.input.MapBasedInputRow; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.query.filter.SpatialDimFilter; -import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.SpatialDimFilter; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.SpatialDimensionSchema; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/ConciseOffsetTest.java b/server/src/test/java/com/metamx/druid/index/v1/ConciseOffsetTest.java index f32ecae3a47..0452bf3be91 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/ConciseOffsetTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/ConciseOffsetTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,7 +19,8 @@ package com.metamx.druid.index.v1; -import com.metamx.druid.index.v1.processing.Offset; +import io.druid.segment.ConciseOffset; +import io.druid.segment.data.Offset; import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.junit.Assert; diff --git a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java index 6bbfd6fbbba..ea412309df6 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/EmptyIndexTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,7 +23,12 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; +import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndex; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index e5f3182a12c..f2fa2849ffd 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -26,7 +26,11 @@ import com.google.common.io.Files; import com.metamx.druid.input.MapBasedInputRow; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.data.IncrementalIndexTest; +import io.druid.segment.incremental.IncrementalIndex; import junit.framework.Assert; import org.apache.commons.io.FileUtils; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java index 3f82a58c491..dc4878a6912 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java +++ b/server/src/test/java/com/metamx/druid/index/v1/TestIndex.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,13 +25,16 @@ import com.google.common.io.CharStreams; import com.google.common.io.InputSupplier; import com.google.common.io.LineProcessor; import com.metamx.common.logger.Logger; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.indexer.data.DelimitedDataSpec; -import com.metamx.druid.indexer.data.StringInputRowParser; -import com.metamx.druid.indexer.data.TimestampSpec; +import io.druid.data.input.DelimitedDataSpec; +import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.TimestampSpec; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.incremental.IncrementalIndex; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/index/v1/processing/IntersectingOffsetTest.java b/server/src/test/java/com/metamx/druid/index/v1/processing/IntersectingOffsetTest.java index 3d5d5100b84..0001c610ab3 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/processing/IntersectingOffsetTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/processing/IntersectingOffsetTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,6 +20,9 @@ package com.metamx.druid.index.v1.processing; import com.google.common.collect.Lists; +import io.druid.segment.data.ArrayBasedOffset; +import io.druid.segment.data.IntersectingOffset; +import io.druid.segment.data.Offset; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/index/v1/processing/UnioningOffsetTest.java b/server/src/test/java/com/metamx/druid/index/v1/processing/UnioningOffsetTest.java index f470e5ae37a..a3dc59aaab0 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/processing/UnioningOffsetTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/processing/UnioningOffsetTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,6 +20,9 @@ package com.metamx.druid.index.v1.processing; import com.google.common.collect.Lists; +import io.druid.segment.data.ArrayBasedOffset; +import io.druid.segment.data.Offset; +import io.druid.segment.data.UnioningOffset; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java index 0b80a7fc5b6..f429c36096f 100644 --- a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,7 +20,7 @@ package com.metamx.druid.loading; import com.metamx.common.MapUtils; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import io.druid.segment.QueryableIndex; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; diff --git a/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java index 2337f882098..c05462d934d 100644 --- a/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java +++ b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java @@ -1,9 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.common.collect.ImmutableMap; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.segment.IndexIO; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/loading/SingleSegmentLoaderTest.java b/server/src/test/java/com/metamx/druid/loading/SingleSegmentLoaderTest.java index f82311609ac..62e82b6ad91 100644 --- a/server/src/test/java/com/metamx/druid/loading/SingleSegmentLoaderTest.java +++ b/server/src/test/java/com/metamx/druid/loading/SingleSegmentLoaderTest.java @@ -1,7 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.common.collect.ImmutableMap; -import com.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java index 8513c157de3..45edbafc172 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTest.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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; @@ -23,9 +23,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.server.shard.shard.NoneShardSpec; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java index 3b5865fb361..ea72386293c 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterBalancerTester.java @@ -1,7 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; public class DruidMasterBalancerTester extends DruidMasterBalancer { diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java index 27dc46636ac..f91aa192c75 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterRuleRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -25,17 +25,17 @@ import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; import com.metamx.druid.db.DatabaseRuleManager; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.master.rules.IntervalDropRule; import com.metamx.druid.master.rules.IntervalLoadRule; import com.metamx.druid.master.rules.Rule; -import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.segment.IndexIO; +import io.druid.server.shard.shard.NoneShardSpec; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; 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 04caa13a8a4..e287d2f7aac 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -22,9 +22,9 @@ package com.metamx.druid.master; 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 com.metamx.druid.shard.LinearShardSpec; +import io.druid.client.DataSegment; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.server.shard.shard.LinearShardSpec; import junit.framework.Assert; import org.joda.time.Interval; import org.junit.Test; 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 af8840d948e..51adf2bb686 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,13 +21,13 @@ package com.metamx.druid.master; import com.google.common.collect.MapMaker; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.client.SingleServerInventoryView; -import com.metamx.druid.curator.inventory.InventoryManagerConfig; import com.metamx.druid.db.DatabaseSegmentManager; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.metrics.NoopServiceEmitter; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.client.SingleServerInventoryView; +import io.druid.curator.inventory.InventoryManagerConfig; +import io.druid.server.initialization.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.easymock.EasyMock; import org.joda.time.Duration; diff --git a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java index 366cde87e1d..62b196aac6d 100644 --- a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java +++ b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java @@ -1,6 +1,25 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.metamx.druid.client.DataSegment; +import io.druid.client.DataSegment; import java.util.concurrent.ConcurrentSkipListSet; diff --git a/server/src/test/java/com/metamx/druid/master/ReservoirSegmentSamplerTest.java b/server/src/test/java/com/metamx/druid/master/ReservoirSegmentSamplerTest.java index 7b768e11172..33fafa72358 100644 --- a/server/src/test/java/com/metamx/druid/master/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/com/metamx/druid/master/ReservoirSegmentSamplerTest.java @@ -1,29 +1,29 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.server.shard.shard.NoneShardSpec; import junit.framework.Assert; import org.easymock.EasyMock; import org.joda.time.DateTime; diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java index c6bceb08e5e..b214721083c 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodDropRuleTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,8 +19,8 @@ package com.metamx.druid.master.rules; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java b/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java index 3944d96ecb9..c09e5be3e19 100644 --- a/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/com/metamx/druid/master/rules/PeriodLoadRuleTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,8 +19,8 @@ package com.metamx.druid.master.rules; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.shard.NoneShardSpec; +import io.druid.client.DataSegment; +import io.druid.server.shard.shard.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; diff --git a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java index 8d46099a016..f24c7aa4d31 100644 --- a/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java +++ b/server/src/test/java/com/metamx/druid/query/QueryRunnerTestHelper.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -20,25 +20,26 @@ package com.metamx.druid.query; import com.google.common.collect.Lists; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.aggregation.post.ArithmeticPostAggregator; -import com.metamx.druid.aggregation.post.ConstantPostAggregator; -import com.metamx.druid.aggregation.post.FieldAccessPostAggregator; -import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndexSegment; -import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.TestIndex; import io.druid.granularity.QueryGranularity; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.aggregation.post.FieldAccessPostAggregator; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; +import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index d9a6fea2c49..fc1e93c8720 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -31,26 +31,29 @@ import com.google.common.collect.Ordering; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; import com.metamx.druid.collect.StupidPool; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.query.dimension.DefaultDimensionSpec; -import com.metamx.druid.query.dimension.DimensionSpec; -import com.metamx.druid.query.filter.RegexDimFilter; -import com.metamx.druid.query.group.having.EqualToHavingSpec; -import com.metamx.druid.query.group.having.GreaterThanHavingSpec; -import com.metamx.druid.query.group.having.OrHavingSpec; -import com.metamx.druid.query.group.orderby.DefaultLimitSpec; -import com.metamx.druid.query.group.orderby.LimitSpec; -import com.metamx.druid.query.group.orderby.OrderByColumnSpec; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.RegexDimFilter; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.groupby.having.EqualToHavingSpec; +import io.druid.query.groupby.having.GreaterThanHavingSpec; +import io.druid.query.groupby.having.OrHavingSpec; +import io.druid.query.groupby.orderby.DefaultLimitSpec; +import io.druid.query.groupby.orderby.LimitSpec; +import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java index b8fe4b60d72..6fc14793192 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByTimeseriesQueryRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -29,12 +29,15 @@ import com.metamx.druid.collect.StupidPool; import com.metamx.druid.input.MapBasedRow; import com.metamx.druid.input.Row; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerTest; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.query.Query; import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java index 94f64540149..2125f9e5b8e 100644 --- a/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java +++ b/server/src/test/java/com/metamx/druid/query/metadata/SegmentAnalyzerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,13 +21,16 @@ package com.metamx.druid.query.metadata; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; -import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.v1.TestIndex; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.query.segment.QuerySegmentSpecs; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.metadata.metadata.ColumnAnalysis; +import io.druid.query.metadata.metadata.SegmentAnalysis; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.spec.QuerySegmentSpecs; +import io.druid.segment.IncrementalIndexSegment; +import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.column.ValueType; import junit.framework.Assert; diff --git a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java index b0512978b77..11adc434cc1 100644 --- a/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/search/SearchQueryRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -24,12 +24,17 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Druids; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.SearchResultValue; +import io.druid.query.Druids; import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.filter.DimFilter; +import io.druid.query.search.SearchQueryQueryToolChest; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.FragmentSearchQuerySpec; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.search.search.SearchQueryConfig; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index ed45fee9508..03813081630 100644 --- a/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,11 +21,12 @@ package com.metamx.druid.query.timeboundary; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Druids; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeBoundaryResultValue; +import io.druid.query.Druids; import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index 5325212d2f3..8636580489c 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -1,23 +1,43 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.query.timeseries; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Druids; -import com.metamx.druid.aggregation.CountAggregatorFactory; -import com.metamx.druid.index.IncrementalIndexSegment; -import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.input.MapBasedInputRow; -import com.metamx.druid.query.FinalizeResultsQueryRunner; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.aggregation.CountAggregatorFactory; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.incremental.IncrementalIndex; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 71824e90729..ceea3d448ab 100644 --- a/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -22,23 +22,24 @@ package com.metamx.druid.query.timeseries; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Druids; import com.metamx.druid.TestHelper; -import com.metamx.druid.aggregation.LongSumAggregatorFactory; -import com.metamx.druid.aggregation.MaxAggregatorFactory; -import com.metamx.druid.aggregation.MinAggregatorFactory; import com.metamx.druid.query.QueryRunnerTestHelper; -import com.metamx.druid.query.filter.AndDimFilter; -import com.metamx.druid.query.filter.DimFilter; -import com.metamx.druid.query.filter.RegexDimFilter; -import com.metamx.druid.result.Result; -import com.metamx.druid.result.TimeseriesResultValue; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; import io.druid.query.QueryRunner; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.aggregation.MinAggregatorFactory; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.RegexDimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Interval; diff --git a/server/src/test/java/com/metamx/druid/utils/DruidMasterBalancerProfiler.java b/server/src/test/java/com/metamx/druid/utils/DruidMasterBalancerProfiler.java index 482274c0b28..e85cea29e34 100644 --- a/server/src/test/java/com/metamx/druid/utils/DruidMasterBalancerProfiler.java +++ b/server/src/test/java/com/metamx/druid/utils/DruidMasterBalancerProfiler.java @@ -1,21 +1,21 @@ /* -* 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. -*/ + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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.utils; @@ -25,8 +25,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MinMaxPriorityQueue; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DruidServer; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.master.DruidCluster; import com.metamx.druid.master.DruidMaster; @@ -42,9 +40,11 @@ import com.metamx.druid.master.SegmentReplicantLookup; import com.metamx.druid.master.ServerHolder; import com.metamx.druid.master.rules.PeriodLoadRule; import com.metamx.druid.master.rules.Rule; -import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.server.shard.shard.NoneShardSpec; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java b/server/src/test/java/io/druid/client/DataSegmentTest.java similarity index 97% rename from client/src/test/java/com/metamx/druid/client/DataSegmentTest.java rename to server/src/test/java/io/druid/client/DataSegmentTest.java index 7532aeb1424..fe327f353f0 100644 --- a/client/src/test/java/com/metamx/druid/client/DataSegmentTest.java +++ b/server/src/test/java/io/druid/client/DataSegmentTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,10 +25,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.shard.NoneShardSpec; -import com.metamx.druid.shard.SingleDimensionShardSpec; +import io.druid.segment.IndexIO; +import io.druid.server.shard.shard.NoneShardSpec; +import io.druid.server.shard.shard.SingleDimensionShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/client/cache/ByteCountingLRUMapTest.java b/server/src/test/java/io/druid/client/cache/ByteCountingLRUMapTest.java similarity index 96% rename from client/src/test/java/com/metamx/druid/client/cache/ByteCountingLRUMapTest.java rename to server/src/test/java/io/druid/client/cache/ByteCountingLRUMapTest.java index fac858d2192..5edf6a669e6 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/ByteCountingLRUMapTest.java +++ b/server/src/test/java/io/druid/client/cache/ByteCountingLRUMapTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import org.junit.Assert; import org.junit.Before; diff --git a/client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java b/server/src/test/java/io/druid/client/cache/MapCacheTest.java similarity index 96% rename from client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java rename to server/src/test/java/io/druid/client/cache/MapCacheTest.java index 23a3bd1d641..fbff6f9fe2f 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MapCacheTest.java +++ b/server/src/test/java/io/druid/client/cache/MapCacheTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.primitives.Ints; import org.junit.Assert; diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java b/server/src/test/java/io/druid/client/cache/MemcachedCacheBenchmark.java similarity index 81% rename from client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java rename to server/src/test/java/io/druid/client/cache/MemcachedCacheBenchmark.java index c63e98e8988..7ef6047d9ac 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheBenchmark.java +++ b/server/src/test/java/io/druid/client/cache/MemcachedCacheBenchmark.java @@ -1,4 +1,23 @@ -package com.metamx.druid.client.cache; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.client.cache; import com.google.caliper.Param; import com.google.caliper.Runner; diff --git a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java b/server/src/test/java/io/druid/client/cache/MemcachedCacheTest.java similarity index 99% rename from client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java rename to server/src/test/java/io/druid/client/cache/MemcachedCacheTest.java index 23ca0ea9693..55c85f967f0 100644 --- a/client/src/test/java/com/metamx/druid/client/cache/MemcachedCacheTest.java +++ b/server/src/test/java/io/druid/client/cache/MemcachedCacheTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.cache; +package io.druid.client.cache; import com.google.common.collect.Lists; import com.google.common.collect.Maps; diff --git a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java similarity index 90% rename from client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java rename to server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index bb7bc307808..5b0b08b2c05 100644 --- a/client/src/test/java/com/metamx/druid/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client; +package io.druid.client.client; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; @@ -26,13 +26,16 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; -import com.metamx.druid.coordination.BatchDataSegmentAnnouncer; -import com.metamx.druid.coordination.DruidServerMetadata; -import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import io.druid.client.BatchServerInventoryView; +import io.druid.client.DataSegment; +import io.druid.client.DruidServer; +import io.druid.curator.PotentiallyGzippedCompressionProvider; +import io.druid.curator.announcement.Announcer; +import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.initialization.ZkPathsConfig; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/client/src/test/java/com/metamx/druid/client/selector/ServerSelectorTest.java b/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java similarity index 90% rename from client/src/test/java/com/metamx/druid/client/selector/ServerSelectorTest.java rename to server/src/test/java/io/druid/client/selector/ServerSelectorTest.java index 9009ec3d246..315f53690df 100644 --- a/client/src/test/java/com/metamx/druid/client/selector/ServerSelectorTest.java +++ b/server/src/test/java/io/druid/client/selector/ServerSelectorTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,22 +17,22 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.client.selector; +package io.druid.client.selector; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.SettableFuture; -import com.metamx.druid.Druids; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.client.DirectDruidClient; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.query.ReflectionQueryToolChestWarehouse; -import com.metamx.druid.query.timeboundary.TimeBoundaryQuery; -import com.metamx.druid.shard.NoneShardSpec; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.RequestBuilder; +import io.druid.client.DataSegment; +import io.druid.client.DirectDruidClient; +import io.druid.query.Druids; +import io.druid.query.ReflectionQueryToolChestWarehouse; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.server.shard.shard.NoneShardSpec; import junit.framework.Assert; import org.easymock.EasyMock; import org.jboss.netty.handler.codec.http.HttpMethod; diff --git a/client/src/test/java/com/metamx/druid/curator/CuratorTestBase.java b/server/src/test/java/io/druid/curator/CuratorTestBase.java similarity index 54% rename from client/src/test/java/com/metamx/druid/curator/CuratorTestBase.java rename to server/src/test/java/io/druid/curator/CuratorTestBase.java index dcc0cfda60e..a850d756ea3 100644 --- a/client/src/test/java/com/metamx/druid/curator/CuratorTestBase.java +++ b/server/src/test/java/io/druid/curator/CuratorTestBase.java @@ -1,4 +1,23 @@ -package com.metamx.druid.curator; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator; import com.google.common.io.Closeables; import org.apache.curator.framework.CuratorFramework; diff --git a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java b/server/src/test/java/io/druid/curator/announcement/AnnouncerTest.java similarity index 98% rename from client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java rename to server/src/test/java/io/druid/curator/announcement/AnnouncerTest.java index c8c21cc52a6..9321385a711 100644 --- a/client/src/test/java/com/metamx/druid/curator/announcement/AnnouncerTest.java +++ b/server/src/test/java/io/druid/curator/announcement/AnnouncerTest.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.curator.announcement; +package io.druid.curator.announcement; import com.google.common.collect.Sets; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.curator.CuratorTestBase; +import io.druid.curator.CuratorTestBase; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorEvent; import org.apache.curator.framework.api.CuratorEventType; diff --git a/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java b/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java similarity index 88% rename from client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java rename to server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java index da72db06048..548b1c7e5c9 100644 --- a/client/src/test/java/com/metamx/druid/curator/inventory/CuratorInventoryManagerTest.java +++ b/server/src/test/java/io/druid/curator/inventory/CuratorInventoryManagerTest.java @@ -1,10 +1,28 @@ -package com.metamx.druid.curator.inventory; +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.curator.inventory; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.metamx.druid.concurrent.Execs; -import com.metamx.druid.curator.CuratorTestBase; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.api.CuratorEvent; import org.apache.curator.framework.api.CuratorEventType; @@ -21,7 +39,7 @@ import java.util.concurrent.ExecutorService; /** */ -public class CuratorInventoryManagerTest extends CuratorTestBase +public class CuratorInventoryManagerTest extends io.druid.curator.CuratorTestBase { private ExecutorService exec; diff --git a/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java similarity index 92% rename from client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java rename to server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index f8d7abed828..1ba6f13c15f 100644 --- a/client/src/test/java/com/metamx/druid/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.coordination; +package io.druid.server.coordination.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,12 +25,14 @@ import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; -import com.metamx.druid.curator.announcement.Announcer; -import com.metamx.druid.initialization.BatchDataSegmentAnnouncerConfig; -import com.metamx.druid.initialization.ZkPathsConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import io.druid.client.DataSegment; +import io.druid.curator.PotentiallyGzippedCompressionProvider; +import io.druid.curator.announcement.Announcer; +import io.druid.server.coordination.BatchDataSegmentAnnouncer; +import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.initialization.ZkPathsConfig; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/client/src/test/java/com/metamx/druid/shard/NumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/shard/NumberedShardSpecTest.java similarity index 97% rename from client/src/test/java/com/metamx/druid/shard/NumberedShardSpecTest.java rename to server/src/test/java/io/druid/server/shard/shard/NumberedShardSpecTest.java index 15a9c898922..1fc9d01bf87 100644 --- a/client/src/test/java/com/metamx/druid/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/shard/NumberedShardSpecTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; diff --git a/client/src/test/java/com/metamx/druid/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/io/druid/server/shard/shard/SingleDimensionShardSpecTest.java similarity index 97% rename from client/src/test/java/com/metamx/druid/shard/SingleDimensionShardSpecTest.java rename to server/src/test/java/io/druid/server/shard/shard/SingleDimensionShardSpecTest.java index 217b18d9be7..a5a9b79e89f 100644 --- a/client/src/test/java/com/metamx/druid/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/shard/SingleDimensionShardSpecTest.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.shard; +package io.druid.server.shard.shard; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -31,7 +31,6 @@ import com.metamx.druid.input.MapBasedInputRow; import org.junit.Assert; import org.junit.Test; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; import java.util.Map; diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 14339320320..3c40c24f14b 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,23 +21,23 @@ package io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; -import com.metamx.druid.client.cache.CacheMonitor; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.guice.BrokerModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryToolChestModule; -import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.ServerViewModule; -import com.metamx.druid.guice.annotations.Client; -import com.metamx.druid.http.ClientQuerySegmentWalker; -import com.metamx.druid.http.StatusResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Command; +import io.druid.client.cache.CacheMonitor; +import io.druid.curator.CuratorModule; +import io.druid.guice.QueryToolChestModule; +import io.druid.guice.guice.BrokerModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.QueryableModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.ServerViewModule; +import io.druid.guice.guice.annotations.Client; +import io.druid.server.ClientQuerySegmentWalker; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerModule; /** */ diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index f8466abf057..2231fe39517 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -1,30 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.curator.discovery.DiscoveryModule; -import com.metamx.druid.guice.CoordinatorModule; -import com.metamx.druid.guice.DbConnectorModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.IndexingServiceDiscoveryModule; -import com.metamx.druid.guice.JacksonConfigManagerModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.ServerViewModule; -import com.metamx.druid.guice.annotations.Self; -import com.metamx.druid.http.InfoResource; -import com.metamx.druid.http.MasterResource; -import com.metamx.druid.http.RedirectFilter; -import com.metamx.druid.http.StatusResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerInitializer; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Command; +import io.druid.curator.CuratorModule; +import io.druid.curator.discovery.DiscoveryModule; +import io.druid.guice.guice.CoordinatorModule; +import io.druid.guice.guice.DbConnectorModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.IndexingServiceDiscoveryModule; +import io.druid.guice.guice.JacksonConfigManagerModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.ServerViewModule; +import io.druid.guice.guice.annotations.Self; +import io.druid.server.StatusResource; +import io.druid.server.http.InfoResource; +import io.druid.server.http.MasterResource; +import io.druid.server.http.RedirectFilter; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerInitializer; +import io.druid.server.initialization.initialization.JettyServerModule; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 2f8d521d5be..fef6ef57385 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -23,25 +23,25 @@ import com.google.inject.Injector; import com.metamx.common.logger.Logger; import com.metamx.druid.coordination.ServerManager; import com.metamx.druid.coordination.ZkCoordinator; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.guice.AWSModule; -import com.metamx.druid.guice.AnnouncerModule; -import com.metamx.druid.guice.DataSegmentPullerModule; -import com.metamx.druid.guice.DruidProcessingModule; -import com.metamx.druid.guice.HistoricalModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryRunnerFactoryModule; -import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.StorageNodeModule; -import com.metamx.druid.http.StatusResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import com.metamx.druid.metrics.ServerMonitor; import io.airlift.command.Command; +import io.druid.curator.CuratorModule; +import io.druid.guice.guice.AWSModule; +import io.druid.guice.guice.AnnouncerModule; +import io.druid.guice.guice.DataSegmentPullerModule; +import io.druid.guice.guice.DruidProcessingModule; +import io.druid.guice.guice.HistoricalModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.QueryRunnerFactoryModule; +import io.druid.guice.guice.QueryableModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.StorageNodeModule; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerModule; /** */ diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 513600f0b10..1c9cac9cffe 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -1,24 +1,43 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.guice.AWSModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.MiddleManagerModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.TaskLogsModule; -import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.worker.WorkerTaskMonitor; import com.metamx.druid.indexing.worker.http.WorkerResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerInitializer; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Command; +import io.druid.curator.CuratorModule; +import io.druid.guice.guice.AWSModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.MiddleManagerModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.TaskLogsModule; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerInitializer; +import io.druid.server.initialization.initialization.JettyServerModule; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 46bc734138d..298e06edb4b 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -22,26 +22,26 @@ package io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.curator.discovery.DiscoveryModule; -import com.metamx.druid.guice.AWSModule; -import com.metamx.druid.guice.DbConnectorModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.JacksonConfigManagerModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.OverlordModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.TaskLogsModule; -import com.metamx.druid.http.RedirectFilter; -import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.TaskMaster; import com.metamx.druid.indexing.coordinator.http.IndexerCoordinatorResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerInitializer; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Command; +import io.druid.curator.CuratorModule; +import io.druid.curator.discovery.DiscoveryModule; +import io.druid.guice.guice.AWSModule; +import io.druid.guice.guice.DbConnectorModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.JacksonConfigManagerModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.OverlordModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.TaskLogsModule; +import io.druid.server.StatusResource; +import io.druid.server.http.RedirectFilter; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerInitializer; +import io.druid.server.initialization.initialization.JettyServerModule; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 167c019c075..d424dd5344e 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -23,34 +23,34 @@ import com.google.common.base.Throwables; import com.google.inject.Injector; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.curator.discovery.DiscoveryModule; -import com.metamx.druid.guice.AWSModule; -import com.metamx.druid.guice.AnnouncerModule; -import com.metamx.druid.guice.DataSegmentPusherModule; -import com.metamx.druid.guice.DruidProcessingModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.IndexingServiceDiscoveryModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.PeonModule; -import com.metamx.druid.guice.QueryRunnerFactoryModule; -import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.ServerViewModule; -import com.metamx.druid.guice.StorageNodeModule; -import com.metamx.druid.http.StatusResource; import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; import com.metamx.druid.indexing.worker.executor.ChatHandlerResource; import com.metamx.druid.indexing.worker.executor.ExecutorLifecycle; import com.metamx.druid.indexing.worker.executor.ExecutorLifecycleConfig; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.log.LogLevelAdjuster; import com.metamx.druid.metrics.MetricsModule; import io.airlift.command.Arguments; import io.airlift.command.Command; import io.airlift.command.Option; +import io.druid.curator.CuratorModule; +import io.druid.curator.discovery.DiscoveryModule; +import io.druid.guice.guice.AWSModule; +import io.druid.guice.guice.AnnouncerModule; +import io.druid.guice.guice.DataSegmentPusherModule; +import io.druid.guice.guice.DruidProcessingModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.IndexingServiceDiscoveryModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.PeonModule; +import io.druid.guice.guice.QueryRunnerFactoryModule; +import io.druid.guice.guice.QueryableModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.ServerViewModule; +import io.druid.guice.guice.StorageNodeModule; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerModule; import java.io.File; import java.util.List; diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 080397a1a76..e5c5c7a6fe0 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,27 +21,27 @@ package io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.CuratorModule; -import com.metamx.druid.guice.AWSModule; -import com.metamx.druid.guice.AnnouncerModule; -import com.metamx.druid.guice.DataSegmentPusherModule; -import com.metamx.druid.guice.DbConnectorModule; -import com.metamx.druid.guice.DruidProcessingModule; -import com.metamx.druid.guice.HttpClientModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryRunnerFactoryModule; -import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.RealtimeModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.ServerViewModule; -import com.metamx.druid.guice.StorageNodeModule; -import com.metamx.druid.http.StatusResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.metrics.MetricsModule; import com.metamx.druid.realtime.RealtimeManager; import io.airlift.command.Command; +import io.druid.curator.CuratorModule; +import io.druid.guice.guice.AWSModule; +import io.druid.guice.guice.AnnouncerModule; +import io.druid.guice.guice.DataSegmentPusherModule; +import io.druid.guice.guice.DbConnectorModule; +import io.druid.guice.guice.DruidProcessingModule; +import io.druid.guice.guice.HttpClientModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.QueryRunnerFactoryModule; +import io.druid.guice.guice.QueryableModule; +import io.druid.guice.guice.RealtimeModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.ServerViewModule; +import io.druid.guice.guice.StorageNodeModule; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerModule; /** */ diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 1a6f9f97381..0796dda3b04 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -21,19 +21,19 @@ package io.druid.cli; import com.google.inject.Injector; import com.metamx.common.logger.Logger; -import com.metamx.druid.guice.DruidProcessingModule; -import com.metamx.druid.guice.LifecycleModule; -import com.metamx.druid.guice.QueryRunnerFactoryModule; -import com.metamx.druid.guice.QueryableModule; -import com.metamx.druid.guice.ServerModule; -import com.metamx.druid.guice.StorageNodeModule; -import com.metamx.druid.http.StatusResource; -import com.metamx.druid.initialization.EmitterModule; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.JettyServerModule; import com.metamx.druid.realtime.RealtimeManager; import druid.examples.guice.RealtimeExampleModule; import io.airlift.command.Command; +import io.druid.guice.guice.DruidProcessingModule; +import io.druid.guice.guice.LifecycleModule; +import io.druid.guice.guice.QueryRunnerFactoryModule; +import io.druid.guice.guice.QueryableModule; +import io.druid.guice.guice.ServerModule; +import io.druid.guice.guice.StorageNodeModule; +import io.druid.server.StatusResource; +import io.druid.server.initialization.initialization.EmitterModule; +import io.druid.server.initialization.initialization.Initialization; +import io.druid.server.initialization.initialization.JettyServerModule; /** */ diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index 78ea25219e1..0e7fc170a1f 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -1,9 +1,28 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import com.metamx.druid.http.QueryServlet; -import com.metamx.druid.initialization.JettyServerInitializer; +import io.druid.server.QueryServlet; +import io.druid.server.initialization.initialization.JettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; From bd0756e36021452c8d13b92af26a819b973be5c6 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 30 Aug 2013 18:58:28 -0500 Subject: [PATCH 60/92] More stuff moved, things still compiling and tests still passing. Yay! --- common/pom.xml | 3 +- .../druid/config/ConfigManagerConfig.java | 20 ------- .../com/metamx/druid/config/ConfigSerde.java | 9 --- .../com/metamx/druid/utils/SocketUtil.java | 39 ------------- .../druid/collections}/CombiningIterable.java | 4 +- .../druid/collections}/CombiningIterator.java | 4 +- .../druid/collections}/CountingMap.java | 4 +- .../druid/collections}/IntList.java | 4 +- .../collections}/OrderedMergeIterator.java | 4 +- .../collections}/OrderedMergeSequence.java | 4 +- .../druid/collections}/ResourceHolder.java | 4 +- .../druid/collections}/StupidPool.java | 4 +- .../collections}/StupidResourceHolder.java | 4 +- .../druid/common}/config/ConfigManager.java | 23 +++++++- .../common/config/ConfigManagerConfig.java | 39 +++++++++++++ .../io/druid/common/config/ConfigSerde.java | 28 +++++++++ .../common}/config/JacksonConfigManager.java | 4 +- .../common}/guava/CombiningSequence.java | 4 +- .../druid/common}/guava/DSuppliers.java | 2 +- .../common}/guava/FileOutputSupplier.java | 4 +- .../druid/common}/guava/GuavaUtils.java | 4 +- .../druid/common}/guava/Runnables.java | 4 +- .../common}/guava/ThreadRenamingCallable.java | 4 +- .../common}/guava/ThreadRenamingRunnable.java | 4 +- .../druid/common}/utils/CompressionUtils.java | 4 +- .../druid/common}/utils/JodaUtils.java | 4 +- .../druid/common}/utils/PropUtils.java | 4 +- .../druid/common}/utils/SerializerUtils.java | 6 +- .../io/druid/common/utils/SocketUtil.java | 58 +++++++++++++++++++ .../metamx => io}/druid/concurrent/Execs.java | 4 +- .../druid/data}/input/InputRow.java | 4 +- .../druid/data}/input/MapBasedInputRow.java | 4 +- .../druid/data}/input/MapBasedRow.java | 5 +- .../druid => io/druid/data}/input/Row.java | 4 +- .../druid => io/druid/data}/input/Rows.java | 4 +- .../metamx => io}/druid/db/DbConnector.java | 4 +- .../druid/db/DbConnectorConfig.java | 4 +- .../druid/db/DbTablesConfig.java | 21 ++++++- .../druid/guice/guice/DbConnectorModule.java | 6 +- .../guice/JacksonConfigManagerModule.java | 10 ++-- .../guice/guice/JacksonConfigProvider.java | 4 +- .../druid/timeline}/LogicalSegment.java | 2 +- .../druid/timeline}/TimelineObjectHolder.java | 4 +- .../timeline}/VersionedIntervalTimeline.java | 10 ++-- .../partition/ImmutablePartitionHolder.java | 4 +- .../partition/IntegerPartitionChunk.java | 4 +- .../partition/LinearPartitionChunk.java | 21 ++++++- .../partition/NumberedPartitionChunk.java | 4 +- .../timeline}/partition/PartitionChunk.java | 4 +- .../timeline}/partition/PartitionHolder.java | 4 +- .../SingleElementPartitionChunk.java | 4 +- .../partition/StringPartitionChunk.java | 4 +- .../druid/collections}/IntListTest.java | 4 +- .../druid/collections}/IteratorShell.java | 4 +- .../OrderedMergeIteratorTest.java | 4 +- .../OrderedMergeSequenceTest.java | 4 +- .../common}/guava/CombiningSequenceTest.java | 4 +- .../druid/common}/utils/JodaUtilsTest.java | 4 +- .../VersionedIntervalTimelineTest.java | 14 ++--- .../partition/IntegerPartitionChunkTest.java | 6 +- .../partition/StringPartitionChunkTest.java | 6 +- .../examples/flights/FlightsConverter.java | 4 +- .../flights/FlightsFirehoseFactory.java | 2 +- .../examples/rand/RandomFirehoseFactory.java | 23 +++++++- .../TwitterSpritzerFirehoseFactory.java | 23 +++++++- .../web/InputSupplierUpdateStream.java | 4 +- .../examples/web/WebFirehoseFactory.java | 8 +-- .../examples/web/WebFirehoseFactoryTest.java | 4 +- .../metamx/druid/indexer/DbUpdaterJob.java | 4 +- .../druid/indexer/DeterminePartitionsJob.java | 4 +- .../indexer/HadoopDruidIndexerConfig.java | 6 +- .../indexer/HadoopDruidIndexerMapper.java | 2 +- .../druid/indexer/IndexGeneratorJob.java | 2 +- .../java/com/metamx/druid/indexer/Utils.java | 4 +- .../indexer/updater/DbUpdaterJobSpec.java | 4 +- .../indexer/HadoopDruidIndexerConfigTest.java | 6 +- .../granularity/ArbitraryGranularityTest.java | 5 +- .../granularity/UniformGranularityTest.java | 5 +- .../index/EventReceiverFirehoseFactory.java | 2 +- .../common/index/StaticS3FirehoseFactory.java | 2 +- .../indexing/common/task/AppendTask.java | 4 +- .../indexing/common/task/HadoopIndexTask.java | 2 +- .../task/IndexDeterminePartitionsTask.java | 2 +- .../common/task/IndexGeneratorTask.java | 2 +- .../common/task/RealtimeIndexTask.java | 2 +- .../indexing/coordinator/DbTaskStorage.java | 4 +- .../coordinator/IndexerDBCoordinator.java | 8 +-- .../coordinator/ThreadPoolTaskRunner.java | 2 +- .../config/IndexerDbConnectorConfig.java | 5 +- .../http/IndexerCoordinatorResource.java | 2 +- .../http/OldIndexerCoordinatorResource.java | 21 ++++++- .../indexing/worker/WorkerTaskMonitor.java | 2 +- .../worker/executor/ExecutorLifecycle.java | 21 ++++++- .../indexing/common/task/TaskSerdeTest.java | 6 +- .../common/task/VersionConverterTaskTest.java | 2 +- .../coordinator/RemoteTaskRunnerTest.java | 4 +- .../coordinator/TaskLifecycleTest.java | 8 +-- .../scaling/EC2AutoScalingStrategyTest.java | 6 +- .../SimpleResourceManagementStrategyTest.java | 4 +- .../indexing/worker/TaskAnnouncementTest.java | 2 +- processing/pom.xml | 9 +++ .../io/druid/data/input/InputRowParser.java | 1 - .../druid/data/input/MapInputRowParser.java | 2 - .../data/input/ProtoBufInputRowParser.java | 1 - .../data/input/StringInputRowParser.java | 1 - .../io}/druid/jackson/AggregatorsModule.java | 22 +++---- .../jackson/CommaListJoinDeserializer.java | 4 +- .../jackson/CommaListJoinSerializer.java | 4 +- .../druid/jackson/DefaultObjectMapper.java | 2 +- .../DruidDefaultSerializersModule.java | 2 +- .../java/io}/druid/jackson/JacksonModule.java | 2 +- .../java/io}/druid/jackson/JodaStuff.java | 4 +- .../druid/jackson/QueryGranularityModule.java | 2 +- .../java/io/druid/query/QueryToolChest.java | 2 +- .../druid/query/ResultMergeQueryRunner.java | 2 +- .../query}/aggregation/CountAggregator.java | 4 +- .../aggregation/CountAggregatorFactory.java | 5 +- .../aggregation/CountBufferAggregator.java | 4 +- .../aggregation/DoubleSumAggregator.java | 3 +- .../DoubleSumAggregatorFactory.java | 5 +- .../DoubleSumBufferAggregator.java | 3 +- .../druid/query}/aggregation/Histogram.java | 2 +- .../aggregation/HistogramAggregator.java | 3 +- .../HistogramAggregatorFactory.java | 5 +- .../HistogramBufferAggregator.java | 3 +- .../query}/aggregation/HistogramVisual.java | 2 +- .../aggregation/JavaScriptAggregator.java | 3 +- .../JavaScriptAggregatorFactory.java | 5 +- .../JavaScriptBufferAggregator.java | 3 +- .../query}/aggregation/LongSumAggregator.java | 3 +- .../aggregation/LongSumAggregatorFactory.java | 5 +- .../aggregation/LongSumBufferAggregator.java | 3 +- .../query}/aggregation/MaxAggregator.java | 3 +- .../aggregation/MaxAggregatorFactory.java | 5 +- .../aggregation/MaxBufferAggregator.java | 3 +- .../query}/aggregation/MinAggregator.java | 3 +- .../aggregation/MinAggregatorFactory.java | 5 +- .../aggregation/MinBufferAggregator.java | 3 +- .../query}/aggregation/NoopAggregator.java | 4 +- .../aggregation/NoopBufferAggregator.java | 4 +- .../ToLowerCaseAggregatorFactory.java | 5 +- .../post/ArithmeticPostAggregator.java | 2 +- .../post/ConstantPostAggregator.java | 2 +- .../post/FieldAccessPostAggregator.java | 2 +- .../io/druid/query/groupby/GroupByQuery.java | 2 +- .../groupby/GroupByQueryQueryToolChest.java | 6 +- .../groupby/having/AlwaysHavingSpec.java | 2 +- .../query/groupby/having/AndHavingSpec.java | 2 +- .../groupby/having/EqualToHavingSpec.java | 2 +- .../groupby/having/GreaterThanHavingSpec.java | 2 +- .../query/groupby/having/HavingSpec.java | 2 +- .../groupby/having/LessThanHavingSpec.java | 2 +- .../query/groupby/having/NotHavingSpec.java | 2 +- .../query/groupby/having/OrHavingSpec.java | 2 +- .../groupby/orderby/DefaultLimitSpec.java | 2 +- .../query/groupby/orderby/LimitSpec.java | 2 +- .../query/groupby/orderby/NoopLimitSpec.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 4 +- .../search/SearchQueryQueryToolChest.java | 2 +- .../spec/MultipleIntervalSegmentSpec.java | 2 +- .../spec/MultipleSpecificSegmentSpec.java | 2 +- .../TimeBoundaryQueryQueryToolChest.java | 4 +- .../TimeseriesQueryQueryToolChest.java | 2 +- .../io/druid/segment/CompressedPools.java | 4 +- .../main/java/io/druid/segment/IndexIO.java | 4 +- .../java/io/druid/segment/IndexMerger.java | 12 ++-- .../java/io/druid/segment/MetricHolder.java | 2 +- .../data/CompressedFloatsIndexedSupplier.java | 4 +- .../CompressedFloatsSupplierSerializer.java | 4 +- .../data/CompressedLongsIndexedSupplier.java | 4 +- .../CompressedLongsSupplierSerializer.java | 4 +- .../data/CompressedObjectStrategy.java | 2 +- .../data/InMemoryCompressedFloats.java | 4 +- .../segment/data/InMemoryCompressedLongs.java | 4 +- .../segment/data/IntBufferIndexedInts.java | 2 +- .../segment/incremental/IncrementalIndex.java | 6 +- .../SpatialDimensionRowFormatter.java | 2 +- .../segment/serde/ComplexMetricExtractor.java | 2 +- .../collections/CombiningIterableTest.java | 1 - .../data/input/InputRowParserSerdeTest.java | 3 +- .../input/ProtoBufInputRowParserTest.java | 1 - .../granularity}/QueryGranularityTest.java | 7 +-- .../jackson/DefaultObjectMapperTest.java | 2 +- .../test/java/io/druid/query/QueriesTest.java | 10 ++-- .../aggregation/CountAggregatorTest.java | 2 +- .../aggregation/DoubleSumAggregatorTest.java | 2 +- .../aggregation/HistogramAggregatorTest.java | 4 +- .../query/aggregation}/HistogramTest.java | 5 +- .../aggregation/JavaScriptAggregatorTest.java | 4 +- .../aggregation/LongSumAggregatorTest.java | 2 +- .../query}/aggregation/MaxAggregatorTest.java | 2 +- .../aggregation/MetricSelectorUtils.java | 2 +- .../query}/aggregation/MinAggregatorTest.java | 2 +- .../aggregation/TestFloatMetricSelector.java | 2 +- .../post/ArithmeticPostAggregatorTest.java | 4 +- .../post/ConstantPostAggregatorTest.java | 2 +- .../post/FieldAccessPostAggregatorTest.java | 4 +- .../query/groupby/having/HavingSpecTest.java | 6 +- .../query/spec/QuerySegmentSpecTest.java | 2 +- .../TimeBoundaryQueryQueryToolChestTest.java | 2 +- .../timeseries/TimeseriesBinaryFnTest.java | 10 ++-- ...ompressedFloatsSupplierSerializerTest.java | 2 +- ...CompressedLongsSupplierSerializerTest.java | 2 +- .../segment/data/IncrementalIndexTest.java | 4 +- .../druid/realtime/DbSegmentPublisher.java | 2 +- .../com/metamx/druid/realtime/Indexer.java | 4 +- .../druid/realtime/RealtimeManager.java | 2 +- .../firehose/ClippedFirehoseFactory.java | 21 ++++++- .../druid/realtime/firehose/Firehose.java | 4 +- .../druid/realtime/firehose/IrcDecoder.java | 21 ++++++- .../realtime/firehose/IrcFirehoseFactory.java | 4 +- .../firehose/KafkaFirehoseFactory.java | 2 +- .../realtime/firehose/PredicateFirehose.java | 21 ++++++- .../firehose/RabbitMQFirehoseFactory.java | 2 +- .../firehose/TimedShutoffFirehoseFactory.java | 21 ++++++- .../firehose/WikipediaIrcDecoder.java | 4 +- .../plumber/RealtimePlumberSchool.java | 10 ++-- .../metamx/druid/realtime/plumber/Sink.java | 2 +- .../guice/DbSegmentPublisherProvider.java | 2 +- .../druid/realtime/RealtimeManagerTest.java | 6 +- .../plumber/RealtimePlumberSchoolTest.java | 2 +- .../druid/realtime/plumber/SinkTest.java | 4 +- .../com/metamx/druid/sql/antlr4/DruidSQL.g4 | 14 ++--- .../druid/coordination/ServerManager.java | 10 ++-- .../metamx/druid/db/DatabaseRuleManager.java | 3 +- .../druid/db/DatabaseRuleManagerProvider.java | 2 + .../druid/db/DatabaseSegmentManager.java | 9 +-- .../db/DatabaseSegmentManagerProvider.java | 2 + .../druid/loading/HdfsDataSegmentPuller.java | 2 +- .../druid/loading/HdfsDataSegmentPusher.java | 2 +- .../druid/loading/LocalDataSegmentPuller.java | 2 +- .../druid/loading/LocalDataSegmentPusher.java | 2 +- .../druid/loading/S3DataSegmentPuller.java | 2 +- .../druid/loading/S3DataSegmentPusher.java | 2 +- .../cassandra/CassandraDataSegmentPuller.java | 2 +- .../cassandra/CassandraDataSegmentPusher.java | 2 +- .../com/metamx/druid/master/DruidMaster.java | 4 +- .../druid/master/DruidMasterCleanup.java | 4 +- .../druid/master/DruidMasterLogger.java | 2 +- .../master/DruidMasterSegmentMerger.java | 6 +- .../com/metamx/druid/master/MasterStats.java | 4 +- .../metamx/druid/metrics/MetricsModule.java | 2 +- .../druid/query/group/GroupByQueryEngine.java | 8 +-- .../group/GroupByQueryRunnerFactory.java | 2 +- .../io/druid/client/BrokerServerView.java | 6 +- .../druid/client/CachingClusteredClient.java | 6 +- .../java/io/druid/client/DataSegment.java | 4 +- .../io/druid/client/ServerInventoryView.java | 2 +- .../io/druid/client/TimelineServerView.java | 2 +- .../io/druid/guice/guice/AnnouncerModule.java | 2 +- .../guice/guice/DruidProcessingModule.java | 4 +- .../main/java/io/druid/server/DruidNode.java | 2 +- .../http/MasterSegmentSettingsResource.java | 3 +- .../initialization/Initialization.java | 2 +- .../server/shard/shard/LinearShardSpec.java | 6 +- .../server/shard/shard/NoneShardSpec.java | 6 +- .../server/shard/shard/NumberedShardSpec.java | 6 +- .../druid/server/shard/shard/ShardSpec.java | 4 +- .../shard/shard/SingleDimensionShardSpec.java | 6 +- .../java/io/druid/server/sql/SQLRunner.java | 4 +- .../SegmentChangeRequestDropTest.java | 2 +- .../SegmentChangeRequestLoadTest.java | 2 +- .../druid/coordination/ZkCoordinatorTest.java | 4 +- .../druid/db/DatabaseSegmentManagerTest.java | 5 +- .../index/brita/SpatialFilterBonusTest.java | 6 +- .../druid/index/brita/SpatialFilterTest.java | 6 +- .../druid/index/v1/IndexMergerTest.java | 2 +- .../com/metamx/druid/index/v1/TestIndex.java | 2 +- .../druid/query/QueryRunnerTestHelper.java | 12 ++-- .../query/group/GroupByQueryRunnerTest.java | 10 ++-- .../GroupByTimeseriesQueryRunnerTest.java | 6 +- .../TimeseriesQueryRunnerBonusTest.java | 4 +- .../timeseries/TimeseriesQueryRunnerTest.java | 6 +- .../java/io/druid/client/DataSegmentTest.java | 2 +- .../client/BatchServerInventoryViewTest.java | 2 +- .../client/selector/ServerSelectorTest.java | 2 +- .../curator/announcement/AnnouncerTest.java | 2 +- .../CuratorInventoryManagerTest.java | 2 +- .../BatchDataSegmentAnnouncerTest.java | 2 +- .../shard/shard/NumberedShardSpecTest.java | 4 +- .../shard/SingleDimensionShardSpecTest.java | 4 +- .../src/main/java/io/druid/cli/CliPeon.java | 2 +- .../java/io/druid/cli/ServerRunnable.java | 21 ++++++- .../initialization}/LogLevelAdjuster.java | 4 +- .../LogLevelAdjusterMBean.java | 4 +- 285 files changed, 857 insertions(+), 620 deletions(-) delete mode 100644 common/src/main/java/com/metamx/druid/config/ConfigManagerConfig.java delete mode 100644 common/src/main/java/com/metamx/druid/config/ConfigSerde.java delete mode 100644 common/src/main/java/com/metamx/druid/utils/SocketUtil.java rename common/src/main/java/{com/metamx/druid => io/druid/collections}/CombiningIterable.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/collections}/CombiningIterator.java (96%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/CountingMap.java (95%) rename common/src/main/java/{com/metamx/druid => io/druid/collections}/IntList.java (97%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/OrderedMergeIterator.java (98%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/OrderedMergeSequence.java (98%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/ResourceHolder.java (91%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/StupidPool.java (96%) rename common/src/main/java/{com/metamx/druid/collect => io/druid/collections}/StupidResourceHolder.java (93%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/config/ConfigManager.java (90%) create mode 100644 common/src/main/java/io/druid/common/config/ConfigManagerConfig.java create mode 100644 common/src/main/java/io/druid/common/config/ConfigSerde.java rename common/src/main/java/{com/metamx/druid => io/druid/common}/config/JacksonConfigManager.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/CombiningSequence.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/DSuppliers.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/FileOutputSupplier.java (94%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/GuavaUtils.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/Runnables.java (93%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/ThreadRenamingCallable.java (94%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/guava/ThreadRenamingRunnable.java (93%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/utils/CompressionUtils.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/utils/JodaUtils.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/utils/PropUtils.java (95%) rename common/src/main/java/{com/metamx/druid => io/druid/common}/utils/SerializerUtils.java (98%) create mode 100644 common/src/main/java/io/druid/common/utils/SocketUtil.java rename common/src/main/java/{com/metamx => io}/druid/concurrent/Execs.java (95%) rename common/src/main/java/{com/metamx/druid => io/druid/data}/input/InputRow.java (96%) rename common/src/main/java/{com/metamx/druid => io/druid/data}/input/MapBasedInputRow.java (94%) rename common/src/main/java/{com/metamx/druid => io/druid/data}/input/MapBasedRow.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/data}/input/Row.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/data}/input/Rows.java (94%) rename common/src/main/java/{com/metamx => io}/druid/db/DbConnector.java (98%) rename common/src/main/java/{com/metamx => io}/druid/db/DbConnectorConfig.java (96%) rename common/src/main/java/{com/metamx => io}/druid/db/DbTablesConfig.java (73%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/LogicalSegment.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/TimelineObjectHolder.java (95%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/VersionedIntervalTimeline.java (98%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/ImmutablePartitionHolder.java (93%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/IntegerPartitionChunk.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/LinearPartitionChunk.java (66%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/NumberedPartitionChunk.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/PartitionChunk.java (96%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/PartitionHolder.java (97%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/SingleElementPartitionChunk.java (96%) rename common/src/main/java/{com/metamx/druid => io/druid/timeline}/partition/StringPartitionChunk.java (97%) rename common/src/test/java/{com/metamx/druid => io/druid/collections}/IntListTest.java (95%) rename common/src/test/java/{com/metamx/druid/collect => io/druid/collections}/IteratorShell.java (93%) rename common/src/test/java/{com/metamx/druid/collect => io/druid/collections}/OrderedMergeIteratorTest.java (98%) rename common/src/test/java/{com/metamx/druid/collect => io/druid/collections}/OrderedMergeSequenceTest.java (98%) rename common/src/test/java/{com/metamx/druid => io/druid/common}/guava/CombiningSequenceTest.java (98%) rename common/src/test/java/{com/metamx/druid => io/druid/common}/utils/JodaUtilsTest.java (97%) rename common/src/test/java/{com/metamx/druid => io/druid/timeline}/VersionedIntervalTimelineTest.java (99%) rename common/src/test/java/{com/metamx/druid => io/druid/timeline}/partition/IntegerPartitionChunkTest.java (94%) rename common/src/test/java/{com/metamx/druid => io/druid/timeline}/partition/StringPartitionChunkTest.java (94%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/AggregatorsModule.java (76%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/CommaListJoinDeserializer.java (94%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/CommaListJoinSerializer.java (95%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/DefaultObjectMapper.java (98%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/DruidDefaultSerializersModule.java (99%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/JacksonModule.java (98%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/JodaStuff.java (98%) rename {common/src/main/java/com/metamx => processing/src/main/java/io}/druid/jackson/QueryGranularityModule.java (98%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/CountAggregator.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/CountAggregatorFactory.java (93%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/CountBufferAggregator.java (93%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/DoubleSumAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/DoubleSumAggregatorFactory.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/DoubleSumBufferAggregator.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/Histogram.java (99%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/HistogramAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/HistogramAggregatorFactory.java (96%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/HistogramBufferAggregator.java (96%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/HistogramVisual.java (98%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/JavaScriptAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/JavaScriptAggregatorFactory.java (97%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/JavaScriptBufferAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/LongSumAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/LongSumAggregatorFactory.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/LongSumBufferAggregator.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MaxAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MaxAggregatorFactory.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MaxBufferAggregator.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MinAggregator.java (95%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MinAggregatorFactory.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/MinBufferAggregator.java (94%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/NoopAggregator.java (93%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/NoopBufferAggregator.java (92%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/ToLowerCaseAggregatorFactory.java (93%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/post/ArithmeticPostAggregator.java (98%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/post/ConstantPostAggregator.java (97%) rename {common/src/main/java/io/druid/query/aggregation => processing/src/main/java/io/druid/query}/aggregation/post/FieldAccessPostAggregator.java (97%) rename {common/src/test/java/com/metamx/druid => processing/src/test/java/io/druid/granularity}/QueryGranularityTest.java (98%) rename {common/src/test/java/com/metamx => processing/src/test/java/io}/druid/jackson/DefaultObjectMapperTest.java (97%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/CountAggregatorTest.java (97%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/DoubleSumAggregatorTest.java (98%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/HistogramAggregatorTest.java (97%) rename {common/src/test/java/com/metamx/druid/histogram => processing/src/test/java/io/druid/query/aggregation}/HistogramTest.java (94%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/JavaScriptAggregatorTest.java (98%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/LongSumAggregatorTest.java (98%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/MaxAggregatorTest.java (97%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/MetricSelectorUtils.java (96%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/MinAggregatorTest.java (97%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/TestFloatMetricSelector.java (96%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/post/ArithmeticPostAggregatorTest.java (96%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/post/ConstantPostAggregatorTest.java (97%) rename {common/src/test/java/io/druid/query/aggregation => processing/src/test/java/io/druid/query}/aggregation/post/FieldAccessPostAggregatorTest.java (93%) rename {common/src/main/java/com/metamx/druid/log => services/src/main/java/io/druid/initialization}/LogLevelAdjuster.java (96%) rename {common/src/main/java/com/metamx/druid/log => services/src/main/java/io/druid/initialization}/LogLevelAdjusterMBean.java (91%) diff --git a/common/pom.xml b/common/pom.xml index fba4b3c660e..1716a004688 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -1,7 +1,7 @@ diff --git a/server/src/main/java/io/druid/client/cache/MapCache.java b/server/src/main/java/io/druid/client/cache/MapCache.java index 743ca26463d..dda2929bc89 100644 --- a/server/src/main/java/io/druid/client/cache/MapCache.java +++ b/server/src/main/java/io/druid/client/cache/MapCache.java @@ -33,6 +33,11 @@ import java.util.concurrent.atomic.AtomicLong; */ public class MapCache implements Cache { + public static Cache create(long sizeInBytes) + { + return new MapCache(new ByteCountingLRUMap(sizeInBytes)); + } + private final Map baseMap; private final ByteCountingLRUMap byteCountingLRUMap; diff --git a/common/src/main/java/io/druid/guice/DruidGuiceExtensions.java b/server/src/main/java/io/druid/guice/Binders.java similarity index 53% rename from common/src/main/java/io/druid/guice/DruidGuiceExtensions.java rename to server/src/main/java/io/druid/guice/Binders.java index c33144d70d0..427512bb9a8 100644 --- a/common/src/main/java/io/druid/guice/DruidGuiceExtensions.java +++ b/server/src/main/java/io/druid/guice/Binders.java @@ -20,16 +20,33 @@ package io.druid.guice; import com.google.inject.Binder; -import com.google.inject.Module; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.MapBinder; +import io.druid.query.Query; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryToolChest; /** */ -public class DruidGuiceExtensions implements Module +public class Binders { - @Override - public void configure(Binder binder) + public static MapBinder, QueryRunnerFactory> queryRunnerFactoryBinder(Binder binder) { - binder.requireExplicitBindings(); - binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON); + return MapBinder.newMapBinder( + binder, new TypeLiteral>() + { + }, TypeLiteral.get(QueryRunnerFactory.class) + ); + } + + public static MapBinder, QueryToolChest> queryToolChestBinder(Binder binder) + { + return MapBinder.newMapBinder( + binder, new TypeLiteral>() + { + }, new TypeLiteral() + { + } + ); } } diff --git a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java index 212e3fa6f62..773dd6e1cb9 100644 --- a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java @@ -21,7 +21,6 @@ package io.druid.guice; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; @@ -57,9 +56,8 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule { super.configure(binder); - - final MapBinder, QueryRunnerFactory> queryFactoryBinder = MapBinder.newMapBinder( - binder, new TypeLiteral>(){}, TypeLiteral.get(QueryRunnerFactory.class) + final MapBinder, QueryRunnerFactory> queryFactoryBinder = Binders.queryRunnerFactoryBinder( + binder ); for (Map.Entry, Class> entry : mappings.entrySet()) { diff --git a/processing/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java similarity index 92% rename from processing/src/main/java/io/druid/guice/QueryToolChestModule.java rename to server/src/main/java/io/druid/guice/QueryToolChestModule.java index accc85fc062..e644848147f 100644 --- a/processing/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -22,7 +22,6 @@ package io.druid.guice; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Module; -import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import io.druid.query.Query; import io.druid.query.QueryToolChest; @@ -45,7 +44,7 @@ import java.util.Map; */ public class QueryToolChestModule implements Module { - final Map, Class> mappings = + public final Map, Class> mappings = ImmutableMap., Class>builder() .put(TimeseriesQuery.class, TimeseriesQueryQueryToolChest.class) .put(SearchQuery.class, SearchQueryQueryToolChest.class) @@ -57,9 +56,7 @@ public class QueryToolChestModule implements Module @Override public void configure(Binder binder) { - MapBinder, QueryToolChest> toolChests = MapBinder.newMapBinder( - binder, new TypeLiteral>(){}, new TypeLiteral(){} - ); + MapBinder, QueryToolChest> toolChests = Binders.queryToolChestBinder(binder); for (Map.Entry, Class> entry : mappings.entrySet()) { toolChests.addBinding(entry.getKey()).to(entry.getValue()); diff --git a/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java b/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java index 7e5349a759f..2e9f6254c1e 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java +++ b/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java @@ -20,7 +20,7 @@ package io.druid.server.initialization.initialization; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Guice; @@ -28,23 +28,66 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.initialization.DruidModule; import io.druid.jackson.JacksonModule; +import io.tesla.aether.internal.DefaultTeslaAether; +import org.eclipse.aether.artifact.Artifact; -import javax.annotation.Nullable; +import java.net.URL; +import java.net.URLClassLoader; import java.util.Arrays; +import java.util.Collections; import java.util.List; /** */ public class Initialization { + private static final Logger log = new Logger(Initialization.class); + + private static final List exclusions = Arrays.asList( + "io.druid", + "com.metamx.druid" + ); + + public static Injector makeInjector(final Object... modules) { + final List> externalModules = Lists.newArrayList(); + final DefaultTeslaAether aether = new DefaultTeslaAether(); + try { + final List artifacts = aether.resolveArtifacts( + "com.metamx.druid-extensions-mmx:druid-extensions:0.4.18-SNAPSHOT" + ); + List urls = Lists.newArrayListWithExpectedSize(artifacts.size()); + for (Artifact artifact : artifacts) { + if (!exclusions.contains(artifact.getGroupId())) { + urls.add(artifact.getFile().toURI().toURL()); + } + else { + log.error("Skipped Artifact[%s]", artifact); + } + } + + for (URL url : urls) { + log.error("Added URL[%s]", url); + } + + ClassLoader loader = new URLClassLoader( + urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader() + ); + + externalModules.add(loader.loadClass("com.metamx.druid.extensions.query.topn.TopNQueryDruidModule")); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + final Injector baseInjector = Guice.createInjector( new DruidGuiceExtensions(), new JacksonModule(), @@ -62,62 +105,82 @@ public class Initialization binder.bind((Class) module); } } + + for (Class externalModule : externalModules) { + binder.bind(externalModule); + } } } ); - List actualModules = Lists.newArrayList(); - actualModules.add(DruidSecondaryModule.class); - actualModules.addAll(Arrays.asList(modules)); + ModuleList actualModules = new ModuleList(baseInjector); + actualModules.addModule(DruidSecondaryModule.class); + for (Object module : modules) { + actualModules.addModule(module); + } - return Guice.createInjector( - Lists.transform( - actualModules, - new Function() - { - ObjectMapper jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class)); - ObjectMapper smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class)); + for (Class externalModule : externalModules) { + actualModules.addModule(externalModule); + } + + return Guice.createInjector(actualModules.getModules()); + } + + private static class ModuleList + { + private final Injector baseInjector; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final List modules; + + public ModuleList(Injector baseInjector) { + this.baseInjector = baseInjector; + this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class)); + this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class)); + this.modules = Lists.newArrayList(); + } + + private List getModules() + { + return Collections.unmodifiableList(modules); + } + + public void addModule(Object input) + { + if (input instanceof DruidModule) { + baseInjector.injectMembers(input); + modules.add(registerJacksonModules(((DruidModule) input))); + } + else if (input instanceof Module) { + baseInjector.injectMembers(input); + modules.add((Module) input); + } + else if (input instanceof Class) { + if (DruidModule.class.isAssignableFrom((Class) input)) { + modules.add(registerJacksonModules(baseInjector.getInstance((Class) input))); + } + else if (Module.class.isAssignableFrom((Class) input)) { + modules.add(baseInjector.getInstance((Class) input)); + return; + } + else { + throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); + } + } + else { + throw new ISE("Unknown module type[%s]", input.getClass()); + } + } + + private DruidModule registerJacksonModules(DruidModule module) + { + for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + smileMapper.registerModule(jacksonModule); + } + return module; + } + } - @Override - @SuppressWarnings("unchecked") - public Module apply(@Nullable Object input) - { - if (input instanceof DruidModule) { - baseInjector.injectMembers(input); - return registerJacksonModules(((DruidModule) input)); - } - - if (input instanceof Module) { - baseInjector.injectMembers(input); - return (Module) input; - } - - if (input instanceof Class) { - if (DruidModule.class.isAssignableFrom((Class) input)) { - return registerJacksonModules(baseInjector.getInstance((Class) input)); - } - if (Module.class.isAssignableFrom((Class) input)) { - return baseInjector.getInstance((Class) input); - } - else { - throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); - } - } - - throw new ISE("Unknown module type[%s]", input.getClass()); - } - - private DruidModule registerJacksonModules(DruidModule module) - { - for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) { - jsonMapper.registerModule(jacksonModule); - smileMapper.registerModule(jacksonModule); - } - return module; - } - } - ) - ); } -} From 6c9a10735673363f560cfecce04ae8470955be4e Mon Sep 17 00:00:00 2001 From: cheddar Date: Mon, 9 Sep 2013 17:02:57 -0500 Subject: [PATCH 65/92] 1) remove duplicate package initialization.initialization --- .../common/config/IndexerZkConfig.java | 2 +- .../coordinator/RemoteTaskRunner.java | 2 +- .../coordinator/RemoteTaskRunnerFactory.java | 2 +- .../indexing/coordinator/TaskMaster.java | 2 +- .../config/IndexerCoordinatorConfig.java | 2 +- .../worker/WorkerCuratorCoordinator.java | 2 +- .../coordinator/RemoteTaskRunnerTest.java | 2 +- .../client/BatchServerInventoryView.java | 2 +- .../BatchServerInventoryViewProvider.java | 2 +- .../client/SingleServerInventoryProvider.java | 2 +- .../client/SingleServerInventoryView.java | 2 +- .../curator/discovery/DiscoveryModule.java | 2 +- .../java/io/druid/guice/AnnouncerModule.java | 2 +- .../guice/{Binders.java => DruidBinders.java} | 2 +- .../druid/guice/QueryRunnerFactoryModule.java | 2 +- .../io/druid/guice/QueryToolChestModule.java | 2 +- .../java/io/druid/guice/ServerModule.java | 2 +- .../AbstractDataSegmentAnnouncer.java | 2 +- .../BatchDataSegmentAnnouncer.java | 4 +- .../SingleDataSegmentAnnouncer.java | 2 +- .../server/coordination/ZkCoordinator.java | 2 +- .../BatchDataSegmentAnnouncerConfig.java | 2 +- .../{initialization => }/ConfigModule.java | 2 +- .../CuratorDiscoveryConfig.java | 2 +- .../{initialization => }/EmitterModule.java | 2 +- .../initialization/ExtensionsConfig.java | 61 ++++++++++ .../HttpEmitterConfig.java | 2 +- .../HttpEmitterModule.java | 2 +- .../{initialization => }/Initialization.java | 113 ++++++++++++------ .../JettyServerInitializer.java | 2 +- .../JettyServerModule.java | 28 +---- .../LogEmitterModule.java | 2 +- .../PropertiesModule.java | 2 +- .../{initialization => }/ServerConfig.java | 2 +- .../{initialization => }/ZkPathsConfig.java | 2 +- .../io/druid/server/master/DruidMaster.java | 2 +- .../client/BatchServerInventoryViewTest.java | 4 +- .../coordination/ZkCoordinatorTest.java | 2 +- .../BatchDataSegmentAnnouncerTest.java | 4 +- .../druid/server/master/DruidMasterTest.java | 2 +- .../src/main/java/io/druid/cli/CliBroker.java | 6 +- .../java/io/druid/cli/CliCoordinator.java | 8 +- .../main/java/io/druid/cli/CliHistorical.java | 6 +- .../java/io/druid/cli/CliMiddleManager.java | 8 +- .../main/java/io/druid/cli/CliOverlord.java | 8 +- .../src/main/java/io/druid/cli/CliPeon.java | 6 +- .../main/java/io/druid/cli/CliRealtime.java | 6 +- .../java/io/druid/cli/CliRealtimeExample.java | 6 +- .../cli/QueryJettyServerInitializer.java | 2 +- 49 files changed, 209 insertions(+), 129 deletions(-) rename server/src/main/java/io/druid/guice/{Binders.java => DruidBinders.java} (98%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/BatchDataSegmentAnnouncerConfig.java (95%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/ConfigModule.java (96%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/CuratorDiscoveryConfig.java (95%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/EmitterModule.java (98%) create mode 100644 server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java rename server/src/main/java/io/druid/server/initialization/{initialization => }/HttpEmitterConfig.java (95%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/HttpEmitterModule.java (97%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/Initialization.java (61%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/JettyServerInitializer.java (94%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/JettyServerModule.java (84%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/LogEmitterModule.java (96%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/PropertiesModule.java (97%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/ServerConfig.java (95%) rename server/src/main/java/io/druid/server/initialization/{initialization => }/ZkPathsConfig.java (97%) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java index 214d3ed6330..67a750ab535 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/IndexerZkConfig.java @@ -19,7 +19,7 @@ package io.druid.indexing.common.config; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.skife.config.Config; import org.skife.config.Default; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java index 93452a77daa..679faaf3191 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java @@ -50,7 +50,7 @@ import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import io.druid.indexing.coordinator.setup.WorkerSetupData; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java index ea274bb2bfb..947d0d22f1d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunnerFactory.java @@ -27,7 +27,7 @@ import io.druid.curator.cache.SimplePathChildrenCacheFactory; import io.druid.guice.annotations.Global; import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import io.druid.indexing.coordinator.setup.WorkerSetupData; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; /** diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java b/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java index 012e5a2e711..ab8d410574f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java +++ b/indexing-service/src/main/java/io/druid/indexing/coordinator/TaskMaster.java @@ -36,7 +36,7 @@ import io.druid.indexing.coordinator.exec.TaskConsumer; import io.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import io.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; import io.druid.server.DruidNode; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderSelector; import org.apache.curator.framework.recipes.leader.LeaderSelectorListener; diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java index 16498286127..0671aabe135 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java @@ -21,7 +21,7 @@ package io.druid.indexing.coordinator.config; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.skife.config.Config; import org.skife.config.Default; diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index 17d3d73b024..f4fe25d8c3b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -31,7 +31,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.CreateMode; import org.joda.time.DateTime; diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 24b41aad7a4..47bfce6157b 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -49,7 +49,7 @@ import io.druid.indexing.worker.WorkerTaskMonitor; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/server/src/main/java/io/druid/client/BatchServerInventoryView.java b/server/src/main/java/io/druid/client/BatchServerInventoryView.java index 4de3354e8fd..3a1bf636aa2 100644 --- a/server/src/main/java/io/druid/client/BatchServerInventoryView.java +++ b/server/src/main/java/io/druid/client/BatchServerInventoryView.java @@ -27,7 +27,7 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; import io.druid.guice.ManageLifecycle; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; diff --git a/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java index 1bd0163dca8..609a47754aa 100644 --- a/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/BatchServerInventoryViewProvider.java @@ -21,7 +21,7 @@ package io.druid.client; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import javax.validation.constraints.NotNull; diff --git a/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java b/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java index 74ec48c9bd6..7ebce938791 100644 --- a/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java +++ b/server/src/main/java/io/druid/client/SingleServerInventoryProvider.java @@ -21,7 +21,7 @@ package io.druid.client; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import javax.validation.constraints.NotNull; diff --git a/server/src/main/java/io/druid/client/SingleServerInventoryView.java b/server/src/main/java/io/druid/client/SingleServerInventoryView.java index 38c01bc0598..801b78c5d2a 100644 --- a/server/src/main/java/io/druid/client/SingleServerInventoryView.java +++ b/server/src/main/java/io/druid/client/SingleServerInventoryView.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.guice.ManageLifecycle; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; diff --git a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index 63116b1f92a..20d52e8e77f 100644 --- a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -36,7 +36,7 @@ import com.metamx.common.lifecycle.Lifecycle; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.server.DruidNode; -import io.druid.server.initialization.initialization.CuratorDiscoveryConfig; +import io.druid.server.initialization.CuratorDiscoveryConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceDiscoveryBuilder; diff --git a/server/src/main/java/io/druid/guice/AnnouncerModule.java b/server/src/main/java/io/druid/guice/AnnouncerModule.java index c7cc0c78ef0..379cc29392e 100644 --- a/server/src/main/java/io/druid/guice/AnnouncerModule.java +++ b/server/src/main/java/io/druid/guice/AnnouncerModule.java @@ -28,7 +28,7 @@ import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentAnnouncerProvider; import io.druid.server.coordination.SingleDataSegmentAnnouncer; -import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.curator.framework.CuratorFramework; /** diff --git a/server/src/main/java/io/druid/guice/Binders.java b/server/src/main/java/io/druid/guice/DruidBinders.java similarity index 98% rename from server/src/main/java/io/druid/guice/Binders.java rename to server/src/main/java/io/druid/guice/DruidBinders.java index 427512bb9a8..4a3b6729380 100644 --- a/server/src/main/java/io/druid/guice/Binders.java +++ b/server/src/main/java/io/druid/guice/DruidBinders.java @@ -28,7 +28,7 @@ import io.druid.query.QueryToolChest; /** */ -public class Binders +public class DruidBinders { public static MapBinder, QueryRunnerFactory> queryRunnerFactoryBinder(Binder binder) { diff --git a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java index 773dd6e1cb9..0f9970a6f30 100644 --- a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java @@ -56,7 +56,7 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule { super.configure(binder); - final MapBinder, QueryRunnerFactory> queryFactoryBinder = Binders.queryRunnerFactoryBinder( + final MapBinder, QueryRunnerFactory> queryFactoryBinder = DruidBinders.queryRunnerFactoryBinder( binder ); diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index e644848147f..4da4eb2537f 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -56,7 +56,7 @@ public class QueryToolChestModule implements Module @Override public void configure(Binder binder) { - MapBinder, QueryToolChest> toolChests = Binders.queryToolChestBinder(binder); + MapBinder, QueryToolChest> toolChests = DruidBinders.queryToolChestBinder(binder); for (Map.Entry, Class> entry : mappings.entrySet()) { toolChests.addBinding(entry.getKey()).to(entry.getValue()); diff --git a/server/src/main/java/io/druid/guice/ServerModule.java b/server/src/main/java/io/druid/guice/ServerModule.java index 06bd69c0800..3417ef3d82e 100644 --- a/server/src/main/java/io/druid/guice/ServerModule.java +++ b/server/src/main/java/io/druid/guice/ServerModule.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.Lifecycle; import io.druid.guice.annotations.Self; import io.druid.initialization.DruidModule; import io.druid.server.DruidNode; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.partition.LinearShardSpec; import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec; diff --git a/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java index f4bd5016430..ba73b74e994 100644 --- a/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/AbstractDataSegmentAnnouncer.java @@ -26,7 +26,7 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; /** diff --git a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java index 59ac2f8e579..d0b24ea5653 100644 --- a/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/BatchDataSegmentAnnouncer.java @@ -28,8 +28,8 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; -import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.utils.ZKPaths; import org.joda.time.DateTime; diff --git a/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java b/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java index d812ab6e995..37107e0dfdb 100644 --- a/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java +++ b/server/src/main/java/io/druid/server/coordination/SingleDataSegmentAnnouncer.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.druid.curator.announcement.Announcer; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.utils.ZKPaths; diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index a30b178cefd..1987f2212af 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoadingException; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.ChildData; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java b/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java similarity index 95% rename from server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java rename to server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java index 079cb92e5bc..5df79357c77 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/BatchDataSegmentAnnouncerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/BatchDataSegmentAnnouncerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java b/server/src/main/java/io/druid/server/initialization/ConfigModule.java similarity index 96% rename from server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java rename to server/src/main/java/io/druid/server/initialization/ConfigModule.java index 4ff54fbf82b..62644146a05 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/ConfigModule.java +++ b/server/src/main/java/io/druid/server/initialization/ConfigModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.google.inject.Binder; import com.google.inject.Module; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java similarity index 95% rename from server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java rename to server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java index 55ead420ff7..e029112b183 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/CuratorDiscoveryConfig.java +++ b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java b/server/src/main/java/io/druid/server/initialization/EmitterModule.java similarity index 98% rename from server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java rename to server/src/main/java/io/druid/server/initialization/EmitterModule.java index a719502c51a..c7b29d3af53 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/EmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/EmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.google.common.base.Supplier; import com.google.common.collect.Lists; diff --git a/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java b/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java new file mode 100644 index 00000000000..0973a249434 --- /dev/null +++ b/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java @@ -0,0 +1,61 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.initialization; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import javax.validation.constraints.NotNull; +import java.util.List; + +/** + */ +public class ExtensionsConfig +{ + @JsonProperty + @NotNull + private List coordinates = ImmutableList.of(); + + @JsonProperty + @NotNull + private String localRepository = String.format("%s/%s", System.getProperty("user.home"), ".m2/repository"); + + @JsonProperty + @NotNull + private List remoteRepositories = ImmutableList.of( + "http://repo1.maven.org/maven2/", + "https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local" + ); + + public List getCoordinates() + { + return coordinates; + } + + public String getLocalRepository() + { + return localRepository; + } + + public List getRemoteRepositories() + { + return remoteRepositories; + } +} diff --git a/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java b/server/src/main/java/io/druid/server/initialization/HttpEmitterConfig.java similarity index 95% rename from server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java rename to server/src/main/java/io/druid/server/initialization/HttpEmitterConfig.java index 97c36874afd..9141f7316b1 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterConfig.java +++ b/server/src/main/java/io/druid/server/initialization/HttpEmitterConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java b/server/src/main/java/io/druid/server/initialization/HttpEmitterModule.java similarity index 97% rename from server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java rename to server/src/main/java/io/druid/server/initialization/HttpEmitterModule.java index 7e408dc9b38..fb460b7f3d0 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/HttpEmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/HttpEmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.google.common.base.Supplier; import com.google.inject.Binder; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java b/server/src/main/java/io/druid/server/initialization/Initialization.java similarity index 61% rename from server/src/main/java/io/druid/server/initialization/initialization/Initialization.java rename to server/src/main/java/io/druid/server/initialization/Initialization.java index 2e9f6254c1e..6de6612ae51 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/Initialization.java +++ b/server/src/main/java/io/druid/server/initialization/Initialization.java @@ -17,11 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.google.common.io.ByteStreams; import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -31,18 +32,22 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidSecondaryModule; +import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.initialization.DruidModule; import io.druid.jackson.JacksonModule; +import io.tesla.aether.TeslaAether; import io.tesla.aether.internal.DefaultTeslaAether; import org.eclipse.aether.artifact.Artifact; +import java.io.PrintStream; import java.net.URL; import java.net.URLClassLoader; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.ServiceLoader; /** */ @@ -58,36 +63,6 @@ public class Initialization public static Injector makeInjector(final Object... modules) { - final List> externalModules = Lists.newArrayList(); - final DefaultTeslaAether aether = new DefaultTeslaAether(); - try { - final List artifacts = aether.resolveArtifacts( - "com.metamx.druid-extensions-mmx:druid-extensions:0.4.18-SNAPSHOT" - ); - List urls = Lists.newArrayListWithExpectedSize(artifacts.size()); - for (Artifact artifact : artifacts) { - if (!exclusions.contains(artifact.getGroupId())) { - urls.add(artifact.getFile().toURI().toURL()); - } - else { - log.error("Skipped Artifact[%s]", artifact); - } - } - - for (URL url : urls) { - log.error("Added URL[%s]", url); - } - - ClassLoader loader = new URLClassLoader( - urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader() - ); - - externalModules.add(loader.loadClass("com.metamx.druid.extensions.query.topn.TopNQueryDruidModule")); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - final Injector baseInjector = Guice.createInjector( new DruidGuiceExtensions(), new JacksonModule(), @@ -99,34 +74,94 @@ public class Initialization public void configure(Binder binder) { binder.bind(DruidSecondaryModule.class); + JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); for (Object module : modules) { if (module instanceof Class) { binder.bind((Class) module); } } - - for (Class externalModule : externalModules) { - binder.bind(externalModule); - } } } ); - ModuleList actualModules = new ModuleList(baseInjector); actualModules.addModule(DruidSecondaryModule.class); for (Object module : modules) { actualModules.addModule(module); } - for (Class externalModule : externalModules) { - actualModules.addModule(externalModule); - } + addExtensionModules(baseInjector.getInstance(ExtensionsConfig.class), actualModules); return Guice.createInjector(actualModules.getModules()); } + private static void addExtensionModules(ExtensionsConfig config, ModuleList actualModules) + { + final TeslaAether aether = getAetherClient(config); + + for (String coordinate : config.getCoordinates()) { + log.info("Loading extension[%s]", coordinate); + try { + final List artifacts = aether.resolveArtifacts(coordinate); + List urls = Lists.newArrayListWithExpectedSize(artifacts.size()); + for (Artifact artifact : artifacts) { + if (!exclusions.contains(artifact.getGroupId())) { + urls.add(artifact.getFile().toURI().toURL()); + } + else { + log.debug("Skipped Artifact[%s]", artifact); + } + } + + for (URL url : urls) { + log.debug("Added URL[%s]", url); + } + + ClassLoader loader = new URLClassLoader( + urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader() + ); + + final ServiceLoader serviceLoader = ServiceLoader.load(DruidModule.class, loader); + + for (DruidModule module : serviceLoader) { + log.info("Adding extension module[%s]", module.getClass()); + actualModules.addModule(module); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + + private static DefaultTeslaAether getAetherClient(ExtensionsConfig config) + { + /* + DefaultTeslaAether logs a bunch of stuff to System.out, which is annoying. We choose to disable that + unless debug logging is turned on. "Disabling" it, however, is kinda bass-ackwards. We copy out a reference + to the current System.out, and set System.out to a noop output stream. Then after DefaultTeslaAether has pulled + The reference we swap things back. + + This has implications for other things that are running in parallel to this. Namely, if anything else also grabs + a reference to System.out or tries to log to it while we have things adjusted like this, then they will also log + to nothingness. Fortunately, the code that calls this is single-threaded and shouldn't hopefully be running + alongside anything else that's grabbing System.out. But who knows. + */ + if (log.isTraceEnabled() || log.isDebugEnabled()) { + return new DefaultTeslaAether(config.getLocalRepository(), config.getRemoteRepositories()); + } + + PrintStream oldOut = System.out; + try { + System.setOut(new PrintStream(ByteStreams.nullOutputStream())); + return new DefaultTeslaAether(config.getLocalRepository(), config.getRemoteRepositories()); + } + finally { + System.setOut(oldOut); + } + } + private static class ModuleList { private final Injector baseInjector; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java b/server/src/main/java/io/druid/server/initialization/JettyServerInitializer.java similarity index 94% rename from server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java rename to server/src/main/java/io/druid/server/initialization/JettyServerInitializer.java index b16e73001e8..44229e99dc7 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/JettyServerInitializer.java +++ b/server/src/main/java/io/druid/server/initialization/JettyServerInitializer.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.google.inject.Injector; import org.eclipse.jetty.server.Server; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java b/server/src/main/java/io/druid/server/initialization/JettyServerModule.java similarity index 84% rename from server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java rename to server/src/main/java/io/druid/server/initialization/JettyServerModule.java index 2c5d81ab0a7..80c6fb7cc4b 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/JettyServerModule.java +++ b/server/src/main/java/io/druid/server/initialization/JettyServerModule.java @@ -17,10 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.primitives.Ints; @@ -32,8 +30,6 @@ import com.google.inject.Key; import com.google.inject.Provides; import com.google.inject.ProvisionException; import com.google.inject.Scopes; -import com.google.inject.TypeLiteral; -import com.google.inject.name.Named; import com.google.inject.name.Names; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -42,8 +38,10 @@ import com.sun.jersey.api.core.ResourceConfig; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.spi.container.servlet.WebConfig; +import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.JSR311Resource; import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; import org.eclipse.jetty.server.Connector; @@ -85,26 +83,12 @@ public class JettyServerModule extends JerseyServletModule JsonConfigProvider.bind(binder, "druid.server.http", ServerConfig.class); - // The Guice servlet extension doesn't actually like requiring explicit bindings, so we do its job for it here. - try { - final Class classToBind = Class.forName( - "com.google.inject.servlet.InternalServletModule$BackwardsCompatibleServletContextProvider" - ); - binder.bind(classToBind); - } - catch (ClassNotFoundException e) { - throw Throwables.propagate(e); - } - binder.bind(GuiceContainer.class).to(DruidGuiceContainer.class); binder.bind(DruidGuiceContainer.class).in(Scopes.SINGLETON); serve("/*").with(DruidGuiceContainer.class); - final ImmutableSet> theResources = ImmutableSet.copyOf(resources); - binder.bind(new TypeLiteral>>(){}) - .annotatedWith(Names.named("resourceClasses")) - .toInstance(theResources); - for (Class resource : theResources) { + for (Class resource : resources) { + Jerseys.addResource(binder, resource); binder.bind(resource).in(LazySingleton.class); } @@ -118,7 +102,7 @@ public class JettyServerModule extends JerseyServletModule @Inject public DruidGuiceContainer( Injector injector, - @Named("resourceClasses") Set> resources + @JSR311Resource Set> resources ) { super(injector); diff --git a/server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java b/server/src/main/java/io/druid/server/initialization/LogEmitterModule.java similarity index 96% rename from server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java rename to server/src/main/java/io/druid/server/initialization/LogEmitterModule.java index 1d84b3fb352..4759cf57758 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/LogEmitterModule.java +++ b/server/src/main/java/io/druid/server/initialization/LogEmitterModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Supplier; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java similarity index 97% rename from server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java rename to server/src/main/java/io/druid/server/initialization/PropertiesModule.java index a82708ee0b9..2efe1e3eac7 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/PropertiesModule.java +++ b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.google.common.base.Throwables; import com.google.common.io.Closeables; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java b/server/src/main/java/io/druid/server/initialization/ServerConfig.java similarity index 95% rename from server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java rename to server/src/main/java/io/druid/server/initialization/ServerConfig.java index 8894161fc33..6188e9681f5 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/ServerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ServerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Period; diff --git a/server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java similarity index 97% rename from server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java rename to server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java index 4f0c7aeea9e..2169177b16c 100644 --- a/server/src/main/java/io/druid/server/initialization/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization.initialization; +package io.druid.server.initialization; import org.apache.curator.utils.ZKPaths; import org.skife.config.Config; diff --git a/server/src/main/java/io/druid/server/master/DruidMaster.java b/server/src/main/java/io/druid/server/master/DruidMaster.java index 3bc359d6dbb..1f4886febaf 100644 --- a/server/src/main/java/io/druid/server/master/DruidMaster.java +++ b/server/src/main/java/io/druid/server/master/DruidMaster.java @@ -50,7 +50,7 @@ import io.druid.db.DatabaseRuleManager; import io.druid.db.DatabaseSegmentManager; import io.druid.guice.ManageLifecycle; import io.druid.segment.IndexIO; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.leader.LeaderLatch; diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index 1a858027ae3..38864822eb8 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -33,8 +33,8 @@ import io.druid.curator.announcement.Announcer; import io.druid.jackson.DefaultObjectMapper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index 2053e90f3b4..e1d78741b7c 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -32,7 +32,7 @@ import io.druid.query.NoopQueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.loading.CacheTestSegmentLoader; import io.druid.segment.loading.SegmentLoaderConfig; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; diff --git a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index 66e7acbb78c..245a3573c5f 100644 --- a/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/io/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -30,8 +30,8 @@ import io.druid.curator.announcement.Announcer; import io.druid.jackson.DefaultObjectMapper; import io.druid.server.coordination.BatchDataSegmentAnnouncer; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.initialization.initialization.BatchDataSegmentAnnouncerConfig; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.BatchDataSegmentAnnouncerConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; import junit.framework.Assert; import org.apache.curator.framework.CuratorFramework; diff --git a/server/src/test/java/io/druid/server/master/DruidMasterTest.java b/server/src/test/java/io/druid/server/master/DruidMasterTest.java index 1c876b53519..a1334718e94 100644 --- a/server/src/test/java/io/druid/server/master/DruidMasterTest.java +++ b/server/src/test/java/io/druid/server/master/DruidMasterTest.java @@ -25,7 +25,7 @@ import io.druid.client.DruidServer; import io.druid.client.SingleServerInventoryView; import io.druid.curator.inventory.InventoryManagerConfig; import io.druid.db.DatabaseSegmentManager; -import io.druid.server.initialization.initialization.ZkPathsConfig; +import io.druid.server.initialization.ZkPathsConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index fdcb897334f..c843cbefc01 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -34,9 +34,9 @@ import io.druid.guice.ServerViewModule; import io.druid.guice.annotations.Client; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.StatusResource; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; /** diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index a2e77204117..af435b77545 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -38,10 +38,10 @@ import io.druid.server.StatusResource; import io.druid.server.http.InfoResource; import io.druid.server.http.MasterResource; import io.druid.server.http.RedirectFilter; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerInitializer; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.master.DruidMaster; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Handler; diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 588402dfb48..2035990d9c1 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -37,9 +37,9 @@ import io.druid.guice.StorageNodeModule; import io.druid.server.StatusResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; import io.druid.server.metrics.ServerMonitor; diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index a75de7d97c4..16aec9d2460 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -33,10 +33,10 @@ import io.druid.guice.TaskLogsModule; import io.druid.indexing.worker.WorkerTaskMonitor; import io.druid.indexing.worker.http.WorkerResource; import io.druid.server.StatusResource; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerInitializer; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index eccd7a65300..5c91c645488 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -37,10 +37,10 @@ import io.druid.indexing.coordinator.TaskMaster; import io.druid.indexing.coordinator.http.IndexerCoordinatorResource; import io.druid.server.StatusResource; import io.druid.server.http.RedirectFilter; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerInitializer; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index f0f477edb5d..95ac2c42d4c 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -47,9 +47,9 @@ import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; import io.druid.initialization.LogLevelAdjuster; import io.druid.server.StatusResource; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; import java.io.File; diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index 760b05c7b59..fa07364339c 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -38,9 +38,9 @@ import io.druid.guice.ServerViewModule; import io.druid.guice.StorageNodeModule; import io.druid.segment.realtime.RealtimeManager; import io.druid.server.StatusResource; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; /** diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 882b8ae1b8b..52b258898c3 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -31,9 +31,9 @@ import io.druid.guice.ServerModule; import io.druid.guice.StorageNodeModule; import io.druid.segment.realtime.RealtimeManager; import io.druid.server.StatusResource; -import io.druid.server.initialization.initialization.EmitterModule; -import io.druid.server.initialization.initialization.Initialization; -import io.druid.server.initialization.initialization.JettyServerModule; +import io.druid.server.initialization.EmitterModule; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.JettyServerModule; /** */ diff --git a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java index 0e7fc170a1f..f23536ff665 100644 --- a/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/QueryJettyServerInitializer.java @@ -22,7 +22,7 @@ package io.druid.cli; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import io.druid.server.QueryServlet; -import io.druid.server.initialization.initialization.JettyServerInitializer; +import io.druid.server.initialization.JettyServerInitializer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; From 3e4a4f5566e7f94e6cdf21337ddba3264bd52e94 Mon Sep 17 00:00:00 2001 From: cheddar Date: Wed, 11 Sep 2013 17:41:20 -0500 Subject: [PATCH 66/92] 1) Adjustments to allow for the addition of new Runnables via modules. Modules should implement CliCommandCreator and add their stuff there. --- .../io/druid/guice/DruidSecondaryModule.java | 1 + .../server/initialization/Initialization.java | 150 ++++++++++-------- .../src/main/java/io/druid/cli/CliBroker.java | 35 ++-- .../java/io/druid/cli/CliCoordinator.java | 8 +- .../main/java/io/druid/cli/CliHistorical.java | 9 +- .../java/io/druid/cli/CliMiddleManager.java | 8 +- .../main/java/io/druid/cli/CliOverlord.java | 8 +- .../src/main/java/io/druid/cli/CliPeon.java | 61 ++++--- .../main/java/io/druid/cli/CliRealtime.java | 9 +- .../java/io/druid/cli/CliRealtimeExample.java | 9 +- services/src/main/java/io/druid/cli/Main.java | 47 +++++- .../java/io/druid/cli/ServerRunnable.java | 17 +- 12 files changed, 233 insertions(+), 129 deletions(-) diff --git a/common/src/main/java/io/druid/guice/DruidSecondaryModule.java b/common/src/main/java/io/druid/guice/DruidSecondaryModule.java index fbafb29d42b..7ace75f97a6 100644 --- a/common/src/main/java/io/druid/guice/DruidSecondaryModule.java +++ b/common/src/main/java/io/druid/guice/DruidSecondaryModule.java @@ -68,6 +68,7 @@ public class DruidSecondaryModule implements Module @Override public void configure(Binder binder) { + binder.requireExplicitBindings(); binder.install(new DruidGuiceExtensions()); binder.bind(Properties.class).toInstance(properties); binder.bind(ConfigurationObjectFactory.class).toInstance(factory); diff --git a/server/src/main/java/io/druid/server/initialization/Initialization.java b/server/src/main/java/io/druid/server/initialization/Initialization.java index 6de6612ae51..96b20ae077a 100644 --- a/server/src/main/java/io/druid/server/initialization/Initialization.java +++ b/server/src/main/java/io/druid/server/initialization/Initialization.java @@ -22,117 +22,126 @@ package io.druid.server.initialization; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.io.ByteStreams; -import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidSecondaryModule; -import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.initialization.DruidModule; -import io.druid.jackson.JacksonModule; import io.tesla.aether.TeslaAether; import io.tesla.aether.internal.DefaultTeslaAether; import org.eclipse.aether.artifact.Artifact; +import org.eclipse.aether.artifact.DefaultArtifact; +import org.eclipse.aether.collection.CollectRequest; +import org.eclipse.aether.graph.Dependency; +import org.eclipse.aether.graph.DependencyFilter; +import org.eclipse.aether.graph.DependencyNode; +import org.eclipse.aether.resolution.DependencyRequest; +import org.eclipse.aether.util.artifact.JavaScopes; +import org.eclipse.aether.util.filter.DependencyFilterUtils; import java.io.PrintStream; import java.net.URL; import java.net.URLClassLoader; -import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.ServiceLoader; +import java.util.Set; /** */ public class Initialization { private static final Logger log = new Logger(Initialization.class); + private static final Map loadersMap = Maps.newHashMap(); - private static final List exclusions = Arrays.asList( + private static final Set exclusions = Sets.newHashSet( "io.druid", "com.metamx.druid" ); - - public static Injector makeInjector(final Object... modules) - { - final Injector baseInjector = Guice.createInjector( - new DruidGuiceExtensions(), - new JacksonModule(), - new PropertiesModule("runtime.properties"), - new ConfigModule(), - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(DruidSecondaryModule.class); - JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); - - for (Object module : modules) { - if (module instanceof Class) { - binder.bind((Class) module); - } - } - } - } - ); - - ModuleList actualModules = new ModuleList(baseInjector); - actualModules.addModule(DruidSecondaryModule.class); - for (Object module : modules) { - actualModules.addModule(module); - } - - addExtensionModules(baseInjector.getInstance(ExtensionsConfig.class), actualModules); - - return Guice.createInjector(actualModules.getModules()); - } - - private static void addExtensionModules(ExtensionsConfig config, ModuleList actualModules) + public synchronized static List getFromExtensions(ExtensionsConfig config, Class clazz) { final TeslaAether aether = getAetherClient(config); + List retVal = Lists.newArrayList(); for (String coordinate : config.getCoordinates()) { log.info("Loading extension[%s]", coordinate); try { - final List artifacts = aether.resolveArtifacts(coordinate); - List urls = Lists.newArrayListWithExpectedSize(artifacts.size()); - for (Artifact artifact : artifacts) { - if (!exclusions.contains(artifact.getGroupId())) { - urls.add(artifact.getFile().toURI().toURL()); + ClassLoader loader = loadersMap.get(coordinate); + if (loader == null) { + final CollectRequest collectRequest = new CollectRequest(); + collectRequest.setRoot(new Dependency(new DefaultArtifact(coordinate), JavaScopes.RUNTIME)); + DependencyRequest dependencyRequest = new DependencyRequest( + collectRequest, + DependencyFilterUtils.andFilter( + DependencyFilterUtils.classpathFilter(JavaScopes.RUNTIME), + new DependencyFilter() + { + @Override + public boolean accept(DependencyNode node, List parents) + { + if (accept(node.getArtifact())) { + return false; + } + + for (DependencyNode parent : parents) { + if (accept(parent.getArtifact())) { + return false; + } + } + + return true; + } + + private boolean accept(final Artifact artifact) + { + return exclusions.contains(artifact.getGroupId()); + } + } + ) + ); + + final List artifacts = aether.resolveArtifacts(dependencyRequest); + List urls = Lists.newArrayListWithExpectedSize(artifacts.size()); + for (Artifact artifact : artifacts) { + if (!exclusions.contains(artifact.getGroupId())) { + urls.add(artifact.getFile().toURI().toURL()); + } + else { + log.error("Skipped Artifact[%s]", artifact); + } } - else { - log.debug("Skipped Artifact[%s]", artifact); + + for (URL url : urls) { + log.error("Added URL[%s]", url); } + + loader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader()); + loadersMap.put(coordinate, loader); } - for (URL url : urls) { - log.debug("Added URL[%s]", url); - } + final ServiceLoader serviceLoader = ServiceLoader.load(clazz, loader); - ClassLoader loader = new URLClassLoader( - urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader() - ); - - final ServiceLoader serviceLoader = ServiceLoader.load(DruidModule.class, loader); - - for (DruidModule module : serviceLoader) { + for (T module : serviceLoader) { log.info("Adding extension module[%s]", module.getClass()); - actualModules.addModule(module); + retVal.add(module); } } catch (Exception e) { throw Throwables.propagate(e); } } + + return retVal; } private static DefaultTeslaAether getAetherClient(ExtensionsConfig config) @@ -162,6 +171,22 @@ public class Initialization } } + public static Injector makeInjectorWithModules(final Injector baseInjector, List modules) + { + ModuleList actualModules = new ModuleList(baseInjector); + actualModules.addModule(DruidSecondaryModule.class); + for (Object module : modules) { + actualModules.addModule(module); + } + + final ExtensionsConfig config = baseInjector.getInstance(ExtensionsConfig.class); + for (DruidModule module : Initialization.getFromExtensions(config, DruidModule.class)) { + actualModules.addModule(module); + } + + return Guice.createInjector(actualModules.getModules()); + } + private static class ModuleList { private final Injector baseInjector; @@ -217,5 +242,4 @@ public class Initialization return module; } } - - } +} diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index c843cbefc01..b88835e125b 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -19,7 +19,7 @@ package io.druid.cli; -import com.google.inject.Injector; +import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.cache.CacheMonitor; @@ -35,10 +35,11 @@ import io.druid.guice.annotations.Client; import io.druid.server.ClientQuerySegmentWalker; import io.druid.server.StatusResource; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; +import java.util.List; + /** */ @Command( @@ -55,22 +56,22 @@ public class CliBroker extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( - new LifecycleModule(), - EmitterModule.class, - HttpClientModule.global(), - CuratorModule.class, - new MetricsModule().register(CacheMonitor.class), - new ServerModule(), - new JettyServerModule(new QueryJettyServerInitializer()) - .addResource(StatusResource.class), - new QueryableModule(ClientQuerySegmentWalker.class), - new QueryToolChestModule(), - new ServerViewModule(), - new HttpClientModule("druid.broker.http", Client.class), - new BrokerModule() + return ImmutableList.of( + new LifecycleModule(), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule().register(CacheMonitor.class), + new ServerModule(), + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class), + new QueryableModule(ClientQuerySegmentWalker.class), + new QueryToolChestModule(), + new ServerViewModule(), + new HttpClientModule("druid.broker.http", Client.class), + new BrokerModule() ); } } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index af435b77545..039284ab032 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -19,6 +19,7 @@ package io.druid.cli; +import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; @@ -39,7 +40,6 @@ import io.druid.server.http.InfoResource; import io.druid.server.http.MasterResource; import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.initialization.JettyServerModule; import io.druid.server.master.DruidMaster; @@ -55,6 +55,8 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.servlets.GzipFilter; +import java.util.List; + /** */ @Command( @@ -71,9 +73,9 @@ public class CliCoordinator extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule().register(DruidMaster.class), EmitterModule.class, HttpClientModule.global(), diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 2035990d9c1..7b493b639f3 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -19,7 +19,7 @@ package io.druid.cli; -import com.google.inject.Injector; +import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.curator.CuratorModule; @@ -38,11 +38,12 @@ import io.druid.server.StatusResource; import io.druid.server.coordination.ServerManager; import io.druid.server.coordination.ZkCoordinator; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; import io.druid.server.metrics.ServerMonitor; +import java.util.List; + /** */ @Command( @@ -59,9 +60,9 @@ public class CliHistorical extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule().register(ZkCoordinator.class), EmitterModule.class, HttpClientModule.global(), diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 16aec9d2460..7b40d812d41 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -19,6 +19,7 @@ package io.druid.cli; +import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; @@ -34,7 +35,6 @@ import io.druid.indexing.worker.WorkerTaskMonitor; import io.druid.indexing.worker.http.WorkerResource; import io.druid.server.StatusResource; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; @@ -47,6 +47,8 @@ import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.servlets.GzipFilter; +import java.util.List; + /** */ @Command( @@ -63,9 +65,9 @@ public class CliMiddleManager extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule().register(WorkerTaskMonitor.class), EmitterModule.class, HttpClientModule.global(), diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 5c91c645488..a7842673c44 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -19,6 +19,7 @@ package io.druid.cli; +import com.google.common.collect.ImmutableList; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import com.metamx.common.logger.Logger; @@ -38,7 +39,6 @@ import io.druid.indexing.coordinator.http.IndexerCoordinatorResource; import io.druid.server.StatusResource; import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; @@ -54,6 +54,8 @@ import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.servlets.GzipFilter; import org.eclipse.jetty.util.resource.ResourceCollection; +import java.util.List; + /** */ @Command( @@ -70,9 +72,9 @@ public class CliOverlord extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule(), EmitterModule.class, HttpClientModule.global(), diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 95ac2c42d4c..4147b249775 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -20,6 +20,8 @@ package io.druid.cli; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.inject.Inject; import com.google.inject.Injector; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; @@ -70,34 +72,45 @@ public class CliPeon implements Runnable @Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK") public String nodeType = "indexer-executor"; + private Injector injector; + + @Inject + public void configure(Injector injector) + { + this.injector = injector; + } + private static final Logger log = new Logger(CliPeon.class); protected Injector getInjector() { - return Initialization.makeInjector( - new LifecycleModule(), - EmitterModule.class, - HttpClientModule.global(), - CuratorModule.class, - new MetricsModule(), - new ServerModule(), - new JettyServerModule(new QueryJettyServerInitializer()) - .addResource(StatusResource.class) - .addResource(ChatHandlerResource.class), - new DiscoveryModule(), - new ServerViewModule(), - new StorageNodeModule(nodeType), - new DataSegmentPusherModule(), - new AnnouncerModule(), - new DruidProcessingModule(), - new QueryableModule(ThreadPoolTaskRunner.class), - new QueryRunnerFactoryModule(), - new IndexingServiceDiscoveryModule(), - new AWSModule(), - new PeonModule( - new ExecutorLifecycleConfig() - .setTaskFile(new File(taskAndStatusFile.get(0))) - .setStatusFile(new File(taskAndStatusFile.get(1))) + return Initialization.makeInjectorWithModules( + injector, + ImmutableList.of( + new LifecycleModule(), + EmitterModule.class, + HttpClientModule.global(), + CuratorModule.class, + new MetricsModule(), + new ServerModule(), + new JettyServerModule(new QueryJettyServerInitializer()) + .addResource(StatusResource.class) + .addResource(ChatHandlerResource.class), + new DiscoveryModule(), + new ServerViewModule(), + new StorageNodeModule(nodeType), + new DataSegmentPusherModule(), + new AnnouncerModule(), + new DruidProcessingModule(), + new QueryableModule(ThreadPoolTaskRunner.class), + new QueryRunnerFactoryModule(), + new IndexingServiceDiscoveryModule(), + new AWSModule(), + new PeonModule( + new ExecutorLifecycleConfig() + .setTaskFile(new File(taskAndStatusFile.get(0))) + .setStatusFile(new File(taskAndStatusFile.get(1))) + ) ) ); } diff --git a/services/src/main/java/io/druid/cli/CliRealtime.java b/services/src/main/java/io/druid/cli/CliRealtime.java index fa07364339c..c6eab365cba 100644 --- a/services/src/main/java/io/druid/cli/CliRealtime.java +++ b/services/src/main/java/io/druid/cli/CliRealtime.java @@ -19,7 +19,7 @@ package io.druid.cli; -import com.google.inject.Injector; +import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.curator.CuratorModule; @@ -39,10 +39,11 @@ import io.druid.guice.StorageNodeModule; import io.druid.segment.realtime.RealtimeManager; import io.druid.server.StatusResource; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerModule; import io.druid.server.metrics.MetricsModule; +import java.util.List; + /** */ @Command( @@ -59,9 +60,9 @@ public class CliRealtime extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule(), EmitterModule.class, DbConnectorModule.class, diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index 52b258898c3..adc6f78ccba 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -19,7 +19,7 @@ package io.druid.cli; -import com.google.inject.Injector; +import com.google.common.collect.ImmutableList; import com.metamx.common.logger.Logger; import druid.examples.guice.RealtimeExampleModule; import io.airlift.command.Command; @@ -32,9 +32,10 @@ import io.druid.guice.StorageNodeModule; import io.druid.segment.realtime.RealtimeManager; import io.druid.server.StatusResource; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.Initialization; import io.druid.server.initialization.JettyServerModule; +import java.util.List; + /** */ @Command( @@ -51,9 +52,9 @@ public class CliRealtimeExample extends ServerRunnable } @Override - protected Injector getInjector() + protected List getModules() { - return Initialization.makeInjector( + return ImmutableList.of( new LifecycleModule(), EmitterModule.class, DruidProcessingModule.class, diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index f7a264f6b4d..1f27ed215d3 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -1,6 +1,6 @@ /* * Druid - a distributed column store. - * Copyright (C) 2012 Metamarkets Group Inc. + * Copyright (C) 2012, 2013 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 @@ -19,9 +19,23 @@ package io.druid.cli; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; import io.airlift.command.Cli; import io.airlift.command.Help; import io.airlift.command.ParseException; +import io.druid.guice.DruidGuiceExtensions; +import io.druid.guice.DruidSecondaryModule; +import io.druid.guice.JsonConfigProvider; +import io.druid.jackson.JacksonModule; +import io.druid.server.initialization.ConfigModule; +import io.druid.server.initialization.ExtensionsConfig; +import io.druid.server.initialization.Initialization; +import io.druid.server.initialization.PropertiesModule; + +import java.util.List; /** */ @@ -54,9 +68,19 @@ public class Main .withDefaultCommand(Help.class) .withCommands(CliPeon.class); + final Injector injector = makeStartupInjector(); + final ExtensionsConfig config = injector.getInstance(ExtensionsConfig.class); + final List extensionCommands = Initialization.getFromExtensions(config, CliCommandCreator.class); + + for (CliCommandCreator creator : extensionCommands) { + creator.addCommands(builder); + } + final Cli cli = builder.build(); try { - cli.parse(args).run(); + final Runnable command = cli.parse(args); + injector.injectMembers(command); + command.run(); } catch (ParseException e) { System.out.println("ERROR!!!!"); @@ -65,4 +89,23 @@ public class Main cli.parse(new String[]{"help"}).run(); } } + + public static Injector makeStartupInjector() + { + return Guice.createInjector( + new DruidGuiceExtensions(), + new JacksonModule(), + new PropertiesModule("runtime.properties"), + new ConfigModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(DruidSecondaryModule.class); + JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); + } + } + ); + } } diff --git a/services/src/main/java/io/druid/cli/ServerRunnable.java b/services/src/main/java/io/druid/cli/ServerRunnable.java index a7ba9febbbd..aa2508d6007 100644 --- a/services/src/main/java/io/druid/cli/ServerRunnable.java +++ b/services/src/main/java/io/druid/cli/ServerRunnable.java @@ -20,10 +20,14 @@ package io.druid.cli; import com.google.common.base.Throwables; +import com.google.inject.Inject; import com.google.inject.Injector; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.initialization.LogLevelAdjuster; +import io.druid.server.initialization.Initialization; + +import java.util.List; /** */ @@ -31,12 +35,20 @@ public abstract class ServerRunnable implements Runnable { private final Logger log; + private Injector baseInjector; + public ServerRunnable(Logger log) { this.log = log; } - protected abstract Injector getInjector(); + @Inject + public void configure(Injector injector) + { + this.baseInjector = injector; + } + + protected abstract List getModules(); @Override public void run() @@ -44,7 +56,8 @@ public abstract class ServerRunnable implements Runnable try { LogLevelAdjuster.register(); - final Lifecycle lifecycle = getInjector().getInstance(Lifecycle.class); + final Injector injector = Initialization.makeInjectorWithModules(baseInjector, getModules()); + final Lifecycle lifecycle = injector.getInstance(Lifecycle.class); try { lifecycle.start(); From b8bd19e87c10f4b1552a4f8ff9aa233f5bb2f96a Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 10:13:04 -0500 Subject: [PATCH 67/92] whitespace --- .../src/main/java/io/druid/server/master/DruidMaster.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/master/DruidMaster.java b/server/src/main/java/io/druid/server/master/DruidMaster.java index 1f4886febaf..812d131a477 100644 --- a/server/src/main/java/io/druid/server/master/DruidMaster.java +++ b/server/src/main/java/io/druid/server/master/DruidMaster.java @@ -760,16 +760,12 @@ public class DruidMaster .withLoadManagementPeons(loadManagementPeons) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerReferenceTimestamp(DateTime.now()) - .withMasterSegmentSettings( - segmentSettingsAtomicReference.get() - ) + .withMasterSegmentSettings(segmentSettingsAtomicReference.get()) .build(); } }, new DruidMasterRuleRunner( - DruidMaster.this, - config.getReplicantLifetime(), - config.getReplicantThrottleLimit() + DruidMaster.this, config.getReplicantLifetime(), config.getReplicantThrottleLimit() ), new DruidMasterCleanup(DruidMaster.this), new DruidMasterBalancer(DruidMaster.this), From a2dcc45a8eadfdef196447f0f42300d59ee7eb6f Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 11:47:03 -0500 Subject: [PATCH 68/92] 1) Remove SingleSegmentLoader and replace with OmniSegmentLoader --- .../io/druid/indexing/common/TaskToolbox.java | 27 +- .../indexing/common/TaskToolboxFactory.java | 10 +- .../coordinator/TaskLifecycleTest.java | 2 +- .../druid/guice/DataSegmentPullerModule.java | 34 ++- .../java/io/druid/guice/DruidBinders.java | 6 + .../loading/DelegatingSegmentLoader.java | 69 ----- .../segment/loading/OmniSegmentLoader.java | 23 +- .../druid/segment/loading/SegmentLoader.java | 5 +- .../segment/loading/SingleSegmentLoader.java | 249 ------------------ .../loading/CacheTestSegmentLoader.java | 6 + ...erTest.java => OmniSegmentLoaderTest.java} | 6 +- .../coordination/ServerManagerTest.java | 7 + .../src/main/java/io/druid/cli/CliPeon.java | 2 + 13 files changed, 67 insertions(+), 379 deletions(-) delete mode 100644 server/src/main/java/io/druid/segment/loading/DelegatingSegmentLoader.java delete mode 100644 server/src/main/java/io/druid/segment/loading/SingleSegmentLoader.java rename server/src/test/java/io/druid/segment/loading/{SingleSegmentLoaderTest.java => OmniSegmentLoaderTest.java} (91%) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 242e7dcb90c..dcef1147d17 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -31,14 +31,10 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.MMappedQueryableIndexFactory; -import io.druid.segment.loading.S3DataSegmentPuller; -import io.druid.segment.loading.SegmentLoaderConfig; +import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; -import io.druid.segment.loading.SingleSegmentLoader; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.timeline.DataSegment; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; import java.util.List; @@ -53,13 +49,13 @@ public class TaskToolbox private final Task task; private final TaskActionClientFactory taskActionClientFactory; private final ServiceEmitter emitter; - private final RestS3Service s3Client; private final DataSegmentPusher segmentPusher; private final DataSegmentKiller dataSegmentKiller; private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; private final MonitorScheduler monitorScheduler; + private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; public TaskToolbox( @@ -67,13 +63,13 @@ public class TaskToolbox Task task, TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, - RestS3Service s3Client, DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, MonitorScheduler monitorScheduler, + SegmentLoader segmentLoader, ObjectMapper objectMapper ) { @@ -81,13 +77,13 @@ public class TaskToolbox this.task = task; this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; - this.s3Client = s3Client; this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; this.monitorScheduler = monitorScheduler; + this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; } @@ -144,22 +140,9 @@ public class TaskToolbox public Map getSegments(List segments) throws SegmentLoadingException { - final SingleSegmentLoader loader = new SingleSegmentLoader( - new S3DataSegmentPuller(s3Client), - new MMappedQueryableIndexFactory(), - new SegmentLoaderConfig() - { - @Override - public String getLocations() - { - return new File(getTaskWorkDir(), "fetched_segments").toString(); - } - } - ); - Map retVal = Maps.newLinkedHashMap(); for (DataSegment segment : segments) { - retVal.put(segment, loader.getSegmentFiles(segment)); + retVal.put(segment, segmentLoader.getSegmentFiles(segment)); } return retVal; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 5eb3d327e5a..fc1e8db390b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -30,8 +30,8 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.loading.SegmentLoader; import io.druid.server.coordination.DataSegmentAnnouncer; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** * Stuff that may be needed by a Task in order to conduct its business. @@ -41,13 +41,13 @@ 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 DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; private final MonitorScheduler monitorScheduler; + private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; @Inject @@ -55,26 +55,26 @@ public class TaskToolboxFactory TaskConfig config, TaskActionClientFactory taskActionClientFactory, ServiceEmitter emitter, - RestS3Service s3Client, DataSegmentPusher segmentPusher, DataSegmentKiller dataSegmentKiller, DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, MonitorScheduler monitorScheduler, + SegmentLoader segmentLoader, ObjectMapper objectMapper ) { this.config = config; this.taskActionClientFactory = taskActionClientFactory; this.emitter = emitter; - this.s3Client = s3Client; this.segmentPusher = segmentPusher; this.dataSegmentKiller = dataSegmentKiller; this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; this.monitorScheduler = monitorScheduler; + this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; } @@ -85,13 +85,13 @@ public class TaskToolboxFactory task, taskActionClientFactory, emitter, - s3Client, segmentPusher, dataSegmentKiller, segmentAnnouncer, newSegmentServerView, queryRunnerFactoryConglomerate, monitorScheduler, + segmentLoader, objectMapper ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java index 6ea5c1c0d80..ceb30ffdea6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java @@ -118,7 +118,6 @@ public class TaskLifecycleTest new TaskConfig(tmp.toString(), null, null, 50000), tac, newMockEmitter(), - null, // s3 client new DataSegmentPusher() { @Override @@ -139,6 +138,7 @@ public class TaskLifecycleTest null, // new segment server view null, // query runner factory conglomerate corporation unionized collective null, // monitor scheduler + null, // segment loader new DefaultObjectMapper() ); diff --git a/server/src/main/java/io/druid/guice/DataSegmentPullerModule.java b/server/src/main/java/io/druid/guice/DataSegmentPullerModule.java index 6b33cf401a5..3fd70ceecab 100644 --- a/server/src/main/java/io/druid/guice/DataSegmentPullerModule.java +++ b/server/src/main/java/io/druid/guice/DataSegmentPullerModule.java @@ -21,8 +21,6 @@ package io.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; -import com.google.inject.multibindings.MapBinder; -import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.HdfsDataSegmentPuller; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.OmniSegmentLoader; @@ -49,35 +47,35 @@ public class DataSegmentPullerModule implements Module private static void bindDeepStorageLocal(Binder binder) { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("local").to(LocalDataSegmentPuller.class).in(LazySingleton.class); + DruidBinders.dataSegmentPullerBinder(binder) + .addBinding("local") + .to(LocalDataSegmentPuller.class) + .in(LazySingleton.class); } private static void bindDeepStorageS3(Binder binder) { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + DruidBinders.dataSegmentPullerBinder(binder) + .addBinding("s3_zip") + .to(S3DataSegmentPuller.class) + .in(LazySingleton.class); } private static void bindDeepStorageHdfs(Binder binder) { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + DruidBinders.dataSegmentPullerBinder(binder) + .addBinding("hdfs") + .to(HdfsDataSegmentPuller.class) + .in(LazySingleton.class); binder.bind(Configuration.class).toInstance(new Configuration()); } private static void bindDeepStorageCassandra(Binder binder) { - final MapBinder segmentPullerBinder = MapBinder.newMapBinder( - binder, String.class, DataSegmentPuller.class - ); - segmentPullerBinder.addBinding("c*").to(CassandraDataSegmentPuller.class).in(LazySingleton.class); + DruidBinders.dataSegmentPullerBinder(binder) + .addBinding("c*") + .to(CassandraDataSegmentPuller.class) + .in(LazySingleton.class); ConfigProvider.bind(binder, CassandraDataSegmentConfig.class); } } diff --git a/server/src/main/java/io/druid/guice/DruidBinders.java b/server/src/main/java/io/druid/guice/DruidBinders.java index 4a3b6729380..5bc84227cd5 100644 --- a/server/src/main/java/io/druid/guice/DruidBinders.java +++ b/server/src/main/java/io/druid/guice/DruidBinders.java @@ -25,6 +25,7 @@ import com.google.inject.multibindings.MapBinder; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.segment.loading.DataSegmentPuller; /** */ @@ -49,4 +50,9 @@ public class DruidBinders } ); } + + public static MapBinder dataSegmentPullerBinder(Binder binder) + { + return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class); + } } diff --git a/server/src/main/java/io/druid/segment/loading/DelegatingSegmentLoader.java b/server/src/main/java/io/druid/segment/loading/DelegatingSegmentLoader.java deleted file mode 100644 index 6f907a61813..00000000000 --- a/server/src/main/java/io/druid/segment/loading/DelegatingSegmentLoader.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.segment.loading; - -import com.metamx.common.MapUtils; -import io.druid.segment.Segment; -import io.druid.timeline.DataSegment; - -import java.util.Map; - -/** - */ -public class DelegatingSegmentLoader implements SegmentLoader -{ - private volatile Map loaderTypes; - - public void setLoaderTypes( - Map loaderTypes - ) - { - this.loaderTypes = loaderTypes; - } - - @Override - public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException - { - return getLoader(segment.getLoadSpec()).isSegmentLoaded(segment); - } - - @Override - public Segment getSegment(DataSegment segment) throws SegmentLoadingException - { - return getLoader(segment.getLoadSpec()).getSegment(segment); - } - - @Override - public void cleanup(DataSegment segment) throws SegmentLoadingException - { - getLoader(segment.getLoadSpec()).cleanup(segment); - } - - private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException - { - String type = MapUtils.getString(loadSpec, "type"); - SegmentLoader loader = loaderTypes.get(type); - - if (loader == null) { - throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); - } - return loader; - } -} diff --git a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java index 16b2d50d597..de86620a57a 100644 --- a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java +++ b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java @@ -116,13 +116,14 @@ public class OmniSegmentLoader implements SegmentLoader @Override public Segment getSegment(DataSegment segment) throws SegmentLoadingException { - File segmentFiles = loadSegmentFiles(segment); + File segmentFiles = getSegmentFiles(segment); final QueryableIndex index = factory.factorize(segmentFiles); return new QueryableIndexSegment(segment.getIdentifier(), index); } - public File loadSegmentFiles(DataSegment segment) throws SegmentLoadingException + @Override + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { StorageLocation loc = findStorageLocationIfLoaded(segment); @@ -199,7 +200,7 @@ public class OmniSegmentLoader implements SegmentLoader } - private static class StorageLocation + static class StorageLocation { private final File path; private final long maxSize; @@ -218,41 +219,41 @@ public class OmniSegmentLoader implements SegmentLoader this.segments = Sets.newHashSet(); } - private File getPath() + File getPath() { return path; } - private Long getMaxSize() + Long getMaxSize() { return maxSize; } - private synchronized void addSegment(DataSegment segment) + synchronized void addSegment(DataSegment segment) { if (segments.add(segment)) { currSize += segment.getSize(); } } - private synchronized void removeSegment(DataSegment segment) + synchronized void removeSegment(DataSegment segment) { if (segments.remove(segment)) { currSize -= segment.getSize(); } } - private boolean canHandle(long size) + boolean canHandle(long size) { - return available() > size; + return available() >= size; } - private synchronized long available() + synchronized long available() { return maxSize - currSize; } - private StorageLocation mostEmpty(StorageLocation other) + StorageLocation mostEmpty(StorageLocation other) { return available() > other.available() ? this : other; } diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoader.java b/server/src/main/java/io/druid/segment/loading/SegmentLoader.java index 794b885449f..d4c050b6044 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoader.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoader.java @@ -22,11 +22,14 @@ package io.druid.segment.loading; import io.druid.segment.Segment; import io.druid.timeline.DataSegment; +import java.io.File; + /** */ public interface SegmentLoader { public boolean isSegmentLoaded(DataSegment segment) throws SegmentLoadingException; - public Segment getSegment(DataSegment loadSpec) throws SegmentLoadingException; + public Segment getSegment(DataSegment segment) throws SegmentLoadingException; + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException; public void cleanup(DataSegment loadSpec) throws SegmentLoadingException; } diff --git a/server/src/main/java/io/druid/segment/loading/SingleSegmentLoader.java b/server/src/main/java/io/druid/segment/loading/SingleSegmentLoader.java deleted file mode 100644 index e24f339b9b7..00000000000 --- a/server/src/main/java/io/druid/segment/loading/SingleSegmentLoader.java +++ /dev/null @@ -1,249 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.segment.loading; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; -import com.google.common.primitives.Longs; -import com.google.inject.Inject; -import com.metamx.common.IAE; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; -import io.druid.segment.QueryableIndex; -import io.druid.segment.QueryableIndexSegment; -import io.druid.segment.Segment; -import io.druid.timeline.DataSegment; -import org.apache.commons.io.FileUtils; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -/** - * TODO: Kill this along with the Guicification of the IndexingService stuff - */ -@Deprecated -public class SingleSegmentLoader implements SegmentLoader -{ - private static final Logger log = new Logger(SingleSegmentLoader.class); - - private final DataSegmentPuller dataSegmentPuller; - private final QueryableIndexFactory factory; - - private final List locations; - - @Inject - public SingleSegmentLoader( - DataSegmentPuller dataSegmentPuller, - QueryableIndexFactory factory, - SegmentLoaderConfig config - ) - { - this.dataSegmentPuller = dataSegmentPuller; - this.factory = factory; - - final ImmutableList.Builder locBuilder = ImmutableList.builder(); - - // TODO - // This is a really, really stupid way of getting this information. Splitting on commas and bars is error-prone - // We should instead switch it up to be a JSON Array of JSON Object or something and cool stuff like that - // But, that'll have to wait for some other day. - for (String dirSpec : config.getLocations().split(",")) { - String[] dirSplit = dirSpec.split("\\|"); - if (dirSplit.length == 1) { - locBuilder.add(new StorageLocation(new File(dirSplit[0]), Integer.MAX_VALUE)); - } - else if (dirSplit.length == 2) { - final Long maxSize = Longs.tryParse(dirSplit[1]); - if (maxSize == null) { - throw new IAE("Size of a local segment storage location must be an integral number, got[%s]", dirSplit[1]); - } - locBuilder.add(new StorageLocation(new File(dirSplit[0]), maxSize)); - } - else { - throw new ISE( - "Unknown segment storage location[%s]=>[%s], config[%s].", - dirSplit.length, dirSpec, config.getLocations() - ); - } - } - locations = locBuilder.build(); - - Preconditions.checkArgument(locations.size() > 0, "Must have at least one segment cache directory."); - log.info("Using storage locations[%s]", locations); - } - - @Override - public boolean isSegmentLoaded(final DataSegment segment) - { - return findStorageLocationIfLoaded(segment) != null; - } - - public StorageLocation findStorageLocationIfLoaded(final DataSegment segment) - { - for (StorageLocation location : locations) { - File localStorageDir = new File(location.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - if (localStorageDir.exists()) { - return location; - } - } - return null; - } - - @Override - public Segment getSegment(DataSegment segment) throws SegmentLoadingException - { - File segmentFiles = getSegmentFiles(segment); - final QueryableIndex index = factory.factorize(segmentFiles); - - return new QueryableIndexSegment(segment.getIdentifier(), index); - } - - public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException - { - StorageLocation loc = findStorageLocationIfLoaded(segment); - - final File retVal; - - if (loc == null) { - Iterator locIter = locations.iterator(); - loc = locIter.next(); - while (locIter.hasNext()) { - loc = loc.mostEmpty(locIter.next()); - } - - if (!loc.canHandle(segment.getSize())) { - throw new ISE( - "Segment[%s:%,d] too large for storage[%s:%,d].", - segment.getIdentifier(), segment.getSize(), loc.getPath(), loc.available() - ); - } - - File storageDir = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - if (!storageDir.mkdirs()) { - log.debug("Unable to make parent file[%s]", storageDir); - } - - dataSegmentPuller.getSegmentFiles(segment, storageDir); - loc.addSegment(segment); - - retVal = storageDir; - } - else { - retVal = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - } - - loc.addSegment(segment); - - return retVal; - } - - @Override - public void cleanup(DataSegment segment) throws SegmentLoadingException - { - StorageLocation loc = findStorageLocationIfLoaded(segment); - - if (loc == null) { - log.info("Asked to cleanup something[%s] that didn't exist. Skipping.", segment); - return; - } - - try { - File cacheFile = new File(loc.getPath(), DataSegmentPusherUtil.getStorageDir(segment)); - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); - loc.removeSegment(segment); - } - catch (IOException e) { - throw new SegmentLoadingException(e, e.getMessage()); - } - } - - static class StorageLocation - { - private final File path; - private final long maxSize; - private final Set segments; - - private volatile long currSize = 0; - - StorageLocation( - File path, - long maxSize - ) - { - this.path = path; - this.maxSize = maxSize; - - this.segments = Sets.newHashSet(); - } - - File getPath() - { - return path; - } - - Long getMaxSize() - { - return maxSize; - } - - synchronized void addSegment(DataSegment segment) - { - if (segments.add(segment)) { - currSize += segment.getSize(); - } - } - - synchronized void removeSegment(DataSegment segment) - { - if (segments.remove(segment)) { - currSize -= segment.getSize(); - } - } - - boolean canHandle(long size) - { - return available() >= size; - } - - synchronized long available() - { - return maxSize - currSize; - } - - StorageLocation mostEmpty(StorageLocation other) - { - return available() > other.available() ? this : other; - } - - @Override - public String toString() - { - return "StorageLocation{" + - "path=" + path + - ", maxSize=" + maxSize + - '}'; - } - } -} diff --git a/server/src/test/java/io/druid/segment/loading/CacheTestSegmentLoader.java b/server/src/test/java/io/druid/segment/loading/CacheTestSegmentLoader.java index 9ccb2675a00..42f16fd21c1 100644 --- a/server/src/test/java/io/druid/segment/loading/CacheTestSegmentLoader.java +++ b/server/src/test/java/io/druid/segment/loading/CacheTestSegmentLoader.java @@ -77,6 +77,12 @@ public class CacheTestSegmentLoader implements SegmentLoader }; } + @Override + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { + throw new UnsupportedOperationException(); + } + @Override public void cleanup(DataSegment loadSpec) throws SegmentLoadingException { diff --git a/server/src/test/java/io/druid/segment/loading/SingleSegmentLoaderTest.java b/server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java similarity index 91% rename from server/src/test/java/io/druid/segment/loading/SingleSegmentLoaderTest.java rename to server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java index ebba47cf5f5..90949215298 100644 --- a/server/src/test/java/io/druid/segment/loading/SingleSegmentLoaderTest.java +++ b/server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java @@ -30,13 +30,13 @@ import java.util.Arrays; /** */ -public class SingleSegmentLoaderTest +public class OmniSegmentLoaderTest { @Test public void testStorageLocation() throws Exception { long expectedAvail = 1000l; - SingleSegmentLoader.StorageLocation loc = new SingleSegmentLoader.StorageLocation(new File("/tmp"), expectedAvail); + OmniSegmentLoader.StorageLocation loc = new OmniSegmentLoader.StorageLocation(new File("/tmp"), expectedAvail); verifyLoc(expectedAvail, loc); @@ -65,7 +65,7 @@ public class SingleSegmentLoaderTest verifyLoc(expectedAvail, loc); } - private void verifyLoc(long maxSize, SingleSegmentLoader.StorageLocation loc) + private void verifyLoc(long maxSize, OmniSegmentLoader.StorageLocation loc) { Assert.assertEquals(maxSize, loc.available()); for (int i = 0; i <= maxSize; ++i) { diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 08fd201af74..f3483487c1d 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -65,6 +65,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Iterator; @@ -114,6 +115,12 @@ public class ServerManagerTest ); } + @Override + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException + { + throw new UnsupportedOperationException(); + } + @Override public void cleanup(DataSegment segment) throws SegmentLoadingException { diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 4147b249775..9962e6d93ce 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -32,6 +32,7 @@ import io.druid.curator.CuratorModule; import io.druid.curator.discovery.DiscoveryModule; import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; +import io.druid.guice.DataSegmentPullerModule; import io.druid.guice.DataSegmentPusherModule; import io.druid.guice.DruidProcessingModule; import io.druid.guice.HttpClientModule; @@ -99,6 +100,7 @@ public class CliPeon implements Runnable new DiscoveryModule(), new ServerViewModule(), new StorageNodeModule(nodeType), + new DataSegmentPullerModule(), new DataSegmentPusherModule(), new AnnouncerModule(), new DruidProcessingModule(), From 2ea09e2d709d8cfd823671de58b72cddc3699de4 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 12:56:56 -0500 Subject: [PATCH 69/92] 1) Clean up specification of StorageLocations --- .../segment/loading/OmniSegmentLoader.java | 108 +----------------- .../segment/loading/SegmentLoaderConfig.java | 24 +++- .../segment/loading/StorageLocation.java | 84 ++++++++++++++ .../loading/StorageLocationConfig.java | 58 ++++++++++ ...aderTest.java => StorageLocationTest.java} | 6 +- 5 files changed, 168 insertions(+), 112 deletions(-) create mode 100644 server/src/main/java/io/druid/segment/loading/StorageLocation.java create mode 100644 server/src/main/java/io/druid/segment/loading/StorageLocationConfig.java rename server/src/test/java/io/druid/segment/loading/{OmniSegmentLoaderTest.java => StorageLocationTest.java} (91%) diff --git a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java index de86620a57a..31efc1b0ea9 100644 --- a/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java +++ b/server/src/main/java/io/druid/segment/loading/OmniSegmentLoader.java @@ -19,12 +19,8 @@ package io.druid.segment.loading; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; -import com.google.common.primitives.Longs; +import com.google.common.collect.Lists; import com.google.inject.Inject; -import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; @@ -39,7 +35,6 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; /** */ @@ -64,36 +59,10 @@ public class OmniSegmentLoader implements SegmentLoader this.factory = factory; this.config = config; - final ImmutableList.Builder locBuilder = ImmutableList.builder(); - - // TODO - // This is a really, really stupid way of getting this information. Splitting on commas and bars is error-prone - // We should instead switch it up to be a JSON Array of JSON Object or something and cool stuff like that - // But, that'll have to wait for some other day. - for (String dirSpec : config.getLocations().split(",")) { - String[] dirSplit = dirSpec.split("\\|"); - if (dirSplit.length == 1) { - locBuilder.add(new StorageLocation(new File(dirSplit[0]), Integer.MAX_VALUE)); - } - else if (dirSplit.length == 2) { - final Long maxSize = Longs.tryParse(dirSplit[1]); - if (maxSize == null) { - throw new IAE("Size of a local segment storage location must be an integral number, got[%s]", dirSplit[1]); - } - locBuilder.add(new StorageLocation(new File(dirSplit[0]), maxSize)); - } - else { - throw new ISE( - "Unknown segment storage location[%s]=>[%s], config[%s].", - dirSplit.length, dirSpec, config.getLocations() - ); - } + this.locations = Lists.newArrayList(); + for (StorageLocationConfig locationConfig : config.getLocations()) { + locations.add(new StorageLocation(locationConfig.getPath(), locationConfig.getMaxSize())); } - locations = locBuilder.build(); - - Preconditions.checkArgument(locations.size() > 0, "Must have at least one segment cache directory."); - log.info("Using storage locations[%s]", locations); - } @Override @@ -198,73 +167,4 @@ public class OmniSegmentLoader implements SegmentLoader return loader; } - - - static class StorageLocation - { - private final File path; - private final long maxSize; - private final Set segments; - - private volatile long currSize = 0; - - StorageLocation( - File path, - long maxSize - ) - { - this.path = path; - this.maxSize = maxSize; - - this.segments = Sets.newHashSet(); - } - - File getPath() - { - return path; - } - - Long getMaxSize() - { - return maxSize; - } - - synchronized void addSegment(DataSegment segment) - { - if (segments.add(segment)) { - currSize += segment.getSize(); - } - } - - synchronized void removeSegment(DataSegment segment) - { - if (segments.remove(segment)) { - currSize -= segment.getSize(); - } - } - - boolean canHandle(long size) - { - return available() >= size; - } - - synchronized long available() - { - return maxSize - currSize; - } - - StorageLocation mostEmpty(StorageLocation other) - { - return available() > other.available() ? this : other; - } - - @Override - public String toString() - { - return "StorageLocation{" + - "path=" + path + - ", maxSize=" + maxSize + - '}'; - } - } } diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index a7b603582d8..f4034865022 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -20,26 +20,26 @@ package io.druid.segment.loading; import com.fasterxml.jackson.annotation.JsonProperty; +import org.hibernate.validator.constraints.NotEmpty; -import javax.validation.constraints.NotNull; import java.io.File; +import java.util.List; /** */ public class SegmentLoaderConfig { @JsonProperty - @NotNull - private String locations = null; + @NotEmpty + private List locations = null; @JsonProperty("deleteOnRemove") private boolean deleteOnRemove = true; @JsonProperty - @NotNull private File infoDir = null; - public String getLocations() + public List getLocations() { return locations; } @@ -51,6 +51,20 @@ public class SegmentLoaderConfig public File getInfoDir() { + if (infoDir == null) { + infoDir = new File(locations.get(0).getPath(), "info_dir"); + } + return infoDir; } + + @Override + public String toString() + { + return "SegmentLoaderConfig{" + + "locations=" + getLocations() + + ", deleteOnRemove=" + isDeleteOnRemove() + + ", infoDir=" + getInfoDir() + + '}'; + } } diff --git a/server/src/main/java/io/druid/segment/loading/StorageLocation.java b/server/src/main/java/io/druid/segment/loading/StorageLocation.java new file mode 100644 index 00000000000..339ed88058f --- /dev/null +++ b/server/src/main/java/io/druid/segment/loading/StorageLocation.java @@ -0,0 +1,84 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.loading; + +import com.google.common.collect.Sets; +import io.druid.timeline.DataSegment; + +import java.io.File; +import java.util.Set; + +/** +*/ +class StorageLocation +{ + private final File path; + private final long maxSize; + private final Set segments; + + private volatile long currSize = 0; + + StorageLocation(File path, long maxSize) + { + this.path = path; + this.maxSize = maxSize; + + this.segments = Sets.newHashSet(); + } + + File getPath() + { + return path; + } + + long getMaxSize() + { + return maxSize; + } + + synchronized void addSegment(DataSegment segment) + { + if (segments.add(segment)) { + currSize += segment.getSize(); + } + } + + synchronized void removeSegment(DataSegment segment) + { + if (segments.remove(segment)) { + currSize -= segment.getSize(); + } + } + + boolean canHandle(long size) + { + return available() >= size; + } + + synchronized long available() + { + return maxSize - currSize; + } + + StorageLocation mostEmpty(StorageLocation other) + { + return available() > other.available() ? this : other; + } +} diff --git a/server/src/main/java/io/druid/segment/loading/StorageLocationConfig.java b/server/src/main/java/io/druid/segment/loading/StorageLocationConfig.java new file mode 100644 index 00000000000..2ab5920313d --- /dev/null +++ b/server/src/main/java/io/druid/segment/loading/StorageLocationConfig.java @@ -0,0 +1,58 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.loading; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; +import java.io.File; + +/** + */ +public class StorageLocationConfig +{ + @JsonProperty + @NotNull + private File path = null; + + @JsonProperty + @Min(1) + private long maxSize = Long.MAX_VALUE; + + public File getPath() + { + return path; + } + + public long getMaxSize() + { + return maxSize; + } + + @Override + public String toString() + { + return "StorageLocationConfig{" + + "path=" + path + + ", maxSize=" + maxSize + + '}'; + } +} diff --git a/server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java b/server/src/test/java/io/druid/segment/loading/StorageLocationTest.java similarity index 91% rename from server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java rename to server/src/test/java/io/druid/segment/loading/StorageLocationTest.java index 90949215298..ca499d041e5 100644 --- a/server/src/test/java/io/druid/segment/loading/OmniSegmentLoaderTest.java +++ b/server/src/test/java/io/druid/segment/loading/StorageLocationTest.java @@ -30,13 +30,13 @@ import java.util.Arrays; /** */ -public class OmniSegmentLoaderTest +public class StorageLocationTest { @Test public void testStorageLocation() throws Exception { long expectedAvail = 1000l; - OmniSegmentLoader.StorageLocation loc = new OmniSegmentLoader.StorageLocation(new File("/tmp"), expectedAvail); + StorageLocation loc = new StorageLocation(new File("/tmp"), expectedAvail); verifyLoc(expectedAvail, loc); @@ -65,7 +65,7 @@ public class OmniSegmentLoaderTest verifyLoc(expectedAvail, loc); } - private void verifyLoc(long maxSize, OmniSegmentLoader.StorageLocation loc) + private void verifyLoc(long maxSize, StorageLocation loc) { Assert.assertEquals(maxSize, loc.available()); for (int i = 0; i <= maxSize; ++i) { From a001c1d32a71b70666bd98dfac0e69e4164e110b Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 13:02:40 -0500 Subject: [PATCH 70/92] 1) Remove TODO from ZkCoordinator. Always load the cache, 'cause I say so. --- .../server/coordination/ZkCoordinator.java | 11 ------ .../coordination/ZkCoordinatorConfig.java | 38 ------------------- 2 files changed, 49 deletions(-) delete mode 100644 server/src/main/java/io/druid/server/coordination/ZkCoordinatorConfig.java diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 1987f2212af..a55341a75a1 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -109,17 +109,6 @@ public class ZkCoordinator implements DataSegmentChangeHandler curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient()); curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); -/* TODO - if (config.isLoadFromSegmentCacheEnabled()) { - try { - loadCache(); - } - catch (Exception e) { - log.makeAlert(e, "Exception loading from cache") - .emit(); - } - } -*/ loadCache(); loadQueueCache.getListenable().addListener( diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinatorConfig.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinatorConfig.java deleted file mode 100644 index c524f152d8c..00000000000 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinatorConfig.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.server.coordination; - -import org.skife.config.Config; - -import java.io.File; - -/** - */ -public abstract class ZkCoordinatorConfig -{ - @Config("druid.paths.segmentInfoCache") - public abstract File getSegmentInfoCacheDirectory(); - - @Config("druid.segmentCache.enable") - public boolean isLoadFromSegmentCacheEnabled() - { - return true; - } -} From 044e43d231ae53615be22647ff682e04694178ba Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 14:17:11 -0500 Subject: [PATCH 71/92] 1) Remove last TODO related to segment lifetime. Don't think we care about that anymore. --- .../io/druid/client/ServerInventoryView.java | 2 +- .../client/ServerInventoryViewConfig.java | 36 ------------------- .../io/druid/guice/CoordinatorModule.java | 2 -- .../io/druid/server/master/DruidMaster.java | 27 ++++---------- 4 files changed, 8 insertions(+), 59 deletions(-) delete mode 100644 server/src/main/java/io/druid/client/ServerInventoryViewConfig.java diff --git a/server/src/main/java/io/druid/client/ServerInventoryView.java b/server/src/main/java/io/druid/client/ServerInventoryView.java index 07b1a08bdc4..f9187828ac3 100644 --- a/server/src/main/java/io/druid/client/ServerInventoryView.java +++ b/server/src/main/java/io/druid/client/ServerInventoryView.java @@ -63,7 +63,7 @@ public abstract class ServerInventoryView implements ServerView, ) { this.log = log; - this.inventoryManager = new CuratorInventoryManager( + this.inventoryManager = new CuratorInventoryManager<>( curator, new InventoryManagerConfig() { diff --git a/server/src/main/java/io/druid/client/ServerInventoryViewConfig.java b/server/src/main/java/io/druid/client/ServerInventoryViewConfig.java deleted file mode 100644 index 0e96cec5e34..00000000000 --- a/server/src/main/java/io/druid/client/ServerInventoryViewConfig.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.client; - -import org.skife.config.Config; -import org.skife.config.Default; - -/** - */ -public abstract class ServerInventoryViewConfig -{ - @Config("druid.master.removedSegmentLifetime") - @Default("1") - public abstract int getRemovedSegmentLifetime(); - - @Config("druid.announcer.type") - @Default("legacy") - public abstract String getAnnouncerType(); -} \ No newline at end of file diff --git a/server/src/main/java/io/druid/guice/CoordinatorModule.java b/server/src/main/java/io/druid/guice/CoordinatorModule.java index 75b7ab93e32..df0e36a53c5 100644 --- a/server/src/main/java/io/druid/guice/CoordinatorModule.java +++ b/server/src/main/java/io/druid/guice/CoordinatorModule.java @@ -24,7 +24,6 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; import com.metamx.common.concurrent.ScheduledExecutorFactory; -import io.druid.client.ServerInventoryViewConfig; import io.druid.client.indexing.IndexingServiceClient; import io.druid.db.DatabaseRuleManager; import io.druid.db.DatabaseRuleManagerConfig; @@ -49,7 +48,6 @@ public class CoordinatorModule implements Module public void configure(Binder binder) { ConfigProvider.bind(binder, DruidMasterConfig.class); - ConfigProvider.bind(binder, ServerInventoryViewConfig.class); JsonConfigProvider.bind(binder, "druid.manager.segment", DatabaseSegmentManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", DatabaseRuleManagerConfig.class); diff --git a/server/src/main/java/io/druid/server/master/DruidMaster.java b/server/src/main/java/io/druid/server/master/DruidMaster.java index 812d131a477..937c1c6bfa1 100644 --- a/server/src/main/java/io/druid/server/master/DruidMaster.java +++ b/server/src/main/java/io/druid/server/master/DruidMaster.java @@ -160,8 +160,8 @@ public class DruidMaster this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); - this.leaderLatch = new AtomicReference(null); - this.segmentSettingsAtomicReference= new AtomicReference(null); + this.leaderLatch = new AtomicReference<>(null); + this.segmentSettingsAtomicReference= new AtomicReference<>(null); this.loadManagementPeons = loadQueuePeonMap; } @@ -731,29 +731,16 @@ public class DruidMaster SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(cluster); // Stop peons for servers that aren't there anymore. - for (String name : Sets.difference( - loadManagementPeons.keySet(), - Sets.newHashSet( - Iterables.transform( - servers, - new Function() - { - @Override - public String apply(@Nullable DruidServer input) - { - return input.getName(); - } - } - ) - ) - )) { + final Set disdappearedServers = Sets.newHashSet(loadManagementPeons.keySet()); + for (DruidServer server : servers) { + disdappearedServers.remove(server.getName()); + } + for (String name : disdappearedServers) { log.info("Removing listener for server[%s] which is no longer there.", name); LoadQueuePeon peon = loadManagementPeons.remove(name); peon.stop(); } - // TODO: decrementRemovedSegmentsLifetime(); - return params.buildFromExisting() .withDruidCluster(cluster) .withDatabaseRuleManager(databaseRuleManager) From e0738b3b22a2c845c965ba65f0c07f762a9d57b6 Mon Sep 17 00:00:00 2001 From: cheddar Date: Thu, 12 Sep 2013 17:59:23 -0500 Subject: [PATCH 72/92] 1) Add tool to convert properties from the old settings to the new ones. --- .../initialization/PropertiesModule.java | 4 +- services/src/main/java/io/druid/cli/Main.java | 6 + .../cli/convert/ChatHandlerConverter.java | 48 ++++++ .../druid/cli/convert/ConvertProperties.java | 163 ++++++++++++++++++ .../convert/DatabasePropertiesConverter.java | 96 +++++++++++ .../cli/convert/IndexCacheConverter.java | 52 ++++++ .../io/druid/cli/convert/PrefixRename.java | 69 ++++++++ .../druid/cli/convert/PropertyConverter.java | 31 ++++ .../java/io/druid/cli/convert/Rename.java | 54 ++++++ 9 files changed, 522 insertions(+), 1 deletion(-) create mode 100644 services/src/main/java/io/druid/cli/convert/ChatHandlerConverter.java create mode 100644 services/src/main/java/io/druid/cli/convert/ConvertProperties.java create mode 100644 services/src/main/java/io/druid/cli/convert/DatabasePropertiesConverter.java create mode 100644 services/src/main/java/io/druid/cli/convert/IndexCacheConverter.java create mode 100644 services/src/main/java/io/druid/cli/convert/PrefixRename.java create mode 100644 services/src/main/java/io/druid/cli/convert/PropertyConverter.java create mode 100644 services/src/main/java/io/druid/cli/convert/Rename.java diff --git a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java index 2efe1e3eac7..3aad005dd68 100644 --- a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java +++ b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java @@ -19,6 +19,7 @@ package io.druid.server.initialization; +import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.io.Closeables; import com.google.inject.Binder; @@ -31,6 +32,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.InputStreamReader; import java.util.Properties; /** @@ -67,7 +69,7 @@ public class PropertiesModule implements Module if (stream != null) { log.info("Loading properties from %s", propertiesFile); try { - fileProps.load(stream); + fileProps.load(new InputStreamReader(stream, Charsets.UTF_8)); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 1f27ed215d3..c0d2925b68b 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -26,6 +26,7 @@ import com.google.inject.Module; import io.airlift.command.Cli; import io.airlift.command.Help; import io.airlift.command.ParseException; +import io.druid.cli.convert.ConvertProperties; import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.JsonConfigProvider; @@ -63,6 +64,11 @@ public class Main .withDefaultCommand(Help.class) .withCommands(CliRealtimeExample.class); + builder.withGroup("tools") + .withDescription("Various tools for working with Druid") + .withDefaultCommand(Help.class) + .withCommands(ConvertProperties.class); + builder.withGroup("internal") .withDescription("Processes that Druid runs \"internally\", you should rarely use these directly") .withDefaultCommand(Help.class) diff --git a/services/src/main/java/io/druid/cli/convert/ChatHandlerConverter.java b/services/src/main/java/io/druid/cli/convert/ChatHandlerConverter.java new file mode 100644 index 00000000000..7e3bb47e17a --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/ChatHandlerConverter.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Properties; + +/** + */ +public class ChatHandlerConverter implements PropertyConverter +{ + + private static final String PROPERTY = "druid.indexer.chathandler.publishDiscovery"; + + @Override + public boolean canHandle(String property) + { + return PROPERTY.equals(property); + } + + @Override + public Map convert(Properties properties) + { + if (Boolean.parseBoolean(properties.getProperty(PROPERTY))) { + return ImmutableMap.of("druid.indexer.task.chathandler.type", "curator"); + } + return ImmutableMap.of(); + } +} diff --git a/services/src/main/java/io/druid/cli/convert/ConvertProperties.java b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java new file mode 100644 index 00000000000..78db82ae663 --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java @@ -0,0 +1,163 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.common.base.Charsets; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import com.metamx.common.logger.Logger; +import io.airlift.command.Command; +import io.airlift.command.Option; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStreamWriter; +import java.io.Reader; +import java.io.Writer; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + */ +@Command( + name = "convertProps", + description = "Converts runtime.properties files from version 0.5 to 0.6" +) +public class ConvertProperties implements Runnable +{ + private static final Logger log = new Logger(ConvertProperties.class); + + private static final List converters = Lists.newArrayList( + new DatabasePropertiesConverter(), + new Rename("druid.database.rules.defaultDatasource", "druid.manager.rules.defaultTier"), + new Rename("druid.zk.paths.discoveryPath", "druid.discovery.curator.path"), + new Rename("druid.http.numThreads", "druid.server.http.numThreads"), + new Rename("druid.http.maxIdleTimeMillis", "druid.server.http.maxIdleTime"), + new Rename("druid.database.connectURI", "druid.db.connector.connectURI"), + new Rename("druid.database.user", "druid.db.connector.user"), + new Rename("druid.database.password", "druid.db.connector.password"), + new Rename("com.metamx.emitter", "druid.emitter"), + new Rename("com.metamx.emitter.logging.level", "druid.emitter.logging.logLevel"), + new IndexCacheConverter(), + new Rename("druid.paths.segmentInfoCache", "druid.segmentCache.infoPath"), + new Rename("com.metamx.aws.accessKey", "druid.s3.accessKey"), + new Rename("com.metamx.aws.secretKey", "druid.s3.secretKey"), + new PrefixRename("druid.bard.cache", "druid.broker.cache"), + new Rename("druid.client.http.connections", "druid.broker.http.numConnections"), + new Rename("com.metamx.query.groupBy.maxResults", "druid.query.groupBy.maxResults"), + new Rename("com.metamx.query.search.maxSearchLimit", "druid.query.search.maxSearchLimit"), + new Rename("druid.indexer.storage", "druid.indexer.storage.type"), + new Rename("druid.indexer.threads", "druid.indexer.runner.forks"), + new Rename("druid.indexer.taskDir", "druid.indexer.runner.taskDir"), + new Rename("druid.indexer.fork.java", "druid.indexer.runner.javaCommand"), + new Rename("druid.indexer.fork.opts", "druid.indexer.runner.javaOpts"), + new Rename("druid.indexer.fork.classpath", "druid.indexer.runner.classpath"), + new Rename("druid.indexer.fork.main", "druid.indexer.runner.mainClass"), + new Rename("druid.indexer.fork.hostpattern", "druid.indexer.runner.hostPattern"), + new Rename("druid.indexer.fork.startport", "druid.indexer.runner.startPort"), + new Rename("druid.indexer.properties.prefixes", "druid.indexer.runner.allowedPrefixes"), + new Rename("druid.indexer.taskAssignmentTimeoutDuration", "druid.indexer.runner.taskAssignmentTimeout"), + new Rename("druid.indexer.worker.version", "druid.indexer.runner.workerVersion"), + new Rename("druid.zk.maxNumBytes", "druid.indexer.runner.maxZnodeBytes"), + new Rename("druid.indexer.provisionResources.duration", "druid.indexer.autoscale.provisionPeriod"), + new Rename("druid.indexer.terminateResources.duration", "druid.indexer.autoscale.terminatePeriod"), + new Rename("druid.indexer.terminateResources.originDateTime", "druid.indexer.autoscale.originTime"), + new Rename("druid.indexer.autoscaling.strategy", "druid.indexer.autoscale.strategy"), + new Rename("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion", "druid.indexer.autoscale.workerIdleTimeout"), + new Rename("druid.indexer.maxScalingDuration", "druid.indexer.autoscale.scalingTimeout"), + new Rename("druid.indexer.numEventsToTrack", "druid.indexer.autoscale.numEventsToTrack"), + new Rename("druid.indexer.maxPendingTaskDuration", "druid.indexer.autoscale.pendingTaskTimeout"), + new Rename("druid.indexer.worker.version", "druid.indexer.autoscale.workerVersion"), + new Rename("druid.indexer.worker.port", "druid.indexer.autoscale.workerPort"), + new Rename("druid.worker.masterService", "druid.worker.overlordService"), + new ChatHandlerConverter(), + new Rename("druid.indexer.baseDir", "druid.indexer.task.baseDir"), + new Rename("druid.indexer.taskDir", "druid.indexer.task.taskDir"), + new Rename("druid.indexer.hadoopWorkingPath", "druid.indexer.task.hadoopWorkingPath"), + new Rename("druid.indexer.rowFlushBoundary", "druid.indexer.task.rowFlushBoundary"), + new Rename("druid.worker.taskActionClient.retry.minWaitMillis", "druid.worker.taskActionClient.retry.minWait"), + new Rename("druid.worker.taskActionClient.retry.maxWaitMillis", "druid.worker.taskActionClient.retry.maxWait"), + new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName") + ); + + @Option(name = "-f", title = "file", description = "The properties file to convert", required = true) + public String filename; + + @Option(name = "-o", title = "outFile", description = "The file to write updated properties to.", required = true) + public String outFilename; + + @Override + public void run() + { + File file = new File(filename); + if (!file.exists()) { + System.out.printf("File[%s] does not exist.%n", file); + } + + File outFile = new File(outFilename); + if (!outFile.getParentFile().exists()) { + outFile.getParentFile().mkdirs(); + } + + Properties fromFile = new Properties(); + + try (Reader in = new InputStreamReader(new FileInputStream(file), Charsets.UTF_8)) + { + fromFile.load(in); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + Properties updatedProps = new Properties(); + + int count = 0; + for (String property : fromFile.stringPropertyNames()) { + boolean handled = false; + for (PropertyConverter converter : converters) { + if (converter.canHandle(property)) { + for (Map.Entry entry : converter.convert(fromFile).entrySet()) { + ++count; + updatedProps.setProperty(entry.getKey(), entry.getValue()); + } + handled = true; + } + } + + if (!handled) { + updatedProps.put(property, fromFile.getProperty(property)); + } + } + + try (Writer out = new OutputStreamWriter(new FileOutputStream(outFile), Charsets.UTF_8)) + { + updatedProps.store(out, null); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + + log.info("Completed! Converted[%,d] properties.", count); + } +} diff --git a/services/src/main/java/io/druid/cli/convert/DatabasePropertiesConverter.java b/services/src/main/java/io/druid/cli/convert/DatabasePropertiesConverter.java new file mode 100644 index 00000000000..a46d5cd6e88 --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/DatabasePropertiesConverter.java @@ -0,0 +1,96 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.api.client.util.Maps; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + */ +public class DatabasePropertiesConverter implements PropertyConverter +{ + + private final List tableProperties = Lists.newArrayList( + "druid.database.segmentTable", + "druid.database.configTable", + "druid.database.ruleTable", + "druid.database.taskLockTable", + "druid.database.taskLogTable", + "druid.database.taskTable" + ); + private AtomicBoolean ran = new AtomicBoolean(false); + + @Override + public boolean canHandle(String property) + { + return tableProperties.contains(property) && !ran.get(); + } + + @Override + public Map convert(Properties properties) + { + if (!ran.getAndSet(true)) { + String tablePrefix = properties.getProperty("druid.database.segmentTable"); + + if (tablePrefix == null) { + tablePrefix = "druid"; + } + else { + tablePrefix = tablePrefix.split("_")[0]; + } + + Map retVal = Maps.newLinkedHashMap(); + + retVal.put("druid.db.tables.base", tablePrefix); + + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.segmentTable", "segments"); + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.configTable", "config"); + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.ruleTable", "rules"); + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.taskTable", "tasks"); + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.taskLockTable", "taskLock"); + addIfNotDefault(properties, tablePrefix, retVal, "druid.database.taskLogTable", "taskLog"); + + return retVal; + } + return ImmutableMap.of(); + } + + private void addIfNotDefault( + Properties properties, + String tablePrefix, + Map retVal, + String property, + String tablename + ) + { + final String value = properties.getProperty(property); + if (value != null) { + if (!value.equals(String.format("%s_%s", tablePrefix, tablename))) { + retVal.put(String.format("druid.db.tables.%s", tablename), value); + } + } + } +} diff --git a/services/src/main/java/io/druid/cli/convert/IndexCacheConverter.java b/services/src/main/java/io/druid/cli/convert/IndexCacheConverter.java new file mode 100644 index 00000000000..ca4d61bd041 --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/IndexCacheConverter.java @@ -0,0 +1,52 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Properties; + +/** + */ +public class IndexCacheConverter implements PropertyConverter +{ + + private static final String PROPERTY = "druid.paths.indexCache"; + + @Override + public boolean canHandle(String property) + { + return PROPERTY.equals(property); + } + + @Override + public Map convert(Properties properties) + { + final String value = properties.getProperty(PROPERTY); + + return ImmutableMap.of( + "druid.segmentCache.locations", + String.format( + "[{\"path\": \"%s\", \"maxSize\": %s}]", value, properties.getProperty("druid.server.maxSize") + ) + ); + } +} diff --git a/services/src/main/java/io/druid/cli/convert/PrefixRename.java b/services/src/main/java/io/druid/cli/convert/PrefixRename.java new file mode 100644 index 00000000000..3e8cd67c217 --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/PrefixRename.java @@ -0,0 +1,69 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.api.client.util.Maps; +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + */ +public class PrefixRename implements PropertyConverter +{ + private final String prefix; + private final String outputPrefix; + + private final AtomicBoolean ran = new AtomicBoolean(false); + + public PrefixRename( + String prefix, + String outputPrefix + ) + { + this.prefix = prefix; + this.outputPrefix = outputPrefix; + } + + @Override + public boolean canHandle(String property) + { + return property.startsWith(prefix) && !ran.get(); + } + + @Override + public Map convert(Properties properties) + { + if (!ran.getAndSet(true)) { + Map retVal = Maps.newLinkedHashMap(); + + for (String property : properties.stringPropertyNames()) { + if (property.startsWith(prefix)) { + retVal.put(property.replace(prefix, outputPrefix), properties.getProperty(property)); + } + } + + return retVal; + } + return ImmutableMap.of(); + } +} diff --git a/services/src/main/java/io/druid/cli/convert/PropertyConverter.java b/services/src/main/java/io/druid/cli/convert/PropertyConverter.java new file mode 100644 index 00000000000..46af47ed85d --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/PropertyConverter.java @@ -0,0 +1,31 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import java.util.Map; +import java.util.Properties; + +/** + */ +public interface PropertyConverter +{ + public boolean canHandle(String property); + public Map convert(Properties properties); +} diff --git a/services/src/main/java/io/druid/cli/convert/Rename.java b/services/src/main/java/io/druid/cli/convert/Rename.java new file mode 100644 index 00000000000..19139d8e363 --- /dev/null +++ b/services/src/main/java/io/druid/cli/convert/Rename.java @@ -0,0 +1,54 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.cli.convert; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Properties; + +/** + */ +public class Rename implements PropertyConverter +{ + private final String property; + private final String newProperty; + + public Rename( + String property, + String newProperty + ) + { + this.property = property; + this.newProperty = newProperty; + } + + @Override + public boolean canHandle(String property) + { + return this.property.equals(property); + } + + @Override + public Map convert(Properties properties) + { + return ImmutableMap.of(newProperty, properties.getProperty(property)); + } +} From f7c10e359400a1ac572209e50963f3675021b857 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 12 Sep 2013 16:37:58 -0700 Subject: [PATCH 73/92] rework tests in indexing service to be more unit testy --- .../coordinator/RemoteTaskRunner.java | 17 +- .../worker/WorkerCuratorCoordinator.java | 4 +- .../indexing/worker/WorkerTaskMonitor.java | 4 +- .../common/IndexingServiceCondition.java | 27 ++ .../TestMergeTask.java} | 48 ++- .../TestRealtimeTask.java | 7 +- .../io/druid/indexing/common/TestTask.java | 30 ++ .../io/druid/indexing/common/TestUtils.java | 48 +++ .../coordinator/RemoteTaskRunnerTest.java | 378 +++++++++--------- .../TestRemoteTaskRunnerConfig.java | 52 +++ .../SimpleResourceManagementStrategyTest.java | 7 +- .../worker/WorkerTaskMonitorTest.java | 189 +++++++++ 12 files changed, 595 insertions(+), 216 deletions(-) create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/IndexingServiceCondition.java rename indexing-service/src/test/java/io/druid/indexing/{TestTask.java => common/TestMergeTask.java} (66%) rename indexing-service/src/test/java/io/druid/indexing/{coordinator => common}/TestRealtimeTask.java (92%) create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/TestTask.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java index 679faaf3191..fb15cd75354 100644 --- a/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/coordinator/RemoteTaskRunner.java @@ -297,18 +297,16 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer if (runningTask != null) { ZkWorker zkWorker = findWorkerRunningTask(task.getId()); if (zkWorker == null) { - log.makeAlert("Told to run task that is in the running queue but no worker is actually running it?!") - .addData("taskId", task.getId()) - .emit(); - runningTasks.remove(task.getId()); + log.warn("Told to run task[%s], but no worker has started running it yet.", task.getId()); } else { log.info("Task[%s] already running on %s.", task.getId(), zkWorker.getWorker().getHost()); TaskAnnouncement announcement = zkWorker.getRunningTasks().get(task.getId()); if (announcement.getTaskStatus().isComplete()) { taskComplete(runningTask, zkWorker, task.getId(), announcement.getTaskStatus()); } - return runningTask.getResult(); } + + return runningTask.getResult(); } RemoteTaskRunnerWorkItem pendingTask = pendingTasks.get(task.getId()); @@ -552,12 +550,15 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer timeoutStopwatch.start(); synchronized (statusLock) { while (!isWorkerRunningTask(theWorker, task)) { - statusLock.wait(config.getTaskAssignmentTimeout().getMillis()); - if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeout().getMillis()) { + final long waitMs = config.getTaskAssignmentTimeout().toStandardDuration().getMillis(); + statusLock.wait(waitMs); + long elapsed = timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS); + if (elapsed >= waitMs) { log.error( - "Something went wrong! %s never ran task %s after %s!", + "Something went wrong! [%s] never ran task [%s]! Timeout: (%s >= %s)!", theWorker.getHost(), task.getId(), + elapsed, config.getTaskAssignmentTimeout() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index f4fe25d8c3b..812eee3d4cb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -183,7 +183,7 @@ public class WorkerCuratorCoordinator } } - public void announceTask(TaskAnnouncement announcement) + public void announceTastAnnouncement(TaskAnnouncement announcement) { synchronized (lock) { if (!started) { @@ -219,7 +219,7 @@ public class WorkerCuratorCoordinator try { if (curatorFramework.checkExists().forPath(getStatusPathForId(announcement.getTaskStatus().getId())) == null) { - announceTask(announcement); + announceTastAnnouncement(announcement); return; } byte[] rawBytes = jsonMapper.writeValueAsBytes(announcement); diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index af28f2402a4..c1382c994e6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -42,7 +42,7 @@ import java.util.concurrent.ExecutorService; * The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be * created that waits for new tasks. Tasks are executed as soon as they are seen. *

- * The monitor implements {@link QuerySegmentWalker} so tasks can offer up queryable data. This is useful for + * The monitor implements {@link io.druid.query.QuerySegmentWalker} so tasks can offer up queryable data. This is useful for * realtime index tasks. */ public class WorkerTaskMonitor @@ -122,7 +122,7 @@ public class WorkerTaskMonitor TaskStatus taskStatus; try { workerCuratorCoordinator.unannounceTask(task.getId()); - workerCuratorCoordinator.announceTask( + workerCuratorCoordinator.announceTastAnnouncement( TaskAnnouncement.create( task, TaskStatus.running(task.getId()) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/IndexingServiceCondition.java b/indexing-service/src/test/java/io/druid/indexing/common/IndexingServiceCondition.java new file mode 100644 index 00000000000..6157e69a6ce --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/IndexingServiceCondition.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.common; + +/** + */ +public interface IndexingServiceCondition +{ + public boolean isValid(); +} diff --git a/indexing-service/src/test/java/io/druid/indexing/TestTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java similarity index 66% rename from indexing-service/src/test/java/io/druid/indexing/TestTask.java rename to indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java index f8e13043872..906e6e6c1e6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/TestTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java @@ -17,37 +17,59 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing; +package io.druid.indexing.common; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolbox; +import com.google.common.collect.Lists; import io.druid.indexing.common.task.MergeTask; import io.druid.query.aggregation.AggregatorFactory; import io.druid.timeline.DataSegment; +import org.joda.time.DateTime; +import org.joda.time.Interval; import java.util.List; /** */ @JsonTypeName("test") -public class TestTask extends MergeTask +public class TestMergeTask extends MergeTask { - private final TaskStatus status; + public static TestMergeTask createDummyTask(String taskId) + { + return new TestMergeTask( + taskId, + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval(new DateTime(), new DateTime()), + new DateTime().toString(), + null, + null, + null, + null, + 0, + 0 + ) + ), + Lists.newArrayList() + ); + } + + private final String id; @JsonCreator - public TestTask( + public TestMergeTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators, - @JsonProperty("taskStatus") TaskStatus status + @JsonProperty("aggregations") List aggregators ) { super(id, dataSource, segments, aggregators); - this.status = status; + this.id = id; } @Override @@ -57,15 +79,9 @@ public class TestTask extends MergeTask return "test"; } - @JsonProperty - public TaskStatus getStatus() - { - return status; - } - @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { - return status; + return TaskStatus.running(id); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRealtimeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java similarity index 92% rename from indexing-service/src/test/java/io/druid/indexing/coordinator/TestRealtimeTask.java rename to indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java index 22f36bc5098..cc69067d23c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRealtimeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java @@ -17,14 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.coordinator; +package io.druid.indexing.common; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import io.druid.granularity.QueryGranularity; -import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.TaskResource; import io.druid.query.aggregation.AggregatorFactory; @@ -34,7 +32,7 @@ import io.druid.timeline.partition.NoneShardSpec; /** */ @JsonTypeName("test_realtime") -public class TestRealtimeTask extends RealtimeIndexTask +public class TestRealtimeTask extends RealtimeIndexTask implements TestTask { private final TaskStatus status; @@ -66,6 +64,7 @@ public class TestRealtimeTask extends RealtimeIndexTask return "test_realtime"; } + @Override @JsonProperty public TaskStatus getStatus() { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java new file mode 100644 index 00000000000..fa0e9072c84 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestTask.java @@ -0,0 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.common; + +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.task.Task; + +/** + */ +public interface TestTask extends Task +{ + public TaskStatus getStatus(); +} diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java new file mode 100644 index 00000000000..2ec6c474e6c --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.common; + +import com.google.common.base.Stopwatch; +import com.metamx.common.ISE; + +import java.util.concurrent.TimeUnit; + +/** + */ +public class TestUtils +{ + public static boolean conditionValid(IndexingServiceCondition condition) + { + try { + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (!condition.isValid()) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + } + catch (Exception e) { + return false; + } + return true; + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 47bfce6157b..e9eebfaba67 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -20,62 +20,46 @@ package io.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Function; import com.google.common.base.Joiner; -import com.google.common.base.Stopwatch; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; -import com.metamx.common.ISE; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.common.guava.DSuppliers; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.curator.cache.SimplePathChildrenCacheFactory; -import io.druid.indexing.TestTask; +import io.druid.indexing.common.IndexingServiceCondition; import io.druid.indexing.common.TaskStatus; -import io.druid.indexing.common.TaskToolboxFactory; -import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.common.TestMergeTask; +import io.druid.indexing.common.TestRealtimeTask; +import io.druid.indexing.common.TestUtils; import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.TaskResource; -import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import io.druid.indexing.coordinator.setup.WorkerSetupData; +import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; -import io.druid.indexing.worker.WorkerCuratorCoordinator; -import io.druid.indexing.worker.WorkerTaskMonitor; -import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; -import io.druid.query.aggregation.AggregatorFactory; import io.druid.server.initialization.ZkPathsConfig; -import io.druid.timeline.DataSegment; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingCluster; import org.apache.zookeeper.CreateMode; import org.easymock.EasyMock; -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.io.File; import java.util.Arrays; import java.util.Set; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -/** - * Several of the tests here are integration tests rather than unit tests. We will introduce real unit tests for this - * class as well as integration tests in the very near future. - */ public class RemoteTaskRunnerTest { private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); @@ -88,10 +72,8 @@ public class RemoteTaskRunnerTest private TestingCluster testingCluster; private CuratorFramework cf; private RemoteTaskRunner remoteTaskRunner; - private WorkerCuratorCoordinator workerCuratorCoordinator; - private WorkerTaskMonitor workerTaskMonitor; - private TestTask task; + private TestMergeTask task; private Worker worker; @@ -108,26 +90,77 @@ public class RemoteTaskRunnerTest .build(); cf.start(); cf.create().creatingParentsIfNeeded().forPath(basePath); + cf.create().creatingParentsIfNeeded().forPath(tasksPath); + cf.create().creatingParentsIfNeeded().forPath(statusPath); - task = makeTask(TaskStatus.success("task")); + + task = TestMergeTask.createDummyTask("task"); } @After public void tearDown() throws Exception { remoteTaskRunner.stop(); - workerCuratorCoordinator.stop(); - workerTaskMonitor.stop(); cf.close(); testingCluster.stop(); } @Test - public void testRunNoExistingTask() throws Exception + public void testRun() throws Exception + { + doSetup(); + + ListenableFuture result = remoteTaskRunner.run(task); + + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); + Assert.assertTrue(workerRunningTask(task.getId())); + mockWorkerCompleteSuccessfulTask(task); + Assert.assertTrue(workerCompletedTask(result)); + + Assert.assertEquals(task.getId(), result.get().getId()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + } + + @Test + public void testRunExistingTaskThatHasntStartedRunning() throws Exception { doSetup(); remoteTaskRunner.run(task); + Assert.assertTrue(taskAnnounced(task.getId())); + + ListenableFuture result = remoteTaskRunner.run(task); + + Assert.assertFalse(result.isDone()); + mockWorkerRunningTask(task); + Assert.assertTrue(workerRunningTask(task.getId())); + mockWorkerCompleteSuccessfulTask(task); + Assert.assertTrue(workerCompletedTask(result)); + + Assert.assertEquals(task.getId(), result.get().getId()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); + } + + @Test + public void testRunExistingTaskThatHasStartedRunning() throws Exception + { + doSetup(); + + remoteTaskRunner.run(task); + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); + Assert.assertTrue(workerRunningTask(task.getId())); + + ListenableFuture result = remoteTaskRunner.run(task); + + Assert.assertFalse(result.isDone()); + + mockWorkerCompleteSuccessfulTask(task); + Assert.assertTrue(workerCompletedTask(result)); + + Assert.assertEquals(task.getId(), result.get().getId()); + Assert.assertEquals(TaskStatus.Status.SUCCESS, result.get().getStatusCode()); } @Test @@ -139,7 +172,7 @@ public class RemoteTaskRunnerTest doSetup(); - remoteTaskRunner.run(makeTask(TaskStatus.success(new String(new char[5000])))); + remoteTaskRunner.run(TestMergeTask.createDummyTask(new String(new char[5000]))); EasyMock.verify(emitter); } @@ -149,31 +182,43 @@ public class RemoteTaskRunnerTest { doSetup(); - TestRealtimeTask theTask = new TestRealtimeTask( - "rt1", - new TaskResource("rt1", 1), - "foo", - TaskStatus.running("rt1") - ); - remoteTaskRunner.run(theTask); - remoteTaskRunner.run( - new TestRealtimeTask("rt2", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt2")) - ); - remoteTaskRunner.run( - new TestRealtimeTask("rt3", new TaskResource("rt2", 1), "foo", TaskStatus.running("rt3")) + TestRealtimeTask task1 = new TestRealtimeTask("rt1", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt1")); + remoteTaskRunner.run(task1); + Assert.assertTrue(taskAnnounced(task1.getId())); + mockWorkerRunningTask(task1); + + TestRealtimeTask task2 = new TestRealtimeTask("rt2", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt2")); + remoteTaskRunner.run(task2); + + TestRealtimeTask task3 = new TestRealtimeTask("rt3", new TaskResource("rt2", 1), "foo", TaskStatus.running("rt3")); + remoteTaskRunner.run(task3); + + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + return remoteTaskRunner.getRunningTasks().size() == 2; + } + } + ) ); - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); - while (remoteTaskRunner.getRunningTasks().size() < 2) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Cannot find running task"); - } - } + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + return remoteTaskRunner.getPendingTasks().size() == 1; + } + } + ) + ); - Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); - Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); } @@ -182,53 +227,62 @@ public class RemoteTaskRunnerTest { doSetup(); - TestRealtimeTask theTask = new TestRealtimeTask( - "rt1", - new TaskResource("rt1", 1), - "foo", - TaskStatus.running("rt1") - ); - remoteTaskRunner.run(theTask); - remoteTaskRunner.run( - new TestRealtimeTask("rt2", new TaskResource("rt2", 3), "foo", TaskStatus.running("rt2")) - ); - remoteTaskRunner.run( - new TestRealtimeTask("rt3", new TaskResource("rt3", 2), "foo", TaskStatus.running("rt3")) + TestRealtimeTask task1 = new TestRealtimeTask("rt1", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt1")); + remoteTaskRunner.run(task1); + Assert.assertTrue(taskAnnounced(task1.getId())); + mockWorkerRunningTask(task1); + + TestRealtimeTask task2 = new TestRealtimeTask("rt2", new TaskResource("rt2", 3), "foo", TaskStatus.running("rt2")); + remoteTaskRunner.run(task2); + + TestRealtimeTask task3 = new TestRealtimeTask("rt3", new TaskResource("rt3", 2), "foo", TaskStatus.running("rt3")); + remoteTaskRunner.run(task3); + Assert.assertTrue(taskAnnounced(task3.getId())); + mockWorkerRunningTask(task3); + + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + return remoteTaskRunner.getRunningTasks().size() == 2; + } + } + ) ); - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); - while (remoteTaskRunner.getRunningTasks().size() < 2) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Cannot find running task"); - } - } + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + return remoteTaskRunner.getPendingTasks().size() == 1; + } + } + ) + ); - Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); - Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); } @Test - public void testFailure() throws Exception + public void testStatusRemoved() throws Exception { doSetup(); - ListenableFuture future = remoteTaskRunner.run(makeTask(TaskStatus.running("task"))); - final String taskStatus = joiner.join(statusPath, "task"); + ListenableFuture future = remoteTaskRunner.run(task); + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); + + Assert.assertTrue(workerRunningTask(task.getId())); - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); - while (cf.checkExists().forPath(taskStatus) == null) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Cannot find running task"); - } - } Assert.assertTrue(remoteTaskRunner.getRunningTasks().iterator().next().getTask().getId().equals("task")); - cf.delete().forPath(taskStatus); + cf.delete().forPath(joiner.join(statusPath, task.getId())); TaskStatus status = future.get(); @@ -258,7 +312,7 @@ public class RemoteTaskRunnerTest Assert.assertTrue(existingTasks.contains("first")); Assert.assertTrue(existingTasks.contains("second")); - remoteTaskRunner.bootstrap(Arrays.asList(makeTask(TaskStatus.running("second")))); + remoteTaskRunner.bootstrap(Arrays.asList(TestMergeTask.createDummyTask("second"))); Set runningTasks = Sets.newHashSet( Iterables.transform( @@ -303,18 +357,14 @@ public class RemoteTaskRunnerTest { doSetup(); remoteTaskRunner.bootstrap(Lists.newArrayList()); - Future future = remoteTaskRunner.run(makeTask(TaskStatus.running("task"))); + Future future = remoteTaskRunner.run(task); - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); - while (cf.checkExists().forPath(joiner.join(statusPath, "task")) == null) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Cannot find running task"); - } - } + Assert.assertTrue(taskAnnounced(task.getId())); + mockWorkerRunningTask(task); - workerCuratorCoordinator.stop(); + Assert.assertTrue(workerRunningTask(task.getId())); + + cf.delete().forPath(announcementsPath); TaskStatus status = future.get(); @@ -325,68 +375,6 @@ public class RemoteTaskRunnerTest { makeWorker(); makeRemoteTaskRunner(); - makeTaskMonitor(); - } - - private TestTask makeTask(TaskStatus status) - { - return new TestTask( - status.getId(), - "dummyDs", - Lists.newArrayList( - new DataSegment( - "dummyDs", - new Interval(new DateTime(), new DateTime()), - new DateTime().toString(), - null, - null, - null, - null, - 0, - 0 - ) - ), - Lists.newArrayList(), - status - ); - } - - private void makeTaskMonitor() throws Exception - { - workerCuratorCoordinator = new WorkerCuratorCoordinator( - jsonMapper, - new ZkPathsConfig() - { - @Override - public String getZkBasePath() - { - return basePath; - } - }, - new TestRemoteTaskRunnerConfig(), - cf, - worker - ); - workerCuratorCoordinator.start(); - - final File tmp = Files.createTempDir(); - - // Start a task monitor - workerTaskMonitor = new WorkerTaskMonitor( - jsonMapper, - cf, - workerCuratorCoordinator, - new ThreadPoolTaskRunner( - new TaskToolboxFactory( - new TaskConfig(tmp.toString(), null, null, 0), - null, null, null, null, null, null, null, null, null, jsonMapper - ) - ), - new WorkerConfig().setCapacity(1) - ); - jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test")); - jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime")); - workerTaskMonitor.start(); } private void makeRemoteTaskRunner() throws Exception @@ -426,30 +414,60 @@ public class RemoteTaskRunnerTest ); } - private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig + private boolean taskAnnounced(final String taskId) { - @Override - public boolean isCompressZnodes() - { - return false; - } + return pathExists(joiner.join(tasksPath, taskId)); + } - @Override - public Period getTaskAssignmentTimeout() - { - return new Period(60000); - } + private boolean workerRunningTask(final String taskId) + { + return pathExists(joiner.join(statusPath, taskId)); + } - @Override - public long getMaxZnodeBytes() - { - return 1000; - } + private boolean pathExists(final String path) + { + return TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + try { + return cf.checkExists().forPath(path) != null; + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } - @Override - public String getWorkerVersion() - { - return ""; - } + private boolean workerCompletedTask(final ListenableFuture result) + { + return TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + return result.isDone(); + } + } + ); + } + + private void mockWorkerRunningTask(final Task task) throws Exception + { + cf.delete().forPath(joiner.join(tasksPath, task.getId())); + + TaskAnnouncement taskAnnouncement = TaskAnnouncement.create(task, TaskStatus.running(task.getId())); + cf.create().forPath(joiner.join(statusPath, task.getId()), jsonMapper.writeValueAsBytes(taskAnnouncement)); + } + + private void mockWorkerCompleteSuccessfulTask(final Task task) throws Exception + { + TaskAnnouncement taskAnnouncement = TaskAnnouncement.create(task, TaskStatus.success(task.getId())); + cf.setData().forPath(joiner.join(statusPath, task.getId()), jsonMapper.writeValueAsBytes(taskAnnouncement)); } } diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java new file mode 100644 index 00000000000..c006535251a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/TestRemoteTaskRunnerConfig.java @@ -0,0 +1,52 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.coordinator; + +import io.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import org.joda.time.Period; + +/** + */ +public class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig +{ + @Override + public boolean isCompressZnodes() + { + return false; + } + + @Override + public Period getTaskAssignmentTimeout() + { + return new Period("PT1S"); + } + + @Override + public long getMaxZnodeBytes() + { + return 1000; + } + + @Override + public String getWorkerVersion() + { + return ""; + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index cbfcc484f9e..03fd983437c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -26,7 +26,7 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; -import io.druid.indexing.TestTask; +import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; import io.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; @@ -69,7 +69,7 @@ public class SimpleResourceManagementStrategyTest ) ); - testTask = new TestTask( + testTask = new TestMergeTask( "task1", "dummyDs", Lists.newArrayList( @@ -85,8 +85,7 @@ public class SimpleResourceManagementStrategyTest 0 ) ), - Lists.newArrayList(), - TaskStatus.success("task1") + Lists.newArrayList() ); simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( autoScalingStrategy, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java new file mode 100644 index 00000000000..399ff4ab05a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -0,0 +1,189 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.worker; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.base.Joiner; +import com.google.common.io.Files; +import io.druid.curator.PotentiallyGzippedCompressionProvider; +import io.druid.indexing.common.IndexingServiceCondition; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.TestMergeTask; +import io.druid.indexing.common.TestRealtimeTask; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.common.config.TaskConfig; +import io.druid.indexing.coordinator.TestRemoteTaskRunnerConfig; +import io.druid.indexing.coordinator.ThreadPoolTaskRunner; +import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.initialization.ZkPathsConfig; +import junit.framework.Assert; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.test.TestingCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; + +/** + */ +public class WorkerTaskMonitorTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final Joiner joiner = Joiner.on("/"); + private static final String basePath = "/test/druid"; + private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath); + private static final String statusPath = String.format("%s/indexer/status/worker", basePath); + + private TestingCluster testingCluster; + private CuratorFramework cf; + private WorkerCuratorCoordinator workerCuratorCoordinator; + private WorkerTaskMonitor workerTaskMonitor; + + private TestMergeTask task; + + private Worker worker; + + @Before + public void setUp() throws Exception + { + testingCluster = new TestingCluster(1); + testingCluster.start(); + + cf = CuratorFrameworkFactory.builder() + .connectString(testingCluster.getConnectString()) + .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) + .build(); + cf.start(); + cf.create().creatingParentsIfNeeded().forPath(basePath); + //cf.create().creatingParentsIfNeeded().forPath(tasksPath); + //cf.create().creatingParentsIfNeeded().forPath(statusPath); + + worker = new Worker( + "worker", + "localhost", + 3, + "0" + ); + + workerCuratorCoordinator = new WorkerCuratorCoordinator( + jsonMapper, + new ZkPathsConfig() + { + @Override + public String getZkBasePath() + { + return basePath; + } + }, + new TestRemoteTaskRunnerConfig(), + cf, + worker + ); + workerCuratorCoordinator.start(); + + final File tmp = Files.createTempDir(); + + // Start a task monitor + workerTaskMonitor = new WorkerTaskMonitor( + jsonMapper, + cf, + workerCuratorCoordinator, + new ThreadPoolTaskRunner( + new TaskToolboxFactory( + new TaskConfig(tmp.toString(), null, null, 0), + null, null, null, null, null, null, null, null, null, jsonMapper + ) + ), + new WorkerConfig().setCapacity(1) + ); + jsonMapper.registerSubtypes(new NamedType(TestMergeTask.class, "test")); + jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime")); + workerTaskMonitor.start(); + + task = TestMergeTask.createDummyTask("test"); + } + + @After + public void tearDown() throws Exception + { + workerTaskMonitor.stop(); + cf.close(); + testingCluster.stop(); + } + + @Test + public void testRunTask() throws Exception + { + cf.create() + .creatingParentsIfNeeded() + .forPath(joiner.join(tasksPath, task.getId()), jsonMapper.writeValueAsBytes(task)); + + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + try { + return cf.checkExists().forPath(joiner.join(tasksPath, task.getId())) == null; + } + catch (Exception e) { + return false; + } + } + } + ) + ); + + + Assert.assertTrue( + TestUtils.conditionValid( + new IndexingServiceCondition() + { + @Override + public boolean isValid() + { + try { + return cf.checkExists().forPath(joiner.join(statusPath, task.getId())) != null; + } + catch (Exception e) { + return false; + } + } + } + ) + ); + + TaskAnnouncement taskAnnouncement = jsonMapper.readValue( + cf.getData().forPath(joiner.join(statusPath, task.getId())), TaskAnnouncement.class + ); + + Assert.assertEquals(task.getId(), taskAnnouncement.getTaskStatus().getId()); + Assert.assertEquals(TaskStatus.Status.RUNNING, taskAnnouncement.getTaskStatus().getStatusCode()); + } +} \ No newline at end of file From 66c2ea2ce3b661ddff6a02cb5cccb387859e1444 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 13 Sep 2013 16:26:41 -0500 Subject: [PATCH 74/92] 1) Move doc/publications to top-level publications directory. 2) Move data_flow and segment propagation stuff to static.druid.io/docs/flow and remove from repository --- doc/data_flow_realtime.dot | 39 ------ doc/data_flow_realtime.png | Bin 98571 -> 0 bytes doc/data_flow_realtime.svg | 128 ------------------ doc/data_flow_simple.dot | 27 ---- doc/data_flow_simple.png | Bin 64023 -> 0 bytes doc/data_flow_simple.svg | 92 ------------- doc/publications/vldb/README.md | 4 - doc/publications/vldb/druid.pdf | Bin 520154 -> 0 bytes doc/segment_propagation.png | Bin 75005 -> 0 bytes doc/segmentation_propagation.txt | 13 -- .../vldb/.gitignore | 0 .../vldb/Makefile | 0 .../vldb/druid.bib | 0 .../vldb/druid.tex | 0 .../vldb/figures/Druid_cluster.png | Bin .../vldb/figures/cluster_scan_rate.pdf | Bin .../vldb/figures/concise_plot.pdf | Bin .../vldb/figures/core_scan_rate.pdf | Bin .../vldb/figures/data_reading.png | Bin .../vldb/figures/druid_message_bus.png | Bin .../vldb/figures/druid_realtime_flow.png | Bin .../vldb/figures/druid_segment.png | Bin .../vldb/figures/zookeeper.png | Bin .../vldb/src/druid_plot.R | 0 .../vldb/src/druid_tables.R | 0 .../vldb/vldb.cls | 0 26 files changed, 303 deletions(-) delete mode 100644 doc/data_flow_realtime.dot delete mode 100644 doc/data_flow_realtime.png delete mode 100644 doc/data_flow_realtime.svg delete mode 100644 doc/data_flow_simple.dot delete mode 100644 doc/data_flow_simple.png delete mode 100644 doc/data_flow_simple.svg delete mode 100644 doc/publications/vldb/README.md delete mode 100644 doc/publications/vldb/druid.pdf delete mode 100644 doc/segment_propagation.png delete mode 100644 doc/segmentation_propagation.txt rename {doc/publications => publications}/vldb/.gitignore (100%) rename {doc/publications => publications}/vldb/Makefile (100%) rename {doc/publications => publications}/vldb/druid.bib (100%) rename {doc/publications => publications}/vldb/druid.tex (100%) rename {doc/publications => publications}/vldb/figures/Druid_cluster.png (100%) rename {doc/publications => publications}/vldb/figures/cluster_scan_rate.pdf (100%) rename {doc/publications => publications}/vldb/figures/concise_plot.pdf (100%) rename {doc/publications => publications}/vldb/figures/core_scan_rate.pdf (100%) rename {doc/publications => publications}/vldb/figures/data_reading.png (100%) rename {doc/publications => publications}/vldb/figures/druid_message_bus.png (100%) rename {doc/publications => publications}/vldb/figures/druid_realtime_flow.png (100%) rename {doc/publications => publications}/vldb/figures/druid_segment.png (100%) rename {doc/publications => publications}/vldb/figures/zookeeper.png (100%) rename {doc/publications => publications}/vldb/src/druid_plot.R (100%) rename {doc/publications => publications}/vldb/src/druid_tables.R (100%) rename {doc/publications => publications}/vldb/vldb.cls (100%) diff --git a/doc/data_flow_realtime.dot b/doc/data_flow_realtime.dot deleted file mode 100644 index e29cd47a743..00000000000 --- a/doc/data_flow_realtime.dot +++ /dev/null @@ -1,39 +0,0 @@ -digraph DruidDataFlow { - graph [bgcolor="#fffef5", clusterrank=global, rankdir=TB]; - node [color="#0a0701", fillcolor="#fdf4c6", fontname=Helvetica, shape=box, style=filled, label="\N"]; - edge [color="#377d18"]; - - 1 [label = "REST query"]; - 2 [label = "realtime_data_src"]; - 4 [label = "indexed segments\nblob_store (S3, HDFS)"]; - 5 [label = "Realtime.working_set"]; - 6 [label = "Realtime.spill_disk"]; - 7 [label = "HDFS_data_src"]; - 8 [label = "external_data_src"]; - 9 [label = "Realtime.local_disk"]; - 10 [label = "metadata_store (mysql)"]; - 11 [label = "HadoopDruidIndexer.working_set"]; - 12 [label = "IndexerService.working_set"]; - -2 -> 5 [label = "Realtime.Firehose"]; -5 -> 6 [label = "Realtime.Firehose"]; - -5 -> 9 [label = "Realtime.Indexer"]; -6 -> 9 [label = "Realtime.Indexer"]; -5 -> 10 [label = "Realtime.MetadataUpdater"]; - -7 -> 11 [label = "HadoopDruidIndexer"]; -11 -> 4 [label = "HadoopDruidIndexer"]; -11 -> 10 [label = "HadoopDruidIndexer"]; - -8 -> 12 [label = "IndexerService"]; -12 -> 4 [label = "IndexerService"]; -12 -> 10 [label = "IndexerService"]; - - -9 -> 4 [label = "Realtime.segmentPusher"]; - -5 -> 1 [label = " results "]; - -1 -> 5 [label = " query "]; -} diff --git a/doc/data_flow_realtime.png b/doc/data_flow_realtime.png deleted file mode 100644 index f05bd0f5ea2f0bd1b3dd0035e03ff26066d4a314..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 98571 zcmcG$Wl)?=v@S{@xCM7eaCg@L!QI`11b5fq9^74mySux)4DRl3XJX&J_x{>-tIk^m z6f<;B_v&T+JZljwD=h*KgAD@$0s=24`b8cDQv4tw zi(mqCGFiaip{+&L>_9*edVYPqi=#ro0lr9VFQjU(U}a?Qq-SdgBIoeUz)bO*h2cjw zT2@-tkK|H%1|MYbjYfs!>u1LjjmR@a z^NB^I289HXBYeh%f=qRtd&BClOd#y<+_w_(*@BD12SQ$bWB$BGN~R{2P}GnK(d1`1hFt zgVBQgO!BiRIH^o9Q$9AiY{mf!T;C(NYRqfc==x|)W?t{Z?aH>kJCP zwI!U5iqZ_ey-Dk5Rh{z(ACt^d#8>r6A|}>sfjon5H?4KF{fKHOBawlQ?ck%(O(241@6b|_kbf`8;i5)|A5BbkN_$?ypsKWZESX>tJt zGYH!2aovYi@Af?dl72RA(~oKu!DRL7MEiGO$HE4YyEzJUDm4W#@81W&%tQ6gbY?*% z*CKaTs|%*3lcs94Id0&8*1DJevdAro_STxA#-}8Qscc6K>gwSKx%?z{!_6V>NglE4 zUXVkck^UGZ?&?)Q`#OzOX^D{f6SpIzf{%VKN$m4)DP~s%0M{q=nsv%*M&jto2CkUI z_e@YMf4&!Bh{?@|y1}Rbg}vGJ@Qdh+YGq z(V+T~XX`tRf@-{VuQV|r zRL%(21ZM)d9Ck@w17OM}y6h234HoYp+K5qG!I7~X^Jz+T z_!nqYRM+~{=FzA#mqVaP2#$I>A>pNjV8i>L$)tJz;2Ym?Es`xyuK#ct7cT3BzcI83 zm5Ns8Ci4v>fws&`FjqQv$P4J822YVk`-5X8&H4&|3m%El+s!z$ z4SeW5uJEnB?JKM~6Ted{R^Qq>YM62q%8iFrCvs35ib%b$+9jk zK+QWeLC+P-P$DHySUwjplWF#TVF@_cN8E^dpE`tm@bq0h2F@wrQ`PWYqL-`HeRgQ4 zNCdyUOPB!8@GnrB*2JFROQuYI2MY_vAo;@VP)9GdM={y29R+h4-TW1_aPNJ409Yr zO{mA|Z{UxiG@8{mo%sf^^d%1p#F>7NT-C$$)Rfm)9^Wg_GP)i5d;lq$B1ES1Cd`UD z0`tN-wRg1uJLY%;>QlrJnP-@d&7P*ehRvT*c6TrQ|i+fFBuPOveleBo9bQ#Z|`VeO#}+}dJAVK0*F z6Y}W|gt|kbiZ0U_67_i92Z`^=m1I6Tn%N%%z`G$F_+x%ruEeBWK1*95k6QqU-xFvC z_vTbdrt2Ahe49`SI)M>Vs!cgzEO?g1eo+O8|XyLWBm%XpJ;7n7XA zYS}@cS>^*Z494B^b|~6vSu(q_gcvtjTv>VPJ zZcWrj8b-B(k;3f|+zG-vOWpR|#vVRBF|>vQh6XG)H6@?aV(g*dyA(I#Itx|U;5?fV zS8*mdCF+}uqas#xyl8Y(n0x|i0D3g0`f$RQVGz-wUUNTckMjIdV(RH4M%<)M{Q;fO z7t$=UPvM48Nv#-BF{u+@S|Fuhzf|@v+zRpN(s(x5!AL_1+^_1UHclg2vh1jOts#@2 z933oX=||ors3buP#G}>%2sR_~+aS&|-myvP)tPR2--owWYLXw-aAVu6vpROUg%=Y# z2k>>dwT2+JxJt|qv`1;M;WjcrTJ&DsS;69-eTU8xaNpwXj}p?#H3)R5>`rn$BO)e* zm{;`9xkT+7#n8BF{P@uwn)`T%wjR&x35^xZP5*E;W}$`-9S@R)+}4I-{4E07?Ngdv ze=o^CmBW??Jdd^+HR^bnISoWF^jG3ls!F4xXYNcFgPH+sT>6drf-An&g`M{CSl5Mbsr~fF+l~!H>;`$)PqZP zuS!rT7Aj1VYPOljF!-Py1l+fmILAf8-OBdckru4aAp%?u<|VuTWDP>SU+~44`XA9I zc*-NvlvdGtXVF?|Y|EwFl0%x*G|rP)dcNlbt~=Z^fA@s$&YJjqJb6tIx90ZQb@cMO zWAT2_n)(pL`9W^ALq^_m1iZSxW~{37Js7Er?sgxzjLj5kDok7qVtIY7FDC_bu>55< zj3joWZbb#EH%Sw`ckb{)wpd@-vHZws z@Qz{~Bb9PvO1IscXP7)6(rZ!PM+RQ_daJeW(ks1t1-rV`Qrg67N`Cd93KB>6nw-ixHGa@Gt_u*&75w8yx-iIIK=!Y`GXBj!0!Q=M;8FnNhury;4F|W&bM%7Tn}SBOYle-G ##~kqu+`do57qg!@-9j z8BjpwlbXg9d65ZecLB^aZY9>d6(yP1r6@1DC|0Cka?jRfr*8%Kc$zffwK@#v+=z2- z(}vJ;iI$<%`R-GCtF=K*#{XRr}QA|G0{ z*V{$j&X?r!6Qq||ML;oZH-;~He%KAr_oL%VH2F|*TR90r*cNv27QWQhqFAln=oU?2 zz}OS;T}4> z)YC)RJRH7na@^;zKN@U^f#UKW)=XnzZY1hMfranE(frogQcfi#NAv6uK1P-VFjU|- z)N6o*UGh(a;4cwu^clsQYyuFbRcZwH^6e({`raKsaDj@K2MDFd)h#Gep4@B56yG&e z?yq)yK&&odVq^oC$Irw$nOOw+1di%+Kz)BzyZahn~ zSJ}67h|ijn{r%Hjns7(aM~cn{<1Nz-2h=PTmCPJgjHG61MVeQV;NQV_2%hv3YO#t) zjRG|}SDm?xlt+N7`WWkbwtXQ}T|t0bm)m_GpUuM&zDwPDwV7HwrZKhOV-~4eLNe84 zBC3aaI`b!kFmAo(mU6LMGJdqc9JgwB)@org710%+W;giGg+4WnB_95T2pyZr0sQbh z+K7XPexqAL;c(lI^l-9``du<&QI{4=%n4SfVoHI^AuX5NmQ-_F-0*_#oJrXyk3Rh* z(#7irMc;A7c_F4r@p2W0*{`z+0I&@n)VF!PA*mUa?A~L=^GhOIzaM|sn(r77y~kv~ zXc$MROj=(zBAE@f#_$~OjLeUZG~69h8yglWBu8F`DRkIw5S?~AtR+}#*B20f1b7W? zT$=ZcHQ!A&1(!P6;Cdz&5nd=NDIO@EgkZ=^#}JucumW|G_aZoHl*?DPZG!wWq}}_0 zpfn+VMH4r@l_gz`hvI?AvC;l!HDM7{1})ToOL=c$9<($si2-qa;vnpsAi>7ngT#7* zg@|2^B^8+gDfK;Cuv*=hQ&rgcDA@I#2c}cBC4ov9&emlq^~;o2b#Y2gmpebAl&zf{ z_MI{@(B)YH8ViJdq^>!O*^w6W=wd1(+QCBqh^RYe>Lb=DT&GQaHfg}^AJZ5&3#8_aHF+(*=fI0<&~cqG!rvrNW1UC-QylS z6;3SJ^SmH3XO<==B|ULWNaxFL^389@?_44y6Lfq#TapZBFk00_f1u6`2xKy?ABF#oUe*gu%Jl~ zuxpp~7O64p)4h;U0Ir)Acd9M&62Sz<_hwBc2>mVZdd>Ka7W2uJuhmIMHIHU1xgy4r z{fvrW6Pva_NAMJP$WAX<3G_2GFyGOrl4^FeC?!1wS1Xr0UiK{@ur9@WR%hn`bfegSROh>wN|8!kE&Exy?xi}+y@zn|FE;Uxs%9=Nq{p>l8Fzc` z455f*q|$GsI&NMsS+oAe;coRYf9yw$6p9LMFRxA`+IG59^M~@GEp;PD%$?E)X+i;{ znCyTqO;6U5ksZ#YtLhS%5f+qO^{>S#O1ea?DuIPYwaX>tAC8Y`X&|9)yaa@i#~s+E zHSuEdhE3&?42}_-D;5MTEBD#Fl2Uu@9QteIE?Wo2LqBo5*Xt(7CWZA~E)H)Kr(3Uh zKzi&L%Qx(?1fGfYGas(PTP9N-FA_jHR1VkDD@{1e30MGzjj20y#|^vIKp9+C;kl%x z#wP_>oYLqm0bmz;ytb)X{E2yDn;bXTG|zZgfsY@5pAcm`OIDtps|U2X)@mjNTC+CFdbqx^AjAHa`~pzF5H_F;I&d=cOFSTjLJB)=@}E|iPif=!HiH;$1YOFK46H!go%rD z++vYqU9_d$F&py86<$|spF!!8me0U$VQ#EaS`Ty>w^vEEFbA7K^mo!@j z)}LCsb?wL7Fa0~)%9TkR$O-9g_|(RMonLKME*GE|Zdvx{y)R#OK#WUCerqG}+Z5RU z4t)K;hw*;SdwYLRCpB*)3teEUpMUfry;vM(ov*emhdf;z;&a;kC=x+YPt5h3_%)#L zULb6HPcV(#||rC(4k$;)WC z==Nm!i1Yn#iDm2Sg>p~fc*HksuFUskok{pQ;E7iPWLiTH-WecyJR$G8xrg3{YdYW| z^N<~qnM&SzVa%UucitH|X63ukXa)YshQI6xktjq&i9CLyd<8|u2Xvi!)g5rg>KALz z!cYl6bk~X*JHOFmb49*o9{t$ocyK*4Pi?k=(Lq!jCjQGY#IfH#5Vn56eh9qwWH`m8 zU4;>u=9K1)C~$^D6Hc7=flBJrZGk*)8iGF0ol zc>CHJ;LBM~ijJ)RFaias`h<}~EzLt*K@Bd#PcB~0AExm059zawr&kv7)*=95F)ER| zU444aKOA-DKTR?PNeA^9^GOSoE59%t&oVS21pb+5h=KI7U<9`L+PH4~Aact3Be$r? zWNer+T}|)OB;RJ)C2Py`D!=BG6i|>d8|jw_`LFB27#1l8!e?~$cnE)>kT^TZZMmcb z23~FDM?fp9Z(1$bGq;oI)}G4)e8-NTC~$1=x#-J>X;9wJaYHcQG`WS)!2aiCaDmpA zA3PXsFzL0Up22Y7-&W07-{*8@A^3vrc(2?2IkoLJ5-1JA$ICseAhDLPUcndH(VC|J zWC;Vb70PMp9$cS88`t#ZA;DtqsH94c`0$0GgkvuKsa>RdF3Kx9B2@dHeci)PlO3oNSrDO_^VXegfswN=o)SaMgU_Hn_`awy2@pg3YKHuNqz!xv}hvOogqHruF zY0WCg_I?~R0J3C7`M0Uz+NF;hF&)tC9w!7^feN2WpX{&@wAj7D!Bn!_;?n--2*ZMb zRvt10!~LLAdPFZC$8NM5wyLwaBb;`Ap-k5Uh?sH^MGIjU-U#?%bzxK9XkzeI^*^H{ zevhVvkXM-w$+z7ZN@UZez@BqOig{j(Bh|{>J9{3jzSqBFhC4o|eQaYEo!|IF@*yz* zT|)Ov1?4a6;e<=0XQb~^5uEGlnRq`9y4v5}kLZ9W-{|77C{xEYrQ?)DXXIc53`81f}p% zpojZoZc*qcF1c(Y;iFXcp)GJHc-p-UY`lF%zB(J`$Iw6_mz6+ke{ZJ+?{-?JoGG3B z{NH8`yT9Ncz{4s?5a+GTJ75}iO{la?WKEjgU-yeScQ8^QJ&iqwg}-U|A6R)r%sHO? zr($xnzgAmdD2G!Yvt^GboFrQ)PjcV#Qd{%fYZ;MnS2Cu>M-tA(p1Wmo?dT%%-v{ml z|2mgiYI7Sd4()lpSgct|8it>5`hLRrJ>@^04mojK(K75MlA#;Yx~UQ;;>6_sL`lkK zAZ6RCr%;N3&#Sz_uO3|{NWr9eykwvo=u~+>@6zz%U5XY;b)rYfyL9M(sAQ(U>YZ1h zndF81Cx90mZ=Eu;Z);j_jTJ}>k#c`^C?bw^E-51J zAt@3+b^qgcT!a29t%G~{YoZ45j&4HSV>XJJhKC8G) zuE3w-X=E#DsD!1rzMEX`6Dvj>P4giNCc*q+BgOn}87*za&Wf|u9z^?|SAxxUYj+{m z9p!FnF5BM*x=$9audcMF(vjenRh@%X1a`F8;^*tRR=pZ|?iX&_PF1w{n~5AN{C^7v z@AGb4v-NR7Q+zA(jR zOrC%;56jdndA_UV-CCx6o_OQ!q;SSFxdsRG@uJSODJ|&F@?91)+fR3N=It5IkTy0K zvb#$pcMLXqR1-^FsHd}4^`8|I)v}ue()y8-j%G}iPMc`UhU>qhp}VbC^17yOxuLde z#LzZ4y1*4-pqjmN_|$GJ|HO@jWpayhRLZ#4R7API81G1aoJ1v^^hf#P@P8>E`@B;W zGQIxYz;i*|1$&ztN}qjMh|ln??wtT)qUu6N?lO#Mc%dR&g{dSLW}-4vT5Dd)q@~ON4You~VEw@0mV56juMQXb!UX(Yo(CA&-9#BqH3{WfqGP08N9;$N zZ*)s)P)|OPDWk>S?ePQcJ=4UyZGh{ir)?{1P4wYZF3nz+IA zNAb{+-&F0y#bJIIQ)o_4tZ)eUu&W@Jx2B5g`!s>{L2AnkpEpfJJ-)2f zxeA}o3WtHE&zGLt_+j7Fl-dXmxOvu_pWNg9v^T~9)v?gOO>y$7@0Ugw_3fH`okle- z+sPP^cs54NzrhJ~2=vZO@;$$O{p!-4WhN@Zy<21T{oy`Ed(liGor4kJ> z@m7*n?k>31T)`5p6-NTZmM`>hv#l-0ss_ zNHw|84if4=En0E)z{UG?;oE&DVvt+9W1X7qN@G2atR8&x1r%RmN3#TnHwa!`bQQ;3 zbn0~RaEAnB{3QC0?wUPT075={u~2)D;ym_@uMc{4q9XFKF~&W0)=h$efg3c?FJYdQ z&zTuDMxgIJkqJ_|t0_){k3R6o!d-V`cHBhSa*Iv9ChQt+5-0z&D-6vj=&`+t|_CICeaI5eiZYGpw4kFIeG6oV- zS|RI^vl(V6@l)75BQNY&)}m^Z{F%7g`!TlKW23`hA#eKh0x8%lEYfQn9)jxV3n$U# z=$^!mdeHrkxViM}NvV`fj{SqUU2*rMGdgW9(CWZePul(B!a)yv1^|`lvKac7+I06x zBZMYpWWrx+`+e3&&ylT6j=oB+ek7ZS`#GN_T+OdKzlvB-khgi!N}q)gsBhF>x8HQ_ z=l)$Fz0sHk<x%9ISXJ&IAEbC9U{nzNIT+}Fk6Pf)jk*QwI?`=7rFRXK zOeBt$;AFPt$0e-5-sqZ{sqNRM_uiLHTfeL-Jv$nylv&)#Jn3Z;thZgV3)dG>lrmW^ zpmkS;Pa6f3rfrj)yoZAUXFOHZb^;sb^rA>Lp;$F7`)r;KT8?-?bynekLR)aC2UuD; zpH6jhJZE{vdIp)N6*1W0DoZR93n^QEOkov6b~AzPVnFo8qmjr&{mPfI5aHd~u zRtuarMvX%X5O7VSFj=eMMZH)ftNv3G9cBOG-gW!y#tNDjEjFB)KYN|F1a(`Jl(;Vb zC4T5i<>9Y`rTgz-6+og_T^cKKJh_{&pb;&1mYXOIj;ZdP*1Ss_bhU8cGrzx)x(Lr- zq`#b}8R<|JYg?b$qd7WA>1zJO>fx4{=t3%}w5EH0eS{;bFBx!`g$ql6fncsgb1gNN zT^o|drn7p<+S%U#K!#8zU1hC4KY2~L-6h~`BT&K;g=!X8$q5Ari08eR>eH}1Vm34D zSZT%3kJj-PtGl`tC{NhV)$wqn;kY@LE+qMp~%iGD04eA$?_SDux$keG%I{BaIu$VZdd)oF{r`1;FJ4Gzav<7yvli zz}nJ@Dx{}H*q-2O7r(WK2fCDv?p7LQs5>ymWXw`;?`8Xtej*=Awx(TjQmKgBfX3WQ zbQznLaMSGpI}!D8!HrVUs_<)<=^X#6d@F^UX@Qt@qozo*9;;d=tu@IgfuK6sfkt4D zJmx_Tih&Zle-!)1q=IZH@H-v7Zuyn;?vXqt=$Wdby)`?3IIAI$3nQ8pjZ&Y5>*+3bZM zWcV)StW`WHM8R%EF8$rCpN%=+d0bPuT;k_BU%Y>sMMoVCY+(>Kj zcULYn&A*Ys^JBe&Pzb2b*6uL~;u(vq%f3GhF~C7TA4!V+{5Qeiu>xYAuKlul#I-}P zjpt7a>3UqRK_iPY(&1apcOO|P@arugHqDnf%+qO|g#6anoy&ONs6XvV-ME*N)F~f7 zO?`BKYe#d=9Wz-n41*ZmwoIrD9$!$AjZs*{MvXUnrgs29A0sr&?Ng{bhb8o{XtY70 zkV%4b|IG1#zAlmylZiJfC{<{E=f3*NIgVut8w-~UYX$%KxZYte@@;nR0&fE+-mq?) z5YWy^tBPH3VfNFd;pPm&-oYI7NI;!4J_Pyx6wdDX?K8|W0_Muix0niW?iiuO>z)Q_ zqS5_|AG7A~0|%z4bggsO@QGL& zANRlNrY{rx9Q;<0nJN2C_6e>O!aYcOQ{Y~oUG2If%s-UDK$VOAS_8cOJh;HZlm?d> zv^ZjT5X7uGe@gcOn!@(*)7H=DtRw>l<6-=gj3XZv;~_02-?k_0W}@iao#py6iL*h( zDHf4}3t92?nVs*kHV5!SJiNZ~@meR9 zJk-QcKrrM}#`iI&Tk@#DY<98|AV{ILy%RF;|K96ny;ee!|BX~hmAwEhHSnRsDGM=G zN<**liPC+k(6R36bxS}#ouSU-4CR5Vlij(-`F7WN?U@zPxGbaN7qGI_m9N?#@PHIF#(~#{!>T8}hQYjITCV?K%)3BS zC!j#~iBecIAuKoSFdOThcE9-yslg3Am8<;2p^Y#yJytM95(A%T8%b+%mA!+M7rAjy zRb7*!+1!e>JQ{`L`mrEWE;Jfi#4s8fP7fMCmCeED0a?X|^6?pAP~H6iLh z!HZC<=sE-BE$@I6AzNRg&FkH3&^=l+X<(-9&PeKYTu0;vy@^sh59DTfirGa-X=*?- zK_%Ers_^;`M>jfi-bR>ijaffRTNKIU!RkWJ#%4lpa?~`^vaY#u5dmJ!;I+NBKubH8 zt&zs@?u;#;`DUyBYm%+WwN}XJx$rOb{#i*07`MT%-zd*R5pf>}!JfFJ)&kb$?sT=* zYsLFYpV-dFF30X`#syjy#iciU{YwS!RLBIG&Olqxsr?6E2UR@_Fr-N!v*WNbo2nqe zaNwti&%D98?mC?f5Y6dRLd3uxGj-Nj3nGZ$kKy|HsGZJxG!e8txzn4^$|Vw$@>lgR zejOxOU~UwQaMTw|RA{7iqbtrGFY69c-aaRAnN5zmHJ=}7 zJv&^&yB{(#OK81!zs=~@RMC}wLZ31((>JT4f#eO)sh8up#n|{p zH9;s*%u%`8LNSwKUU~WeF4np>R8Na~E!lT^8`V!S^C2UxG>xvauvuoeDR|l1{F@a3 z{qROKOFMM$U}g|05J3oPelhSSMk`N@5yq1428`3z5u z-6Cw7r9y2=ve`2y4g+4KG5=*uJHj*$nKDj2*`lUN?UeTiWuCk#)>k9a@g)jcj2)*o z#)AlJ>1y*m>SJ{hJs?ut;LL>k4&~N30)s4;f71gEl{CJ~VODi*9wwbXDVRG6Mm*2j zNnk|Y#I#n?sPUAFK|A^DXFW^kqs>*{OE~TN{h&$}f{vWdX=8~9%~yh!`mnyIv{r`8 z+DaLZ^q+F)FU9_6TI9xAWte7`G9`SQd!T2^kDfF1^>rgkd=m1VmBl|WfBH3AoEqo$ zpj}F$A92?sN2MaDE-ZA-G2SJ?xb3DF^8SHac<6wTV%wAR5hg%7qCX$;JIq=mXtV7T zazKeYN@(EYMjVFDxB()`++FP;QPiC~@KWF4{D5zcJxZbkE0Wl-78d97&rfIVKxkbOXUc|Iy5~8(;2w5yuK%?SGY*cauL`!en8j814sF&h zyf;&r=j8Hww4Y(=;$X@-at4vXBLxudxUoJ2oA zp|nhU`T;TFX^8TDA0vmf^{ZPIwYj3ai!RjU%CjE5^RRFUy{}%iFqD{EYP2K~FL&*s zkd>=fWDE1AnzogHW<`o)0cJ5fK0tCNplvhw>{a!9Gez+4WnW)5133oBJB2eNwPNqe z%F5UQ6rH^2o*3FEbFr2)LFLVk#g8L*_x~aNMIV54r|kw$J))kTzE9p$+k^wTp}Nx^ zFFgZ2|HL4H1Kto5bi_Tp+4qEwCn*AZlk4PvSUn*n8jvq()CHobk*=e?XM3KV8GCmJ zoT-cw)IYULpA7E!G;SOvJFnQr^-sqNF()k9A!-Q9{t;h6;_yJe#Iogb^JstC=p*`j zkMBL(?}4DV(T&?T^37;VLh8dGK(x5d53>Gz5=dj!nMG1qdFKDmcr@VnGRe~milQXd z%_}Ve!PI_#TQk{ddyeIP3+=9U;z$miTORRhuHZp`uj0`A3sPmqY2tbgil zWxaWbWRCD6J--pFW>t?D(oPq+e(TE)q+W>i+16#eQvHAwohtVX;{OxBmn$>?l6Pr9 zzz#?!P`hKaV0wx81Nj2RgIZE8+0k6i+tf#~`7G4Isu!KjjKuDH(*gAb^|@IG_P^@Q z+4%)OP6CW8OaO7O?f&?uScB(X5l}zuP?g}$jfUIqcV=V^+h8wc`s;L`qQdV>&Qwg~ z%H#e!f+;K@=?6%je3Fgo!hI~ znmAkY{u7~n2#_zU4}%M+IuQ=vN=3t zNb8A?b)}nc+V>N6N522m1Pmw5mRk-V(Ecj>qWl40rOwLl3XT27U?9E6;CYr=tK~+G z*-+Ir?np^`+{4&t%LT(wPxmb$X6jUosp~YwvqflL>aUt~bs@7!S&h?uAe+K}j$So@!jAO#-i!9CNtWJhPJ6u15m zxrCH4lu1=)G)izZI4yn;U@H1=iyz$|uEH4kI=io~_3n}$NEv~EDdmny&zDu!1D=yC zWRw??4S9e|%j6~vHC9vlKmIJ68faB8TR_Jbc$TSz#+x)hkPp3T%JXsDA}?*A*_jF# zV{7aipE2>3I;)BAqq1WjHtb&^ssTIUl*rX$T!0j#|6{QF{HFsycG!|iAF;BVxn0&g zY!P9zmJ-Hjm%XwK%30OF9MljeP!GD0VqAbUcj%AM_N|;!ZBZ}9i@qRKja7KJUaEQ5 zVtMl@GHKgNK5p_13(4KTu4D&@I4g;s;56#b77$op%yERRv7?F=-r3{H7vemIV3drM zRF_uM_I_FQInM-*DL9Cfx1rCrCtLkflS4>3{A;JhYFfr6H*h^KGj5ozWqkUUA_u^# z07HEfN{7nPB4&;4o!L@WL9RfR4L(OY(!2YbnazJPKl(Mk)F+N-&h6xfI*>eWX(t30 z2FMp8leJnG%zk-ltWn?IN!=#&#T=g6STlGClA6lJ{cAq4Kvt^g+%$2vXGl(CB2P%h z9hV*=MWB7!K{W3dd3ay(1b4ERs%4RpkXm}tb2MhF1M*hco++3!FHQpShoK_$$eC~e z`C{nf)Ab?a`a*~(C1178i?_=_X1%(;U4TVFLM!kV5FA{6i%7+%lzhV9M-6k6 zRz-XllY3@cJ+NMi4~n=Wgc9>M#n#^YLXp{ z)iNtQZnW8ga6AF{*FUwCRbv)$zcaU*5mr}X!YsmvHp4%lj-S$x*A%*EJ80DDr>smq zKiiUTRCcL4+T)unUt9iH>Y}9kizKgg>BeY6@F^IfuXh!i|MUmFkBYDf`%54LMf-oT zoALi%K@G4sGj~m~GI6SnNWGRI{b=@K6x!rw*IqKI9Yv4fV4UNJy}5HaQBmRpp+MX? z?V;+3Io)YB^P2~yHx*YrjNt@Q@i@kLT1ClUyR`j9ET=qv(kJvj58Br(|1l8*g$Un! z`JmH1TL{aF8p}@k4pE`k3s|E@;}xOGCL*nxHmj);7m|at*#&*VizfHj%6{;Y)#NQe zzGoO$Z26b$M)MJvfA@h0B?}@&43>7m$>yClI`7wAkMwozYiQ?Yj{u%7vhqgAjmp+~ zqqpc^{eS~R!of*+Z9EOg_4p}S7gsO>bcv{K5<(PIui$@fw%PjYG9n?)aHx zqG2+hdv3U;Qa`U}I$f(0yn%~p58n5--;n5=kLG7Am$aFuEepXqsp z?%ca5%>G8C$${fjEeGB3li6DWL#3w3IPoY%s5KI9%3%vw=4KflP|JjD7HQ2xiT|xj zKKxODyDxdRi1>@P%vi0}b`%jit|ug-9tc&Suyd}Ld;I)rgLnL>2yrO)@Lf_cjh~Y*gMn>Iq=S= zX|TKAGM@g#&aqU0tU+s|+evE{RY`<(AaTM%7j_^+&-%1yqeU_EZX-R#@z^^eoaTxg zfSUUHggUn34%c18vcAxSgzKMJyAv(5vQA*|mmG)l;*xK?V9&VeBW+w7fR?Xtd&Su2 z<74i3#W3fr1L;mSCe`jzc1M`Ug0#)(a$wGO3>OyM!aY);%QDNEgqnBPJmtV1WzHr~7Fpg)s-KE;{B%P&4s1OaBt0EH=kkGy-%Cj47#4JH z_Ns_uaR2ipfailXFuf6h6yVg5Cw8s()B(DFeO?=r7+nZQE`1MJ=N z=Z3va=Ic;Er8;|tSJ_SLURh6FVQG{cc=YdHL|fR25b{(-@uUwhN(sejQ0WG`_T+0Y@gJ_*@R| zLlt1Jo!#jkyNx-@?L)YKN&}%iPfNYdpvP#@O@-;Sq~y`gM~~w_{btc+S|Y3b%P(^s z1^KrX)(!d>q&g+%@u2(zMYHnA^Wd%O%H< z2xZKXCO$_9n`+PrgJ(pqk|Cg=C{w7GN^LgkuUMs6zb=U1Uq_c8%w8z?0VYxFbDn9* zE;Q+#XEZ8?ZoTjZ2&7@}QlomVpzopAt`j6EjMWA+WW;|}BIh7|*baTj*=9sYWF!b= znU~_M)Jo~GyLyG{boTM%9v3z%$+NCOmY?WPJ{;`L$=wB^>BBZ1H)+diu zCO7W3VN5rnU9o|gWrAMHp_Y`P<6JqZ=+=ZDXf zmWe^U(+*})&=?*b#mNzxZ7yJA{~+oTEkeD!ztGNB5=&NLV#VBqh#jOi3g26z`|iT2 zI||2cLfJfiww=bh1(p;li!1yIR|MT5Y7>917s2?jHS$Lgrl!KXh@tE?5!(3h^-~x> zzktp2IMf;kR*{)3yQF0mp^uKQW+nv`j(oM`v-~|$hu-5T4OWLnQi;hGft;NIU>#CWp z*3r73^H1!8mp(?nk=4)vF{aD1f2Se@{6_+DIG=p@p$47I9A@UiLgVdEqVso>h}4z6 zzV~q1RdZ^7Dm5(`qg%(VTOfcrk*#*w1vLMYs(}u@7cwh1!=u+LP*L57Kj*v#R=KK zAxuQvMex=WDI~b1dW_6uRc4jtBDH>5moqgfy-Eb^Do;XNebc8lYAo$b_eGKgUP!`! zx655rU7GSp->aOmax-L_h;cLcuv^$4ncq!LjlK(-u;)`={XC&<(}{MJUpssWIrjd= zC=8U<6G9brgK?32F80c`LFR)7XcC%3U_7Tuq9C8funHapQ_P+&PNi>4oE z3%?{cj{T%Bq@*J@#w)GB(XJ_t9&a@S0t3b-#EPvJArrkdl#)V0ann&p(yF8tWge^6jnXn2yYF{O6=2fp`r`h3W%j;PnkRPK3i$( zUegc16V9*bFu-Iz`AE_pLHCN8W;{$&3gG6RSjl^LtikI0aw53@qkkJRuaNyN3d6w7 zSQ2$L;jA^NJ(A}~pW~JJ8fd9DQa!}xe$+I(pTtz#Y1)w?SDoQWk<0to${!An>(|e= zQ6FN;dF^oYR*ygXl_&f7zj*c@Bgio9bmz1vD+h{=m1I())rWXF1`T$#T`8}3CXye& zof~0_S2?$7c$0!Wl-7O&@-6bF} zbaxL@(jeUuLx*&CcQ>48e1HFQuIuobnb~{o)%U&EM(KYo2;()l`St1CHc`F4Vf>{9 zcPzH&8EDK=^|FngxzFVKI$NNkBNJxxyt&FHpxODn`22AF)+pN4;Ns>0Wx%-_oR3H| z;wH(l*?dCsdOrC%%q;ola{gUG+FSyR{Hs1#pw=WQH4 zm1p@+^>Ijmn_%`w0qSYFSI5WvZ|BN2C#_%5*mW&_D&~(i(NZ^I5VE`#YRY41%d4`l zfhmj~7po2K&My3Sk1C@VSM0`#Xz-kcaf$lt@K4~(-#H;`*Bo>lnH`z(is1a-U7+nZ z!}wjoycmf#-_2`2#TV}2>rcHjUf|%dBk0+SWfd$nb(j+Y3Xsq1F=4B#@`_$9Q?yTgNW`swBhNfM_AfleZs zm>7{nKm}FWoLRFx{MuRK%#T;C6s-LTJtUSZeD=?q1bn)iiMg+u{?G8^<>CBDWs5~J zyt>5YMX!0hCCl25N=>G>FxAxMN$d{K`wI6i3&&DHq|l!vLB@;E?DD4Qu0bAG&(&3D zV9)PqbP`Z8!FfnbDquzBem3<8ww_`g-agfcf%C!fzKevfCMji<4nM!9BvAGYL>ngY z+EdWcp{soQ46f@FExpaysu#)o?w5FScj?A=mEkcY_~cNZand?mKCW&~G-{8um|Fi@ zxmzw!wWPLO<0&CK6wjzVgI<%%7}BBOVg1J;eZ*;1f5aSx)Y6?4xaFh2{P1kZM$_-n zKX3KHTD__%Q~Hy`Z!Q5h%4ZWMe};%~FgI zHl#-?CT(Xp@Ja%WKGkeixE)wFkeit4BIy4(kH?^FAq=P&S7x6m9xKyd9v&j~0^LU* zHgEc%ZsC$GH&=`qs=CB631vt5>s})^Lp@l0h^;&Zy8k-08n^%6am~y9<~Xk>nA)y* zGoTeoTq0du`hfb4k=nQ$dXdCt^m2DE)gV$w@TK=H1{=&J$Cmtmo|zZ8&mC{SIl_7T zyO*`j*SAmjNopOd7+aD8qGh0O0CUv!_&cG`*aoqqcN}EWzUcBj6HySM)?pa+x|IsM zz>CabHTyf2#OG6_vw3we=2YE7(NS@H^ixJ#d6i{cZ*P1w=>y6F7Gikcu6OaRiOOYz z^H1l?WXsFs4n5F7k-t>DNz1o~zV7E_$Tei&1DZ=C`N@K;WO+%vjsYfRN3``G0b;7H z&7@LdoG|^%S3G7jt5&AgG8moUfjwz;EX z6zjBnz75)WpEY*RM*82$cR{|AH+=p?Ed%3zu-r~4`it*D5qA)BB)zUC<9Z}5qnl@$ zBItZ#KY-k@#K8~9^?UYwqYK7C{;Smu&0(XqT4v+(zR%ZR))13w4;+eTn_s44*tBJ^ z`;gNyVW}&xMrRLJBYgf}`I1XFkL?n8^(~nXog(LAZ!*Uw(_5nX(xt*iI@S|yw1WHn zEuDkyuP}>m_2D1g5$(2^C~46&6GwkiFx{Z@1LpOiyzK;qlC(oU$Cs%f#56=%SnE)Z%3f`kdsc>>- zXS^ayw`#(GvgTgNeJYm#k20sQ{kxL3*(h?m_W^;>;w0T#f4bxI!AZ zVr-CIhQx{vXI#Yqi2gLeGEPXz1!6VA@nXIoI~B zw|*FszVC<~f?-P`WjB{Y=VIC9uJ|%qcfP+xj{7MpG1?^V@-y>*UFR+mhfyrfWAKUd zkk3dozsu44o4iufOqs_W%PRsbU&+2Rhc{;Dl5ODmp-B~WEAbBx;Fu%tzh%g27sKo{ zFRdBKSp3-j7iN%vF<2^jb+tr(IYsHmD>V9)9G>(-7*m57?#3Uf;-_*Z)al=y12Vz=KOUux8xD^(mW zdtGMcX3u4Mg&xL_uT3eKsCC<-lMqhrE~#4*=%jf9Tye|W9hU{A)|6RQ) zXj(2|#EjRRbmq!K&z`S*M~S9>8bt+~<=5hMg>gFhv98dN?O9$4e%vw3wSp<+49{pB z(mXjTFL=_U6s^wc{xc^i%nv>bjEy$F9$9t2s4s>r`keiI%}nOHx6m~si)di&UXK@~ z8XIGxt4;4=95bw`Up1U`T-BP1^iI}CwCcZ}0yHKN{w;Kz;J?FN?Wzg)kj-YOSf<48 z4wm}8LuLxL&n zV1ejmhrcwJsKFpzJJpcTG(KSBz>x00YjaThf19McLPc(e85m^HLKOY~W%oUfuA;zwMwLMP*M|jqxnddpkZ6}Kq^7`^ z`hMhqrs|8H$A3&lZSp=#u_GA;PxaP>6fe-kc&O6X_x{L~I1Ntm4!kK|_=kO~cmnNv zEO>12A-+Sq8+r`KOV{Uy!&@dP5`C&QP6@dY9uESE*wvR{AFnrsCu}n=oK$j?r_vI^ zi4Ru~4(`C=xc$(BYqWjYFY=vy`SYE;{lfg+J@2lTq-ViZS>3(m_ zp_#pD{T}v3*P+ZCL5R)nN8F6y1}OW5>zS;I9z^JJo>=gUThw~0WF^IVG_S)eE~D^j?<8W1?M9 zd39Kq`Yvt%7t72)xo$`WctCT~JZew%W46h4>oMuEUFISl=u$^Qh{`S>nX`;Py(M(z z+?3UzJvl9&biE1I+n(CBRt^#do2a;pHMHE6wy0$ajheXvCj+$sAvmz({OU7pIy?;` z=g#fu5Vx%~caleVCG3yDmBOd*LK+?WA``miZr5h4K zmQ^^c%)-vj?gUYhkgFL2!jAba3#0(Gs|(~L~>)LuKa^r-B}J6!0*Kic(vI63{a zf(LQIN8Zv)Nj`Bpma5;uWm-ec6V>D24$rp%SyRMjH;3!yI~$xuhfjO)p>E;~9Ea(9 zcrIF!XZ*TgU{FcOLyso)Es#uhdnrvppH)`5pjOK2AT#>uP=m^LgU6XivA zb?EU`A!uWn#N}`?x4>2yS zq_=PI%F=@(r5VrN+>!XWcj@>k^DD(aoJQyi83B@ng#7TArP8!$oEW^gXmmNEV7)+I zwdw4$?_!aSh_4=6v(Xu1!zy!SRtLmMeG^xRAHEQ^k6oSe&!$P?P`1G=KLJwYEkZV; zPj?18zoD6RWW@9K>!YG7#VN`*V0Wz8+}V;Amw-Rt@ZauNMNK1@65$whN>1Ie=%mq~ zq`HeSHnEo%01@(KiJCb>gk@lL}t@?`L& z;SGR1m4aJ8ttie_d|Q&*2Q?Bg^`U{MXxz!`Br#^bK>Enc8tLjhj+m7G#;gEL^j%vB zL`opO{*?fodMp)p;AW#;CD5rATFH;wOSE18(^{FQ-<>{ZuNl3Kt*Z{X!U`5#7?)0V z?=9q!S1(Xlu{->CVK^CZ!EpiRd<6EkjFU(D0Y$WS=9-P+#7>txeAb%SS!vjur$2tC z2E*eAj_<&r3(*D}Qki)|YLYvrba>13dMMkd%k>}cFDG#)kt|LL8#p~k^yDUFNeJWL##_<#7vk?k z1KN^VV7RrqOeZaJl=g5=U~%3RLftsUOsA|HE;Q=sV&hn6)j)5dgu<18v$CQgJGp#T zuwRm?=ASQftr|=D0Kb2z*b4*e@z!#?<9SbClT&uzTHAZ4kpx!eu=Fo9{qilUO`rJUXy{Ss zXB?%|f$JieieIzX)O+))q3hHKmyF5NjSSeHtoH7_I~2~H-x4_Q{#NhU^V;p>J)8z> z-BPq~Q7u(9lyd2t*gRgtUCTK>YqWXmk7vH<~BKZOY2PQ^5oh!_E;gZ4G?LzQ9S>GI*EPAJ(^^)#hGeF*{`DJ zt#!s&FTHrDV4;&bB~cixUK;LJOh|#*vS8P7GL37POjI4H?aAz`n;dAtL}EsEbibzdbkEC7iSyvj z4A$QKcX){0&R1}-I1YvSw6EZU6UCg z$U2Z3p(sevBSmUmQ%Uo&!|7hGvj5Kc1-6?>r%lk*TBvO7vSvMD0;*n61L^ZT^x(et zaE}VkGs5@G36`IH-3aCU6HjEv)|)SVp577AS!KVBHmbg~;Nh}i4_#{B9pY5b{c!UO z@)gaMet3b-t!-dn;9Vp9M^wxK5!vq$>?D!il5>&GeAkmlYb)Pc3<~J`3&@{yz&taTblDPuiYgSTu~W;Nk@2%_MoyitqM=~k-$_G|M) zj7JMwy?V2h>^0;=rjk|3zD&hG{dF^&CKVE zc0X(4#&~0jHAIBv`Nk{`D?A)ejgsl6(pxuPXSf!Sky;63tVat|wMnaI0v)~bK72qx zJS%WGY(_t7qL;2j&^A^TIPy1;tj>8f3yY&m;WiHGqY>Apc^ML8t)e@4z%e6mH9x`5 zlUX2rd;pmb3zfvP-73X3#TocwEtouA^h^3Mc(Y}-{yI7g?jf@Py%h!PGaZ!>KT}6^ z@f^L=rasxOz?QY_+WuZ!y;H0SjHmu-X8jcb{rHn!Ls1^SaA&EsFCF5kOTT0o*v z?GfxLi#iVX_K%f!(fq`iCYERA>&#egi@Nd8 zvQvBoimEce=!V`PuVc7u9Gb8`QmO50LHzM94NtN%m#x&h##a=0O-!<-7lWTO?^_eg z5TgO6xAP&!uHiYG1=Kq4hN9i&WR^EgmM46h9+>*OOAAl&6COCpT#gpvgFk1|aRS-M z_#$?%VD!3!%Q?qP3&4IDUF!nt(c01-^0YiS;06kE<2nhxkRg#nPx&DYk#zv09U)F( zsnGXRwPP+aQMGazD#F|anMiS&zLnE&JgtJs++N)E0wZwf+$q}43fiX@pI)tbj^+m7 zi>jl#;+DJ{mATsTqPGKzlop^2L$CgIo1LJoo4j1Ku}PCTr!|*qp$NjA6(Il_xz&YgPX9J)|Uy>LO{Op zB0ljX6kAdop*!NGSr2i~w|Pur*>Vsk_qBfT;nMOP&$-}pm{8yjCPBDjg)7{Y_}>jq zo6}tjToEqH@KR#CwWw*fO6}H3_t>5Yn{-%N2R{0es6^gr=RbM+;f8gxcT9a`Z&>!& zIQy#8D}3cHMi>$F03prUqU03OcC7eCJbIgJc5(;wZ?MTU1ZMT(kTy*>?cKKyNwCG? zSDc(DFU{8p%5rM?=vQ-A~s(gU% z`z=Vd2HUs{{GlJzOO$0G9Y8P9J9q{G>)BeqthM@p?d&_Km2E9(HonhGW~kh4I9%>! zzNW-+Qtvt?wzZ{=kTdng>$BI(&l&zFl6^t~)^*gsOCvXE6IsbB^f^Ob6Dqh zB2!H+Gr7t>-QaT|St0lPY#9%iK*J^o7kx=anD8}lP9@qC@u6qdpXZSPT~_Q}2U;rq z?A`}qXPuu^jcmimPH4XH;`k8HQF8ZcdHIQNSX6fFkzCz&e^o`~Rz7$E*xH6Ka8yrwHbO1{{ZruJK7r>;%l}t5(IYX^zAbM`1 z%g{QV*MoZVYgT1V|3&|=3G?-qUvW;ScWgw)gO@i*gyDoGtvrFam(Cbrsx!o3SHfy7 zH-mgny_>F>uhNqZTNV|iedx}U_2LYdco@?(PWEhUDw73Pi5>|RESZvqs{bbb7pPz% zs7+j$zn3<(>rJ%dd#n>`cfkTGCxVJ8=DsOndcN!kZNYK7D{i^wbgvnXLDN`gR)yO(68j46k(e6 z2!36w_~PlJ!Q%>jaNt~7*$t^#n_9U=;A*zhkZq!|%(NDEK-5+-=f<0=wn!5sj<`{N zo#khuPOKDqCD&cCu*Xs_@1Ws$Pi#21Nw`)pvh77Ir?xgd`?cWxR{ftpRzKL7#{xML zS$rUsr4VgeT+@XLBOgy+P^J}{FKhE^5zn_H6cUYG5fdY2#u~F&qDHkJ?pj)X^Bh7&mEaCbEoB zM6a&3;n8~VBzoW07t_!wJYw3E;h*0J2LHX7Kqn0bxereseD}2$Gjbok(E!n*&cVpK z@43}i!p(yI%(T#5rp*M?59y}t_?>lQN@5@%;IQT_1tl6Kaa4{}xeW3_7W@l&TKhqJxt zr(yQ%MHY;u?2~np+n;Iz)V0SQ$J+jw9(%8iu7<>)ab@d=rf6X14n3Xmx?p;sleMor zAlhCZKQB%fL|2Zhvl7**?@T$i`Pl7D6ew!=spmSw)lE9&(CZJ=`;xW_t3U5I1T$`h z-VwF9D-RJY8Zkudz`_*B^j_w>3sw|#8R;~8VG~F;vSO{^FO?AR4G6tYuuTb8BNwLfTp^4{uQ+JKlz$|!Q|W_Z_e`+;riwWx_Hq zMrvHt4eUOYTky(xj-hbv7THV6bL*m#URd<)WP2t0CaBsk{7w(${b=_uW^8g zt80DcO!45J5cTgBPX6h)0GnE-4%vQF4V+_gCgP-LHYe6rT!`9g)#4g$_+YZ65JICx zWt8LLUuKy0=R8WI=qc7`tW+s!u~AsR7Cx=VbyZW86Z z=xNU{2pj}o%UNpU5WbF&l_?Kbk$dV|5Fif0>$ac{u-=>B%P>sQt}Rn8pH-Y0+AU+(#5b)1 zHYTHku;gvl$HqTh*@I`n-U*M1TKafA1TDdnT*pyozp{TBW=Or7aY*;aVaRGP>vxi@ zTk&sLA$p(~8q$3`zbW7;V0{`%@5-gbjUHH0+q#ZmzQ^(I%7=T1phyfve?7Rh%tTsO z*RCCGBI;{=@|grf#99I^ro%CEU7{w1p6A1(!gG?5kIIW`ANZHPd8f{k4o>BR1`9ygbx z#jb!(^E62KNPIu$Z36X9xsqnD_a#uCt4L9fpP_!@q#;|ocm)r4kwCcnxW25#Q9P;g zXmO)JOsv3Nj_0lG^J1cL@a5)t=bm@tNyms-x9D#zU5Ea~P>i*p=WgMUqOO$yvh20% zqmjEJ1>)IN-|loFuezj=Fx*16b5b5R1KNAiO`>8OcdPYjq)Y}<^dR9@_0L9%W_miq zgcI)cv8oMbDbwsxIC~V|-m7ZnRi1e5FOQCpA%*_P_va);eZXBJ+l6?CQkr0!D{dY% ze>WT~CVOC~&AQ8(to+IdU;2mFe~wGEv=i`NH1@hdHy7)Fu=5nkAClA`GkY)AV!0ez z7}^}Xw>+4*viKr%473?6R-1uph^~HGT<=Y6N`hZHO)YsruO-G4HSMPqwv$6XOl{C{ zVwD=2P4G_gUCo$TrCPwg*d^)sZ#%?tyQ^1LL`#a)Cqi?W;bdafBDxsEL6L<$CGBLq zx87q$Pu&Wg=BJtypY^eaBm(jg&x7<2{o8Qm8Io~pFAD+;Zni&^e$mu>CGSFZv#pi# z+D`F}>&={3UAbk2_f7G`zTjUorB9}&fqB)%@MIJGiqhgR*tBN#)_Ffvxnlo7|9C4% zyu$UE5B2L6Y1+Q-@z%)_tMkR2>q&q9@n7{47Q+K1-s8FGB%kcwk5Ci>xZ8uLou*s8 zB?VdcQ#IaO!evEDnuKI~SgLf6*9srJW;2N z;ji5=Toy2|{cgy7?P0J{W>iRE8kCUk_%MTZ^uQZo(x;ui zz(YFNDBAWXDhfMi@KESGel>nFT|0bJh@-}EC0cf!sqo!0?X1RJ;xg69Z!G%75~7q$K(9OQHUAzEts!OSK5XS0I{XwA4O2g!rM zU)X>G6Cg?4p~+8LP>G|^7efa5oiP4Pt}OA0@XK+_EI#tj_P|!Qw|SB8u;Zro2@Utn zO4jEI1QABdhru$Lte9x@uZ`r6Jw3-qiR4ZvWL<5hWf(MB(By_?;v9yolOrYBg-voR ziz@Ion&ODQ-X467&VXtkH6BeGdS`9|^Hm5aR^r}o<-aFaZ7raFRbj7o`;b>kSW;R! zTmR#(m#f5354rP~Z&^#uTqYzL$#y%;j@{HKtuj4jVbj>IQ$m-_Iiw?W>C>56hUi7D zy2yQm2l;8zW4sq6q(669C?~DLbp9_I1`!rk#9ng300@^+X~QEOR;bZY)*I+%`Idak z&2XT!@HGBp7*!-msOHB5J&V9g^RpRZ$BUXk|H^FdBW7Vg&k3|}i#-eN*#hLpWC1#n zSS=j>U+^9P1$C)U>VzwyO$})Tv%db1LE|gKhXNKE_Zt+BUym>4^dj1RN4;^imh*7T z7)3oQBlJ50dXd>@cN&)P>OCC!s`G8(KQ@EXd|iAPUW>VJa9@{Heqj4enL(Cdkew`% zff#sl#=jL5W^=cF+uGRT!<44wY2ayX*NKCAK4o-pow3bIInM*S_V2phY;9gHeF2@6 z121IUj_S^ZwU?I78hFRPpQ+Y9a+^C58+hk9@SDjmpW=3lwXF6!z%zR7Y^Kap=%Q+1EDf#eY{rP>#*l2_ z#?d5?SFn8VwAOW2SH0`d{R?RYheX$xjU%?Mx&<{22iw;}5x~&e?tr66Q=%#l%v%t< zREvBn*CP^2DEO$%8y2HjVt~e}c5Wxys@mW*`w~pS0L8&Q(hc+-oF^p~H#nq31%+7_ zzS%zAh`ly#BYa7RHx5r!BFC()`Jk&5n06;`DU8tHul^_FczHVcgeG~usP1Z zg@`T6w{^8ojIRLgYMc^eoa7#QNpH|8%44G)Lyh+_vlSUsY_UU$rnCPuuF7SEZAJg& zYvr6kts=j4!xpt1k8cml&Q-hX;4cOH3cPoBI8-Hsc5f_re_5L(()H9+p<%6#m%9uf zRKrN5$row3p3{E)?q9?|Pg2zyi)&H+nm~)uikQUVTYkZ6Pe(jaMdcFVXnwdkI{Z1R zFBU+F5-!;cN%H5x@zJ^5+B@8)vOIm|4kE|)5Mrw45oQ9~BHsXuW*~byXNs|*TJd>M zPvUzA)zOs;X{Xn(?W(Nx{;V7MI`7{A&yPXcJKT2re+`MlZ`9Ef$Vu}TT3#WhN&)5H z+U_9c_876#0#T1I572v%3P;8XHRE|f*95>sMmFb6=Qf<<F>hme6ge|@&s(z_SOddPCQ^ES>S1T%797nep#K3H z0V**-?_3z%MIs`l#iliGVX~_LIsMt3LzDnE=@vnRw|dZlV!U7v^S$(N{$ku2(}#?% z>@yF$?LR_R_gCVcjmEePstLzes9@|Rno$*6C=n+0CD1SVxw7E;=H#f}M**PIRO>v* zkRU5&98UGDWy1jisj^*Fi=Mw1!Fs1|`xl}1=i1TJcL&FO3~y2?u|1wor1bsF`1qI+ zK1~Na87C9rvPRYE>)^5e>V=|YUUQE8dh!WH0`N6`=#hOBUv~apLh@K z!gp#&8s8juO6Zr;M4buK;SciPx-@v2S-AF|;=W#*9A4HKeKTnBJ1)4HCazEepn??C zoqTJ0`8NG%vB3*F+$$v+IBi5DGWct6Dx_}wcYPEotBU~H3P7K(9!7nPmu6a7&uV)m zrt&)a3mf*2ZhN(cp@qY-_9JM3H-mOJo?v|n%i1tdegm{3W#Ow^Xz-Gjvu1EJlmfnU zBi3AQZ|jg*I@2t0xrU1bfI*Fpl69vYHkpg(YLzZ?r^>aI1q$W&C!C95rggv~KEO8| z(Rm;G-9pyYJ|7FX@-54Ini~H~+!pAFQDAlU)W~{e#+YaXT}A(t>s;K{o%YydzP}z; z8d!RK1)$SeCI?*Rn)!@c;z@elJPc`4;AW}yU-FGQXCFX z5#nf~e|xkjs%xSS+PY*ScN~rxULPF0s}vXn0=B&jz-ObCazxh+<7`en!1_X1qRor| zyNXV7$3ZLNWOe!ozG+DIjvcgsgPh189oXl&wJ~7f+EW!>w3tM0O2s_a4$^1ej;xBwDv_?>mZG$|bV9OVuJ z&BMj!P~${&a)DipdnbD$wWlhT%T&gc<&d8l+uFl$Oqd)SQ}Im)I3rk8UyzEwwYsL9 z3%)EYSxalpe-8m}M<*vIXJ@-8YX-XDhXP>2TklhhAL7<97^s=vXWvHgjNo_S(wma| z43XTBh&=RUKxE&MufQ-gc&Lf6&+W&PUrpg7pPJfj;BKp|^>G^jyxqWg6owrw?Q4f6 z4MV^N;mXZ0`tqtQb`I0o2<1}rDZKIOFw5=&*Ic`meRg~g31CMl2T1@hghDoj<$AI( z;90Kk#xv_%CNF|BB6$g+evJU62Am1FOOB%*&EhnLfqQJ?bn)|ti_LGT5zIcRA7b5y zIbA5m!-ejNJK@Qav>GACy2rcU8=2eXQfpR0-Lv^&x#CfpZz&Sc>;( zc^>S7BDaW|!`gNS6|#k2DF*?SiRh1D0qg=JeVid@Ce7DPZ~b-FRh1(Pe%5)@zlqon z3IHTu0LBb%#3O2@lNRp7u?k0umu!ymw_MSjA<`vm^<(l~Sk?+=qz8_bpIsSILC>Tt z9yaT$@^=B~w$py?EwJi`!)9PGReY#Fy`GQz+ZHmykoXSo4xsN>;XG{bn0~<^#&kpz z>k0Jv^Uwj>4=DhdP%AMjAMm5wf6i?W|Gih|azsi@uLf5=;7{cl;6aZU&&XD$p7y%| zw6WLwEU%c%KPy?@cqn1`t;0cDp~wdcsIbwP7IyD^Sc1Qf!q+HIc1bys8F9F4Q!IqC z)qYBKPyd1`Ya<$~t*CaO7e?c$4b`q%bglV@Xu`WliiH{PSR8Fekqr1lO~&}wP#*zb zFE~RAGhPq6bI`_hWYAHNeqDJ*!Habm1ZwtAG^_1CJ+@h2HU(B_e?o(Y&5G)}VszB^ znW;d9WxS{8%bNMB1YDK36#fu+9>5KtPzoSPE`z@a-_u|1hCXhs|UfeJnocHpAJ z&Jy;G%Y*}vsUb-a2r58^?NC^3jNf|UrFRsHw#O~d9n}|!yxG>wpBDk$Z%EGLazZlc z+iI2Tgj2u_+AILNj>*emW|hCq^9Lry_l5z59a;7em^j_H0NxgoMpqFu(~!uyhXd79yg5RJ?X8mEm{Az z%Tf@wauXo+3(k{rLQ)wzxt8e4Er->VFSSZ~8y>{Bi2|!f?o?ff~R2m7eVq5_L5LI^;C5U;S+sPd88~~&Wjw=JmCjgDL zpzlBD^z-=2;s@_Tg7HclXMsXHstxl&FaWs03hUPL(8D1GG61Cpa!Qplf&5<)3Qp!c z^?q@-bpQNpQR7WD^{WTG*T#LfPxxhI!H@!TI6d&D!Jbl1F8SZfvCw&r3oj|I+VtB# zE8^fAyt2Wsk#2KBi6wl1nQz0Nn!UklEtu7s>6n}almzI}1KwX1@D+|ou9%i$R$TxZ z3u8u{%$nQhx<1J~%4&`^+&u=I^GjgQAN5b{JGCw&;gJt~Q>CB@-BLy-kG4uIpBqkD|2v;cjkphNW_LJZr!em>c9hWMf>bvY+f%&2Ge z5E2p!FcyIS2S!B$((0FFJuc80{Lf`7(gOH_FTB{C(QaMNI9(z_(f<=R5^1+K>7*Pa z23Fbqx1*4HXY|FPmOONLss)j%Xdp#9K4_~>n!Z5U7pPZ7hX8CEj5>#_h=1qE2|q_f zDS-OenxN)z2@k?>m(v&goHYZ(lK>QckHF03jizs%L1(#5(sEq(?^8JcK800%@^bP& zK|_r!O@gA+yNxG~2MetB!fF#Q+d4CsL;x51Z)s5Y(O@_N#c#m?T1hA}ub!y!&uV>q&%+$C#Q|B`zxztTybGIQsoe{0!=33z z`mLO&i#X8j=R}jm)EB-f>?NQQoI|y!<;^v0GE}OarM(GjN~A5ZKh!2nKsPTONR5z? z{Wcu>eUo8PsphNDdzGuJiEaZ=l#1|~fZ6UDrz(?ul!$yBMT7GKXjz6Z0J2igHoRt4 z+_Ke89X8V9G#b+L<7?fJO5U?OG5MBRP)^!$k^+(99IOY8$la%YVh`7d z=6-C9)DaTga=JTfu)J*B$~SJ0sOqspR*E1yqMdT#ExTZkXP8=cv2|TQHqj$;o$w;4 zM$ik8UwF^cSS>tNVQI^A;*9BoFVGT?1H_tg^JRJ0vNQ2(w61#IBc`GN+xk=4&t0E`u{1FLq{-}k&-6q>iw;bFR>VbAy3Ud@UU=16Qr=M<1JkHW*8(ybA}1gV=?;pe=Rg z%kBzM1cl}VGCy1r+P^%=!r0YX!DDJSI*yt0bIyYsTW5!}4j5N~v#!PXpcmHh!!y0? z_%tDOZX%i+_Ie)+Lmpgpk)lJX_q?rm`^dB*Qpnc3X#4LlDAuB4wm=pO)pp?XlDnrv zn^0_-8p})D1x^pJE0WmDE8mIl7m)r_As1V@`j&pTlFY)7(g(8Ry8efm{Lf|Yn4`|V zry%VvVty`hJbK6XWXW+s5S|_^s>y$}0z7E2)1;u61cb;5T(6Tg2xEa_HTt6FB8D<| zQzTJsX)0g`6tQ%xdDUE}PbD-19o3y6+k?Nj#=)R7b~X8t%=$#xt`L~k8lBmao|8UF z_>zdJu_i_)3?H@@J=dbSlaK2rz|4mJT}tM_^SJfk?D%dIm`*x=l%yP?gJGLFdmcz%WocmwICytSF z1&o>N^DcCBx%tB|Baj{_EJ!aJ5~7pm6Zwi)?#t5_5sUlf!``K@uj>T|I7)Smu~^)ei{vBh!nfpKO)PeMFd1P((4c}jAnULP4j5& zCN}xkIV46y({3I-5A061?d~0J*$j3=^OlCk*y$LTOTMMiJV>Q4&^@=z0NSn*v<6gq~!`uj(QKTkpd^nK7$p^MR7Nt-p8 zb-M9VNQRh|&lA=B@9MyOZCQ>YL-Z*I>FF$YMjM`Xwtlm?C815KL!Geu_?bV8uQ=zE z_C0il^aNj>A=8ap&c3on_#TC{yo1;HenkJK<`tJ{DxNkp#N_?vdtzfX;_gvNdECLk zlA^XUO~l_Quhjy+c*y>Ln0d*QJtzJ#V~k&mU$gY%QtoP2<)yHQnXc^g_nJY4u~EKE zPSQ?7&KLYy0nFteq2{af%A~Gg&HgVpcrxPVDuiRh3PNiQve|-SV{u}b*6sDm*_qq8 z*sR5UT6jE`pE+l~o^73dS67fG&Op~8uPJnR{bpYf6Q%kVSBIz;927Xj>nu(}=8teE zPrrq;kz}rj>5cXgl|@?WW^s8`PMX!#W9FO!OSa z>*HmmqkbjF*|Y);?y3Pryr@nk0GU-RwFZe&FmWQNU8;_(CwC(;EZ-y*2%HqE@$NY_ zrgPuQB>8r8n&zxzRkz*SX@J;J7n<4h)n}n{Xl8W+%Z~YKrMV`upU=Ct)h*2G8EtK* z2|z!9nGMHUU{9P^7vr0*N$jqlBW-RLKJ%kxm{KA zy(M@;wO!$UA9UA~Ruv=vq$OVEBxksv|6nPIIfi^$TIWwugV_TNzq^=;I_C89)Y}?edV=_?~e^A z=uatqm7~dvxo`4huVA}&^idlv4R@}c%xHenL1ZwF1UwKd z!Rtq!)WER~w9|!U&#xqK^*pepAe#Zd{`#T#POZ~Ed;&BloHB-ViJ57Jy&s$zYU=!I z4mJ;dENG2V3DbFGH*Nj{UR=ZS;zXX%wfJQ64b4NbUFHVSmS1GNY3!+5rTS#Ei7AQQ z6PrSg9&go^P266)XtYOvR&{c+p47Ip>+FJ~B8+5Sm#r+l|LAP?e{>1I;;$Asbd5a1 zIiqxQy-heJruGlpy?~# z?nl8Y8a++wO|H+^x%K9>7)@U_`1W6+CIm$vUR(ct-o!!$G|rWj?_Q9?Y}SG@1X;y+ zGpi4ku=%-zuYLs!l2EVuKU=V~*(Yk->aQft_ZZajGx6Uez&38&dc-T~wjQj{DS1r? zJM%qteJntBY+{mNm{s0Zu*^Ck<}n-jG|;Y~qkN?*6boHS^h;wNH>=I^?k^*MoGh_Z z$Y@2#V`w+r5aU+TY&B%syGm(!+b{OAT$nswk*mCCt-qww^^3%~Wd zbFP|(l8*nH!O7b2B-Pz2#B5xvXkPxkoktCUJQSVlmOi{w#N4><>+WB?y&C4u22HOG zY5pyF(c0v|xxl$=JHN*d`(^QEV0CP4Guh75odu`zo2*xumw-i;rEN+8o^Gp8@y2Tzt2A1 zh$Pv7kRbQLKc)L&rN;O*)>_L zZclAlc%2p;DJ-kripUThR-VbVL#A!o%sZ*`=bn7fP;r!iOR9opF9PRo?Wnah92Rr3 zV7Rgzd^tSvBdCaeHbOrL@xs0GKUU24|I3O=Tpc2}I-qD;?7=dUnI<-Ghxte#Ap)<`ArSz*c6~#XQG+0arn56R4 z4Bz>!*;SCy?e+5lgBc7TvBP#ADlsBN^=W!L8I1mO&5xOD-L$J!wgIuR^mIw}&?P!n z&|Zn`qTSbWSxx6>a z4oTu9nkltObn znju)wcEjVV*E`-D^#g*8Q>y|oYYr;uKw3)mZPK2GHNp$T` zp-MKXY5aD3bNK`RP`w5^v0e_f$u9Dzulg~;1zjgRuxD=Coe_m}}A@X6hLBPKj z5wX{LJ;sF%tD0I3y_ALwRLhSLzs_vL8Bf$kwS7px``k$^JdWL*^WgoX;LrI*u5+eN z@AdhN&0Cxgz2XQbOPIrb+E;D2Ikxj`1Q$UzqSH%>EWf-r^*=YRT?Z;2#}d7141(V(5mgHb3J!L)~W*=(~GiN;<$c#EuBdIApO>dw?_dxn*;7wH!{_9 zYpnGFWO*+K#P&|S|2?Atm&(BN|zV++kWJzvvOYo7@RAVh7D zC&@TrtoG(3qxa$1Ly}`2LEaBIok~ADz@L-JS}#JBp2jHt6oX}2YSOJE`eJD%cJyqc zwHK1{dHZ3;$m)R=m&8M^GtGN2Zta!hnLErVjLl{=E5UW;hG=`YR2a_fvbmz&JP7G7 z-)xI9Y?n&&e`~K;@2NLaq)`1@X-L%IK8*9!4)$FZ|9;|1!yhzieC~q3bco&L=}~Gg zGh%S?R{mULKk|r@DW|NJ(&&#V9jlDWWP{J=6nz~M>XNSBl%t$)tv7vtbw|qN-#*j??a{nhGEUav0Ak)xQ-iEs%jyENvKLc^`S&bLE#-*Ta5!OH5nqcv z{tl6)JbX?fBnTI&C^XmYBAxxM-7qCK+#%dy-^W_+d%)>>p;dIRNLGDz zylm9$_-ZR-E$}6&Y}DSJJYt=&d)dTEpJOh%8N%08}q;k_z!eYAs7= zDw-44{c&nkmC*SAVd|~p+IoWZ;VQ1BxD_bw?rx>HyQa81L5tJk?pD0GI}{5}aCi4W zaQ}txz4!CJ`8#LNoY|RYXLp~O8HIL$f|fQbJ2ZY9m!$1;9!sK-EF+d9kN17o2yP}} zoPQEGxIsQFmGmi7=!-pRSzMeFA__T0?Q^W-dMB<)-c^mR5lY&J@5(*R#*}?`OC?2W z5Y1Y%BUi0@kzhNWsiD{Z1r3BjQ791be7_In{oF zfKH@rQ&O3LlASl4?Lq)|-4LQ`;_ska-6l^?#Oi)``o=y!AQ!{I0*)A?+HF zXVo5NlX3$Pa>otfg}T!OCBbybLGW_)azkpaeMQ}eE{dPAK3S4eIxq|q#&=JM6Wq@^ z(7%)sAWo(>>|16+W-*x(J#LQqjeD`uADZ70grR-+7m+H`ChK4r0Bel5D*8_|4`U-{ zZYUVms~|E|G$NlWLT~hI7OkHAFX=cwOjkjIAE_?}Sc+}Q2f%buQj_i!vi)`U7cl|5 zC2u+9=k*I5Vf2|RkAFp!p73wB5`nVZQn?U#c}H0vb*7_(LS1lPRz)HBl}0D<3FyYO zT3q-Nb@xi5gF^oK{KI4LYeCYOAuKc;BCwV6@y{lz$UV*BOsS(pis$zj)O5>S3NZ(T zZUtWTv$U7mvHXSrZvLF_%k%h2mbax67&bn9M#zf9Ju1bB&v{HgUBQO_Z1|7HC(DJ! zjEDWMRqGwzs9g=0v#l%0MYP+cENU+Jmt<+>NK%D- zGDKbAL8a;9&3vHA6Pd9+EMhy?A_={u0H{g&Z0F z0v+BN4zpRDe`h$>4%_z3t&m1`e2E!;5}P1hD>JH$wiO$m7#}%`O4fnp2iT*QsX;*B zdzc7tz~0`f84J~#3c8ATrHY#e!1`YQnU_9Rm~mu$J!iAl>|}q}p8XGYNXzQ2O?v48 zl}CY`2{EYmjSIlet3S{rNf;atyM?vfBOgzadPq`qq2FLMs zKXt&K2;G)sTN9Ay1_4hMHANm$<*tsQUWqn&7tPI^ft0dm(MuvhyH5N>95yt3KwYx~`?;YG z@|ZM#sR|7zgopK?8~5ooRCv5C9}lN*{$ZgCo-L*Lrx0XqF@|f9$V=4@y9+y+{;E*j z+XTf-%{KwjyBi^WoAU6cqaehj>p0pU-|6w}NkCZXTILjUsK{TtKyDs{?zUUs+LW*#kX8po(K|cRTKTcn`9$SpWH2?u}+HPhH>pNPZuk?FUDGioMQX%e#m= z<#xhe%t8^QaGy^^SR0+o(aXAW#qWjSNXOBExtmUw!E=HB7_oy(&BTrt`LYApmk^b1 z>`T0K@9nB*Z!!shxfV*xHG|f7wijDGE8F}XJ}TLYnw_*8us7)m)q&k#Nz^n2`L>Ov zViNDlkc~k-I7=FZXad;@h%6Zx_dq+nVL_-D=hsjO$MUC@++7uCz4+KzXS0q9C5vFIy3#?!QBO;Duh3;g(=%oztT-#&p(U5VS|luff2rxQrgq{x)NJ$`v`MmA48Z@ z_)t$Hb16KH5~|yNOSzdW!x>Ir*97rN2?w>mh-4^ag=?!mm{@?XEi~3}ims?PLYpOM zKLyW6FMlOn3&m%jTc@$g0k3yxHJLF@V)=>RmKojMkDc_#W?0@adTrITiz6uj z?$Zj^ORrGe3))iCn?*(*?HkaOe$P&GETpjf2wT1JgNgH8M;VKRZsaQhQdb}H^r-RHC*%2+Z&us)MkyH*ITpYLS-%u2x65DB4)_sBj81EJ?F(k?+6 zLfUD*K8gNNAF$gF6F7atb=r~e;`RA3I|u0++cl3$B2!_^^;d)#s31x&iX?=p*%<5m zI)Gwf)?<9FfTdbkDp(L?HD)4l#9Z6fmjMe7whG`cO)tax&%0$FguUF~;Y!}UynTFM zF~s=0*yE2GNtn5n_SXY2hddi2=LL=AFa>xh5BgV)ZuHSh?axlECNN){#58R;wuI(S zEOi=u76N@)Rm)4W+d}Y1H)CU*+cuxbW7gJ-%!N$I#YinXIB*|E^_I{PrZTn~$WMYHp_SiD7YoIkQ{IhMkwJBr6+Sl$%F5%27k~_&@L|soJ zSd^<1ae>io#mTZ)r}}M~b^1tPq-kfFO7XrU?q=#px|^A@+dW)PiDQDc*Sp3%p|r-W z(8qxp#QACq)ZH?nzz|!0lV$2H%z~9b>xm2b_nc4I$q759Q>Vk`-wb$srl5#Vl5J$$ zjh_tIwGDC97&YIAcXvwPB>9}3M{gBKwR^RcHPT}$8^;Ty0<|4We^c{^bB}Fy+csYu z7cy>#q;0VKHdr8p>ECdQ-Gl$-oRK=}-uEfeZoH?@2{@38Q#;nJQ=zR|Vb_<6m)UIh zCKw#dy2AKF6)+PSjTS9RK97 zn&AE}Nskg)%Def69oayALX>yhXQwB76fNIwWBA5=fV-B*d1BSIOi9WP)so%u?=*KJ z&*nnu)C(OKg(GjEs%BK4!Hy9wgSWg6CYQ(dZxg&rJcjfORCE?Wi*5DnLuFilOtY%haH4^Nlb=*n zy9#q}Eo|@qecb#42PhSYd|v{ZJ8!LObT0ry!}8a7h*7n*G44}NbR$Ho2B zV4RiEIQGZuCUM}F{hMdAks%4VRAp$85qqcz9i^ZXCo;?!x*w(;`E19|4|TOOX}0HT zNywY)TZMK9RIM`0ykE}hTmINt!0m0VNI|Y=w^6ZHXM@8`MVIvn%eA}X`VF7{Uk*!q z5a4E#*J9j|h3CQ+KH=rz&M=i(}Nk;NdnS z?d`sXb}s9?M~9Kmo1tCwoDD64+7E?;c#V9B}uZUr9q8f_M9qGVnGGlb`iG>@&21RZh`KKzlXEfH)2gbF#1 z;eS9soYeO<>GJSEI%@c_qQQ_Mgr!)Y#Bb?Wpc#(h!%r;Q?@7U`t_2Eu>ocLZ0Jjm_ z?%bxwwAHfUj+4iIFV)7l_IY*+(wv%7YY}(ZsQ6CqN~ENaPbI-hl2vE{Qh>sA`9 z$bFS)Cj}-62KL`s>xeeF9KAbTZVi^2NgqUC=JC%Qs<_^YmX?1I6WNzI&Lo=?$fIa6 z+K&TFI?Ee)BIiy9)?yq_VQx{F8GC0Z6|dO7CC`)o@^6&b>1AG*#dDv5!GMQ&CoAo% zpp);?JGrQNlyafBMd0x0ujE~I!-tN-%DyYyYh^v}HU!N{zKaa$H-oY~1luX^?EZL< z=dib4Q}1*x>>Z^G)Y^=sNw;h`myY{oA=>orscs@qkLW}DU_|wt2KkZz#ka=ewxe-%w zPoAU&5$G3ffm7n=@mNLhQ9Y17J2?>js6H-WB%=|o*#$^?pc|qUzn8x{<>Birx=hPr z9s^0Xqh^QWnbmlIzbcgXa)2M}z9#Z=h0(PPzuW|JSR0b6O?d#u|D-~#0XQZDZK_aR zGdf(Alc6ms9Ny^=r`{%^0)cJa9$j?MzP2^X)e0@m(*PmVA8&hN4VR=!V2i>JTJKYh z2Zl?%?L^z>J3!yiO>T7^jCwNt!|UG0Av#9cYVP-wt=m(2L?^5v@t*$tQshP|_2Dli z9i7b}7XDFfYf?Ufb^XzfFJdM7Hr_Bh>O3BE{6Vr@yUw$H}?0;gLU?Q)|iy`ke>vL-;uR$UDz>p?z>Z zip;g1ws5d*05TkGdw0HwS0GAmhCW7pkb9m4xPofzP9%f~Fj;AbD-3Z`CVmq5=Tryt zoiqdcEzhv;p2HLta0Kv$#nYc~eer2YMUalTieETTvYBp@CrmfxycgDL;%B)kuS5u| z1NDsYv&$0EwR(hKkPK;_giUq0vD_1;f4B>q8$`Ag^tYH}fuf$p?uojoudz+yW))Ty z3QtNHdO)rmAiGs`BatN`t!eQNLAT{7xn!KUElESZfH=zUxppELW40L|l7F96sQ~Tf zrlV*M?$H!z9v0)bG>kgD`v`G@8>I$?WOJ|3yYBZ*o$@fRzliZvMQOoWh)Zv>uzV{s z(dauW?H_{R@*-}m9aY^7m|fx|w;vGp3h{(96&#Y2>-avgV+*pHhR*-p3y-4Mo*N~e zMK{zSj3zkp$_ z8CL(N#Matq`}b7$eHkdt@%z^Hkah}-F^*MKcyA)R~ zXt2!8Ov__KBh@jBSU+Ut6eb&5ANX|kheJ&Kv)!-k$Y+5D0z%9@VW~(J$d_6E2k_4F zqPvm;k4G%`E>D6p@lWAOzCmDB!?xBFwl_3GfJH3*motl=Wz~e555XF(;pFI6XP?md zB`W<+fFMvWi7eDiIVy#|95b6Hn1o$(UH{iyyeH>9y)hZS>b23vo z@~Hp@{1}QP=Mt<|^k+HO85^f)nV*Rh=H&=zK%CknCeTWtkMii|N02|&vTYMV1349B z^=AQK+tc$B$I(wOOcBS1?Mj-VFl4t><@-{UrZsL#gp*B0Cc zwKZ>C_-LRCFsrPqLO3+MKEEX zgcaOkz<1xafj^;G3!@oA9*OKLiGuP!6Y`5tlRJqq!JShhiaRB-Ujo%>9!b6$EyP7b zc@xz11pQDORWHYzy#APoNZ#u3a_VI}Ae-BV`Pd;`sL5tiZHa1vcB*liO?>7cXeYjiI_-{0yKy{y zB+f^)-lSll_($KEY%KUwBOTbQXw6}F>cEcrGxg`Zb0b+9<&b(s&>KTS=7UjVVq)W- zX#NTNfACGo-Bg{+Y0U?M8LhqqhM|?CON0mYO>|jLRLAb7Ps8RUK&_%+4d&DMA$HG9 z?={8;LA7)T)%mq{rCIuy3et(yCb()_>8%rFKl2-1;>SkVj`jBcX6#yN_5^?q@$exKyqw7e76%5^6*>lp+w30??c*nVp(lz zXKz+JR=Yctx=o%Rhv(dS|0s$EXn1gQqu1Wn__epT!liv`NKmB=0(Z~P8Iuenla0wH zpMEn(ZT3o*6ahZ)O+vWWeaMIm%Wf--;7Q3oVe37$8iifm%!kj#GMvUxa-PxeMIWmB zj6&_6XQfYgeFyZIwi8GN^Pgc?RDv6w;jrWnbjsip<;fNJ`XjZvQUM%h|BkFn5uza+ zY`VLy;i$m$&5X4uLxG52)}eWIAXIiQMv=_#M57sDNcgxA#=5ls;{rr`+kk6yX+KPt zRYX*za7I>=9^T)ZdM-D5k5Qy~-r@A_a5R*Ms@ts=TlY{3->9C3OP?33_rE>Ld%`uP zL#?x2EeSt2DpQ$nzWZYrs24$Qy~P*T5(R@q>chH0ASlD44Fe-)RFcHbASwALLL!|m z-5-MvH}_cpH}lK)1%s?V=XB_Pp-jpYMBnl&lk24fbT?~km3h6H6d3=1v+!v)nPPkJ z=XVV=@SL};Gy{SAt(j-*u^}mEDRY{lkhS=Fz2o|#Q{9=^>Z`h~I4_Jp%aPZlYx$wq zsQ7UY?%fv)!u)JFmh`pL5`~=BWd6qq0*65W^8iv%|APP1fTt{sx zk`;Sl?!mymXw9TNsa{R-J|GC;N5?|a`icI&3djK32^*X)OPh!Ht7^=u#-;=Q# znN**x!VgOvg8gPVO3v{*@Y~*&psSqU&oOUq?`&Cx!qURJLj=gf~F_i69+-^m$T9TQEHIa<-c{vrskFK#$DyUxkk93S#qHt_y zHd`9IKfeF5t@RMqMhM#e`hUkNWb(MDI|#&MGqz~saOm3SKY5f9=RT#}cB9+^@3yu` z`?r#=;S9Vyix-VCm03YBkpeW5gSL{<08ZiBuQmt!ZM&>maKrhoVzV6W(W(Ge_oGP9 z;2p?SJ`zI15FQjEAzeoQwxR{f-TtHhN+=u@B>51zV+I%FvVf()~)GfjZhVuG!c0IHE8BPCw+vILo>h?`SZDig>YSk~f z63`P(R1k_38W^fjOe~iBM|Mi)y?)!(@rQcxT$8XJWM2C7R_*$h%0*)(R?y?OyWl%U z2rQqC$PiU4GIrRoM_2Y?GrU7~SBceieZ+2g?g&HYMf-oo4}ahEZl-5w{l3F%Tz{^VSUy&z{yxttRHP5@BJbpp zQ58W?$4TIZ?1PbArzy==7n}NOC+#Dp6x8N|CEEuBQSfl<+ap#tf!(Az%6 zLyC#vuvPt@ANW;NARDQ0#IMHhbYNHW2#Bea=1qL9^hcLPcc+oWR68Rq*8?+D2MSgL zm>U!$0K~I~w%MVle_w~_Bw7mF^;gUZP^7FEC;OwDq zs=$Me(yPpI!X)s2m4IKu=S&H`oJrZLa5TUtX6V3PlD{v@i%6iu@V+LgN1dsHh* zL~L)pDBpRP=17)PziJ=)t)s4?Z1FV(N*QkE@!h%KQ_{xqm6n_yJIX!0IkzqF_nuq{x1d-xg}^%`nhZChK8`$q?Pj{7Lmc z^L1>Dvoq4Yq-rBqBhk~-n}&TN(ZBglEBqTay8kLnH4v8*U1VJZZPA+I&-_gK?A`M|4ur&u;*T!exkjvlt#a%4u%G?u>OW<_FVs!pr*4c zSY|z%`L<75xUr&cFZS|9{?!Fzj9_D6fb&_h)I}3WefjR+yI2nHuhvBWr%HVA?k@go zqFHcswkFVf)(i8w-R)Dh{nNMq^!UHeV(`;RR&0Oms}~_Px`}3v?+tc^-5&TnI`0I%eq%fgM6g&$f+)|SI!-K4?*)@i zwSVrVcLe=5r7MN$m^bD2J;p;n1q_8ALmnZv92L)aph}VeSY+^OhZ(Q7Byx`rgoO+J z_dD}&dWsR!FEM1VZ@&8iLpj*jO9sr#gQS_k!LTS`^~LuqiGX(w3`V!}=VSIh2*`TL z_(4|8#YuRh=DAn7{YIpVsF=zBHU5T4r!0i=puw3$1{*}J=@o8M+i}S6Cyg0yuT<-M z$jGVtLIkM4pb4HHZVq)Z4dl*}>ii=7b&fZ076h9BWufLGfH*9uH$0Q6+KF$hF*1eLXG`xNko7C z9Au8ED$mvu!4{t1Pdk{gA7Orr7_E}kR+mJyX*TIvw%n;ju-@I@9`WDzn*1m4#J0iE zd(!qtzpy(99&AF2U*9P4`o`Y(to4LY7w+et=5XHr9PHgzkGy?OzSU+=RGs!kE$*Zx+$L9gXH1fi zj|creDu?h`^X^!{;-Q}Lm-fKE*4@qN*B~nXBQX;wt3Q)S2t6s+jqWe7u((3=EzEG@ zW9|p0^g*|vECxi{g=i@+l(z<;|i$RyRFAT%&Ci*V= z*y%mTMdu+bhLOMDV-$2h7lrEwOKo0TEKisaM(TM9x!V_ahU{R4Y~dTXCYOE+QRV9g z>5jkdgtGHy0TQ8z|LdO0LZ2=+AP|zCpt)V#JMZF7&o^63p53i_*a`HSn3VduCO>*z z>MntZ(#U@qRGJ?t$d7vU@D`2;B|09$Gm!gQ>W&;TVy;z*XQMtj=T+OSDF2lfVz**< zir=gt(fQq_&)?mb2gk=Sm^+q%&NdTBm`+NM^6xfGavsx_1Wi%7VY^e#Gz`ZY-hB0% zDUzCut8phBD_QlfWDct`PuqxIPKTSIW%_gJh2Aoc%pD2MQcHCabj9Kq?mbHX2xYn> zQ&*RCtR_0ZJDa3??=Y0$%4<{3Ejtv8NW3NGIy1?F!$ZB{VQ@yjR&Gt-GMliCNt2g? z8;J>QQ7zCTbQnh_oaPi&lC|(@7~p*K_HU+eIKiuFolh=)_K_J8z==Ltc~@;cM`COh z#_4slvENmWw*W#8C#Txz(?Ff@Jyd|F`h@!K<%Z1w-I`{lR9h#01IWl^Aos1aR8iH= zLp0uT_T9S14K|TXRNsI`jH;AMZ zIv)2~R{(;3)&HF#9wSHY={`jQ|`-@TJ)DPNbz*}jnThVt>cs_ zNe_^(>HBP{qH*c`rPQO5JH@~7)Kaf^0mvsY4slnzEPmavkEmnJ)+P?7Fm!L^HMgjIPTpGhwJ@lg zHCVM$yA#}Dn7-G)q=iFl9!uJHa;^DD>LKOXf%@F?2!B62x{~+>s9o7+H}wa$lT+v} zPk;rD@nok;WX$!+q$t*bRkM*3`H~253I+kSrJ>)^VflF^@OC_Q3(qr0Ia|)QWkR@| zdz90+yJgxrs8t;2Y>HJaFIZ&WUPQLM6l`iDAuzy+!K~K_Lb#si%(=2F6=p!CJhiq#}L?QU65I%60`uDM-@m&oTKAM-K zOnk{Xh~H;iLF?Wcwc7QBHI|Cl%!P%)5holUg&*sQq?w57_@Xt-%{-)@|7;EJ@#o3J z?;@NmlCBX)bt4>12;#L03^|H6CrYp2oJ?>rT&OAlQ4ZAM+YQj!R?xHSdkz!r7E>5g zs)ju(**yhx`>!G8*LW>z2O;r53+7STtiie!=X}vfzL|t2R7|~A??PAC3;U=!b9gcj zDbhNGfv3>qqbsL>T#Ub8AsiN*C8!q2BT}-?kT*{sPV1Kh^?o@FsDFSzU4(;toK zM>Y%KD?ZbXvOP8-(~(i_u~bL>mA@wI8Q8Z1Y8`|mkBCes%gA2bG* zi8j4=)SeT`fHT$S$Ot7%Xr#78MHz46V&PW5QQ|g~-0qjOhNIa~hqQf`8}t%GuZE1%cQ5Yn!Zj*&@lk0>2+v-z(yY17;S2O+vgKLK2VRcEcm!?@E64(r3z^mDAPvu^l{_P z37bFHDRJ?PX$I38P$LvB>gPyIC(NnP(rzhdec*Y=8_K+;H%-Ej!uN)$pe5~|vc>uQ z>nu|S6XVNMJ%6>4gKPo=C1~SUEWLtQvco6U7SEw*U_#eAW&iPSL0u41#I*nyQ);j# zq1T^1TrUTUldoh_!p8jSL9%neCW~`l#Ce{2`0q&IUp|b@O&SuNiJMzy_3u{vpq^o@ zh^jJkBwD`vX*ITPNJ}cG0MWyvyb~82JyzK!#v9eifwr|ohBfKv((2G^9upotf%}aj zpQE#Y&y}RRL^UreVcNoq!P1t@C^i>KE0vUgx@jrppU;b2etm+zm*gOANo3a`!ObaA zL@$eMF+(cf+N;J}{31A8HXL7;e6GW-s7k8JQ~_BJ`dn3Q$k<$2u9f@FqWm$Q>!vvq zk6~o*MD^p2H{JK6dZRf%E?Z&pA^3uFeFw_|NA!l7Xb7zS{4VCh{^d2s_0_#NdwVnw z*obVeCH?v$Dy8}SM@Zo8hvO7>y`g@nx9Qo)I*`XvJWJO7s*V=c49W`=Cw*r(E^@6a zn@cllcxyB`cIZTVo~%q!6Tw8*|%4+@9 zVbY0?+w_~=;R%qnW4M3(@Kg2K9QfEn{hganr)0DK(57_frW;CX+BoxE9^wP}O5^hn=lpsfPo6ai z6X`X2*|BB^!L&Am>DrKivoFAo)!2})j#<-KA)=E

ZpH~k-xZvqn^ z%!X0Y+ZBD+rai|TvqtLHV6pDoYZgNG0 zo5t^r(q%$FzOnbEjy`1Yn52-Bm{doKiSKk$%=Rx{t=tKkwdc;jC!##OJV{stX79A55&Fco|BXbwF^ zY^SpmpTZ10z6W$l&~Tvu7VxV=liwUhyG=1|(Q&&E*g?><-Z%(&ik5ArRm(u${v2H$ z!gJ8Ri5iPT&g48aoUk4Yd(Jn%}Na%ZG{jxlyBeoR9x3i3Is zX)6a;`0B6X!a~HGx#+}GMUB&?ICG~*hBBL*r9ob1-63O81AMsaXxmDF`9Sr2S}9Sr zfwt|`vCzvD{~n&$Xoolg=V_~WB9YDqs<*J6=ntLe{=E=fA#17?g|$>R1+tLA3vSuO zpvyTB0q7Svc$BplF)@iJx5i21&LsTeh4M5)%$Vs!E7v*yJu0gs0oJJ@C@?kx71*6% z;%(CbBy${Q)25Gp$@|(fxHO4e+s?k7X|{ZS`JMvMC&Hv>6DAmbl2}cq2;*=vSr(!C zg6ra`h$E_W7Ld~Y29BRLQ7=Yk4~DpSyGDiaU{0Z(vYCg%p72|8KSEfpa`=|_WAu`Z z+ql*6!!jFoL%jP?VoMXdc_MhA;o!8LK4g!afIvu3SnmqL@@krHznkj}tp=9vER#)Q z&>;j-6~15-{7wH%ZxIF|ds$%I6`P6zqY@g`8oQoy63{JGX)K6|C7cX!hiiT-z_ImK zKBLA!9QEbvQpXdI<7PtRU>h0p!(f{~ZPER?=}t$ru|bv!!{Nw~e|wC%FSj+yx7I2h z-hb$9Hcbw5F5~!ocigp25Q2tiRWu>#vLq+7w7Bqqp5%hPtlKp?_<@V)FjUoR&h4E9 zKzx+M56{YxtK*R?Kiwe=-lP#!;N-U2j@s%F24}~GcygPChpRy`8RJPsUjr*B#SsbB z4_x)6ma)(l#Tzm*9dk3@98FdLSF!*rSagT33W%m1>-Cug#NQwde-lh7ITzl8q%AbC z4C7D4Ivo#>4P+-)kCFL-Z3o#Kd~jwPysn4Gn(XIiW6Y~)HJ^@hu4)?Oou4146ljmW zPEYiD>gzmei|aiys$b7|r+y%EAC+lJg#Kh=6cG+ZL7}UD%x4Q z{cT}{0G5+7C+@1DljVo?bPM%E*)LEsy&yjd{x)-11&-n~_UmXsOWmwq>~JUn#{ny* zCAcF32&aRs}Lxp7!z+%BGmM0)@XTE zaYU+QSn1hv>uihZi_Nq1H|Cuoz+<&8UcuDAQTfmvE<0gzr;PDI-Sc5D#vE?fuzY_i zN+d!0h90U5eCNNUN1IhL-l7idAJVQY0I`TJ=R9;BR84{eMVkJ(B_sjd;=w+Cnkdh| z<~Czhp>N(Zw<&>-`v?RkYDqE3h~h}1#p9av3c!bH@<&B`BrpBv&k-0lX;tt}kc9@K zjsnB)%el6wKayYSC}kqJ?6v^R8K%`N zF!;v>U#z(tV^i|zV!_zf&rFKbCx)VuIXP#Sv@E0F7Yi5fpNgZ;k=u$YO!V>#3+Jche!MeX z;V5WzlFFjKXNn6tw9}sQJq;d;4EU+fTD-XxkUqB~T`5C6TEwl@lvqmUT1*i5Jc)i4({p-Wq7eO*#!AllRYP%x_sQe%%J|>3V0XEFmC+!(1G-`0 z>afC(iFJUXJ}5vcy0~(+zZR=Q6ta-+=%Q=NadK9DnZO7H(P(B%at$l^Z4u#S)-yOI zJ|7C-H5fE_Kcf*;E|tJ;4XMdEw@n2o6nVHXTG=XBT|YHM$A=CX@Ro6!ow6UPsV}%| z*exa>AeVM};t}3FQ9rKya@rij_KmuHv2>ZEzhwUHp9jvtw&CA5`Xl$`a)663zaN2 z$UJg;AE9&qA9*7lF>`dWavLE!U_?a5CG~*AArYk<~H8rIpMv;kLVG3X5N`- z&Qk+uaboLp-xb5^OgWSSA>QsO&?EdwJCN?J|75u?EnGt77hzA9-K4%FPNC-*IDGlG z?33!c)ruY!K`vhbjr-iUc;S12HF%qxipR9#NGA9|h9V+Ixjz9Nil`yk$-=Ped07X% z!0E-mxBE~t)5{r73j(v*1wU575Dw;9&Y2}ZBs4LFa1|YM{}&?j%!u*Eb_T*XF zbPN|@PurXOFXN$lMI1>WDp1NZSekpTxzhTH(ZalInM) z{_RFn`t>lBMjuv57Zp9%FDFzgUZ|-??_@mZDMCJ_EW%%KqeN&vHv(?!>YFUEe~IHW zX}MtLb4be+p6-q%(YKlP%`W{(@h;*7RoURmN7S7U?;@cT`VGQH+Qd)v^r>qlFyisT zr&gjKVVXTo?D#WzHsxYnQkh!R3`#db76!PuUGM zk8#6@Ga+@3X`@AgTEjwf{T60N4fV53M=+`i*_Y2TfLYq}`DY<=FFJm7QlZ{$SAf#& z6nY-NuTek!uVR|!Y&TL$z3=~KMkQMeIWBSutq>u&B+sUWv7kvs35EK0f$A2Ntv>`t z3DZ1y6t3p1)#fqv&2k)xN3`(LYwM%Y?rBnj(`!5tS^njYEPYcIS5)Ni)lO9EynaOy2(sZ|9z(=&moOOpC^v}uQ3@Fi& z?>1Lx$mo=M>Qv`4``w`w?u^T887i=5J%3wpYan>u>Dhq;{!x*4gjxUNok0e@p#tyA zK#H6>uBrN-TwAt5z_R?b+4-|nLjoB{Is)SOmOIXd2-x^@m^L(ljE<&&HgW>&^V3k# zrfcH@o=t$hzYNhL+?GF)$-^yDycLn(*1%jhimZ0Wr9Z|q<5AZph7Gh6QdNyC$BdpX zh=R)QV;`B_=;NAtdDQUAC5P_*$0eWrF{DqaiQqV&U>@V#QDdanNt9%}sgCJC8OfMZ zd;OKI88cYEd(+U)kRO&Ywe#0J+v@c8N~NKs{lPl6uZt-Iu!#FmJ8jv_MY(!5cbru? zM&Q?JGwW~ps8NflWx)N2%`9wwjG>kh;1$g|lESKSYQ$~h&tVl^4Ny3LGLv9pk>hd= zAozjDVOM6{Q{o|>!4?gqZUc?k@5JVlqjcqBhAg3j3Cx0qAoWj9o{u0rvy{l7%j8Mg zgC<*Z`DlE8x~_nkwKW1UVH5`F z-T3?gUw0vX6pWYzd>GM~+7DqzThz(g&TyV$svJ{VO;Jo8?##QGYQz=uB9NlVgMI|x zj)JZoRJ2z!)J$WTp{q9?)m6W zY{1gK9M#<;ZW^cT%M6MrGSrf4C{5m`W9hlqaan)NFJ-4PF#*TIE{e#&qWpgQ4s_Y! z5lMZs-bd5!XX3AAb7VjxxN~;IY_qV8Y#|3thxwxr8FmfA=Q=X*JXHI~>M0f~g|k+J zqi&|Y;PmBk*uWY}V?2alQpFyBGm0U8j@Kp1Qn;Q|b0JRH*`U&|+-7Q+*Y~+G(cgLG zLN+#y$Ey57&M*6Ebo@}zuV>&(N=HKB4{L&em1EfwIZ%GJS;kJxzVQ~hJiLp_B$B+J zT}Bm!EjK65HT&b5uA(-RDvuT)eFxWF5ZkBZHum!Q=kgt;v2G5_CF7nDl;Ny4+ic^W zAH)5i@-_2$JB_pN{CRZ-yuHt7S(Y9SjxHtnsA zgUTV8__Mh;$7>8ode&1D7p*Ae)S9ei%MX5k{mgkCH}g;HoKD^Zy0Z?1Iv({6nJV@m z6x3u01>@~P1dJQxYiJd1#+n;ZV#R>GcJ>*4TbrUryHH3F$II$KZqgCISE(e0nA`sx z7-Xv`NSY0xZeF&qsce>ClJbufu49&>(k{$KM+&F(BVWyBnN^~)U82RmM5R>!tGY>8 z9>-{Ohs^~#DyoW8geeot*M5TCPwxnSiN|uVg%2} zRkGRKaCYDCwApN3w^;Mg9S6=v)%Hj+u=G;u`tCUmsTVG9{(Rlu)jm9q5>oDz6kX;8 zEIyQoQ~QT6r%^|HOCR!{8-DJ_b?uJ^T<4hJ^YhoKTiENx^DWMJJan(oZ#5TtT41@1 zdjPxSd6(3RX1?Fl3f`$z_C@l0dOSRk3s=4w5^^K9)0^q$&uM!P7>BV==hs4ThwfVt zOKI)WQ6NlZ59-ybCXsNaPZw1-@a^EpUXRO{8svad&i)Yt2?~p>cB2f!M&%hAKRHm#gmY{pV%N}M)i5jCP6Kf#B}K-S?%#eoxjLgomcqWac&w&;3h+AB!`6Pme>WUkl#| zVZX;%xAsxQb)LS;|H>+p>vZ)U=T3yM7Gbq3Z`|vd%$qwO=QIo`K45BEzJm>qHK*IM zJt=#t3Z6Z8w51b=maPw!2qcry-CY*R%)46g`{FfYofZfjdoo6ooh0+X=g> zX^~g=ft2=cMN= z+2Q$1*XykLnNGJAz*Ha2De*xiMGU{&;i~wZ40D@XhB8}s!X#TR_8~Ymv4yg%v42Vb zpC&Y=*Nt!$#8kyj&y-*OX}2g+nzJ1xTk`llagclZAc80RIQ21Od7s4FOOC(5nktAK zAoxx-(6oX|C_lk>9X)It!B_+z++Vq5XxOeJ>VIFO(eb7s`Kis~dWl^wBAs86ewA@! zy~vl~|Hsx@hE)-Ljb5a?yFuxcZs{&bN$KvEl5P;CyF!92Jp~N9M37<37|}V1EfoAKilQt==x&%-SIvx-u0+d;QU$EKSH6G)JNtr zk{{?|wt9@T6i|hY!Gv9+U&GXnN11_&BsRt+9V;4ygEwX78YKy}U=32mpx+hvL7vP7O;R=oi)nE&$_jsdRLqkUcInHR2qZ=$=G2VDPuWp?8)LVyIA`kQ#2Eu(aYr9abZH{OFyg`#5 z#Ff(n)!5kh2Fdai!=jQVjqd@~F(@#a;+=cw%U})H^yz;lIcfkX4-IK0`Gj<1;~;5} zy95rlP{;)7=Z&C=oeitP%JZA}zYm(sb)pm$)VGu8Kck)jE+#H)Q6t0Mt7+>@?H_aV zO>S)p3WPZW7@%b2*;}s2M0IgL=x`&VY!=(Y7{-J|H67kqN>^4jThQbh* zI3G?8~;10!)Uh8J?8AUS^CzSw#^_@mnYXkslDJZHH`04GGc1PZ*8UH_=>U=~L zj%c_6+|B;vUr)Hh^Xc7~4A!IJW}c7SDzYO(m}L^3f1fs#-h5~l1cz&F#RlXZo*6x7 z$4{Hr0t#~RFw>JSQkPc{x10?O_G&BGE%p~x;!@ZU{Iou4_9|NZ6+=t|pQ187@DAMf z98NQyQnN#pj;5m_5dE=imLY{_J*aQ>+rivWTXJRAq7|l ziD{$B{N}!Yj+A~gbTD@Mx_Hrf*RvRcnfx9{&vFU%S#;f4$XzVXD$!QA^FlVHOmN}0 zm zC#RsZ%=mttY@Lbgl!MqpN+vSU&)hjz2USn9QS`RVWI+1DWSb0D|Ky1XmE@3-1ecy< z`!k%X`|9+1Z*xb&PDqFPX1iVpZpw{VW+6=OKRA^#hayMs#|^1FP|YIh56BYBo?T4r z&?85I2B6%~4kuAR31|RhreAL;7656U%LPO)VgP7m!(ri@k$Go5sPdHufDBYRybEL@ zE#oSX0gjwbTwEe#;!Lv6(XU8}ifqK=rOEOA2B}W)$y92-{}%=_ecL76_6!gL%NGhd zlxkj;)aPOhWd}+<3K~~Yr`_%@xrs0rtEy;uv1)}V{oeC@Z-FotN!30`*QnEZ5cK66ZFB-mC~JaFSHMHhrJcLG0eBHVydTgN z6}`g-9a7#^aZF#eK!d>E-O0seZ1%+8{XVHcOgbLWk1rD{G4*5DNt%T?NAqHF3di-u zqV{;MVs`2PTIl#rPwNOOGC=PQuA4Ce7MAz{_PbZJ35E}UYyY+8a)7Fwo)8TD?dj%I z)J)NVKI~RD<+;vI`E)W1P5PohnlH|5?_ULPX_V+I@{>sqleXI-^M47c+|d=JEU}+EV^ep_c>1`X95z^71sF~|*funW ztO$QMwgcXEooU3!iR1c*C(#~J_a={AgE=Z_B7F4Nk5_E+lmX=eQ2> zFwWwqI0c;-;EBW{iLWlJm;GH{9xN;-!+k5#M_mQ4+5zV?Ye;nNAE1MV4iWW-BcvfU z#yJ)E$>i+d;lzPsODHWWM(rQB3^*Y4oh)3Uzd4&kf*qs*I31C9EgFt#8pre#%W(C; zYz=X5Lc;=Y&G-Ka%i-RtKnRS?H~=P|P+AD;1vp9YX30G|fo*@N!z5xxtOiRx$s;=H zGq(akGAAXeoBqLcq+659&>aYS>e7()!##V>z%y45`Wi7s!tmp6_;NQQ}EZQjqL8{{E@`Z zP2b#{k1jDpwCema8#-pqskzqgFBov?u##dTD!!P=klma@0^sxeTCcuO&PVI=CoDw4 z_Yp+JVivetLYuRz=Hp?R_9Sl2U%+tK$FiP}yYX0wCRv$7@$Xw{$|6G;;v zWksKnu)CExvt)>^))7o&;p2Yj*g~TNV&a*adf4Kl`eM70FLqs)b)thB zF>_ub1cPVhtHS(a?XF=6`8piCYGq#_nA<1kr>fAfD|0-)iS;2~u_`+#v5(O2F;h57 z5c;eZyTaZ7iIn_0TdYb7_||8pkl9FozAYGs$``gKwe-|AUfw~RFbJRVv3`i=JhU@L%$iGWfF7D=9G`@Py!5<)C zgZ{UF77G;k+3d{+&NK*%HFf^;TEM@`G0gJ;ee2bWQU(sQ3XpX>J`@@k^dzflr`M7G&?qN$Lp{O6@y?^z{zK+F!KS}Vh z)s#Xa@kVEVo)K%$B8p4BM94A*ylcvnkZ(C)xKdveaZ0p~Hec5JV{DQlaYgsu7Jnlk z)H>cDyXO(%QAeMF=b&+wu*JEcmXz6f)01}{Qmjlnno2LbNb5{|7TyQm@20u4;tO1J zdkc&*mkOfbhi7p)G^gYRq5+VmnTgT5Ye98VTFB{>ojLtHvDTQdHO^yr)#GyK8oq+UfX}2ovR&*z|LEW!oAAK8gCaCx- zSx2hJb9iYNOp7!ST@Kf#H{|h1=3(<;O9#{>nP$rQ<2^hn7`UegGQ& z)cWKg>nZwArtaM=%Ff62rlU+3A109a$ycR;%3_<~r5?PQJxFVuQ{f zK9L;kwa6wKI^Ocy|2~!Pn%mL7uspI@jH1Njxn+6Dn6KV^NIOU9eSPL7)tGKoqV7QW zY-V7z&a`a8%DgqsQIkHM@DSQ_Ye2ex7(wMeN_3ophx9ivrA{JLj{DuK#gNx^CevW;wSI|F`?s4k^uL`S<2E{{h`P7g~!}_DFx)x+=m#uIe?j|jCniHvJ zPx+M^l+jY=FGdXeNH+WH+Py?ZI* zR;gUwhTt{`_m6gWP3GAR-#lEH;R&F0&l7_mEQ>D6aDXNU|7Gd?}A&kyk z%)sgE0ecQwba+S0?qF;&VvRR>*~wYje~kjC{DgKrZwT7epI#xYqiZVcP)Pi-C_6+` zOzVd*N7pU8l;|{CsI}#SiwSBiYw;)UAcj4x?Zo7!%^VA8$1}3*{SD227<$I}9Pr1$ z3SEE?XW4?$Aj873W)wD_M0gBAM8SzUqPgIPBlXoX$0FT||BLS8bpEHOpC;NU6F!ex zMyq=Ss_2e+4lRea{gkKU-sCP1S+-->b&7Xq2-_;(Vild3LgGw5dokOk@p<7rDnOoO zFk1=aMSAd+R4?wG6>4%0Y`d=#-#;$Z3cg?cVt=#e9mmKVchf|JYt0x_{aF4tXySRh z>8g=py$LNwrLcmGC5z%|NyrES&2fIq*Iuru0ywF!5l>)F@~Bw45esycjFL@z2LB29X%Hu}8Enw#PD;BRpE>C@EIx&QzZ!+MggW zQA2R#@yDuVRGWDwdy+9Xam6F2IBSat{0Fh-iKmM3*Z3b#-b0w^(9uD!jt26`l>9yR$JoTfscA zOELev9eYG0-)S)OiTLTq-DW^$X~JNB`Wyj{K~iNZd%qWq+N*=q#b~3i6YT{X)lVvX zKARs(pbbmio*uc^?X6tNBA(hNpPDT?xwq#kr|mhveX@V@h%ZWjkBG0m(8F1kiksU- zAHgQDZu<8_>3tsMQxvNCgn>wl;N1tRowNzVi6GR;=2lI2kl_kQ3bTAB*g?=_>$$sf~RkSjRiCz?5__eyU>#xdAHnKj?D9~-uF$13+bJj@yMCjbYz z3?DEb-r%_@(oM6qIaTlVN1y>2S<9DPU>!y>&jmiC8P9WtaKD4Tm2@+6OS7{>8Bbk& z9`X-63!l>1^QR+MU!6fC*xkFjT&1z&xsdknq%n#(!wDp$+g<*LaEQmutp)ffI;kNz zdP_D&S46fe;IU&WQ`p;X>h7|GmIebIClBIiuV+q?g3S1B0mfc!S-tn|MYk9Q@C`V` zYe5h$GE15$`P(geZ7k}c9rF9ji*4dw8_V2iT(nNy>X)wSCRH{Y6A?)dj?8j3cY~Iq zt)+_WRvVm=md0jFRw^EGS8kwQ4ZG}n^2gQQiaC? z&-&3v@I`~+T&mlXkBqnqY~xUjSjexj7_YG7{t zaNN`F<7vaAWv*a0t^&5Mh;-hU8}Yv7*!*qBxNsjofxELJd?OD_Q_nO|WfEP+!x`aq z`{W3fR8M`ybX9(J<+=l$5DSujxrsIF@6j5GU?SP@suylQQJJ3QuCaT7Qu*R@dzHL+ zbtKnHe`Dkf^{lPXDfn^VufnUfWhg@C13AiT9hNp_2wXf?S4Q8f*!TPYW&!YO`ej1O zGSsTR9YD3NG}Svdj3kk3HXpKjEai{L#z508yo*ykjmSaw|0-){xQ^E(n!kpKBNOG` zB>-bseC|WZJ933gT01uT>R)zbfOv9oQy|hhk_6A~<;>wKfud@d91Na`p>v`#7$50} z>_lUDFkwZp{!XZyw_i_bJj`@3I-q02$0yB7A5__xK5vr}CM^(nl5w%g-s3tjmW!9u zYTG-S{Kfi6@Qi$IgWm(Mo&8L4>aIt*+oIK=G%gf5RwTa{9XVF4&yRKCB!TW|jK$(5 zM3Z_q;M>8JJl^$@1|suO<~o3cHC%i08}`D7dvqwecvs}4U;21MZyZZu7B%xzDQXj>pPG8<}-^;9=-mpiNV^qe`ZxH)!J zP=mLE2>ZJ}QB)+hoXB_t_V?KNu+>D*M5BSz@XFV?zIrdA{Uj7BQ5L&7qS9aWVIWjn z*QzT1YETOX9+(Md?6{{erq{N$j@IG$0 z0B3L>vlToPEQaD@1m~x=caP@}yLnBB%z8X>@jdumiB2TT>}U3Cn8f1t8Ji{hPTxa? zziMy_WH<<}G$gvj$Gi7Rv~pG;_tI_M%;^6<+AEPz9LU^TnM?q2H;z_O516K&eHd+V zHCR^*Don-w1&W>uEBZWAz065sI(lFBw+>{E+8g=L@pJ zWTm8WiI|j2+VwKudwgt9*X~6|$x7!V@!C^0;)gindyvuIPDzzKXa)H&D|??&)mxk< zwhRSpD{6kNrBl%pFB1-0H$+chs!(`u#DGyC0`gT}81bU8;gs=*=n0DsoPhh^5bkqn zKH5{kllKm#hlGp7P9zQ@`7BaakKcD0psj>nS!pyv{>-AsrYivFM)d$a4x` zLyK7DF9w^4@#t;lm%AN>R1>sG314XgFm43jwXa2E0w+o+o0eU{QZY?YXjiWJ6UF5= zq>?*-Ag8>+OV={uFwJ}yjf`sqp{OB3jL&q-d0jMV4Dml;dg_f9GG7f;8BO-yq5iGg zKbM$$3B@`KLFx(`^9IS_p)W}f_L&vr%747q0IOry77^4FO?kdxqwu7NI61-w+o5M# z1iQSBN@x_*`xQ=I9PqmHySMgz9ck&%V^~8aVZ($i5MbQD9SCq>Y)t!E)82VHq>aY+ z3D0HClHBlCN-SKLp)@_%EoH6m{=yOZCH|g%m&jDI=_k`pGHZO5sfGRwgu(7HbKnrS zolWbfYY!xXSqPiR#FJf*WYkJ!ZUW3`)GEUnXe2ys{S;BROXh=8qzkgL7WZKPQ-Vc?_zT#6qy^s*Y_qoiOW-18b*CoW)CNb@5Km%rVsBbr{VN3TC5^_ zKMY^^Xxjqg22^Th0Np=)ZCY7b^DqT3C7YR=Wtc74)RL+xSDzFRiyg%rwNO>xQ264Y zsJ-_k;ymu|sNu_Q13xr zqEp~}BdZ4buvU*I3?WA`_%7+SS7rk{N~XQ1X8+;|C({@?CPVjLKZm4!`*nHlwl@N{ z%l%Um)vxUx2cbjG@pQq6AG)}5!@1>!>wn}V%`AlmWbt}2t8j?>eFPip>RBtR&^MS6 zrgX=KLZp9@d_&#vDEC`3N`{7TPvWo(O}PpUg`AwK1HtkIpynj7t=(pRkwJ8^MTsh!55Vzy}ADhm|7-Kwh% zsg_8R4C?Y%?skpL1<#O?2pcDWodg%~j)+XG7H|Z)v9>Y|T%NZ%o$|OtslV+^ql{2- z%{6_YUFqR z7PZ%3^>lD!d2N_@8vj0GGbdHjGe*Br%iSD4a4Zm@ZS5J4MLeF^l-lBsDnfbRv49HC zCIAC=W-TF}292=lPPC|k%?+{Cv z3n-bL2+}|^!N2DmA56BB57;^qxkC#DPgP3M%pTICrP*uZvOGB1@rrQS#1pLfbyExv z2dQ{BPNs_5#%t(!h)3#q_Vg5^x*ik)4|ScgzH6X_<1mF|!9N@NlR2XDupCce@iC1F z6MEXrYGG?DuL{+Wp7t~zYmybvyY*6XX8VAy+tInBVf7@Ru(;M=mq0>X zM%{Vx$=`P8S-yNSD~#riqcf9l^YP7wq!XzjGMv@f*fy^&QvZ z0ahEB*MQsC?8Bdyq!a^bv_SIdd6)1{OkM+`C)zT)9krzy>*yZIbi69!zcS2C2^dJZ zKVSF!PKbar;VUQ`f%CCkR!olsU%^DkhFg9csP;)|o=SUAO%D)^(m~?1<&0MN`>8Wx zBcW8<0Qr>~4wVN*=xokw`DMMsKyCxNvhH(AbkuSWVaZbT3>8rfK~z_r;RUr&Ko*SP znf6*IHhCF}V)&;PuN%Z{g63PZGplbjOzVmgu__uK$)-$rA+lwXhFgEE-1#fWMf`cb zu1!@zcv53I{8}L|`#ObPM^E5Nlac&=CC_IW(PZB^opY?aX#da6f-%aMWy*;9+d!!1 z?L-dDEz3;^4RL4#k8cE|PRw=&UDth2bV#{L1~o;QQ(x4k6=PHuac~7c8Od_JH!Rg-Pi0tif54Jt74QqsNjR;c;~++C;MvoiaP}S$ zy(u>?T91b{kRUT3=n9OlFlfy|P@N{_FMT5SC& zZ&0S4sfil^`9-4FIB3?Eh(s&CJf!?M5q zqt}PGv!H|rkg(R0A#9?N?dTujx`~jOt9;YVH$Mi)zI@KCGMX&2R2Jwso(K2+=v8H> ztGH3Y^ySeRNOon94p|k|cUbzvb5q>h6W-8VO>J~_XUAd2 zPz)+jRIWYk1Jgi4I_!x-t?TW6FU`2iGO)pVPJ3?LB8X0F{vo^itM|o5Ip#?EPef0t zE6vj4BNO7>ip8*YnuUJ7>O+tU{KW5>li$Gy;J>&b*5+qnNK*ZT<%2UQ$CWfHGlA+6 zF+U1dxT|a}5Hg1tm0Q{|OO6y4xlHo>zF|*WsAjuAq!V~AB;OnLvl-cRe$Wu=gCF{` zkI~ktpigeZggrBuR3AMYW$a**`Ei-(s=Sa1C`^|AZmwlI8?XtbrKO#fuWDHVmUId|r*k+e5;s$r3O8i$As)|iye!6-^gdCHBS0$GfS;f@ zqL+(|8n|q^L@SSn+ONH&KHn((rO`v|8l$FE7QYQDyTteJGl7-=gDrYUn?2P^*yjO1 zpL~(s9KuWl+nLEhP|lzEExUR|YOld?pA+j9F>2d4$OOU8tM-1Y(c^f!3(ZVRY;O3jq%Kpgf=FOjqD#C5#EhmO9WZZzG`xyV0y zZjI|t_DS(jJKJaJ`#l6_ zt(L5W?;hosoc#^2Uzy85>l-=VE67h@(n}W##g*If8hWV+Jfj6WLm%pMY4NB}# z=t*}**CWSATBFQxzRXnFry8x_ciInY1g-BTw)0!@rPCcX)v@TY5u(c^ggF;rw;2E>zVL+ z94Qr`P$%}DgLH^(cU+%NVF7)x3ys~iyM;?54`>9MwzapO8P@*qFyfd)I70c|qMB!h z+ropFWRa!p8eT8QWNsNv3I54X>FVP3@}=^RuUv-fhSu%n^Am2|?dI9;c4LwuXg$IYKGgl=c13`6MG{n6 zx{DJ)2FEry@5k?w`#WVznZbei;@6{gHgv)Tig_+;ySE}zMjd>KMybRA0wjCv%bw4$ zyf<@X&UQq?MVX}0bk0cDl}|RSp#PI^&*4nf_n{(?QipKLQWN3BIB&KEP2RUr z&mVevANL`jQYUzN_6(EU-jUz|x$S4~jy)mq(g#wRYt8VRQisZKPf(4I2ogK3Cf;`k z7p`@@Mmzhf84-Akd6?}<4e)Fdk&-1Z;(+lzR)J3znutZ|hm>xe;|9dR$G_-+c19?SvTHDw_T}ULa_i*x`aC=uhU%x&kTVe_qPVc|^;r2P;u`+CR^{Qsq2vt$JF;I|d&#wQg4Z&A4X5 zHz=W|>f4FJD87&$Cda(lk-6aV#?GYtopJo8sG0KOH&x|DquJ&A?k2z-j)oKN%@s-% zy$sopV(=-k2?&WBRorV8A;r+J?E|WPL8lc5tx&1WdD8$1!HeVD|e%$S`JHkqo9FWJIer(u#R$& zvtVni(Z+aKsN2Jo<0E0DiCAw#+JU9--jthbx3{1uV;M6#g&i+L6EHbPv*t+f{><3* z;A@vexIp(oV3arf)`*@^t;yNOH1?4uUvEI>9Hl+l#`tR3Bi#XKgzhtmX30Do7(cz1 z1}>GWTy!C2C(sZ3mKAR*Jk=>R*=&!Zc_IA=&+!$%Q7;-hCg>VW8prhh=E6fit1+D~ zJn>?)0b`srH&-*J%LR#3!6m>#CMUS&+(O1o!EM1{IyClgyXJ}6q`FGd1Gv?LQx8Z2 zuiZqF{|lE)=c_E8uS9N`=2|?W+|HW(S{W2}$>1m?hx+|myV2HZmk4xWvS}z@9BqhQ zL)rM3C!XLX_YsLR%OW(4p04G)r{QkN8|~d{8`*Pc9ws&-i@gRI2u8dU4Nm_AJh8%} zOyE+8>LT{pz4jmE^$(MCHgmMNh%R;L=sqZp?X9<7&&n28q~p z{9O1&Fwi!A+4a$wQ9*+FcX`@zZ?c2y&Er@zb3N8L+u4UgS6qX?dg${dCq9Cp}VmLtyoDoQNW! z4ef$Xm!~GGX0Nr)S}$a*@6N;1ox082bbO#<_0zkc=2jQl7+NzkIMUmfNMwzoY$`ZW z%?LKfvOeBXd)>kSs>^JFe(R+m@#mmqgqN!XgJHw+)AQchiPS?~Op}yN=iV>wPB)`< z0ZBXuW}|Cl&M1WscD=s@t;-6(2L=g1QO}>4G4DT+{)Jg6LS&^*ZZxmtzaGM3f?7^!1%z7!_alzT7 zU)d|iNuJ1^RWnMWW;=!WG5wWq`V99-#X^4WU9pgsmFZ)gq$2AH_*pwQ$M)qzgQ(?n zEg{B{jfG7WO7laL03Ph#?S2Qe4P2nm$`UEJ2T}p}lh+^Vyjn>XQH|rYizeP~ww3X9 z)?K?_ETK7}-;;HQ*dZ{Gl$2|kb$GYJU(tLtpgdm-D$2oYwIwtSt68sS(hC7uH+xe}bS@{aDojPoXecJD z=pC#Oo!mrj$lg@_`se8th$GlddSu1_NqA&K@lYkLA$J=voDg=2qF(8O7TZ*_2mp7Lv>Zv-HLmm9AV)4#l)l01V<5b!-0HWsa(Wj zF`q4Al(EH4O6J#IkfwnVBB9IoVRy0hXh`j88jvakHt|P^coR_Q(;0k+WeyxdZmo)m zDy`8{Xe~H@<=f2OyX(iiOz{d?G#+t+@F(%z2{r@k25pIa5+^p{ld$xY-BkE8($9_El{G4lv%ls^ zr03%E?!=0)5Wo-hzXWdf2EcTMhs>wRF+TlkxM2L^9B+y3%mGR@gQ#E8ytiFQsUKg4 zC|A87QeLgK|GYm`<@~~xcKhV%}{&bs|aYbqY7M&NPF0{og;+ zE@6a_ALMP7;rxI4t5p#pxW>X;6Up|(uYD?Dp=(y1^Yr|fW*rlbjYM1lxiZPU_nDj0 z;{Io7H_6Nz*eVRBcvq3+7Pg2I81AmU zTi%WJb-|4n&e8bb>pA!U1zO{Cr$)K7y0NV`uYb#%k23}(ECQ2@%{Q>B;d76Fnp|PD z+0LE7`YjdMCv&UYix{cYQ!)RjHQTef)ZSXu;wd9d6ELS>31~~lOXi-f5C;%H*00WgcQ^sr@rqTM!iN<3B}&lR3f#32<$cNDyp{K*i!_edp>pzjPrY(6D0q zr??Pbk)edqpoDilu%hQ!9k$-fr6u}*__!}CGq8any!+)4DJj$qayJ3kD#6aR+y~ro zIoLC(%hxSm8EEEzeu<4^+&=KEvz;mTx2LVX|&&*#vQ)EB7t9oR(lLk#f3XnUU_ zJ^t6dP|BU|M6YN;r3l#!;$S8*LuH_$-EnjriUu+S}FA+pr$$xh%fRBV@+Uc+h9|o!VVr zbhZd(v&0+nD7TBmD&0`|47S75-@xs|a&?Rl-6gv(Yon6_@AB;tpHhqOsCT>g$Vzz+ z33YHBA7y_{uBS0O`p^)1pK&#zuxWfu?gio{3@SqyVMK|AEz)WFq*ZH8-hj6xGZDnd zQdGwU(?HA!j2ozatHyc-dl|Yq)Q?$G#R@zmmFJDd=#v>{L=i^24=keQLU(3GgDpSa zBc^EhvqaLC>Vt5*Osp)XtZI9KK;)LO*4>G{R_SppILn|fY3l$inuy3H{nO(HYoG*4 z|ABV4`xPfV$MBr+3NP`NUIUvBJ%@jMeEg~*4jxIH*AE}r6{DreDD$tmWvkn0fyY1s z8TL8RtzYX*XiGi682BBO0g8Ixr-s^uLGLI1k?h6qmI9pVIsq5Dg+p(B^hjE;4!PQl z7r1*gBs&wzcPzHNE4`t)w!DNNUQGP0-TGc0pD+7K|ASz@A0Tfg_UL4atNf8WT>cJQ zrxu!Du|;Mq8CeQuuxiU@;IWCXCL)JhgH?uxi`2476jr&V08Q$0?e@Nm8xI{S*dqqzrlqHF1&Q6o zmbSq>5N`ix3R`RD$NphP1YtCWTm99(3qn|jV+odW0k{yGDMNF$8e&uLC${DJ(;wTM zV8;ll(AtU>XkY!s=nBuV?4_S#e$@PXL^||b1#{R?v69(-tzNyEQzX0?+}iWAXUMt(UG~wP;ke1RRUx;r_=HA zz?a+0g;c~A>ME1i8d#!2OhG#?9L7HlK`!W4P2wMrCfoHi?m`&$^|hMRZ9f1lkkjv< zrHPzTWex+PjF$X19q4IW+|4^;cPGhVer1w5TFHlw5n^a!OdokUG598Ix?VqyxKSm!B2q)%eDCP=!+wFEa zV6(*9CE48FY)DV5iS@^Aqz4`ZSk|0K2-IvOM%b>%NN$?97o#BYRzQSJ;^c5k+)hVO z$JWQn9T>>wm|}WCz=YrjAz*hmGO4hdL~t>!2}Dqh-}71X3ZoTx0A}7beIuNBzkbuh zkYRcDuv7BXO1TZ$MeK@9;bX}rmQ^4olz|6#rdwL(q^QR06lv?x6(#aI2(s~OoQ^%W z?au_KkHTos=uE#{o#tWkl6#?I-QHwDDs`E7vO1VAf#eRNBW8UJ^%NaS`Ft}$!CI&d z;<3l=ay(KX_50QgPQbk%8O2$5ll*2vFOAAC1C?2IKyfK1uJ|u#QS_ffNIQ3)3ghYY z2+|3zbGll*LbX8!G&UMI1Ld&5f+>@EJafY1o~D@|HITlU;ZO#^-Q35)ubCO1|Iu3{ zY=`ttdCB}Y=K=OL4qSk;Ru&mOT`pBLkAJS9$~>RK3TstVn|eKX^DQlg-pTpop{>$} zj_*lJDei_zbQ(Rw%Sa<~McZuea%Ql+HZRG?REG-fnH%mwbqlvfc>@}NJw)SCSo9@# zpJ)|b;Z6TfH?U3_cLqq2dJidua*du(X>5&{8>9e-N{Q67c{dgS2lpJGsXSeD2&>6` z&L#MZVvK4?UzJuZiqv?M==4n-r6g~duwP%?kRH3;$QkIMbmkVD)nT6&eWA}Bf8>ne zAn{hZvhG{@YfI?n*VX`|6pQX&3fK8Tav24rx(EpdM+PEKx$#5FPy14gMp%l0Lz6&f z#W>GTpZSQw?_dcjbpMSydmo0pv3A#{uoMr%qatROLZjqmUP&I}!^jW_IO%~MY0D(lBFZ+&QR4~(>D zOIcYs@y}fVzllT^H@*kd@T{@3^9k*nS20Eh=3(D!b`;T*$#y{Tb+o&JvVIDK&wmh> zsHq;+Dw=645!%I!FFyjdA{QhK9l4BsXe67|@uqp@+ct;VL>=gnS+c&{Pw6U^4YXYy zyLTG{LS1!^ludn_oCyan8Y6zo3}C$?Zt z3&1ZVl=EJZ9n98}8>leYVY{y-d#Ws1!aG6YAZ&SL#ybOgXD^*%>DjbixInA@i%MzW|X?$Pkz>eFmN7_hHFBP?0ekb z9(d_^8x$JGW*Ww;j~Da{oOX!J$C7VLKatJ9&gj}!h3b4}J=!i(ZbvC!%>``hH*^vw zl$P<=E?BzCYP#N|*F2I}(H+Y%5Kh#fk+bb2;;YUBVvL^P55Qf%3Hj3S25$WY@5B-+ z;Do%*$tr$))2_uUxVeYGLQl^5e(QTVI-G~Cd1Tzy z4Mr1FR}LLPLIs3Ac4Eofhu$)-BmhP&W$?}S;P?Z}jfdId9l_S1V7RsZHEF>&=*AzQ$r3#DOH2rOFKgDB_z!9sjRPbS^Grr>*;dIOt zFROBYC2EW-3)_&e9Y}C^JhvDjXAhM$M%1}p2ExHhVP-fAq3XvcPQmYP75&fu!f!+o zwo3#uMypy-?&3@B0&dhF&H$Amx;M`cjWfrG?X%96st;!Q(*uQM$3+r^FvW)>LSSR* z)*0=j%@TDFp{}m}CN`!{oDcd>9;TEEFK$>SC-mR!Q>X5Vl5v|}$sD1^ZyEOcx6kL= z@D5+{fw>G6-?AR<7`OE|Q(<1$2OR1>>>(qL68HW+wydiES@e+m?MwQbGWzWTW<+j- z69$&)tlu93IC9Vv0!%=S`Go7|onIL{hXJy0;__?D%kPxC1sc-l03q{7wv;N>=D_PC zbPE;nK&7jZIa$^d+rDB^O9&bc!lhDKMaaQjIez|8ckQQ3rx0T)bf6&wndg zOA^AP<97jR0NRK2fKDQ?hP8J@)wf2{jjAj{d-EahAGI*vwyCGWGhr^5%%AH#< zKlvKLo!OzjZ>!D#vtLg3VDTZiIVM&c%b%sn?e*-sF!VxNzimWUXADF-k@2UX%&dypv zp{zJve7P0Qgxe)x`wy=uScnQU=oWUumL7&5LT65g}6MPU@KGBcL-R>bJyMk z--Yd+k0+IYJE%3_`{k!k+&__71`4oIR>Gm+dW#YO^WbL+LcjNCo1%6&5#|O$WA%S7 zh6-qQGZ&AM=#HMGcP&VNQm(0Qh(cyHqBS6q4l|_qm;D(S@j4E71vSd3r98c4FS0duybkzZSd6FR)W|-s>^MsetTo7nW0SI}pQ69l_ou zdo)2z3sD~It32;)Cb*t8p$$%SYCBoee&@Q!YN1N+Txc`Rm9a2u>KpmF`ln(k%ve`d zaA4t-NTSpHEaC&MQxij8IndOeuJ|C;}diUvNW*XuekRf9xN*z zufDgN)vb`Df-?Q&NRoJlzcWRS3(}4{ys(fv(A%Saz9VDQ^=xIvCHF{(jZ0yQi@=>mkdP_FV0B()zhvK5D=D%Q9YVj+FLh@+AJ!54Wu<6D7RPLq-ZAO& z`x)^KbTMtajh>VIAjto|qeK-+Y(yiY-g6Ewy4mmRBoWKrK`r2uB_uns9R0duoDP=i zuKuvKE^xD%MjTwq<0lJo;?V{N@6Zos+qXY3Mw4Kd>%8PKn#wW#7>N_L%aCm^_87BV z_0aDvU4iOYx%c-Af4FZaY7#vHBF54zerSXp1#7lRni3Ur^N#5%BE!+yuO_iYuZD}m zV6CUb)mS?c6Rdvq!$4^A({myG=*B=i9R!kaBQ_r>w+KDz{hvgCR4)REvBHUD*g`6q z45MOFJPh`m7?ld7=#{30raMY6FK-Y>*KUy8HTR)mS1=a&O_>Y884=T96E<&oJ?5S6 zDO7*tiyYB+8SP*FV7zK<0$yZ$?48Wqs)*tHLVw3{*VdvwMkMdP_Xb_1bneyF@>cAi zIULbZj1zSR+U8e%>lbsjS$+o&YBo~2a~k=`AdD-^+aRhKlk#MSYX$9V{mNnB?ra)Y zaO`=CBzXvGDl&52B9s-?JJhfol!Oohase(rb-KSJ@W@s%@$<_jLEpW2RS4=qAR zlzLn5VV_G<_394tYo#C{}b@43ff=fR#VWeh2 z<6_7n4S}&37_I$UMfFiY%Pa=@kP|n6&mHV*2`n`i;INn_h? zY&1sWG`7vg#)gfp#6qfc|f=Q=GSzREhUP!Esel8sn{*eR<>G1K>FURr8#ZCPwI917Z_ITF5UQK~a7 zq&Ct>M&i;k;2mY`DDFM}-3)u0x?qV}&+*a?_YhFEPvOv!SYZ}#O;1RlA5kjFQ%h6Lfi;}d78eqBE=o1CaFuA4|2)p8N{Je~bpZA5hr9yufyxhKX z`8jW;={_aGp8xeYUrr0hBh|lLGBfktZ~uYNUa%@zQpiBhcY0rj{XVWn4`I!&Gzn83g@vu~D#mscdHLaw4W% zVlqwbl^v(Pm*4=}zG<03Y5t*g=ctjiQ7_Ht<3O+VjRW4mrEtaLOZ-+IxP4Bvb(cQD z_m($pCq6%_QC~@sp}y*@nqE<=T1jZ>d-zzyC+WpR-J(qCv%+#!LpuhV z6iYD6liht2z{3hp(Mq6r*FG`hyrdXceNLaKJNTVg=qE%RTJ=3)Ebp2j&wjaFm9{@L z2(#;q^O6gE<3;eUZX$rldf=ec*_B|&3%!ExhXlNgl|E07HeON-neX!m&bNuBKJb>E z1Jq%w#{6C*lZAi6D~^{`$5j4R7^l@6yNTT?Dx6=Ma^S4=@fKJf+=BhiuvOC&Rbec7 zZtN9wV?S3Echg|Xf@Dc^PN<}u$40|fLc+{`on~GJoM37-i^`0#eRoG9VO6Co^gB@| zeVh0+%>=UALwyB3x(J@WHQ)3dYxa0OP=YwWy$Qa+8+*kkR-bU$Q5+MSXdCb(qK~id zv#y4Q*;hn03G-XtTB+bsq}UCkqwG|)tce_JnXTkUTJa!htMN8G;rF)brsInifj?GS z#c^ppDTbdup(S#}?wdW|d;u<{E#MSD2fO_%R&!_gl4(xz{V&Nl8oq;opo*Z!$G|QATNmn3D zwp=0UQxaUrzIP^?RtnaK<-{44rO<)5VEW)1MJhDE7Q**UBe&8A;ElzC$&Aj(6D~+h z=68uDgtK7pk%Urf<${1otB&7%u3o6p3|3bF1zZjEy8BB)@kG=+ zVKTk``4Kg|VB@`>;)pPhxaPpk_Iu_z|QjD+1u-v(YPD&6V|A1%*QA?oH@;U8dCb&{o=hH z9*-Q|CjvOZ@M)z^3q+sc%ql5~JB}P?f^!kngU3y1gvew@$fsyxA7kpxC_w2u6Rmsv z=bN2jZw#PJ%hEHi@nct24?@7_Gwi5PwYoe|=t5Vl2SCPzR%em3NID!xGnfkb4Gp(? z;owq+oVB@IU43un_YH^Om4@z~M#Ow~A}g zp`Q=j@5=zzJJK%Uh^_4P?fTYSs_ae<>DmJ3z~ZuZcKN{k}J{@JRFQ z4TtJ6GkA9qFDgh2B2eZM5%KazJ=)h}`iY5A(kS!$Ep?T{4OYKL*Dhp_;Sm}%JnWz+ z_T?A7N@n#sPM9s|B=ptX*$U=VR&F(DgX#wyA?E%(>l+Ezx)GQmxt1)tz4pQf@wroOT1{#z_Myf|i+=H7!eJL5&-cy^C z#R8fav07V_S#?J|1i9lX_gdxh--RXZy`pj`ABm^4ieChpgdchiuIM` zMpU76SU*H>5)K(qjQtX*o^DAzQVldmT_Ds;Ob$^Zs+U?b!{bw#N@nrvRHrh=oEoMG z3-~;6^Zw1Yv1n}<{fae*$#w|wxk%vl^qzu!Cf4j#psz~CFj?1o)YorlPyo>a!rSZS z1@vY?d`d_%1!}K*fnhWw?g9+)^EN~{V#Fe}qD_!sblGx$Y+RW}UHcqizIW7s^fJtu z50wKPHU^iSqiRBH?U$mZ4rd2&4=X=(&ErBHY-fgy&K;RkO(I9E=JuL}ecvbhY@NSt zxjvS~uH9c_msL9qB%NjQx3@6LD zNPpdf-09B3x}r>Q2)L1LVBT~3drDfv^;vkbdT8JqAlAc#wtezl&RT^UbWW zyT;28eA?^AsUi_V}E|6`!=m~utbh)jVDJ<@uTV*&GSoft!2b9l>KCoA$jJ$dl(Nn0j(^L@>Tb*Kk zM6!DGNFMLMm}YYL_~6$?-^^CcbRDs+L1KsTe(qlX`w+!hqus@_sNZYCgb^>ylfA~n zsn@2fU8$p9gSfmmm!l$To8t0ssr`5ntguaiI+wkFIA0*@-Z*mMDp`eE{aGM49DVx% zKALDWYZQoyG=6^l8-&-~j*xKuT!(jbijsefm$|vc24g>XO!Psc$>OA*9c~)^)h!aq z;&o8IAT-C2y8@;L!VWG(8yKx-iGO_u3GjT`{5dZp=gwq}`-}kXZ|_iOzV$;WPdKYt zWsOILfV}^c*;n@HiIgK{s*rET#>>bOcS=oYJJsI`@0)QZnYvX@rfqPLU=J7R%;RVx zPwKlRFk(>P&&sZ&^?^HK{bCi5WzpWTP@TfG2l@_2 z9;G#m#VH=<{`zTDk0i;6B5-rhZC#g@F7{XVPH1gAE{0(0vVE?J>b6G~oJ zdQUNpo?GZBwxJTz`YQHe!1gIKV#h6$g0$3S9Oz5yXWNe=p;QzL9@fWfy@~7AyW!GC zHg$sQZ2FlK0WW&jY{(yvDp%l}sb_?FLrN3RI{)zBc1s4!OHNk(%<5acgXVO%4?-Td z4*!>SIA%WS=*JFn6hJOJ-TxtC@r^Hz2}8T{?`l!_HZmCYgg>$FP0{u0G+*pZFyejZ zta7+^LGgcUz-CWh;!CFHWa~`XIHJx%TDYe-Yv|o_D^gyM(_gQfo(jPEM&z zGVJioDkIS+c1oYkj?uEbyNr?LReHQPh8N?N@9vX^4_$AO|13(@Npb`N=$VI$%R-}3 z_RDO|R1Q)-9EG2qSpO^*HjNYsl;Ab$p}Qz)r zWi3ETH|1~=DR061P4MtoJtR4xdgrKC6-D=!{Lzvqj6#IEVl|x+yR++2U)O=p&hB!c z&cnAsf;SFKcg^f$e7tw62x6RntutM`sKeetHqBXBdk&$!k`KAfW zPbrfaYyt;Ud3O~o!Kjm;W^Kj_G(@kIE7y`ZD2-&d;w#G{rAH$n_ogGSt6cDv<_~q3 z5Wf^%f&YV|VT5akxB9%-lWQ%iph|neuoJ~0MYGpYf_&rG;Zw>S|3Gu{YNsE}Ub0fxiet#ZnZhJQruRWzlsp-kDW) z*bfFy2TxD?{d;aMguo}{|9C4|`0&nHZ07aOT4L8I%Oa8B>#c54jYU%YMsC=YfUG}-Yq476 z8Dz@J^8_3r500g(0nTZ{?>ozEBJCzC&?bwd+sYbHnG*^$2mW)6WNf7KJgmjD=upEY*$H6Zd<4)so1z{ zV@&ehkkZb|o=0M(W{AB5W=s={8ZTCpzEx)sAP3_ATbD1hwH*n*KG$JiX=EoT023m_bRq!6s20Q!!%s>DmMNOc6! zCC06}m-WTxz*WgdX70;=EtY|L#=vb!J{9jW0YEDYt5BdsnYA)6_}6 z^3A-yL1Xp8=ZjOfW^V}amRp-<8Hx<<-f**G-^06Z6QQn#I8E?}SeUjBo)AAGdcmZv zB!+^hyl%^$lbD)*OM)e*^5d^r6b1E)xf63zQO1wwY4b4R@J+vL1?&-l zYav*HdJMm-qUKu^(apOcIipUTIKK0q33vgYmsaF{8nn=tC$sKlr!Isl_jae!f6&fJ zB^p$XF;T|;(BzuexjoB_VZW#FmhP6`H>D+cqk~- zu6iN(D<>`DK;_To3ITDXWS%E(HcNks|q{@|@4fa*RqSR@$U zYm- zO8pf&#jbYz*Xp}b{8(-u+`%MwP{p||6$gQlG&UDsUKsX; z_?tir?Egwuh4yAN{ZraBvWDv-yIgt%{+TR2FX6QGsCm|Ss#rOC3yFV)sf}WB=Jm;B z6OLyO4dHk|3!Ua0ErYND`1yKEZVuz)>A5(%%lY;7NKkMuBLD*$4^>x%q$#X~SgPW;rRVej>M-tl4c)y!v z=hsQ(eq6T2wjweWT=rWvRj$vW4$w8TaVTKIS3lYs%Rl&h?fktvuT ze8LP3e2@B41nc{;8o8S=NT8aD6#Ef@tQ?FXIKgJlxmyoD2j0$)>;5S&=Gw@XA1wuA z`$Tg&N?G-DT#dGhSzXyJk2NRIS4|anDw7_`h(&qw<$_eeK=vgN>?nJeE)r3Z@Q$j6 zciS~z_v=k5Re$E`3X}&J6i(aUT`8vE5e_$w0o{Bg@10*HEaSGXg8U2pp!7e$XXgW8 zeUO#R5SOIR`9%X}XW`U&KEBx*0`j8q^Z_;<>Wc9Ha zm&Mg|o633=fd;omsp)g7^0O#m`!n$TRo@qTA-9iczGda)91p91X{L8%0&3|iJ+r)g z%RW;TQ@S1Be8Iwy8tk9w)JlEHGU}A)TRnc6#CKkKZG7|`S2awkj_BYLKW^nK=lQg4sXnOze)i{CB`%|Wa0a$&% zb?wE_E^s|OEB@Bjqfq;zaf!?9QhgTYQbtf!Qn#$2|E#2VCzrPKnMOtaph)720WnD84IeGn{F9nTzU1xGRja3r%mwA1~v zrey8OO5|5Mlo8*Q?R%K$o1?d@Mu<-e7t8JoR>b1`K=d4n>%Sn!&wn+}w^m4y1 zM?o4cyc2peUQByE%4d!1Q_kG%6ZQloY7XhEC-#-RU7`7V7A%7Fkf}-M-`=MAO|!Ez zE8-cebbWbeU1lQu85?I08vB^Oz|22CITCOL-`r-9bK^be7Da2nP{}3;eGoi;NrPrw z;N}(@lhi$< z_N~o}6q+nvd3g0_P-eH!L}nEC$T@IcLfqvxvp=)5ehIwAx3|T(d;QsX*nWECp1!Kp z_@zr#$lBKvt;{ul9WZUiB!D?_zxY`WQ?-X(Ifru3vb5!I=$cs9%@Q12?q)uo5Q^}7 zAj;4Yps2ZXduIp~)~B)l8Y4B?^6FY&W_#At!%}6gW*VcvAq?&?l(clocFr(ta3xr> zcfim4X;~j`jdstGDpZ!*0IVH@q-H1Qj^Q(9b&4D`D9!O*xP_K1UA`dJ zsW#ol@`{sv$BgFiCzJG!ZX4o;)P=3q5+1Iom4k{N`_*Zd!(4gM8C${6nV;|CRTQo=7jVX!o9zb7B)l&~krkkDtI{l)Qa)2SqA z!uzV2utL~Ax$m6lJz&toFl2iU3^}Dv+f#e&uH^SrE0JaMNR}Vto@~reiEcy3kOr@ZkxnVYwVmyi zsFBBow98w)G$DnzbZg$gO#R}8ABL?zMLYkvZMt%D<=9RhAld{8(<_8M_BD0`d7Yig ziYEp5b%$Q6{0wACOuhn{=+}x`qk!cFeu2YmG7!ittnFxGa>w~)&A0JNdQoP!gc~%# z>Pb@8)9TnQy2JfwSo03|=lamKFXm(18eO-q&9)}q_rS~P^Ie0|G--;}py`IkEuD_o zCAj9U$9WkU0V%6jK=NYq2%B|iNe{JE24xoG*Tc{TdbWBED7{2=%(UDLlhV7uA!E zLlvnFEq3f3b1Qq!X~&rJGbdRSA!|%Iwife$QHEWWwDk;iLmB4W(ilYd)~XxcCqo-D zQwdPy|8MHE*)XX0|J)=%6*Jew0U|6C>qKNnzZh9vT^S`TxC)kk1!LrVYzCptBY)ohKX4`X7GtE{@#0vWUaB39t}o9T>3b1c4m~9)`gAp@aY3})7^b)lfb-~^ql~dX;l?C{a335n1+4g4a2RjVbp9efP zcq&B9ZBAwr1%*ZSm7f!T{CH(k;o~z3XEvDn!K73<^&_R6gvBD0Hme8tR~k-Zzju(~}EIp64FtRS;@ z_cPs`WHb!{Q*NmW;zW`u&|Uw_+2Oq+KvzrbS&>{5`Nt9TfeA@d47jlxRBtns597R% zer!$941=nJ%dUk9w681fTe_b_pvOv#v`aP#7?qohM3&ws2Acw(*7AoOL8n4ak-q_h z-|@Wy2BxA^WcE3mv!01kuJn4(6|iSXiG?v8eo7DnQI2{MrLpF9W(<0HIBV8U^O#vwdwA6@AeR5id1^Y*7r*k1PHTMBf}Eh+dOjMvF= zuvXFcv}W{*i1gT=HCruj;jLLAfV4RlCS4S?NPv5U7qxoYIlgl2w}Zig=`R?1oBM)0 zt^T)NYJZ(_b^n(=e}8?aoKGa0wO|NX+yU9{yJ0e^YTCtvWls!n)2Tjk$v&ahV@vlO zY0H08PH^ZSIPGZdgetmkz)ewc7Eqk}_s#G4rSH=}N9{*u8&*VHNB53U>=iH327&Jv zdhqRuMdWPcJO@Z36tUZigF@V{XWW&`e(Sk<8ZCl9bvjJm;dH^2{ueuJXE3`vdle`Q zc3Fh&9K#@>m(+6Zm&CfeQ9wWv?xaeFfBM5H5}5E?nF9MpJ1s{#nmBbk7Dr9xKi=2E zUg2==%)lMVO)@vFP~bmq^Y@OqYn337{c5XGJ=ha>!H9fyG4B^o8yu73*oh7`sILPf zJ`|vVJyp@NlLC%KXA!#`w=tykNt7>s-*MNbKX`V+z=|SN6y;0Lh=B!{(MjCalV=WC z41@-T5rWqm4hV2%^G^f#r{08b^b}C)iggFx@(eT5Z*fqe@C{PIbNutlR!wA{U`rMz z7KeKmDL`8K`-(cmt!eH?N@TEY#FITVQu0255^6%*SO4^!|7E<<#r$`M2k3FT4^1jb zx)8uYH&KT$|I_vRn-{~T5Y9gnXD-#JkKKRwCLAN)v}LAQGY;?rQdlElriU^H9o zJ?{fJ-6=HspJ4_sY#QD%lYb+bDuk@NG9_ufzfR%7c@b-6RzAU~-5r`i1YiTgk~Hqh z-oce_mi=kfMsO+JLIN8?vo;0+aq%BIn{ZXufa^(eQjSLYMve{qblDcfLS%q{yP&^=GM4?668S6COZ*q$J;1WsZ5s0n>aS1?iMEBMQ1 z1Ypc?R2-bc7+{}~WA!%&z)&=6>lBT4xA?x+gTZDr$wzj~0DHfY%Af?;A?n|CaD8~6 z0ADT$H%2@5Q@`t*kmxUhpZUT9x8VTaj2yrS(zafk-)C_I9Px_ui>E%mX(2N449c)| zm`9PojHDKuQRv3X6Q&k!F+o#55YubpQcBK#tzoYfcqWtoo5}kU;M@|2F|GBOT>uVm*>p^Fh44yMQM=r&Vta~FbuC#9?pMc&c3u(cOa98>%NTZQv? zVz_rLe6@{yogYRLTxU*Svn%gSVD`GQnRd1$Dj@QrgloHnBC-x#d8GR*nBpSg3jZ* zS}1H3{Sg#>R59V8vnViLnnuzJAITf@4^y3TbIRap*?p66B1*vj=MI@SJA)&0j$bul zzsWT4rS-GiBX8CdWdk_-PSBGIqiO}*TCThtp|Bn$P+~SlkD_NzL=?UBNUEMAc{Jh2 zDB1LkH2-968v@%O%-};=F}uJs#7l5^G>Byoak63yX(5PwXB4vF3*zX?92HoPh~D2C z!|OMK7@@8fjZI@`%4fEQ&}u%yQ#$whS&l zS8r;JTsJmPTN)S3?7#?7Q5spf_>xVV@p^T9(QwG!=IcHS2IU~?x}6yFQz&kPJ3-RZ<1S* z`A$gpm&qm@ffC3&9KJucT*DX{uejftWXEZY3`iDvJX&i^;u1sS$Y|(Ky_B-pqVWD* zm!n`rePL|4Ke}IjTk8qFijYst{0<9n!yKik5})2U(EHGzd zobWL~zfj&U|7RH#6BIXWR6=HB^Y?biZ{R7zBF=Mjahh9=pla|a}? zJ0^E@RN_R9ZN_O*AtjUG_rclxR{z;+#^$IiQV{=6qjPB{BLsmwh@TIBKYlSJTJE7H zXdz5N=#$LAB`9*lrE`xZy`42aB_cZXo7dXD_G3eJ00Th{r70TPTsl?~Ch5T;-lpq) z|CDoUrq}0KI^vs3f#fm^`@_K!eZ5S>Li<_-!@g@0WP^2Vk2%o2NIxm2h4aj7)+dkWLi9*nTYw`^ECK?HA zxu#irVf8MqJIWK=nCa}@u9P@MVzRM(A?sC;l16df&JA3H_LOk7li6(I;(7V%J0ZV| zHBTbQByQoOA|AZKRk;yFVi>}Fc7i6N?SEW2r3pADaTS#J@6;fept8X85YprA_sfta z5kvm8*QE+X@a z$+60fWMXp#9|UO@iVplOH6i0stp8kvsoR+q5!s>~uah}IVxomAAm=qu9SLa@?IoUe0@W88ZZUd(rUK+9S|afi+yI|79p(tR}=zJn`gB~R|6PYG-*1vPnr zF+m#1=6xMpUY%3+*5xYh)FM)QwX)mc){?i?@86V2Ywk^s+$EV_d%d3ihUy_2?@^9q zt_O6!wCrr&61}e2!auv6ef-vzh6NNUk;XQD0}Qo1rJQcEv+|F!tpfNvVcwzikQg#L zEDNIn_HBG5qp5g?dof8Qww9aFXut?hhSywkJV2%9FpoOR-mpE5k7_^tSi6gJa9|tz z*9*S8p$`jC~J?g395#!X}+|US+*30R4yTOSzL1T-{$XR@;oR9P5L5+&8u60G< z_0Jv(cUimF`EWv&A9l9pQF_~*W3?x0Z?25xC5%4W^j_`Necbwcf3RHry)ffSK?Twk zfXBT;n&ih*Qaus%a#Fnx?q3rc?{%tyA*b)oT+xkaD&0h(jXFnwgYhbVM}Amvthgag z+&D5ps|$y^&|F%jPKpJ%uL^E#{*r0=?9l9TAk=?a&M*<+u>zGg3idpTBf;0BR6M09 zzFsr=WnwKDZX9SCDr7!xuN5%Zi%99|+0e!P_ES_)j+_ET30}h$RlJE zHB?DeAc8-kZPx#_>)-@P{rc85dT57CX1jRd#lvqnQgYTq)zWKY^P@HBa-baDv%zz_`DB1ZOL0e*t)jRH+f#0jXoK}JrqG4|Kfng5AA7y&4(-d z?mS9u986D6@=ljHk;79f%8ZtVo6v#+st3Jilj5%uU!*o~o2m#nv8)Rehm3l{cpt|b4@Zx%k6~X>fz#uS(`5~=2;5Ycv zBhb*6AtF^V1YwHfG);DI&X&_)SAG7WHu3=dFClbwv|s1y)iB93Uf!m?wVEV3uwUs4 zyY;DM(|S~qCcuD}O%Tr_tSy)I(ALk)e@;Pq7o$6mACnd)RjdppLjbB|l!qN(1r9M1 z1=Qd8QhnsFGM^}X)G8y@nyFC8ILOv zQ@!fB$9zr5GIP%DF;}HHcF9k}`mz`mtJ$ZgsV(EXm7z(Bemmnt^90WXr2C_EWbQ*d zuV?C=$bb0^Y_S+^7MUy^+Fhz7SX|fj5+hve?Ws^zZCV8VacwEoCvYWBe{b~Sc^uOnHD(IY%DJA{R~ z?LMxwpIeu+b0ij|J4h4orkg}RT*CAd^8VVLV&He-itB#d&;A7ZJErW|5m`w8cj0fw zlNKt_>bHB{+oDi0cmkgGF|qixxicXbomva$4TjR za1Tz8{#*Em6tI>c22I|CrFXbXNhqC;j$TpOHz)S-uO|5C`zK1IS3|}dahe)RoZmr3 zYd9bmO6DwsyFaQ+Vkj6eT7RbDkj;HO;!c{D^sO(=Tdc$>p*nRgqx)^vJH4V~Cq{~O z9;@lgPs+Qhhew=^le`Wv_#SLF#anaOo>qysP^gU`t2d@L5XlQxl#pVt7PjOPIBMj5 zTyw2wK{F!k&g`wo4nu}J>wc{cmkO|CjKkI!-&5@2$^x51I74{e;}srBHV0-AUI2>B z`(A!htF8f(HAej$gl|>rr?2sY^)J|$hf1T5&HJ0~dS}KIq!*8S z&Fxd#IYyXV%uWr)07TjFb1}=+AcPk-%bp6?rrJ`aiAG}y>yviXyFK@Ig5@Uo&xx#N ze&}=%QJo5)?liT7M+laTZWfP=Ec#ELo@A8!Tbx46)xoxu6@zc5M_6pxl6-@}x+mh{ zil<#O@q#ZVk<41`Zm2sztusATyA`tA}B7!-}SDFoPfS9e7aYDQ&Zf<> zZ;FblgDAlxnPmt&-bl_LNv18}5$GhyP_?>Z82n+{!L!kg2ZZBQ+Jpy5# zF7@{|vQZ2`!n1lQMqBm_rAtK#!Q5BNYKRYT&J+R|<;%+fX&J7=FIPyGm*%K*hqdRv z;N?4Z%#ayJbvido=xYxJr2Tle)xUWF4Gn0v(rbc@?s;ean1KI+z@|iiLyxZa>gTT( zIfdu^B2o=>3~d7)1Lphsg9n`)f)>KWA!@Qs-!6j%vEQz#m%{mfT3G(ga^Lt`1%m)| zy)+yz@l}Vk7SPxA#A&(Q7-YmgHo5iU^60%#D=u2@)uikn@b&O)ag>wn!96l0;abX6MD4x9tM|n7?C(UulfC9V#)AQSN(YY55Ep zv2V7y)?#Y0|GgBeTreBM{Ha1+r$NV7Pg~kee{tHi_I?fiBYLfNbM3j!=GIB|>v_r5 zz^Tdz>d%^6-R39VBnzYRJZ4VMse!kt>|A-n!Xm>B9If6Mmc-ukVC$8X#ZwcBkaE@3Dru^IEWhRp3#V0;jorT zykC6tG^P`qy~asToO1hA5D{_IJatro{$!K<@`bS}m{A%4?Tz$?D)4OI+G9cpC71)L z?(!?z#9c16{wcBVsFDDpo#PfIT8cgerHW(Cn#U&9S-R$DuGWV$NEx!*U@i~b7 ztr_w6K;89tsMqDK6uTmz)y=tMbEYn}o~Zb*Sg6PSgm`1S$r0$U;1vODiFDjTPZl(v zoSNU_19L1jI2jCYiNV831Gu(mu0@AIrFO8&t?$2@$Hfzm6OJ!#XSI`K?5+U<|N zd8kx~=zL(krEloZY!o>D_x0FC+^n zJmDXYiVX(GB3sCk1cT}t2ts(mO}NyhA{{B{30hW2mVl0|dz=h%(ME+60ba3=Pm?^E z-ElKHE#@VcxHoppxJ|$9Q$u5=9l2de9$uS?bl%$^%UTA?eGb>VVyuI;TJ__`Al=U& zkc>P*t%=M7?Su7*eAu-F>(*a#8hW%?{88svu$Rs8%50pzUzB$sPAqralIFnt(aMdT zF;h96G8OF6(bi;K9;yFg4Gpr|^i@*x@N0MZ0;BR^Y$HUn<24o?@c3^~%QWbp(!^l_ zvvZuj@FjmtxtSvR+fa3Mm+L>h1@#CN13p`u#N~F@e&A-dKF~fBjI%WsYNc}h#gUoA zUzLG-5c3YdK@0zO=`Q@YZ0NdaNmP6XZ<)_;`s!Y@5$=YpYf!Inq<_%0uJz@3L-VR^ zzVu>DZ?g^g!B?k(`S6dT*O`{acH2*+=1^|Yj`*1>E3b^83)}MM!~(v!adp?!@E2<8 z?=AcYB)rezBO84Z9{y|AA;h~XLh4ZQoj;TEaNi*Q#naG zlB2$$h|)dm1@I*NqSX0a!FbkSjM4n%u0QxbCwB`%i(szVIQLe4Qh_@N3i**`WJHN0^W5194;l4IIA0S^^B-;8v5Qo9$!8{ zu@36-^K*ZoeLA|xUb%CA+4W!8g)H*3hoJ(E>+0?`U3EDZit_6N+>K`Z90=SmkMeTA zl0elPlF*YolT&;So(r(r@X$K6u@Eo;FE}$G_(D#Co8Elxt3vCq=RKAsOA0|gwKq;H zri-%>S8zFaelG971eyP3&3tqyAS@X<;|)w;QKfr)$oW-J6RV+r!ISl-BuUkp=5Y^l z*>dPFd&zCxRCdiz)*F}r)ENm)rcXX`{=xHHYA9@QzsV1cyn}FQ(i_Opvtz!W=PNS& zkPXQ>EVSoG^7`3$KVfXs*^%sX(3OVZ0 z@i!<0A6rM?HRzxb(|qAz3p5%XcL~$4uOX?stIUqLLVVD52*rCjSHuGhURe??MW9J! zanuDI-jmo64x6m2Uq|@Ba1@ZZ78wfutJ1naK=5v`4!o?qg)wYqzuc|7cv38o2S1;MkYs zE4EhYu17!t`I^rLA-(upOSamyYh%sO>m6S-h2xeC{JIte?Ke97n8u=N>a>G-81U$)2^yrNINe5S1?c3KJi1hFcE#E+ z*$=nSIQE24dY|k_Y4NH~{<6M%h(f)MEqY~*-L`k&UQhD7jwWJ5^_eU&dGPDoqw|N9MM8D3;f}F%(*35BNSC5q?m@Gz6Kpa?pLJ*~Y=W-3DEVMQ0V^78!q2eoLn(pKG z1ct%f?SB^JWKP`ZR!W_spU08DQhhl$%3);iiiHBnQZ})}(0^zMYkB^Qze?fY+Ahu$ zz>r=DyUQve;2Fks_YDLyx&TUeRT!zIttZWvT03Q|M0D$mJ21{I0X<`BDb9TM;Z@Ji ztxf$X%Bu?QD+qyf42^e`$zI3}?fqtbozr?ECAA(ws%5`m21n))ht#ojeDQ8xHmyVyA}xU?#11ucyV`%TX8S$R;0KWcXtUA++7L*3KS^ry2JB6`~ANCW9LVX zV{*(S_sra~u5(>$(%Q4JiG?#bNVjF-wU9Tr0V@owvf9my;8Mdhcj?QbpOkxCkRXjG zFz_B@!~scGUR1ye>G<@o2Yv1yLOyM~mh@n>wM(Kg(en$xrna)X$*w;ca`$WiVUScF zeKDTs*sF7ZTsu{~)Ii(Cks#%3dOxb)c!=f2zQUHd8rPpwe%^MR#eaPpDhXQHZ$ssJ zcB%Y@J>S8K8W|>%vvGiZ8cv&VUU#Y1R%1`pU%L5tQFA$SiNrDPQ^Y43=ZHdH_so2b z#VV)3XSDEN`H-=aV~G4xyLb6jlVPbU%sRbBcF?Dw)dl3l!vT}NUeKw4&6u@5&@W&%*?g+o3 zZtB>>f`r6He<3)PyeN`EgUJF@(_0QJ!#QACsd5&IX+L+CChkwv2$6a*Eq_wRIbd#=D-%~43gIWWf*GYIf{ZhT~ zS%tz}&JlTC&xi_^PIdxkmYI)2{!n4nRE6nrT)~*(ecjwz&Ye7~F(b2=GmyqWYi$h@vq$l@_B6mkR^ zb}kPt`IezmXYQp zE9qH>%Bo&N9*_NGc{9*9SP9osRdw{fCeS{;9F~39VvrT0)-1I*|5k0-E)6(F8=!%r zgnEe))#LgJ&zRWgBEn*=)0gTN>R-%j$sT9N8^S5%1LGAhxk*mR?lDqAH>}iW88=sl zIl?Xi@{5Fdgj>2R-PiqaIzbJ;=z4ZTKQ}l+OstoY|M1 zr0JtYy`gj06s3BP2P5P#3$<}pKwF#!C{U9tc+roaIPU+>H|5{Sn0}sXF0qZnJlU2I zHagZCHKGHGh$LwMt7tvlkA7q$=+$DKRQP`$=$vn&b|NtTsa-5lxMmuJS-1i{lGe7H zR=|wISAubNQru2BWA5#A!U~5MjC&gX_1DVp>`mZ?BRkX+fDhzRnT87`a z8nh?|0vOya#^!G4jm<5>4NsIeAAU%YtRs8+(4_jcVmjyMSe?*diW`bKuEh`BGty7UM7IEx0Z z7`SN=G&iWYl0!n{f6e3U!yavN=0MfGQVZP;Er>}mbvFlp7-h}p8HeXsi7NGoxM_Sj z8LW+m87byqdaXlx`jWu;lQ{ry?R)!;3jif8dc}nz-+~9Z?t*v})QwF*C35epDzkPN z*I(9WCS42A|B%x#@-6hvrk{lg2`Q*6mp5!P<{*AXt;_kbY}F*R)oz>r*G-zfngTkm z5gsY1FRhComwnB{_%I|M69_sWj59%-$c>eObUJFfsc$vag!43ln_Hg-&CWlr7Dg2t z`N^%-s-Qp>H-01ALgp8q3g)z69jdW_oh&X0KrZEcbscC(d&1&|iD3^a?E2RF^G?Qh z2Q)$*lCT&ASsgKy43vo)6}6NVe{VJomoHu*HEf;^ePm_NYhX8}7EdLGTL&3WQnBkP z%Nyv=QXbn5$i#tL3V+l{s2ph436wobX0_e0SB4Hc3`tW8#%}xx3TE6)icK#Gs{==q z#kr?Ov}Lkzd;2``@y7NyL30fDYHkaMj2lK;nUKDiV1xU9OqX^LMn>8$v^%pSoFU7o zgz%zeS!z?g{IT5FZT^t`4EMrh#>|Xav3LSa6h>*9_@lKKT1QITjTy|D{71VAA+$Y= zNxr|)J9B}{w$m_dk~fjzI2@-zRv@N!MIN(Tipa%ja)_?-?3hCD^R<8Sh>^&FuBmD* z6SSI@kNe0e0QV+kyU7Tr9U%O|SjQfRGYp8!w%lhZN5O`u@}omuBP&E*$=lIXZ{Mgo z=Mz;EhhQ)NxQUIl{o~KE;N^Zuh zs9wpt@cbM>`s%PX8H`kV?t07Xn)Db4>g`jc~DJ72py2se^(JaYO@lb-GnUmt^Q1)l7{lBTN+@@2E zlsz8^2o9&GXToEvoU?0QqZ|Jv^>Smny4}#^A%Y^^q)IRh(dQd`tc3c_;tam58+1~% zw#WdDT}m(rxm5?rwdpjLCp#u|>KrsI_enxdTKGhcYwP1YvEPJ-uZU*y)_RSlGk6i( zs`(TKe*eUYZ7Jz*A%LfTvUl^Z3l*=zK&1U?pXyyV`)mq)&8XA;ICU1@O&^@;G}lp> zf1J>?hAKxZHIoMg4j%I8jG<}BuV)`{UykhZ<7t0=kwn^W_3LbgiH<;y5{{7l3&}fE zhwvOFG^b{+1V2`d0y0)ok?PoQQy1pr%dLog!3rieEQvzDmTx!>y}+ncG(|hb$rfSO zy_5E2>vq(!{K9q{!7@{nExU+~oC3WwRiqDVa>>msg1N_ft;l{?jk6Mq=^V17@521I zkknT2oSog_;%GOwra=GFmf47RU(VBs+0L{P6&|4CHPJPQdh|^K(9It^(!a+UaX7rs z81n4jKwJbmGq85XCiNKN#J&1ixi|Nhk~dGM2rZ54gY7* zEzLjNz^q&}X-gN+9rvHnbjbvf&?yCJ8dmu$l5QxcKb6J#4zVkIed)9TAPhcS*4d88>g&BCJ}@WPQ`$4a z@jDJag-B>mE2HChS+bWsx~2^N+>?35L*B*To@E**JdCEcMinZQqpEs=I>`^W71ps@N8o`*I>!*@Y?UhH&*U+-%r= z{~if=c*T$|Mo5H8PPkS)m}uvOq7y$4drnCF`l4*5 zzl`|kv3!Xie6_9O3_wp;tN79%yh6tq$ODz$!8+osSEuF{;3O2vF7p~300M8MV<3$MZN7fo# zLV5M2kAwc)S$gYagJd%5u+)ViDy%R zhwvEsAWs)fNs4nTs=c!xJox_AmkCAc!4)@s<9%X=X?DEc9t&kR2^3lBsfbh79CaKI zj%OjQ8*C&4*oGxImzX&8galMy*t?bKgNshr>ho1@z@pxfv$&2sBABM4PAi~ES=|On z-!2)S5yNoV@vXnY5WoR%ekyX)osdh}hH$#U6%mhAdDA^TTe^RbfAbBU)O43uX-|Ez zfegaZS&|bWi#%)MpM>~CtPR9~*RRy3*yv`F$58G2MWRglca{Wdagf{z@<$@tueb#w z@{NR>q`X4OmJtF#t|0MInwn z=i`i^E!ixD0C^&AlRU}`B7wf$>=apTjPR%Q{4}C@l5?hB4sVPXf-QPN!+e;~dWG^k ziTtw!>ig@zmcgCOIohEg1?%*K;N;N^Yoo0K18PxbighfKLykK>M`%Y0c;pukO~ zaz;>z?u_fWwpQTrNRjq9fY4$7o=~VY_BJH&3MF;G!ZhR-=5E!NyIn}}vp*}T$9)z3 z9mN52j`ik}Kkh77w280JI1Tic^p{~Q}~9bDmrw|c&|0@XW05P;^= zCk;Mj7SSJMqCMwgP!^#iG8#fQI}Vg^8DyG?=ur;-fxQ;T{4nGo5jcQ16G`97Kn&lp zz7usT^|X+3#%5xWn0@ZTN{R9nwI&C%^gD^ScLaXu(upwKxEq0fFow7;^yu(gQd15U zEeE@H-yBQsG7j9$mV!%bGIbbvV7sge52Ojfm|caS3^KTL^G0)xS=#${jacjQD(10) zc3Nbslr6ggfRk584>dzn^kSe;v&x5}uAnE;h2+M@or(Ws$ZYyH`Fi*1mNW^k!y_~!37VPZy$8b$V~ zpSl1+8{Pn_oyBKSHtyg-5lTa*+iK8PKy|Ontd`Ha{RzCwCU>ocESc zAY@975-Wk)hl7fgcVM3+TDn))0NK0#)+kMn4Oi+p#0CF80Z6bGVx~!N5YgQ(x_6#C z=msnv4f3QgcqETHo_Q$iTNE1~?MySAPQ!srks5h1f{>VwX0_ebA`LR|#`0@4maKaY zKX8)NC+>k;nOuBR;#-&s%Ztsa3|Rv+C5SV7JaCLcS@zSN-Dak-deGsumq^SOYb#8y z1wu-kUqsf}Ty7Sn&CP`jd@{uUe`uqSm&8Hw(5C*8*jTS(IE??wBCuKp2dDI5gqui6 zp(1r+YB+Uc70JhCArv3WboMjRc7(SKLR<$tE{pntQVP!B>!URQa${E{pLP^=L|^`u zHG*q<_$w0zG%yKN+nYfn91%0H*Q!E=aGLvuK=giGtE6FHykD{wH4@Zz9~ysX1etntF(Juh@nJQFAE|lzUBpzYT*|$W2vdfa-d`0F6_xqm z6IH^2YI8PG#j(#Li9o()aGaq40RM2|;Uj}(inH5wp8TdNq!&|oDj^RbAS=>^OEiW| zIVB5;i~@hpc`52ev{0#I7RM7inp&ykli*cW1Ny$_wp-mLA983i>`R`LsXmrf{504;M&WY+PD(7gTRI=J|&`D_XfXKX@Ob%bgT`e8@qE zR)IFr)1HfexVf0Y;qFWe0M=qBoV=SaJGN~%;`s}nadw0`uI@aF9FIDprr3JUuJISZ8M#mpxk1SHoYL;l0b(a zJ;=B*0hni&Fslqk$>ox>lh3V17O;AY#F|ox2!CQ-b$-v$58qiJRpJ;?@zko~a_%^n z%b2RrJoutDi{3x(u`AKgSsJ8}Unk!Ye+<|Rs56$;2wkZxCB7EKye@7kL5!MHHhB)T zYgN8`*Zsa3$cbq?8trUzBmx%pA;pSYOvKdsOpog-D-b8|r8d1F`KRszis*YR>i&8a z@ds-QuC%t{?qC^oaoKN{O{agNvebP&wg_TSR~I%sy;M|;cG_O)j)mNLF*4``UwQ31 zbH`3E&ja){`I-ST+(hCTIYL35G6kmziD@0XY9Gu?wm3k4mLtrw+6{yH>Td^wWw@X_ zsUza8_2;i6?Fid-UByx`X5FU&ULlQN;s+bP)U8Q9lbnlvKjrem*e^cbX=C_T+8>vf zXDyi~n@GsgEhFB@ot85+lxT(L=TbyiLMzvR&pseSQ@HLqJ z@KnqyC3_$jGL9DBQBiW1b1IS~>oK~oAaC$7UvPB!f3u_ukI7)wpGE9CIX^5c zT{R`6NGPrQdj=_xr^N*9!C;Z$p}sY&s8iK)hU`6>6H!rxYh3GQy!~uc0fv)`VfjI= zIg9rqIPWuMqLOllno8&$p6jtqkTrpR3#?Jv#qG^7O3;<+HDn^f?KFUXkk{Bi$yhvn z_K`Y;^@N9MigEb++8WCzeF$l*3$oQm;m-QZpv>46IQ6%f!uZBi^5=ZEaEYx6qvm7m z7^rLCH>1Simom{lD?1$dn9lSBoPW;70Q?wo?U!b!Uppc=9reG(tgRM4TK~Iimu-*9W@AEB|nPMx7YP`Pg3T# zIUYT6xIkO@h+6=Q>xgN>o*#Fg9mnn}Mow#FBBcw2TD z?be=f7?8~{sR@S#$xZp}j-!#xFbhDXGrcQ%x^NU#- zb2pyJ3Y|U_#v!zjA8^o*ccUGM7QL{mto4vKK7PvmCf&UM1v^L=EoWzt&{kr`aCvVp z8@`T(0xQ_lO$#jWtD`3^R_o}lwBNRG^X_Art%yvxN2ps1>426P2e?}lMNYc%N^;^% zttmzfTdNYqY%@O{;Q|1=d9PRJaLtdxem;cN-76Uqkk@X?O#24~JtsCdL|)EW^v1&I z*-u5UKU_A5-QTSIh3Idn$S|^|m7bY*Er9ICnJ8wZ!VI&)558FG4nY2-Zo?bKcOW+F z7*crc7uc1`PmnpN} zImAT`=3wiTd$*RJ0e&byg-ifH-M6CxKdC=SeVDJ?o;xkf@yq24wFfpY-!u;*J*H;k zK#UWKZq)x{oIol{Sy-|!-2ig4lx$+0vUbN(0wb>zlMBj0jB-va=d^Zf{wVBqmWioq zK|}OIsZ#?}9=?+vkSn?a|Nb19AwMBiOZ9oQDkcWJM^>jp=&g})|7?&_fr zJ}z|h2EYp7KGu4I)L^(E9xA6p?`@v%)j1ciAGr=jC?p>a4;M~&_AcWzq|i{Zohhyf z?;2`48~YJ5yFP&R2Bxm~2n%b0YKvVOOIS&}IA7+l5)D+}yH(@^J%%OywQTfJ#Kn2Y z`BBsA_6UiR{Wb@yH)1=#3c>jI5RP>YY)uYDq#)?{1Dr^M`H}|gZO+|ofQ(-W_s(iI z6)JUWvB0(~U$LFftctXyw3crlj7~1x+T#H%9SJnY`zkC91RFUtM8?=!`T}ffiTHS1 zcyW0M&Ye6{wX#oY+?^riJCN8LjZbDj@`wiM=pmdkhE$J`NuZX{1%B~3p#nOVKnpAg z?3VT|0kVFFg3};4wodidL?ebh$`WL`-GkUtaiY0Y{w!?lVLMz;E62oE~AFub?BFA(aV`HuYp!5SJWE1V*4FB=Uz z@hcMsnj0z`O;?~)z=bAzZ38klm0#qD(!!Kyjt!|BeNUhl$uDiy3cZ1UNQbn5uvial zz0Fr^J@#oY#ywghV35CetV0Xu5ytWTf)gH?N;o}to}Xnk#6<~?Zjv|b(`0TmzNwsG z@dT)vn(~PFL%ZNkh^H#AweXOxb-DJoa5WCJG#9t&E+^a!;oo;0K43{rMRxbw+|7pS z=BynI+S-_Rf_8^=LZddh2V-Tn^GOw2GTvdqx02)M@nEgn*RFm3&LBGtR_$C>=hTcW z=&OphZkX=J?r;6Xc|D2?+QMk3@j*Ms{Sk&fFh-AxiLn(Dka0`z?Hyh)E6_O9m2qZ} zO{Ktfve-Q2ZH=|K)VVf_bEHfH0v-#lGgo|TSMmX55=f|45thekUy--M;2a)Kr)Yff zI71sKOf(#3YDITH2enrC3Tb`tngU|GF}v>WV-BS?c`*v3^aKi^FAMYMWv0CEVQbcI z%WhxP_Wg*-QSXRo|NctAO;QV+ea%K2Ozvx-k0=y**HZmgWdN@uUz@>Bxq}#;>Jz~X ziO}^&SwgPr@QgD-u9*cts7ahS;BQ8+PPC9Pryemp7uPComg>VDGyZAp{U~_KBf0k^ z>b23Sy0yn9pO2Q6ekF#CFs$W4$LFaeKfsY)a{DM9F2sG3cnIJni9^P!a~^>_)7HNr z?R}|z{)3;f09MLuVpH?XoU&*9{eo{i`-Mw7_{@!?x~`7p={kq1^B4a8&uI)LeNNXa z6Ej==zovA0EKb*&cfP6dPv2#H@sV8+KNjMerl|fl!TnH`9H*vRhZRXU;J<>|5YDN8 z_9ms@M9S!7&5=o5h)t|X7Mgb-w(@pmj8rGTqW zF&x|#E$!>qMcBS4Ddmg51$%U5;pL$FT{R{SG=alwr}g&CiHb2xCYK`^lA;zhwn z3MV+~uZ8y|TITfyIe)ao8p>!{KUsvxwVOAlThJHH071I%soWypd&}A2ef&+i3>1JjeHY02b6w`2>s9C1(DcVl8SzuSN&@GO^(Oxtc^i;Q=<|v`UPpK1=mLB!3 zzO1zS+aOPY#0Fh#5-NR85+dC0ge{o9aBcH(mKI>)9LAsTv}~SVGRI3{A2K!4U>csj z?4dfUnwl~x$aSa~ycWutAmP^F7)soxr|B{_Btq^w&)Z$cRj=sYi=ogn0C-lynEH!u ze>VYWQj*asiUg67pt$t)RT!D19S>Ob#c5mh97Wa+m{%@m!^XcmqB$NC55G?@nH~pU=E6P zC0E;08(HNvkldEnd3(Sej_Qy)=XUsa-1erx*+Y^(4lp)2JfZFJ92|%ki%_)4 z`q#+BDTDcx9~{J5a^<~9!>u#hl6e=pzA!eV!VPgcOhC#8$?bcee(qGn22%XtK^j}v zqM%~OA6X(bO*6TEJ=EG!zGyQ$lZi4;8-AzOgCdy*;vXPU3|6NnTRamc@PXAr_I}u| zVrwx@t~m$#I0(&bj1sIBeOjh5s?-;z`GqZ2yfoKHnDh%I6XgJ;=FOyo!`wV^jEveg zU1{~Tnf`ME1T98NqeU%$n`)}Zl!}-8-C`5qdqYGTF`Iu%er`;0L#i}X4!HkhM^ms= zy_@tpWUqZ{AS7E-t#SFWsrDo;T6<^ds7$0tsI!x;SHEKM`8S`nc*pnLuN6>!v8(o9 zodRV_2yQnN!>aOUU%a$RcMxRx!u>?3z@DZkrWWC}TqkU~z^AT+?>zEl70m>8!~@RG zEX8>5l&(>h72h%+wCywoTPhq4HOzC(gG(}&+rV-Cbyapj*(B>xMx+ko>zOI=4HSj- zp*L2yp9HanrJNLRKnf1FUhq4gb_7`jYahA<8Ck;VHi?OEu@D~gtQd1;`^xjfQp@d7 zS3q}gs0&P@PFh%1_E?(vU%Eqv51tGCbQ1w#$K#0i?;JGCmD_WQH*)+c%Y_e$ym(oV zNvbZ+hZs_PK1pw1W3?_|rX`bxKB8+Z+0Lc9_OYEJ`s$wlHT&>k54?U3!*pW=szGtQ zGK%l5Gs>?~JysI-|Lo-~d{bqBmKz~ym(>j$h+WRQB97DhKYxVec|FRHK10Je7{>PEtwKQvo$y;}53|a( zh)ycsOwXk8@Qi)xhB-`0c;Bf7# zVXe@CGvTMMM?6IP<{AO7kr5ol{#8Q)#&3ag6d0f3J)GO8;B8W|Kxhl;T=SoWy{l-I!_4|4Mnz~ zxY@w}%n`(DsPGr`>sT~fQZ7DgEA)xCF^l4KBXwgON<@WNDQ+qbgv_b-SP%#KtSK&3 zmwIzWa$?k$zyft^rTrgJnyQGC4~5O&<(9A*#o6OqL_IQIOm3ex3>F{@OH}4qle#gm_GYXw zVs1mU`-e5P;KUSS@3so<7qK#ml@se#gKP-5SUb}bi@ufM&@-lGt*`^ksOnTE);`vO&cS3Z>EGup`$Vv0y{fW~b6LMr-lA()Jj^xye z52FmSG=Zh21Ix##+>$N1&DpCAGr_a3xlJP9A~AntuZ%K%g$ z<9eMlHzD#AI_<3w;@Bbi^#P6K*wERf{y903`Of4<)Fw>Nw0Rv;Ws+DSnhPW%U5wG9 zkyMa+jNr$0N06K+0@-+s>^P&I$af**o@55ansL8(J~Fob3V7-3H>qNlTD<49i%E)K z*$;vxrNe_N>n()eey+xiWiEcZs&ALp9uMMZw4N}Jbjq3QJTtaONe!u17vvQ27QjU* z!@rsd3~T4jeez>svwaWMOl7tZG#EuqMEi_dCl3-!e4XF*Qq#b8khyz(Oh}D5#=>=+Zl+4s{a^?6 zkg48f_!}RD353do$i5d?^-juv`{d$Y(>(wK%kGJEVGwZ*f0zW-ykfNYD)+SfD{Lb6J$ zDE30yYusIr`2$7B4;g|x!bMusedcmI&2?7T3>JNKk!`RKnM*yb)sojPHmV&Ovms?E zQgiLFE;az}9c*5f+|Vin%{VKr+q$>F~sF7TzhPB`kc)7VswtQhH^E>1r3~ z#E!+@X`Ia(e6*53@^!z*F~$#XO`#PDh!xob%s$|v*xB&xEi(Smb6)Ro0}Xw>$nH{PO+Kf=>`UoVSAC$cHM|SzDG7ZAkodB$m&75W736$ zIN5s+DF4oEjexhK6G}|K>11m0cu=m=LBk+{xNF-oA>i?!1~ZQzO>DgTzvC71k!(L5 zW&D4pMa?g_zew{f&%U_u1SfF+d3HYK@xX?taizB}ecPo7C!KuRt2(A?ksgFLYT1Bs8p`tYq6k#EPiFjb-rlTdUP!OoseTLA($c(xygo@ONmh#+2mK$EBoPMy diff --git a/doc/data_flow_realtime.svg b/doc/data_flow_realtime.svg deleted file mode 100644 index 873010aff3f..00000000000 --- a/doc/data_flow_realtime.svg +++ /dev/null @@ -1,128 +0,0 @@ - - - - - -DruidDataFlow - -1 - -REST query - -5 - -Realtime.working_set - -1->5 - - - query - -2 - -realtime_data_src - -2->5 - - -Realtime.Firehose - -4 - -indexed segments -blob_store (S3, HDFS) - -5->1 - - - results - -6 - -Realtime.spill_disk - -5->6 - - -Realtime.Firehose - -9 - -Realtime.local_disk - -5->9 - - -Realtime.Indexer - -10 - -metadata_store (mysql) - -5->10 - - -Realtime.MetadataUpdater - -6->9 - - -Realtime.Indexer - -7 - -HDFS_data_src - -11 - -HadoopDruidIndexer.working_set - -7->11 - - -HadoopDruidIndexer - -8 - -external_data_src - -12 - -IndexerService.working_set - -8->12 - - -IndexerService - -9->4 - - -Realtime.segmentPusher - -11->4 - - -HadoopDruidIndexer - -11->10 - - -HadoopDruidIndexer - -12->4 - - -IndexerService - -12->10 - - -IndexerService - - - diff --git a/doc/data_flow_simple.dot b/doc/data_flow_simple.dot deleted file mode 100644 index a30f8ad73de..00000000000 --- a/doc/data_flow_simple.dot +++ /dev/null @@ -1,27 +0,0 @@ -digraph DruidDataFlow { - graph [bgcolor="#fffef5", clusterrank=global, rankdir=TB]; - node [color="#0a0701", fillcolor="#fdf4c6", fontname=Helvetica, shape=box, style=filled, label="\N"]; - edge [color="#377d18"]; - - 1 [label = "REST query"]; - 10 [label = "Broker.query_cache"]; - 2 [label = "realtime_data_src"]; - 3 [label = "Compute.disk_cache"]; - 4 [label = "indexed segments\nblob_store (S3, HDFS)"]; - 5 [label = "Realtime.working_set"]; - 6 [label = "Compute.working_set"]; - -2 -> 5 [label = "Realtime.Firehose"]; -5 -> 4 [label = "Realtime.segmentPusher"]; - -4 -> 3 [label = "Compute.load"]; -3 -> 6 [label = "Compute.map"]; - -5 -> 10 [label = " results "]; -6 -> 10 [label = " results "]; -10 -> 1 [label = " results "]; - -10 -> 5 [label = " query "]; -10 -> 6 [label = " query "]; -1 -> 10 [label = " query "]; -} diff --git a/doc/data_flow_simple.png b/doc/data_flow_simple.png deleted file mode 100644 index f397953b651ad1f869976d6439dc359c2c7e921a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 64023 zcmb@tWmH^Ivn@(+cXxLU?!nzPxVyU(f)m``gF68lmqvoSySux;&Ufw|XS_S!`SZs8 z(SvS!_g+xd@y}qO{AcKAZM(HM@?WX2v;pS=VVh*O_Zf$C-Zf$Q) z%FWEh%tcD8U~EdNq(-Vu2bTZKuncrPjGK&t1k4%&DijEYTf5-BnrO@Z|fQd*%pp*9aQiOng>4TNpdj6BJ=hEuy(t5q-xHkWHVq^q5{S%HX z1XXlIA1UHVT#Vk067&JKJh8uw&JFfIZ&zrSNkMNSBvl$jU_qD06q*HtE+NCGLgxeP zlarGZQ&&%(GLDBYtgS`i=jSf~l7_^?iwsLr^-|+s^17q*A*zb#G}{H)a@W8SL6e23 z!^(2eX9T}4_DwV=Oc|phtzbFfk`-)=hJ)^p6_ywy8>Yaa)AR?653x|>;J`f;IS5sh z9+MN5>i^RWiDA$^QAIhT5|Kbwg;-%y5&W}znpT$UpSgu9$#(zD6)*aq{P-Vl&^91I zhh{w^ClHu3C@>ooq{5_~PB1Mi4og-n?Up@df(GM1y>g3BNOaS!22oR`s-lWAs=uee!q2glpjZr`%NQRRx`4UWcd;Va& zCt|k`*ckAI_#q5jYc?X-lOFRuZMpjePb$-sYmSIKqKUsaQb$Yl!dz!0v8lRR&{jYtxfqaxmW-Z6 zma78H)Kn}zB12rXN?;E5Vk3{@C9UfU8D_@pSq+|nqlG;<6n8*cLYv3Xy}@T5SE8^N zEvwgbA3ns9Fyua>|K9R)syzseaSSP0s1^mm^dqCG6$()Saju3dpe~ zM#Q6eu28XX3#YA;6!uw4@qS9++yp-M%ps5K1DDZd$b+P6hY5Cpwdtwfs6277d`Q^y zp#&%b#o7leYU=1y6TL=1;o3hikF4@86)?OY1L4APzLl0VMP_+Jre<~Ck>Py+sikLGg2Yp^6g@q9(39mYwDM_rMq-}wx?Q|_bjg!c$p_y- z0{$q;XA>v|iZJG~Q(KMv+7~b!-(y7Z{mB@lo5PgJ>&>Jz=?LFQZLBy->j^JttCd*C zGdvzgaJQG*(92#103H!aBnO6FEoOLb=N4N9j!t^=WRRxIV+8rMQw|%<;m1-K5=IJs z_R7)C_l5|jJmdK=hc0W})w)leePpGMpbOr}!5(Lxw;^Q04USmf)51+z#8uS6IGT}X z$UG|m@>%foIDtt-H$gXItbu6aj9+eu-ri~%fKi-8mIJh*spyx>z++!8d;aLU-y;6% z#*f~;QAJwBT4mkXAl)n_XKMrIiI@^{7?F&Yu`5-tZ{ zPxYn?6hbhLnA^(^)H`-Dz0*(WHv>-;OW}$>9mtjRbDlObq#01fo7R zmma>(eSl*8UtZKeB?kW_uaK*<8&m(BVrIrQ({pmVDQdAQ(?d703g=8?={bE+fc(Ud zLLd|2A@1wO^4Q^cyn1jOItR2m(b@`Y4y(P55q0O(+1bGS=0;>SiRDkStD+kaS978W zarB%iN;8Qygr$)4-NO?n|9yX`Nay1)eePw|h7KG7kY`p1i zl}Cf-)Z0`vR%b#GaWECEzdDgVDwO_NoJc7U`+_p$@d0)zcorl8n+Mglot~z@+!ZY? zwejN*m(Bh!g*5I)`x8RlKgH;~HslXMU zt$v{=82!GmZQvE(9;4Pg_#?`;_yB$gswybKgVF_poOWi`KG9OX5H<|tRvz@(J=={| z!PVC}t6I8NUqXkRMOMofAy~~eu(YKqNT&3D)9Zzaqh%M~4MP|Fp-;H*5E(8Gj|ArO z|hIdDMMSqVmhQ*?h2@4vwVI3_ci*=KV#r|O=oLF!F4D@O4&d`nLu%p{Yqs& zYWJTjyab>lO)S|?9m+Zs5hrpF=0jRQR#wVatW&`8&{4fSY6*aBz!nI6$Q}YZzLYb< z)mtwDAk`Sd7!`7rJDB1TsU$Y`2)qs`A;(_d0MOoxX4Z_%dCETaRzVlcTNME>J~6#+ zzWZU>8L)zH1Q(~PeO+jceiztiS;nsN*-My1k9y&sv3Y2d`}qE>g-PGhNd6_$PL+zU z&J=Hsq^}_V4EBZ90)K@Rt4mXr$79|Y#oZS|k57dX2tyLloQbJ-6+&6B304{4_6T~q z+vq%~A^|1!OnNkJ0b{}$x`|{ly={1_w=uTS$ZNk>ly>{ZvxUfyc^Z8v-fp9P?jJmRmAwh$VFFj^5DNV4cOQ5f zl`M0pRnhMKWh0pmutCNKI6(i@ESy-iI>R@Lat+8rOP__+oVF-p&RqG?p8EGE3iQ_I zvzpQM9e6Y3z8<|`>5DW>P2(ZZc~?FJs#+xY%2xY)8GAsg?0$tpC6PVW6R9MensBxmCn6{#+j;TeYSs`cJITe5^>6rjA5marwW#C z9ZQhZI8*D&?sPlzcW7OzQR3HNlFA;i)29?f& z7v+Vfud*-0I>J2!y&wrg;t)RRolL>ftPnH1G21(qgUV$MOCgv;pZk)S;NXOjF1+3V)os^ZH)><^5B&+$L}Q6R8`QeN9@Rg z+>$1D&F@RX1$&*2nzaZ$(_7t6=Y#h)`RYnB8k#3x=X?kw#TRfF4Y#1NyyDF?O6W2nUGFrrd_01)v6jw-t&~1&M|z{&}0s zyDyUIO3G_m1MUOjZiTkCAb6@%HV!>a&R6FSMo44}7f*GX-}RPp+8?^aU#bx*^$nJ6 zB}$en;sJHlcy?d8_n=b@J&LGrEC*He4}Qx3Ux4^OADF@MYFX95F8(M$mlg4Vm5YRl7|}jd^`AmlPwU4!Umh{mPLauzpW_|NY8-b#7?5EwUrXc+rZlaKbfz^d`G|dDK|PYWL#+ z8(B2GQ03+2rLM6NgAWljb6_T*W0G+xSgBGov)#UxQ7vrpr%*3Y>#^Q_M~H{RrnKHW zW#DpJc-z{91KN{-%#9YbMMvN`QLCXF8XN0R(gpy8cs~`0zoc+Y8?6Thz#R4kin*K+ z6o`SqwW=cRav5}&U?;X^U%)_{%IfsQi(mnePF+(I`{P&pFSqY}0qx%6M@}Fdl#Bt& z*8qruFJnLwA+c`BGz(_YaJAy0$b@){VwcG{eSTUej(=k#P!VV>wYJmL@6Xwc9lzqe zG`nN)Wt-G%(DD*f3G`Q|2SP=MfhN?uz^o|B1Z9mOma*q*f*AK);QNu&oXe%RcJ89o zClV$tI_NChk4%iAr)ArR5)x#^I>-0y?+7r2C5l~1mgCZ%Q%*(|t$+rZh5Mj`FNZww zBglZsZ@&wRoap>$J$sf%9gh=}DkQ!l2y~Bw)!EZ4)(@e@+y1{j+Mt0-*8bP`sOM6{ zl30CQu!3&KzKrg=NUJ^%AkWC3E0tZ8#Z*8bI{C1%4?{=LBx1?vz}qoqI!`t>Hjs`cif9!Nbq3t+ zkj;FZ$*-+sF^d3AmTaU$za|vnng1e|!mKq*be!VD71*l^vXyi{jUi7LPel+Qabt?fA>ID@~A`W zM!R2QF`mgOy)90IsA^a|;Hg$|3k^I=C)W%)J;UkP;t+tO{l>Z8X*5a=! z4F#d#pUdO#M4R`fK@`)3rVtn%G72Ed`ETieBqA{-Wqu$F8$N#_anAq~t52BPwvs0b z=94b@$6+fVLAH*uEFzOBq(VP3?}j2v6h+%+kcK+2D8Z6xfP*Xy=|tl9Eo$mjq);i< z)Gq)|m#w^nd1XM@H@}E9V z^{*)&DRWHxP~e)`fd5B?<6XWDho#1wTae!MO5C{}vAEG<6{Z?6Nwb&z$2@S5dHv@! zi`QICQ2JSqn0(3b1+@&4I+FYC1oa%-bcH5mOd#@}3<)xHXCjG!mHp2*F*Vx)v2s)& za{rn|g?&=s{CXuDz|s(ZO7n-izTU4JoG*u{~Li2 zP7X2)MW~Omg=>vL*Iy;qld4|nT8CO(4@9GhK(3%q8VZU~%u!Rt$H(BDhzLrYLL!buAu@%~G>S za(rdY^z5SDC@C_(^&JyfauJxHt}1`T{HOXm40O|&QT~v;r8TL^Glh@|MGct|5r5wG z6XzW(^l3)#-z7xicm6`j_KBlj0h_tA@O;{b9)|tHUS!)xI&yek?9;iuN_w-X#F)Or zU0=?Gj5D<-g>yuGE}?@0-C3g7n!$ZV)??;WlyW4B*)S(_A)x=^d5i2H#2bFDBDw>p z#!1EeMmF|<$!ahC{ti&cw-uU+$rZC(&f68q+_S;lm)}oYemq`vf8Z{+Y>?k?i%`Cj zm~5~^>pB>%uM$kgjcpC4QZ~Z4qpkTX?F_WZL&%+(DMg5zC5+_%Jh@(Af*1zoQLAX!Ec}v=> zoLip(JXi)-K<0znaPmQ8Sp>WQ{RhE8!j9m=i;F_F;K^Y2#s>BNS^*M_v)kn!!MtN$ z#LeL$R(1l|mWw!}+4vacXbUF0p45nFH@gTDoLAG?o%$!|8tH2DeC&ZjCJO`5VU45$ zN;jBZnPM^%Ltzb9JGLs_R;Ni5 z^bxNnIq&iDpe^OSyhO$a{Mb-z5;cOyj4d1C=BlBy?+EEtrfhAP2El#hbV^(e@G@NR zd?Af!P~`pdrbB2HP$m@UtZs)`0w1F!yqq%Z?TmdyNN(8We?hAh!38=^ZX9$Jy zw$9D-0D!};-jOi-CD8mL_R9w^^wN1i9qfJF&LK_G}qOYzAkv_WfDHk(;3Z#2RjsXU9{z0 zgfNk(Gu+|Pvm;Yuw5s*tcr8MxGe$#b48vt~JcCrHe-?Krtn{Pc?#scPT|@@X(=pTe z463O=3zl7!%{J`AOI9p_RIg%8umjSwp~SGH%5A+VsHPm!H@i0)tP%U_4SpVn+Vv z8Pr-G6Z_C^`=FtbW+~r)Lb{>|fBAdHvYn@(>a+I~(l?kaW^GoZj2DtbnfNOm_=F(1 z6Z{cW)_%m?UI=FjUfA;#d{0>_^ot&QtG_5_d54p5lyJ_{#v7=389jYsGx9mgalCodHW*;$W0$E&^Ro10Q9QP9y9t67}mKY&_gn`7|X1RYKV?JTjC308Y5(pg9;>6bRv-_=9N zCCCe+_(s2BiDBHLPl7ge1xAJBGNUmXGac+L@4DeHZ!=w2Jwr~Q~4 zJm<-hknj=aTio@EC3eQKn%Lv;SJtewTioVFw<^U@Pfq_#-{Jd#7W^u#)M`!+}I;%goW*u^YzbHP5QTZ@qA{D)0 zSAA?eyg?le)$)HJ_GE!8*f$Ms>9&+aRTZK21CVN3n`&dK_^-F24h2F|<#5cit+w`l zpNCupVsxG?YUrd1B|U?EAV~QSbwhv*NiJA|HCR6+J-RUvkd46P%XYb^P52a27|Vhu z1i{4k8xY#WHf`_as$L{K4ZKI=OLoCZ!VT4V2?L!8H2;mtzVn5d{ar;m|bMEQeV_LV*O|MKT-%*UxPauZr|T@6zD|C@r~^b zgk`VborcDTKH+iohX=X>z$@EgLXkm`U7GhF=`*TSD|yI{JGboaO-hRch?_oiMS7)C z_tx76iH4UpYbA;o56`5V6{%cPV1SqnJcxdg?S61$8OB}>WO>$NMM9tIJvmA%`cv3B z*+eP$48AvU6#NHr!-LF;&*Zf2>njV$6%S5fmK$>9j)TR~hU`RjlZ02KkEBZBtmsaR1?2E}|ehsG?rj>yvzi+G0WbL@xiGeHz}r||AU;jr)^hSIe>u>|;!>il+OgP$(B6tR zGdr`Nk^qFfK|Qi!LQsJ=^eFO%=2FxM5Cf-f#uQ9&j5v8kZoNVhzseV7om-W9 z&|D^nQvKKN(?FpjnE&=lom1y~qEdz11&4H^t#J}onDIShZQTwORq8~X4UJzLGj3%- zWeN@<{O#DXzu}_c1-zV%DvdIH4aCP^mN`RWOTcmz9TRd$O?GB+_>$=>piBG(kkm>Q zu=r48qN1Yuq=3REx#l?h%w&CKL3zlZkpCKq5TN2kL=;K6k1+<4>#UB`@d>3HZ6=7| zf=Sew5!;6%7utYYhmde^aG(L5t*xyAfx_ShN#T8K*=?#xBUEhHoF3;wBh{tzP*eo; zb&+9@J9MB4ly2pJ8TBp@D^oU53oFu)(*<=bDdT(}i(1W!_zJlGZ*>pm5Sg6#2 zw;p2a1Y(?c{wBB8x)t_>fC5X4cS%>&{zu8M**=#7Iq7%&!d9Bv>)3!98;WzLSH}xS z;x1d>K&P%8Unj*_q{9z`fK<)vFR2NPIg^F59eIgDKVY)_;sn4Wp?App(BX)FEVyRG zN=Xg%h^8Vt%$ji3Sw4Oa?x1My@YI-&*AE>Dj&@~RSSTyblBIw*-s8a&L6kea+;q1+ zZz``4v=Fk9(CE_i6}Bo2Kx)t(!4iQ@lge_9i~zV!qTER}Wg&GsHc>{k-lBF#Yj#bf zaVg~+j3eYJ#f5AGQ1>rGUTX0#xCfHUu5?tu4}?Y2p495OmU@YH4-ZzAj{z!~hzOam@*Gaa;z{R& zGy9${44)QGHC_@KJ;eQq;4!1E;1ns@4#&Xp3UqbWq>wLdOpIu75Du5z)&mw)`HbtwJ6 zT8F>s4~}pDw-z8(J;bmrYV>Bax4~S_?Nj)*V>AYjW@p8J%?EXoNf#KVFg0Dzi=TzL z_FXo^+~Y=+$3*RGzM~m(X${eo(Bz{jUOqO)W72BN_2jyjbh14mJ z%F+kwK>OnqrIASw9@q&GjyM~0Hrj=U8PbY5}ZhadJwp-Jgn=-P6iUm!#77hT0^!|t#PYE!G3ZB0bgd4F7 z7!T#hea6TJ2E=d}GMIyWibf^7A#%>T(=yI0xlw|_QmmUf&t1;1n7}0B?16q{!!56_ z;ns<(-h}&iBh@39#$H5$pCq?-e?BLdgh%Kffif-Ao6gi)16W?dkQXtsb?hcPhAX2G zr@-6pX&AD&t{;u9$Y##mto6(CkQGk=rL9SAE?VwXCyR_GjruXk+Vvp8M#2=Xu34+q zEw`X+ftK@uSUBF_CMWMD#T-#fTuAf!O6+KD_i_M6x+nfVb9iFH9PswHp{Y^WQU_=y9C(X(v)h?o_zc11 zxAFnDM~>+=OP*Fn?{sdy!*V(H2NSVMQ_dLzA6Ncf-9VuyJ%Lbjq8QmfiVq zLTL^?%dy^8;bvoCe@QFF($NhX+vgS99J8{xogMJQHici{8oKT9tOq6*xGn6lbyed0 zc5|&8X-weaSqhc5YJ_<0x*G;X+zGkw0Za2h)Q$ zh!9!xE!}Q-12-E(KFV+LD?Qu_>k)p~S&=ZT%023jX>z$!lql*hwhhOO9L#@~bEb2H z@fi2qHZ}N)1XG02yziXDLqrq!sKKcFnZe+!^07bnp&(OUw!u~K;6v|!l|?>%rFTtL zqrdzs=|dj92N@Qw=S_?r-)F%Avb$5R-iq|o7bR1e$QNQ0A}#3%RBy4{b7yY5S*&8U zcwg(Wd)3`Pot5Z5*DKsU5`yy0I36ed8^=9*np}7;_FOAG7GNto@i2oaw7Rf!)l8>H zo}A{h8XdDrex}2(_~CuDTPh;eESF23B)7 zbAYgjdO>1LgGs|xEE59nuH}TP06%?-Y{&j*ouznqIy*VS70k`|HK86j8j>2ZOXxSA zQDmZUz$d9!xZY4nk&4rOgPQ!{(zmdPVB2YD3)aUKnkU-S`>M(A;WI)@r^r|ga3ELT zdMb;o`^P)?90@NNf}-BgMiQG7v3WJ7r3w{oInTp5x8K^vs?$U}#OK?44(pJ*{1VVE zV)%r)N)N{q+DRs8_pJ%=r-7h5*nbnvIkv>=s@FnK&jTFIm2g_L*r&CAa~7=^mfsn< z1wFiXK0EGyEI{b_=FX2KGz|ueveg$F0kBy>*SOY|%+jLCGT(owBx034f|o%dh5JR$c*2X@^^$i&`&VAZ?;@XNC2i=ribf zwa)ZQcT9eh;L2P;q-0Ts`&TNp0r{9M0YB6+%Eh`|-u!zaPITgG1Wo_Fd6=|ZW+@tv zb%G5zz{;=HK%R*w^t6S?ot@hw^I+YWl5G<&4q|o~6Brrm2RI4SmbC9QJqov1?=iT2 z8q02C-3||9JKqb{ZxTxuxJ*qhBZeQEoE?URLXgg);yR(PM>CgG=T;BqQhw?NUeWxl z6wdnc*lf4mj4m&Lit9pvLIs@4W%6e-3)FuLD8yN*RC(852f-K%g>BwVyb@lP z-_`&e?2z3x_*No!YTd2zivVwY@Bo3%78A>dt2OhB+TFM7DDhy+Fzd<^;Ho=+>?ay> zq-w+VyfO4Gi{Ty*zcSsl5vAaVQda?gzW%yACFM;iW1ES;8_5(5k25|>Zs+hT)<3mG zUxf{&(lxv1n@3+mIZ^TkYfl94g?vMONm-~`1m+s-Hun>8jvc9-tSER3a4JZI{Q&W& zH+Dp+*Kk;_uV|WOYi|dYC-mXK87+#QAF@`nI?XZJ!>H#E44;%LE6zhtr+b6qB|KMr z-d&xoGynv{Qbf#__X#HTNIC?JYgiKMBQlFYN8fFG0S^mF4nu1~Kbbc1(1FByGS^q4 zJ6-f2JvDLDWeBedJ9|L=tL1)8+35%)yX@Br^=>-46oFH1xP@8l>h-IeV=%KH_V6ho zi*U85k6+OaV+|NQa=&_$X=O%N`!ew@JL$6}Dsk-H?Vk>dqgc|$I-1mXgd|*E7e?Mo z)o8dh3yka?It4=;07# z(d@Cr0yhwVm8aa)&FmwrQuw0+354Qo^7*-H2}3$g7xe_-cpOw zH%@~OQdz$wYC+zw+baSS8s0cnD5s zPsyG#6z*@MB4}lw7NtwYgJ*7(tlkom&DECP#RUFZF;3m%X4+n)_TJMSk_F^$yjTk# zjIE}X3QBmhFR<9jY54n0)@WsF2f8$;F?&F*dS)DMrj<92WB#tnn-oK#Sg0Aez>rJ# zA8@bA9v0ZUM9kzN`h^|+qp{khR(SsPbmsx1_#X_8l7U>J2fkGrn5Gphoit=H?CiZ} z9WtJ%r@V%GAlcv$mf^BcY)WNsH5TG&G9!qlZ=zQbJP@RIPF_-*(eYxXGi!Ga!@Z2G z-`(5xre~xY$aO`DOs5GdXrR^N7tefs3@70*<$KXNB~wb5Y)VfWLHtuWxv+fDuj$nK z-QyxwWDNQ4Dbf`{a~yY6bo0QF#E11HIHx7fA6Lu@;GNjRz(3$l7wksJY-#TEwXeQf z{YmGI|5WkBDOXJiib;g|Rq9H>{oP5}^HFQGaF(aLr<5U$?NYNX)G7|nRe&*4B|`-D zVqOa!J$Aawwme)jUdqUeH`dZjnrmVs!25z+F{dN)TelIjh(MPm|1BUoUP?D(kInL| zZvoB*Qb#q}E;+^77=aCq+m$Qi4GKf{>&z3vb9VPj_HxnuS(W;- zsgz2sM7u#wu_>8`>7C@rlQwH1mYURqyQfb(qpc@YaJMoAzOLV+!Fhj=AAI~ULf}9x zit-uu`qEr?7-jhj0AKJQ;ZE|h$p1^Yb5heTG5kzKENG;=d!YGU1UuN8x@Zf9TaV9i z^zx3dWUF^m&8YM#mTiggv${1;CpFURVxB<26kD3+H{YMKU|Zu$Jj6+Rn0nJdz)@fb;Z)>bmtuZJGY zqN5nR*ulBv+a7_P(?F>w;D`Q-Vj#lEF9MWYLr!C2_Q(kQ+CnLYAMDQFeE42fz{cmq z1o)3y$-8uS>?G#KJOKzx#COf#^W8Q;?B@(WTDWJQc^|%)_s37I0j{?hkw+PSfaS+% zvukAB?%S8&D^pqb=Ogb(x36DBWV^+l;$WVZV7R1A{k_e@6$Izh&VJ3HwVnRx(9Qi~ zXy3=!h<0&yDs6YopVdA_iBdO_H0 z9HiC##J2CoW;1hqc6;$lP zYlEMek&ForO&HpzxsRg9Hx$?IELxF&gii|DZMG2!5=8g;IE*=K^A<*E9?oc2rPrZj z2n6=1)!fa)VrY$TZzAYK@=>&ya=SJk6s#%f|CcE1Y7*P8XM!K>mRd@TusDZ7AjLq5K%?$9hY#^wxMw5s ztnn0g{0(_l^Ab2S+&HDVv1{2p^l9Wkd5P_3g47m!7kaq>=H+&boaWtoQ=UC&xgm8W zZt(+6&L2RA!yORB_r7Sd4*kHLPO8+l#@11tpem!>dzpGjK5zc65V>;mfqYU^$~TpS zgE;1rJ7Nb6uQQo;IpWSLk#2|s=}jNxuIUd`T$|6{MGO(IpE0P8hrcfkjNMjg;cExV z^C9CX8Tb?dwDu6OgzNw-F?k8x?t2d2KE82i-EHoDXFsnuX=g--3kp9Kn zE!3|`CxXh#$3>nN%>bgPm{m60VWBSC8FRuK!ORME%wcwQp;PqZQQd|t27IPWH^NA% zZkFUcqVwRzv2j>4bfn2=hv%31(_V@#QolV**dPLkx=@+6rBnH{8w(hEjmUJ*p5Ls7 zUJX=a+ojurmjqR7KpLuFZ;vKw^`lHQ1|X?gYMwb8|2T?n+C$<6^wrV}!NbI1>3+8l z{{l??HAiS%AlNPcLipci9EaD@B>n?N=P#>`A>WZ3Tb_6Jbh<+J0h=oQY*GmI{=)sC zFMn$rQ4ufA-yFAioZy)Svx5fp&R(_{CQb@9b1zFE*bxH4%UE_cX^%lWP;7v&WWlEb z=CSFuK7wKXms#`nxc0GU=BvHpM|~=bKO#zRY`0-~wpL@8|I>Yl^P4(jn%$n=X_(|& zbH*p95{%?cPH??+vZdF@eZ|`%35ERUH7yrlv=*+>haKC_th^K#AJXysb~+|^7Hb*K2AYqMMy>|lxeG(ZHe z_A8xSZxW52C!d8+s%9PLP!G@y6C8%QKT8al{MF|nar;_&em*E@2{-jXf(q<-4B+gQ zLci;?IoDE!l7x;eW^ji$)TEEsX)pYs^tW0nMMjNG<1|_dCM>MAdWACb+N;+9M?{O{ z9zPumu-6)`UjujMV!mUaP2i|%g8nrfvalvn#DR}iZa)>z*Tc+6y(QgTrK91aBqv*t z)vlk`@8Q|fJgTUn-Sd)o!}TPDA@EYFGPQXiFkkzms`TymjUgMad+w%W=Fg>KdE5-# zUS&OLclgF!N||))U$$66ioCQo?{w!6iB}xZEQ!nesxS(63&ZvY&S|1tE(19DJ|`hQ z4Z5Q=t97tGzkq~^(1j_HG7t=Z9F^a^PSCT`dwYxbVyF8^o83a-|viWEAaL|P$axxYE8ud#+ zEKKKqBwj!ME2#I89E5B~1SU14170P~dp~m~D1mKqnf+5DAq=T%OF5QGB1GMMrM(lk zXJVt~X*weRKLa-dOGXQRj{AmFV)BP%@Pm2|==mG}^n?F5p1%1%>*q%sv`}+BH9shL z{#EZGM383X=U6927XsTv!&POMjnYaV6m(zb>{5fOyogADAKiw76qs0;D*LQ1XXr<# zwxKN?QZX@!9ih?_QV~4Mzk1=LqoYH{Y@2HmevGyS^`B{{&)4<8 z#`>!?hIwF#nwynUaPi|WaOzr_y*#ql7RZdJvX-gnQp_$=XDokx%kc&uMy@r|n*B0v z>+z#ZavMfF&4C#UH-~j>e0=+TS+xLI<~Gxy=4{}B+dSs6EIK4}(~_2soqh5fi#h2XoBHXS zn?bc36~0`7)n{6%3~retT3YhCn^=$3UHwf*GIPopE$Ge9oP?N4(qy{_OEg@xg3g9i zC&(lzmu#yK`_1DPHZz@jU_~Xs$$lAz(?&f;a~n&=zc89<{pq!P%|Ps9tie3(TD7H? zbf$n9HVmXGO9n1?!FX!=GQFU8y;vdGoh~B~;g^2{wQ%chvsniDeEU|`G>@F5IVGqF zu(nbt6Zjk7tmIhq0^RTo5x>WgSiXf_RKI9uh!+)BC?z*FxG=Jw^^XJBG6V=vw(spv z!~AChC73!_(gCipX=jNirlZI6tS0D_GV2x~@n`uUNVu2ZVvT;r*Nq3{e(ZT}r(vtz z!~rcM;!6w32Abqh^*0_T|6aL=2bk)k{k`zuT7 zG?gY<4Ns;QlI%93=w}E&(gk)@K0?(P70xJVv%fRWcbnhNx&T;s8F=YOUm++q^R&&m zAEIi6h7)F`)lS599x3#%#_)q5RG5I(O zznld`R12N1bkMXloiVxb6lYfIlrE<9v|eqp+1MKT`T8(H{~?K@aAh$PZ8_H9nno>r zp$vDSP|=9)AJl`;OT^LjZ?RlqE4C;$>5J_JTpvYEk)a_ieLgbxcY4P{L8Kvo8nqyz zm{v!#xZP?C*%55fTkj0>#_`JNihUfz*CZc;tWXqECZtkQ;RdWRaP`~&U|M882pso? zMb+51w72-RZXh)U(mIc=@FPprf|~l2syXTUFn)Oa)AV8=(yJ&=J@J_<9dtt(loV=N={Q^Rzz87 z@HT{{WDw{E7o08B7mXq$vA$v<8m>X-cRfo;S<=z-FdVdYdW3@lfR3mpXs*7}F3b?O z?DGs}?KS61yF$IM?k)71g`Ze{yQ*AD^8$ zmSTqJ@937&POA02e%)XA)0!(l{|QANX*?7;JSit9W?4Aea$+iPCp<1#Mc0nfOiCDt`%w9UzMl8@uMl-&^#+x);+9%K#g~VfMbIqlMp1=( z5`#Q|x4d|ZLLSR-GMC}dwj+q$2Xc{fzB6gU_#zO+E*Phg_=a^n0a;qn3t8frcUFM$ zBgdtqzG%-o!i>a!WI@6bmB`p^`nq-PWdzst=;t(B+Mx$GR(ATg9hG^!Rfce;_v32y zP}T2s-OEvESxy%hmpm@_QXrAEtQdbx@1m)P%c#>u*Kqulpe23I6Zd5O?izLU)sFcX3!iLDi1s{>=1emjvAyid_6IH6qY~XLg=E= za@J3l8RlK}9Eu$dzoNY+D}jpc|HbX5u`sfCYMAD1Z%^%!#SGdQm>^AQ>nMsC>q4z? zpGl4B?Ng`rNpt4O^ZeCN4#ttyZNP7bc&^{CG*+`^5NYBw8Q#YnEp{;=@%-PBiJdlV z;fQt?H)+VXhsVXhL%@4VuCyU&E7PerDRC{Ga@f82V;!7j{dK>6bw0Q}n)y>XWo!-A ztn?|2!m<_2MFLMGo8V@1vNej9m5_OjEF?TBrm2}aMP~i`#l5P|RTdQ#@GhS7o%r`H zQ)C*pX9J5|AI;zniWhp9WTNzqHaT_zzE}{oV4jeM!Xf5j-MtlY%{@!`v7k|qn&GLE zc%9mtPbl1I2TWEStB8)n8B_v|7rhNZlddBPJ-8%r;alV5yzxOd$Eo>kGuHsvyf;AO zAJU3OPDZ6-pA;i}`5AP*+3I|b^q`0!`S~CgiY#+lhh)va%xgP7ez3xE@k0rwrspxx zVYkLhRyMp-=VD|#i}mI`Qaiv*=qK|z7GJT=q0SxEMVB}N4C5jCH&Oa_8&4+7X4v)5 zK8qTD5?`#TPzcR$hG{2!a#5Y}i~E03$4(zxrwWq)ZpBhKe?Ajc3Ea?Cb=tMQR>UWc z9^>1pRx)#abfH80Gy%>dt%%#ttC2nsnVEbFO2q;TfMnuK^YqJJX%E<-xg4N6ARRUk z{&Bt_B^zE{t7Pioqn|;P+x(riC%pac?I(#K+Ln8PMYV;{z%;`r5^-@;*FFTg- zFQVb<`}OlbST?CmRw+*=6#hsxl}j4G7Ahe*Nj){{US&N|P~n?nx_qKs#8%p+@!JD8 z0QT!;=Vjl;VDg|9O%qEv3dCk?)O;s^Qu>3gF^-=x@K}7=^f8@p0Y+9}7LVVfK+3id zr{|E}sMb2QXD?uo7X|@a+ z23pKwLsSOAYS`{A+3ED(KXBL|M-q$d(?2dh?|`$5aogw3nYtUIiPQoT>e0nZ*OX&k zY;Xf{T6oExRkfzOq~-bNN(Ac1X1kZ46Q&nkGDkC242$}4%rhE7A02_vsnju3RRIW~ z;TvO{Kf|elHH}dkDTICQL3k}W10?EA^#T;xE3k7|bE2a#ToM1W6Fa>x@^rH2S!z~M z*HI539+YxAJ-RbMn3|f)d@d-eFV?r8vs%Ox^m#BP&QU3{u)mmYF6A#_UjJdLRix52 z#D`aNVGz?h)`6IAA1G}k&Q>Jc!(q`ueNVG%A#q3`!ALR?>|JX=J^>4xmTXwnz;mX1 zO$W_>%VmEQ=Ia)x7D}IGSGV{>+s}#brHLR~gQD4K{a*SRwlxrY{ZEIw7~blypZ1=0 zA0a3cn}T$pr)$V7Rw0uPqJt?ON1)rDH`4lr=B=#5L!JGr@b-u9)#2YPTMjJLWBz_M z*LHV9Ta+B>92F19|BG$wIIw%iN}vZo?GsqpnldZ^I6qS!0Psx~`A87YXP`T^vKfYa zeSR6;_DZGiErj`+SLSsG20^m1fr#$m8ekH&FzO&+ZDM(XX$s<|ghm zaA%5AD};=Y4zJU?qC-!;9bK~EUO2ycC${(ax}}@tPT@iffO1@5n1t&U= ztJA5xfC~K5#zMMaki6x7e(%C|u#LG^3CYdQJ=AC+VAj=d$BN5}PrUX=9AY2_zAV8- zMySo6f2ERTVs^7RQg=xwhG^mSjQ%7RMpa6mh^0wx76rO?#Eg+sw!~6QSd|L|{Wh6w z@zFfx*X`*J=CMVJj%nIkt^GsK@|LMrLn&Tq`DdqFg=N)JIyF+UT(!Qhc^2HLfoI`L z1^J|;X#Txm7ekC!?E;^vCYYutM=uJ}-WJ&2~M+K?MLf8PNVXvUQp}5mw-t zZ$B{E$rA9+NL9W44PqptYrALa-} z=qo;x1QdyMrhD1^ql-_m+~+V+sA^|%lW1ZhvX|GL+AwN6*<`6C?M8N2zqCikdk zmh7}h$4?{k^((YI)$cA(?~*qdh`nTV*#M13BL(f3*AXnvN9}o%iJc21{<;KBV;7Luc%BJhiTIz+@Seo2#SYG-2d)m-;-cAw1@AuJNUWp_-;2}=V6){{Vd z=v>V)VV{T@9%J!y?D;VokEVoF*xNB8f80!hj1=nH4mAR@m;`MYM7{;V3JfD|OLEuZ z)NIl}yVqf_n)IP{=M20DTOAIv9Q6Bz7*R^_sh+<0gPoy)T%i`DVVJyvx(vB1;#Ovy zUmVt=X*|Jp%L^EN4t@2uBUd|(ILBZLs~2c(SB6*_Z5oLgddXeE6wX`cz#d@6Pggl8 zYZ_q=d}gxHokmQ>Mlv*-LmT&f*{Yd$6E-~y$|<=I=*TYEweZO9CfN15A55&$naLq& z{*b^k(7lR$Uf6OGbS6fqGvTIES&MYEy%&S5*Pb7&!WEPK=4+>$`pJZ0oHKj;V>nUw zAbNZ*E~`5Or=x*#kZLu640-cg`uMYJALwqvi^+I3wRs>+5~l6A2H~b;S2L#aw{7FZ zcu~-IUoDS6s~T%(+XM?u!y3fC{DvXm}w4@7O+P&OglkE21XIB>yjN(6j zYG)9)(QTXQ>|{_U0O~&HtNas2kfKU!82pyP$;u%@ZD?j=eh;@w-Yz%%_sF&*kh=8z%#c5WjfnGXWpe*J_7L)7ZXH zC$KMC9MBKd4B2oE8KYMwo9IXFVM;H}47l*JVv=LVG&Qa|FcPr0jA@k>OSSh3?WPv8 z6LA6dj{yW{$=8~kq))AUY-b4Dt;9@ys3ZN&vx$RBJeIEYeQt0!$eEv-_bdlA{Uh$^ zHX*8I7=;jLqgF@LaDaBrI>T=Pc$+tHBa4ww;~tTgc*?B}0|~AEwp`kn^>=4&=P(*2 zcoV|AsQewQC<;QI03Kn#MT59>NLlfT@!6 zUQ^cBdGDAvxo7TW{e5n6PD!+7ljb+6(`(bz0iYsy>m)kE2>GER-WHFX9RJb#q=fTYn3bGR9HD1L60Rzk@FRLi>a)>o-s=n$Z#pb#F~ ze~w}CrAsl@URC|rmP=eFOxd`P`Fpt9E1Qw-&i68mXfaP09v1utao%fLP%S0)ninw; zns$QQ6>E+qL`?@KK$;W3-qsd=T_QZqH$#=Vs#@Yope?FHqxW%^uWeibW74Ps9Wbuw z{tlrk(qZaEHQ4UNmujiN5ML050*Wm5nHNO42@>9FsR!Nz?~*XXaqc8{fE|1fMps{7 z8Q$?G?EteWxlZMh#cBeX_eoMs}*s=@XTfAgM3MIP{){a3b^Fff^B@ z;@%6GK48=E)Mnj8BT}{+pupMR-5-b0R&~U-fz`>QKZQDkXC+ha^JWW+<~+k}vzW0N z2s7PVPEwr9^9M$Arj~%QxWe(#0)UoR<7kxo-w}%aQU1%ao&KIKEMASr(XVIW<#%`q ze}>>e267=A)K10>e@nQ=GK;#{UMNZB{dHX_G*wf!DChQywO>n}6 zzgb-9Zdu7J3B4BTodGCmvGNOqM{Okz*p-tU2~CfmE>cWL+02@QRM_M*>J8D9&VcdNnT`vQ(wlNANa@{WL}Hs@#!z18)Ipp z?{#YR@A6Nb%7VI>*w{nF4ot-7OX5xz-5oCgK}_HE)}7}0en2jU2x@BU2lKOU=}EuT zT7PF~0=MyZ*T}jf)jB)mA14PgyBq=A7lB58FrfNo|7hGraV6~lOgqL#5lq#P)*l5S z*6heTBgBO~8q-MM&rTRB+8l}b;q>_227sH%x<;WjM#B((3!6>UsrJ#maj|&=mPhsY zhc^lv!rafk0>;MuSKWVMyXiAQ?K&NT$nj#3waF@&0b8SxvzyjfK|vb%{vCmUQ=%Xy z-HGE)rZL2fw!|LE%RtGVw52Mv=S?P3U-hv4^G){0?7@MOhm@q*haUJhHMU9~KQTX~ zE8GQFQuOO|-`G;(F5tD#ODd_Q1@~gi1u3yeTkNDkwFu%3maotWqx-jQli~;3>x?9m zmbZw~bcUnpHg$T;6p=mJ_9}|dt%)iGS-KR7$Yc%txf?nxh%&O+iz}<6w_#nsmEu}F z2VBn20F zKgp&}QXSu0I;N#p$`ntgRqAo;+R6h1zYcr`g2&>FQ}ad8*zF(FWp#VXd&wsSQys2( zmn(+urkX#+Nv^bO{$ju!KJuqz%KF+D|KEE27o%ipJZf!z_hv?G92_oS$-V0rY;tJU zPz?H=j{5bh^>mKxj*%6+D$pf#3pQqy(pUmOmx^b;AJ%hpSf>r>ywB$O`3h^jTjy<`rW{%wu`{H{+ALDOQUq1Amqjoz{t7WKlTE0RX zvneq4{6TF(`8m}Ew?goNT4K4QOuI+}n)4460glDe8t(01V}lcx#3F~I<)vRXjfggb zzX@81`WrQHD5{7iRPc;h4P25#cZ?aFsVf@oe9nj=6e}*&R2)GH5-FWZ9cYMCR>Ohhf!^O4eduOSz zx@A9W#BLyc(!v4N0s9k&rz@_>iX^H6y4+2N+%yX;qUrI!RJu=9WuL_C+#%57he+Iv z;YOUl-guh|wg;zN)D*+!GbONor>!>E--;29EU%p=T|GPFbHY0edpYal0EduMl<%I^ zf4Ca$S;)SW&y4$dg?z-r5VFeumkYxFX{gC#v!XdceCV-$9WLE$>AHM)d0yyhmsscO zZ+c4Yv99|XLeLD9exyuis6o3jh0-UHN) z%JLM1VQ;ULxHcfFZXh5A7z<0fv0`+F2~b=MB+@A_k}W~pt0oXYfrL^bMi;b{3`siuz^+ck{ZRi(!T08->7NKY!5 z{ZQ>@QfGT0NO4GE&$E_SYoJ1eq zs>a5K@0DQ=fMl!Y>{Nocs#*R1u%2fo;sfXmcbX||0n z8#vhconqk7!(6hB!Tw>91}OH2)MvNu6Sf40#U&-IoOo4A?3*lKY_$87QQV2KTJ#?} z>Cv{~w!Nld)O{w1r>3zL(%R{>Uxk617+vWnywp~#zuumhZI;l`O597eld%t%2H(hT z?_)Lj-R^FN%t<3OaGTUQ{kZGUkQeQZ4CA>}8DuK<#?}#-XHGuI?&g+b@27-tn7>Y0 zQ{!m#^SO}WnCF#7HGd7ya@$I~NmyGW$sAkgz`0VJI9xkz!5|b=Z{mi!`lF;{7EE|y zPR>5%rdMjEq|H#-^-U1l0=K`G^`M#20gqE2X=8mtdl zEf6p7g&#&V9i>D;vPKnLTt51|`m*6jJB*7zByb(NLOWR^mMb1#D#oJ@yWOfd`aHGWh z^{7Qh)?24%5E*OvkF^o4-}GalzV6DvOh~uZtp-PHZXM)Bl0>AAyj*wL1>-WLq|NL= zNd=;vrHT{*p;Lak!@0K-kurV6#a=J*J)a2KsLQjF{5?;RMyq+{!a<>Iq60FTcdZ$t=$41^Pa^SC4DOdgjRRbF@I5x$p=q3IHL2bp1e z>F`U(-yN6Hw`xt6u|YZV`MjHi$i4d=hgWCGiq#guB-(6NB{s>)Da;0bEa{iSn8!}4 zq)V(!17m~GXD+MY(JnUF)$`d;9rG8rm&r>$$6AqTd@jFYlO~JfdJ# zZ=$J6wV2M`_EIO40D3Dd6j<_C>fCR5!qmMwd`Oj;7)t-RTjy~~H?`l&(F@d`SZ9Hg zj&ah#IgM#Q=0)rSTGig{)KZw~2Te0F?P2Y2*>{CGCrzS3^%bfM5O{{RNlEd=-b4Go zcTW!YYn8SQN}cGB<=Dk)y6&BEeDr zNOTUQW)0Ddz0(RBI>teZ-RDuWY?5R}db8cVyF$y7IU@fC$9EMn_fD;K$qj8q=Jb2M zh26P{hg>Li+TBrEjBNM3vGMyoN-Y>Sam#l}8eVl&oUFb5q`9D!ne#~Y=18WcXBryw ze=Ceaf)Q?Xn+G$rECPI@J-7daSeNgc%cGZA)}Ch$j)xKoUgbRya!KP5RB!GYkUKs& z#gq?OISRU5_K(<4R=ei4uh^(wEGUK%Dk}jTP@(cA#-RBaz5mKXY)}Q$r3?UVI?e4W ztXQ-;2A*OD1vd*lM$YvfE)0S`b(AARR_h~~X@-((ixw83 zi>W9PnC(tf3H@A9|1hG@9#UXsXHK2p(w0*`WO-NH{5h`&CxF;o+!6RKS7*dcHzTQ2FGHq&Nbf4rpA!W<1{I9Y6^LS{g@T!bZ{rWE80*v@bh-_lN(Az9jgVh z&4o{k7G2cTa(W`=oo9UP@fFn7^pqpv8b>J37HD)y6EqompNXD+vZZr?g z>9l(pF`8MxE>D@1xYI{cW`!+qd@l?n4OHE>I>_mt+#@OOgbz z+os9vlLUNUUAqixBFByasx;A9$$=wo{T^On)S~;8$moY-uWx?;SPwBGz~Hv(La5@@ zF2}A|mSXBbZfIW52Se{Sx0s_u+vXrjfF)9ha8?{$ig=DA{&~puKzM~VqDU2Lxn%2g zg*%g@Q6}6%zaLh^tf;_7cD0q#eD3qJ50qa6wt6b|%)$bAYT$j(JmG zk%CBI3qXo(^?mph_>nP2fDI{dukt!)9+L=nBji~U1>X;{WK^xO*`q4du&6&!hdJk$ z5=A*)0H>2=;(_~3rQGO(GU0C@2bgtUt#WNd@Dm~f;1UzANC;DhyHpvV`+lf7;OOyc zm;TqyvU;VB6Ll597UOmD&%b0|_TC@l%l&~c(of`NvOdffsjA2DK zrJ2tKx{Na9cQ)W#sFSfWDY4!tY%|SrI_Yjl1u`tKs&syOW#G{tEUUR{xy>GbclZ>v zIjUtDf3s#qc#gLI&*E5lGK~LGHemU4nWGBIIga^&SP$DuqmrYP<-6p&idFSj6nO|o z;-$Gyp*%-6hSgRqU{zs}#P$UUz@or_M@VKCX0;|hqCn2WTo^ z5QV`avU7o@kD>^jSvvVCg9giklE+W-gaxxvaI3=X_e@QLHYZO^5olK!SO%J&jHt!AX@u z5Q%E3jg(CE)i)sGnX7m?df<c*0#AXsGgd8u$)tC%lP5-t~EiUm1JOIUzH-nfxC+kCt zJg{c0A5ar0UtN9Z>&L`uXZP%>qS~a`A?dthil!fys(vyEjFwA$q22SlyF*m{68US6 zh$`$6Rn0mb(SSHmujG1m)DuHdGwhgn^wVEjZckFk;)(XMVk~IH+At%AH_n}G#B2qr zzIAG|7|vPbuBTO_A95wUucPA$X-UyV9iX+vN%|JYrQXj6*kP#nnT2__Fh_XD4o^ip zFjA8q5acK4%n1iO^A{Q!tO0h-FPG~6DpYUEMS9ZxDapG0nARqju+)f+Gi6G~CN-iT zHSB;GmM2-gE8k`x(st}*kaJnBCDw60L#|DwKPFyaFLHB&Ch(q9(&N!3x{l`?a5mgd zbv%(ZzE61Kg+4b)*I%C4YkbHeEwzRSJ-tQ5WaW}ll8@J4H-aK;bJCnT?Fkn-i#xK- zk3Y3uas+x|5AKvO6YfPMyjauSBtC!1?s7ti&xM_BQcyrNI#1(%`xy1DVtIxxS`0>pTFkmN>r8|4+gT_e=qql^?& za<;ZxbLd+%Yn^6LD$x>Hli~6c;PS~ zGs7q$46WLwVqC|=afs@Ycq4o$!|vkyzA15gAGfoJ6Y>VL89N@*vF~z_-N&8O@JSTv z`vV|%Q=b;8UIyy%s!nZ;zC+wc-aCASVI6Q>bO2F!C-|T%nqNjY?Dmy#c8RJ znQ~K96w#2Auei^k&~Y3q7)^mA3${aM2DV$>?C8~FcgM5O9zMzM zk1Jr(B=hUfIuLQe=ho16?A~0I$WCz3W5%kX#T2k4Ts&>PHP_ +OQ+tFdQWOo)it zph8$&x_+*sLG>RUXW%B*%D3cZdPSr*?%6pPFA(9WuapB-hCfHAQ){qeLw$0b{(9?9 zwC1*U3<6Zwelm`8R-(b*{YCcpa6jRw(xz7;0>Nw~pwdy`m5UVQAzx|{9H6NT9AJh-3S5%e#nDUwKyYrAN&s zQTL5E=i5CgaVNDwivp%sBSiy3&@RoyTJAyc7K^c4dq8}n$Z>3EM><+UcZvy^_SUN< zCCCAN2aDuHm8+`<`YKq_#}o4Xy@jzCQt98YpOLNmAfF!*bR-mF-m+*1mw?Vrj-0G5 zK`hEd6?U$spF>R6Ysp?Ay7_)UA1*J-gsPogNx|={Al(*8ZQz6~9gaAyRv4ogHdzlO zxGisl;0|>;GzC-iA$^oCYfV>!vM~7wtXUmAt2gLG8-e)Tic{B6K@JU?3OuY<{aM>b zR!-uXjIeQSh24EkICU3+*r*p_&HJe~#~+dUZER5FAcOmE09jtE!QgWU+s6Tr#hIM< z9)O_lG)IoR>5(K+NbmLYG_>%1{FwHm1-w~dtW1|Z9JI6+i%S!M5vpV-b4h`58`IIs zUsyH}w{Krg)-{s{AL5Z_T&E8hwbdy_NOPC`2oq1bzhS~CXL=F`lX2lB%Dxi>!9V65 zkqralcHHs_zLt^&ZKBhjcq;n`@0BprQbcy6YpXA?b27b7O8 zW*!-2i``%5|8k1*?3%qm=GDKmuchuswR!lj;-C9eU zHQW=G+xa#~WU7&k+CDG*m$EV(ZrtKSN4IHXwgje1D66NyML-AB>Hyp4n)~{%7k{l4 zr@r{$D55bA--zk)kkfl zU^L%ZV*9@uUWHN_IH2{0@a;j@vJqo_fLQirldzpk`*VaqZBr=^^0X7(F+O}Mz*!nh$lR=%m!lKp~AQqh|zne~W4`cb|Lo7+;XB*2t1f zKaVIhnv2*_wF6Q&b_mc?gOZI90Jy81sOaY_`=#-t7a+57#D$bUX(`4e7!^stxHVr*xYloU7}6`Q zpDwgaMCOQW;E$LkKyBh&#iRH2IfslJf5}c#6m=SbmH>>(SfY&E<&=AgGP^XShqATU z?@TUSeG9H0DwdAMR4Pax-s$L@QA763UyTD@G6c)#akWA78oeAk=uP_9@p^k0-fo@O z;?Lly-z)4)Y9m4$P|3B$t)Ob(jX)f@-}DhYY_^SLR|VeB_~AZacW}X5OzZYbukFRT zx;K?NoquT1M$DzZCEGYD+4-cYysC&XZdiGJ5gM?dB!#FIw*?no`#5>Xob;uNp^p5v z+?p|w`nQ$Wxgl2O)kmF64|#ZF|@rC!@Bav+>p>3dvfunQY zQYXIK)sNz4xQ69AY~Dd0!cxm55oywdeR(T*mj=5nDy$izm9wtQ_SxZ%1GT2jIw}6D z-_6O~h`W*IZ{T;nc+Ong!xRI`IzvNvx2BM>?%P$hf`SclWIwLE z;OO^S7Q@9f4H1Z*Wbhe2TUARBykF?IV@drL3^Q$8$_I$tr}Ve|&T{m3F+Z~$TP$qX z)UAx7GR992xQfz2Y%Is2+@uYhi&s(>M>(-2sC>9pKMoj`7?@~MBOnv@p1^l787tad6?=A&f5glM2$`uP2$yf|^+hVS7IJ<>RA==L_6{i8tA zb-)QrXoJFZl&W`L-w~}nhIbqCz8?mcP47nX*iqV_>}bulh#Cw9jS7j;%{Iull#9;( zJc13w@OD9DevPy#0JoSqF2j8c$YtGLaXyOu=CgVHy3%x{tMf&TWu;~poP5?>Pyv%P z3A@-^{@7Akys+p4?~R@}tf!23VxfP)+-+0(*$V=ZdPi%j7iYxcr}zGiuIMAYgk@0} z6dr#W@xtNU`wJ_$FYe}vw+Ep{&AZlioU=7kZkYGHw+!~-$KBRU5E*Pu&F<$f+Qe{g z#nwMptr`3Gw1z`{<83XRDliLwh|VI^?&Qqk_8ga&OtJ- zdvmd-K(W-hZ|NConF9)8N>nYkT}zheItJ2t63C=ZiUIyc_RllE(viL ztOYNw|FWa75%6Ft7bwz)EP}r)a6y*YYDC(sORIdG|HHug?u>VKWIiSD$}`j_QeHI8 zX!Z}u5l~AL@2MaXy1>m21C;V(-%{v1xf`p@YI83MqVrytK0h@|AgPMQ6ax0mn8Cbu zI+9-S*38T0{asbWbRCH?ve!8MwK#ZYFGU7hYIesm@M2=A0*^{J?yF8W^>>(pH1# zt0E7dwnDLE_MEpp56$QTPf+|RU7ogaY3W>weA8pcIj0gv4{}^LN6r;kfF8F?9 zn`u3<)Q5MfcX;adA)+E@V~lC#Rs4|==p72-(sT}i7DZ~rj*uXQ@+bLahVYgl`=w_j zlCl{!+PdjWRp8fB7}AOn=6gPnIkLrd{v?fGPP^vMEJs_risFfz!~uRLBh|nM zgK)x}gHiU4kjCIoTE}MB6PHB(pRyOvvME_FPT0xtcg!XtEsAK^Uob_96G&nQ`eI4= zIkZ?5(a}vbd97iXaMNRzIZBj8nekj)OSI-MEA5JFbLksykRnUbJAl$@I-z&CB@=?C z?$-t9cr{WhPh>N0JIr6H#U_5PT>)cFaPn9W-T;AC<^%&D`n=k;&RHS2<_QtepbsYS z9pDYGuSc^~utAEXB50GcVwQq?k23G@g;=!681y^*fyw|6ewp5{8fpHi5fy5-;79#G zxBkB~h0;?;;E=dTvd6^oy`xKt2-i2nW!g|8a$qowBo|UoPw)2c9Klsdk}3mye#qih z$qy{6?PUhN_dOsMPL|Nf$jI^`l6h3HHak4E&CSE0=tDIIXr)*%iaXF4%%k2-Tx6vL zRwu{?tnUmiZ!;GdAq+3a4kn4PdY3w6*bp!U8xNi|VVi>AtzZg7@D002$6EQiO*?<*K6pe#JY##?%YHC(8em zDA5PB<7fQ!UOn|cg%lqBTuK1<7Lmz++k^d?>NR3mbxEQ>B z2sXbks?)#K8-^=smbwR%%)|U!IN}~#4>oqv6TNUL!aqtW9sQ#r;lF=e-ld#J{*NFT z26rwi`K7Ij3k9f9r2q7Hc4Q}i%Ks7u**E{AI2t7^$G@^6H(Q9h4$z>RQk3i_GzJb7 z8JiZFiLA{y4$@B-o@T52=Sjb}$gX9Q3&Qy3X25H;$t7pQE%6MI?Mi(#Y^6t|NEqL?&_nSTC+xzB;|b4!2T%c$cC9 zps*Ypjq}=j@pLGf{6PNPNnKbn>;1q{1=9W?JVNxaH&<4B*!(45&$fq~uffVbygK6P zZ-LH>R6$!=r8nBI*Gm}2bGL)s=pdrlv6p_G`mN0vS4YSDX_*S^1&Pbhrs ztoLuJ*7l`(1s^;+J?gE@M7ufr0~C*RZ0xFJU{RlzP$E%9--7UdEY)C@A zI)_~}UBsslS>!wYm>&JsV9|(k$P*7DZPI_**ReLRiwFltWHV}$Q7xBR+%Q<7Ic`@S zLnV>4df%r^3<9oUo@0g#3x|Y>31-=IVY+%B@jEi77tapGjcbL9S@;J+ImC>5t6cIF zx<5tPpBKh|NDaD^SGiV@P;=CD&c0kgbAdSJ2QKk-j6ry4xKUA8T zGqS}o-$`t}0!7Zr8zPg(cPnWe5imQoPl?S;!HTiS_OGv{b`Ux88oN~=9^3z+l3Mar@gK}ch|&DQ#`6P)P(NO!KvIf%9qwK zj@?yf*R6@gkKWJCnQq*NZv=7o%k9RwgfDJ{sEI!e%>0M{&O|-V&ixIzeW!!W+xAT^g;V{wWD9wDWhHk8+*a@`V z@^!grazK?4hM|#t$%+4JbK+@875f+(?z6pVXBR%0`qc(Ae%^3s0MMi@d7;o6 zjD%!x6EJHlF{@ZlxvslU5I;vrnruF7DLL&{m0Z8M@1Rj)tniMgY%P#K|J~n-<>^Nz z%iv2kzH*_mPgFNPgtS{;L}}-`6W23v)W|C2%-%m%UlI5vO=up5~I^Ll$xKBs}(S6xxeIL5P+Zq$kbe_ZZO!#Xl zPeGi=yr^@}DE&j;lyQ1J!J(pv^i!j+98j+l7H5kJBO;eHFc7km-x5}RPGh7G{PLnM zoO|>pA8y)j&qiPL`Q2;ewZy2_XD0&;T275VIy;+yd4W*dt0Lu}sgKS>h~v)sG2I6I zD9@A62j>&2HM`B&bXyxKrPPXSH=(C6x5hXy!3I5+s<+$zc;gKmG+0~mlkE5l5Gd9t zJQ2x$!~w`d-p^Ju8%+~)r|{$nQL)-lQ+32x)t#WFBlc!?t?q$we}h+Q{$X!pQB7th zo!2?u41bbTCh%wOhy}I?QwJ&HMECAUG!Z`djuUYl%aL%MrYw=!Q3{#* z*>8VCUu7_f(E3PvosVmFit2gtG1a@x__?3NX9SK>f@KLZDN?R_lNqZG?P3}JM*?;4 zs><+9$;;C7L=r*q}DS}gUa6DboN$oap5(GiwE8cy>6aXCGxvl=S~EvdUvg7esgX+uTZp5 zX6|TMokU5`em0qgGP|efq3d5$UasYR=`nC=H>N}cT(h{_USK2t@)##Yn$rjUglq10^3C1`(0*8tq76}J;;_%yk zBEuSWqrC0cQ9fo1jc*hBwBo^`|ICZXL+m$Exv^7vr(1PyLX1%N2l}T;Uo3W-`g3qK zuM+4!ErGwbW0$=dx#8M{_eojz@f9J)I5~!vV8`Lg5(g3gMJ`%jc7Pq+m0)e_PDEg{ zVV+3P5-y2`zig1Uz6}gY>{GL|?)qW8aBVFq_?ahj(N5(=nkozSt#*kw<;^E9-SD@1 z8-x1)ri=|a&)9FE0DK)We}z)AL}hWc*a&#T2#Us35NYP0!p_E?orAP;zwF1F{tD6S zS75WXVkqP!KhQwJmmmedU8T4Z3HLUa8#LGtkNVl2(R}E^Yu0SF(S!D*90K3ajdy_b?fWRmdGw)cSikE&O2@T%sATTJ15tP9~ky)Q$KL0UZT{W+hlA(t_BN4L3ggAeg8>_S$tKd(Ads0wb5fSbT<{YtIklf_khL%|uI=p?c$1yD5%ufJeZO!LP z0hppimHt9zGx~g3Zpgvi?8wv;i`G*Rm{%_e(pERQ*fAofp!-s$Dounl_G#zb^JD)L z@?fCN&yGTBKhM;ts_cZQexgF)wWpYFW@#E^P0`@v)=6pd*m@%7m%qH9Eo3-MK7f~n z5VooeAB5T2Y*!isk1~*7FDQCm#+h*FyA3Dx)VHRaua z817R{LH{Z*=7xsTYzih>wCtBMm(gS?5YxtO=UA9hxfJ~#A{{|;&Nwxk$|G`ku&=_Y zl(}HXY6dgXS|dEpH;i#X-5LcuC9T4141<;E??w}(;ms!d76v;f_7HVz{sJOy61!H- zWF%tadhrj#Jbn}+wD#gTzbt5u8-eXTdBsO_yy1s6ls1uTEM>sFs+h5d1k1nE$c#VThri}^rK8W(5IY~bld$tP~T8p-dX8ZZ*osG9#s>rI<_}8zFSWsmZ~;Jyy@Uq zVy#*{g*gr>niR04%7S%j{O|;RK<~oWO}|| zdvr+mAC@m9AzOKMn@Ppo&KKpD&uOE4PE8rJPpXo>oPIg!_eyc!tfCo_#{R#^$#^HM z1X^bq4hGo6>(TSRf(v^&6i-QQ{^hU4)+@@4)&s9bRVD_U{}Tu~T!}S~ND~EIWI6Ev zMFoP0)S76ZW^E>(w;>TvS-{7s*I2GZDzgydsrE`duvdTQr$H0_g7uXm@xq8)qkgsj zv7=+?6Nm;QVn*qf>>vVZCg4=rQIJSloSCI^johqSk}QF>9?p79UBPdZrDDcHgzB}k z>tnMQI%eH2^dDACoL?p{hFqF?UE%wp`VLaX`QEcq`i;dy#moM^Y4GEP>#W`1h}FfxpwEBb6v)yK>uCn%b|EbcZ*oE5pZE zgf7+H_4AK3&wdDEGwme&`3jQT6FmcLvP42_e77F?l_4RD4LVm+o1JiXYkh6-LBX;C+q95dfZ4JC$H#i$y!) zOi5%=tv(DHgC{2w!*$)m^UDCSv1wBVOZr}>Lvl3tE1|V!y`_&2*TJskml>b5v zM^-2OdO9GwX*^7jUST|dJqXF*h)VFY4Vj2 zk6xiGzu8!=FO|ISrsTuFYYjFzx7EqIp@51P?!9UtJCyLh{uGN_FU$L2=e64j3{hiFjwjX4ZiN~U}rk`a{%l`!*62T|L{P*8? zWyNPZgopE?^0nV^Km8-!OG0W_^@A*ueByrTCqe}S;DRU z)D>Bju7RLee>B?1p$EfAvlH)%O-3O$Soj4jH&B&pqXv2Gr{7*VsYQQnL3+iQXiJCs(#a9^hwN5coNLWitPR* zkSYD(bqc$aB8ej4KASFuEBoN3FV6$={P zEBVHfI8CwRl?!!-BsXI(WdM5{8e5sCOp(_U5>nANf&bHF9-(jq7%F|&Vo+#CE54Fq zs@rxGk!Gf7kd6?YkYf-OHynn_3X&N@`H^`AC9$|}L^e58wpp`tj`1_gR;Y7fGuiX1 z-jiZmMuw3ns)@xT1Or<8Xr88wMBo!Olc|GKmbSK0?+0sKx=}r)DeG@Mi7-;!X(6{y z3Z&+_VF|m0GjcRNOgwE;?OD?~&NpMWYaGnsZK0$~AEd6?Q2meb&!fiE8EkL=rP9P! z>I7MS{-@Jimy=5ef+-}~{f612%R!_;>^?Tr_Dne+D_33f$F_4f&w8tued%98Nrf$a zZjZU;o+}Gqev#~RP2k>A?vxkvJyrI2&w@g;mFLr+xd*>r5^RUMO3RgD@+2ZY@F-O8uPJdmnczuiA8|accxbV1z0I0J z4bl@YBxxzPHQklO8yori>aI88*DxKPtmkT>SM2eiOo2vFQB$;wD;Y{q{ts(k6;($N zY#Rs;2X~iX!QCM^1ef6M?ykWP?ry<@TW~+Py9IZGyURPtyZ5g9?C<^cnl;m1-91xN zHG5BOLp^_SX>lO@3qHoja!ja3z23>nkW_|hDB2OnyUPzE`Dxx&!A*P*!TqHq{(5L?*$=IiUr_RoKM_i9OPS`XO#8c(9ljV=b7%||nswzhEPCUIK zx966P0N)*aYok&ea=CF9($sEK(l5#F9edKO>m(jT%Qp?X1{FOXTLiaaMvEDpqtU^tKLWvNHG9;lX!Gd2FpVPQZcKIFkYPv@WYB2m^lwb_2-9^$$s$4 zKHChNh5k0m(@%>Zl>YRHFl~6fdy5gdETjV=h7D&pqRAW6^==n=K*qlffJ;Blq3*gz z3=DL5)8Ter`wt?Ed}xyxo5~*dFKy2cUEO8SJz?}qe zAPuH%agJ?`mfzsc6~d*)UUwh{(D`mo=L#&seB8MspPWKOyHt0O;fILL1S?W|gwO{> z(QA#heq4901=R>cxDK%*n6oY#z^xdvIKss-Lw3p%-wEn(jkTibXzEeWJ1!eB%2O8Y z#6Zf6P>o1be9jclvf1HZw%+h{8eKF>K zRHwrJ{d-Qk7iwAq%m^Z7PwCO%wT_!<6olcvuUN%2Cs$jKtJd56akG~fbSe10*%bp= z5)yC5o#gucmx=M~#|O{H6J@9ly`Zh()8W34kBg04$B7O`{RVcv*H>~^#6goarmYS9 z?*j+!$9) z`qWXC?~t{OI_pIJr_wF=&00KcpD_X0iHmwW>PWxn&dPr!y)(q845^Il?wx#ZJ4f!z zlI~f1F}lg=3@y!^1wIC20`eb!fFXz}jo$^j$?pzeipIC!IA4BHa?#kvBF}edAVp83 zIB9=$M3x_sgXQ#=AK2F#ZT+-E}YI7Yjg-mj1M+(NN||Q|%xOsKqLw5ua1yjhr%L z&&MI@?*rgCSS=#kZPhr7Y?3pL<=VvVyV;RzF@asOn05Y5zr6c(df4=ut73jqarp)^ z&F9_^Dr*tH0!LGmqOepPk4#_&)c@SsYxyZeSXpLS^V-jSKZW2Sdjq7(XMv_EQc=;+ zM`;xu+ZtI$HT$T2!H#v;t&vKCW1I@{I&-p9SgQ~?Ho*S00SL9j3i#dg!F1}+HC0Lb zS~*+ll|)iu(EmaC&@oZwSMwO$IqW!hG?F&1G=|If1BXI$_|ke*brAr=wd~H?3?j?$ zRZ*6t8xt!NF3_ofEZYb$(%q>&Jq2)J#HdjI4+k;g967kJ zqm_{kTnn@B{iV`|LlZDryZM{P$Df->K2b1u(T2b*J)$xJGHa!tNIufY@rZlFTSUG_ z%=rsgP7lAe^J{z!8|rx*@0ZD6TmzDy%C$(x#Y_UG+k>F^yLf{GBZyFHULzY`KAV%4 z{pCu)Ou3(u?b4dY6TW`TDhAhwIM;7}z^_0@t+#5>QA&?kftIWdp`*aUpni+8VP_B; z?HWF1bz>MjqZ7v(G#!*w^Jtz6|AeBAV+%q*Z&}K%I-Hpf4F5)Q<68DJSYmX>`azLV zfW_hCou0~&a*WFHBH5E`UJM z``(SkCXlV0a3A3fcFosrVp$}mzN`xot^h%^Fqq2&D%;+6hQ$D?UGs;hwk2|RK=vs& zzVk+TeZsTexyT%GC#g)uju??=ZVGPysls6PDfr|w{vY_43G9ZbnbwF;MZCy2s3)^QGd zU|nsM?{3)7aFE47Jsc_UJrdUb(Y{uj5nw^t3JI>+{tmS&_(Hp}G%#p1oG%A7=$z?L zmxb1UcCv@r5#~;8Mq9V3sV+3us6Et%8VU+w03Z&!+krDn2E^lmD~ZqvI!;iNkUjn9 z8r+!N##ZNJ$5Ho77IOSVk}$#jSCrs4b%6v9^;m~HuK?P(I8m;CfZ>Zd6~1(^{y$zi6y ze%OPm@l@kA%LiVa^ez>$;I@zHje0pB7&m%Xy1C{0A6J|W7=Bh0QYN)4c|{XL_xr1^ zPTNS~78P%=lSF||{yvPKFFj)ZxL`UxLKOe2n45J*vp{lt6wP z!8_7nHIw%ir?AY)6sDggNt3^f%_T~yT9Q(m-UpoeVJ%pF_{3;%miq-5lV3bYj!7LN zvf)Rz%9S<1gb2$VvS+XpCQwV`MmT{bVJOAH7K1}u-s0gqW#F?Zh(wn%uAGiE2~8l+ ze@f=e1kHS81moP>TEQd}pPdsF);|{Sb!up3htP{tb~bR3nwlQ)9CG4FG}T+Cl9;cJ z&3sV2D;U685N&yk_oF1*lHq`hH1ZeRFxT}DTLVbzNVl=J(j?D|TT=8CB3gd;Nyi6hh6YxOeeR7h@weHIKE~vxi>n^k}X1^gc1AD zY!&W!1=|2vfJ5cOjYrp|QJQcSo+9$IQ-EOm7e*S}4}lv%D`==26O0owoBMc2t+}K> zhDi_NFWya9gNl3maiJHwD>qMTiUWE7JCA#R;hpH<&X@}&I z0>TdxQdLW5ZlhwHKnN7mcPJ5{Zts2P{74AKvVU3f!6;eUgrbWv5cU=|;VCvCjf+N{ z>|QUSokVh;7H@Tx6kK-7Q}6q^fChqnia)cWu?|A~w#jBi^!C5|w5yt<=K!!ydoHq{ z!ns|Ys zW*?V-NM#U?&67`bun&X8A2Te#dZRHX*NN)d{6jw)zT5V$;rS>c_y_MOCliaG`lS{Q zA%_KADomnjrUw4j$>5*8%?eURLO=09K6R|3^0Fh{=L!UzXfh&KFp61Jtmf(%gm6$N z)PCoC5{y_zMm!F0R(^neh#BOgNkUOI%S954HAK9;7g8aklM_#CA~Fi;_Nn;v}+@&g6-eabRPS$+m>8jg2*2485@!`#XKUdUC?Z=zBl zBpvSwxTV`Vy{A|jcnva~RXhZpf9-O|u@1c(cqyrmzrA6pn+WaB;W7I?m9vyM7AvS9 zt%3Buq~H+N;;){wCc1=4S_^e3=mcGN_v|nTL|jI*g5c*Q?&$~k2)WpS5iW7BgG{K3 zeG}ZUkBf=%w~08s3o6*c?9}8dO*Ki{k%O5jF&A2`fJig7-)*-`c{o^^CVcg=lY|Hh(9mi7B5W9-gPy&g_m0qf^Ez_5Fp8%Vk{~hdteffWa zU5K_dV(T=aZ$8bS?{qu6dY{NY1S2m2{SzzVx$Js<&`v#mMs3Tb<>HJvrUNV;(^Fl@=o#m@I4aJI0$K{|HU&4Wo9!cvHJ6oVY z`GWKDbqBRU+Y3-Wd!T1+kCVQwuHmy-^?Hhdw;0v3d%wac=whud_HtI_ai_R45~mnM zP!v!Hmyd^rt7W{^YCw-fAEE(*nJ@}du~-Tq#TPmC;_NF;?FNjxD<9Q*$QLSi3K@M} zqSR^ek?$nzVPa8gcG}K8T$~#G)eVCPqH-)>Uy&D8c0jwvINfCXTK_Udu=Awl$y7+Q zUFHf#&xF*Ks=IQ5ASUVw!^BhuchxRWA%i9^Aq5^jn|W}ZViEMKO#O#c9c*C-I2Vsh zfHFH=zS7U^Ix_a70rvnQ8ZjkNSgv6Hq3+4r61q@fOsV=wihUQeAkCZWw8@nK8@({- zP8r-vW4utS2e_|0z`u}}6&5^;<51j(37zrSJFsE!;ZxYyr0wTl2!~)= z`MkDM%}bYfLj**|@UB18(3h@y=+I0`HPxXMt$S51EjQQAAxq=$M-b+5uvLYU>kwTw za9gkpP5>0FYgx5$_JEp4f%4e7JNe4}Qr9Y|{IfWx466pl?_~8p9}q{Ur-)jS2zHdM zpy1*noE>@3;2Lei`y|Yk2~vS=4IG`(`APA^scn;yccycWvMhHKStEpOSl%kT(Je2; zD1^KXBip- zHx5hus-@Btib9X)s6Drvnct0Mcr&X(JVFR;rp zp157x=ghkfXxB&|Y@nFpc>W-t1973-2``go+Ipa)!5V!&z=o0|TIOW3u{A0`S-p~6 zb_h=pN00T0mBH>Gi;<{YPG7RkW3KMbW8l=mS8Fp4(6%Vgj*kajW8S!~hO|v8VFa7y`;EZNVn)Ak;%V<0 zU-VpeY`q$@G}>L1x@EzU*e?y<>%6ohkjP&2Y}}FbxXT8!SF z!p053Z!8jf{$4IED*2_{X6>r){S-^~Sb_eHeP@X7%%d^>-pBRx@7E$OcB2EIEn^Me zB;fek@Lq$X1T7E3jmwwU<3BwpWFnnMie5e|xn0@w(LeX$>E3?5DBr-n!5t^a&$%H9 z;M`fHkh21^Af`B2W*#W?-r4Dc+@-?2MwQJmPjzGjm6-~nyp-DBQz3&HjH$zq;k*7M z3(COrHIm2ZI%b4iH_2}+n=s#G%+~p(ffzMd8hWh2qnkd~IcEF!|a;MQ0?AQ(7SPl+$sRzuqx`GjSXngqiki%PPUCl>^ zBGz0lS0@mCMx0ZjK*XA$dtzh5WVpQ03k`3mbH)h>KADEU*jof#BJq20lqSBIPZ%QB zOMfuLxJy9w2N41ruREyBqouWJJ90lJqUaNTmIJ5N={^lw#RNE&dqG2NT4(!DG>KpH z23JM*UMbWi^E|@Sym`XKUXJTdmJ>!`6I#sc8X7S91wV+R;QWYc*@%$JyGaM9AZZw=8ZtHUH;>_uwK+UvWxK6An#5XWfp1 zZm%xO%5i?QV2nK|p0171^Y2&iQG&zub|(UNz?h+(p@yS9K}zN@#Vau}=tD>4O}yJ1 zzkJCD3PG3eE&PdrH<(%NaS)A<>)u^FFf2UPhVkh*oNOyOnUe;f)~#=PPbq8k26n5Z>skGHiw}x_ z(jxjXY|CEk)9*=wa*gb}K>`Tj1PNrw4x9TbwZvS=IY5!V7n%n-d?&P@hyw^?8%RDFvNxH?a0T|+S z4-y(obdL|4Vk-5|DDz!kKwqVT3B?WP3}*V?y~kwlOMT3bUT@MQHy$V4J2&p{`HVZy z&kvqDS2!cq?#>#H0xGVqvbtdSs3WFF-LmSMvpX7Ixc9}l;1x^?niCb5xBa3w!T`Nd z9%g?21X-JBrba<*xpv~l(o}cQ4BbC|1ab*E_v|}IZtf}0&}3F$ZgBWKwR@IXzn!lj zzA3DRJa@xF+vgfURbP;7KHwf2sNi@M+oedHszC7q@P(*sQN)K|Uf zYUIN?%tB*9We{TrSm@z&5NilX{AxL6EeFjM(6WfT^~JWZ27{q$--|@g?CG6rS=T3F z_)ZP3ncr?wXVaxZpG0eTH-asnm9& zrpZvb7R?3^X$sSE!^lQ~FZ1C8V|$TkzAh|gP7Q@P%&i4N{)tW7L+3lB?%Gf57PDeP zlZa0>mV<+B#!nV=hUb25*My8~eucy`z0(EM?V-299ZEz%?HiG!@eZlm`bpi8Z7jx9 zqyVPHn<2vJkXj+uL%_OLxIl4b63?#&#n{#sy?_bm6u*#~5Ftkuc|W=urWy*N6+1%Z%wy2Y zP6H6}2_y9-Yq}G(LX-2dOu@fbJg^13VW$JRzHW$Lb@X*>(-~K(q$rJZ9Rpxms{#6i z;2R1ENeyDexgV$-7=Ig7OJU!oyEH)0o(x(#LdH1(YKM}ISRpKad^Y6#MPeqV*S#M! z?7(35N#U`>@B!013)H?KR7_0u#NVzu8jThD2DozVp!rLGSxHTNKd=lFAuQ zJ+kchcoLmlU}`u{m)`u@|H67I;?7s zkoUQ0UmC(%9zk9W71uG9uBbhf>!j3 zkZ=_g6PJA9a26KMcg+QZm9CKo;6wBM=x#ZdiLY=i!`V%>?`g09Ml5R9D07-2m29>C z;R)I-{5Dw~7|I*cb(+sRLMJA~P#>8#A8NkW2E8@P5k&G(n;xq8;3qbsTFlSc$E>)Hw^=+ljQ2NeqgHH{?dsoWL5{^B9tA-V!~2*?^kt z&#bMEa*T+kpR!(3Jga?KT$IgEjgV2It{L)XSp4o1bk8e`kDFDZDw*scpyHlK$&sq| zi6B^2pAu;#w`T*gc<8+_rkI&N37&55M2(z0a%#@#YFyGEW^RbS%?V+m(@LdUKs| z*j;Q%coxP*2Pz8w%fnIU(I+u_mOz0-k1?$^G~&wryz7%*-~%7=V)ogN$Evf>;W8}I zoPfH^bfK-nOFIQJT5o#rCdhTjK)tpssABu8_{egn1nnM(*3>>>z(V7!yhMFz{}~`* z54n*0jAl%fG#nv?Q@k&i-e-*Ib^D6?v2uV6a=JP02H}(lp|~;FLX-o6Y+iGhfuF{q zQK8K>GPkBwzu81CzBd*IpCshC6k0E_WDC`O{uuJW%$2S)8eOV7NAzMww;9RWBbp?{ zwM~tJ&Q~Zu_4;FfF1J_1mxidH+m1K}e?VN@V((RHEPgb_kNbi?Lf{`9N?ENnUY_T} zWdddhYYly?h(^NY1|93uw%_(oGiXX&9L(Rn9_00dL&6Zw;Jk0EZBUe&s?2nThZAc} zKIkLtvX_`c55UiaXzDgv9YKO(v7EhO3kDq2I1k5D`)M&p;;nBt9tG9Ko|Wo^fnOd- zwwYxS_|y3?8*lKjkq)$BCkTIcI*^mH41Tct^`QM=wyf_NN;WAxmcj7Sts0ZP`Qrqj zlxLf?+4E;Djm@2MRV`bsTHzC?$cnu9Mi#ToH$T0J=13zG%gOr)AX4ws5EuB z2JliAvgXn0a*gAY-01@+N_^}Ey}}hGYm+#&-DiA~sP%|K$Wzwc(nZu84t?F-iBJ3t zL=SA%Vnf8c9UHmiulSwMV#_PVx`=Php9{HbpGg}}`NQ%D5G(kZE_Ckj5gx6Fmcvf4 z&U8C~f6jWFSvZ%(?Dm4uzh*^zgXPRldgk+H4~!L{zj(6F+GvLl(0TUdYG$8cn2)2> zS3yo%z;2x5{WUNP+|7h52kqFzx#vHwK51Z@P*mB^osR8gGn&YcRB0 z%_4Y3-525$`{d}TA=HXBBmb(tT=fA?>rbhjs%EV%=H#?9DzgASTza&Zdzis%oZ8UW z=3rN0pY!OgUS5ZJT=yj1xdWe7SpR2+fr)Ory$Eee{mKYW$%?EU$k-o<7p~`Fvwn$F zMe#`QhIf)2hpL;K`QG^sQTRRCJoY)j*dEtEpV*ZEi{y3P?H7(q%-^Lk$rnKrFU;+) zUcFlMqBBC5J{)L+(9W>hy;`hrIdBKR&0Oq=BWazzdZoM&1IWUXM_aa5wnjK{t}w?) zh=%$O0@-ri#H4#?FomQ%HFds++1?xJ1K{3?&%^-=DliK{og&={4vcweYzEA3HzeP; zh#1IxZ$6ldf|3Vwh~C!d-m`yUy_6ihIYO5wN;QZK!{j+~Lglpwx}QYT63x_F&8m$l zy1JhTA*+iH<~atqOC<#fND~SUz83I4?CyvIBawSOZXeV*HM};dWsfVY8Wi7blOlMw zHh7q^a&jxgbUxLe@<(F{9MBIi6+9+Z*ygS~+-~&RnJas9wwOju@-#&m-AJ#Y&14RC>~Gwcs2wnScT<)pEXel?8ruZ- zU(B+bY63LfJ5OW4?&CZ(KZwU$_Oyo04p5AI-4pttt$~$fzWi3;V7Ax1>RkmC+iX&< znfxm3IyIG4N!1F?7BmCP$VG2<0eZ1vUOiWl&1U{URDtlTU}|r){!>3PX-X>Ie$Emi zftipb6W^H6+141v`7tBh?~|a?Agu$QympZ#1iudr5FHt*h6IJJ<+pco$j*k(Z_ku# zep$dR>nEeIN#S1>sg-$lyo#>J~_r@jXWO<^~UL@RYo9{3#{aKnjBe#NDXClnfaTxu?ktC#kO1HfY(`bBY5sgODo+QUPFzPs6A875E*Hy>`oTM zr3X{%Zur}T)MU3OKSd~MR{_&@8~v{(K0HA&e!d!<96_apKNGL4S7q8`sjD9Xx{wIc zL9IaUU3Ch@>K(EyY2ED1U<+Dk=T(dZk{wE}uN`oJXu0XRvWxp~@m}a8zb7QJR#Xd# zakcs!8MR>5nS2C2a64bmR=&n$!O6%{5(Ha7QaB^-D@|~PBN^y!U#yz03?y@4T=cAJ zn`(&W1myI=)Yxl}G>Shle=f6@;@M%0So%pu82y2d7#V`J8gYXfL)k#~VowB1q>>E1geBm-KuxM=QmW?FL z&eaETJU^tHsS5lo=C#*iBabPIv9tyfic!aTU8Qj<1(B01rM>F98MQw+mqdIs*8ctv zJ{NY){$@^QCcP0po&;!4P=LH&uHcq{LGQ%^N%rhv5|&gpMW6)OX8Mcg4bnWnKT$x zxd=SOzoT(r%o0IZe8OW0% zLV-fcr31_2#gQwfQ;><3cZ_)jTuZYYE*u%>iIN3l1|n1{b-M8Ud$ zWc@!(f>874<@JOhOQ;x`%gn(FOTlqQigm&kS%CZoOdf>3RqhzON-ub@kY*tic<%V- zK^B$%3GZQ1>XW4%Fl6{KfBRi9Dp$(!Ie=Xp{wL~ws_ z(_h0&7vsr0Q0~>h%4i{|$_Rgjf3?CZMJ#DpLV`@S|5}+8_|M9SPCrb$N8Qc^$sM zlzn7ot~yj+Oc5!bf9E0CZ*+>gK(2X2quPSI$`kGYsx{v+BmYT{-CEDEYnK2vg0G(X zp(jR{)$E2F9Zhd&zOMCm1)qw&1@Tu&!e8{-P@R>F=H*{M+g2oDPq`(R-svi|V}ZI#fYX zotIJM2m_LVirR1BV&CicXhi2mwsJs;jD8MvO@N!7-B|9MTy1HD%+j9AZibVX3 z7CuyJ6TAV88+7yExE>dz3~JEpp#?X(!fqP@`0r_zjaXK!;eIdJ0nIAIwu{@Xno^zo z*r-XKW7qj1UTJ z#9Noyxb#gOkp5jEz#uZ0VB}U26Dj!=wTHEQnDLAGIVFef9FFQMTsOydqjF7C z1xF0Ztl(t<9Er21yxh`dwYegkReG~1R2kKoDpZ*7^Lg5L&l(3Do>_#zpu@{%HunC~ zH+Hbn^xvOHYM%mmCCSji6fD~}eeQ&;g+JlD2^{=B5?R_ep5FLnL*Zu+BQ3#y^uDmH zJ?)Wwjjwy^y*x3^yd+HJ8&3eP<$5cpYH!{H0+f=%3Y$m9N(isZ> z>>pz$r4%?ycX4TjiRV-lO}&VuUvQN3m7 zfh_*s{HP760#7>KTw&xj7Qqy#IFRPXY}|mhaE3C^2f)O2Ozq-bGqx}KN{~Ctb^aEc zjey|q8i>)7n*$?_^I5GI(eiZaOIle zzWsR>j^D=xu|T_)FD}9}`(*3diJ!RBCnn5XbTe4WXL%S#8)IY^vhil8`|I!e59>Yg z8ZJkSjPKWwO?P8oR4-CB@De9F7!`g4b{2XayhpJ4eXk+y^{EUVz02yze#icM7F)9j zBM+Ez9d-5g+~0fL_Iw=Ij3PO%q{9&c?%nR7-f#vQxFEdq?ROgB!pv)>ML)GpM1II< z-CFF;WDfKVnr&)+_M(uuz3M)hFQ3ZJ)}WJi|6d_$CLMfeK{M{y0KE)lA~9CwV|+HZ7MW_+^ftnNa{yA4w}I!YqWW z!FG!Q%qbF?Tg9SySjb&!u0H7y8gU|Ax{6M%8(h6g+4m`p6SpQfJMVW!>87IZdiI5H zM6iC|7WTnPSweW%YR(^XjLV4rH&r9=Ja)F)ePW~0l%WnFqqa-{^&+FAztD5Vx11Jo zANCs=zpE=b*iSV_8^k5~LDJc$xPmrnPxJR_!UO#_Z{R&7Fw}YHB z%_qtIY4Tw;!}aEC^tTR4%`$kN7(2ZYxknHGm>l2q!9fbgpN9TOhiAaXl&7L9m7Na`U<$ebJ;CXMgCL3Lqm8UY?Ir;z!1;fzD_7r$#`m?wY>vLVl8S0RF0r z&bW-Izgse3=$(fvBfDuaug4=9xK-aoQzClNfA8ZB1Qo(%E}&5~QG%vYsyTrOLJ`0C zWrj%l9M%8GXA>GIN?-2FeV%#48LspXPL=FDYr)fO!JD~8GQIJF9ub_I1VdxJO|KVt zwNUC;CpI&J{w^>4$h}5oNIkAj-Bu%^*E!y`oy*cY~Qxe{maiA?$l%Ry!NL8J0n7<%Ap1 z{Rr0NLoHvy8DrSZa8@|h^NCt5)jWg!q_Hg%8pFDihKh5f?q_z5(nDinUqk==; ztjFuP)wNZe2dz_L+qWey+G4uzzlvs%`Q$W?m9|XR6{|SY6d8NK8}fGcmvED6ZJlBG;FKVk(*{ z75cW59a%HYcnJE844gKd$01bSDWvr+QM7v}3N6|~k2?g}1Oru*QSUqC2Ak4AF%UHZ z0;WY?bsX7gIwTCZCIj@obAe}s*jOv#d4Whuj5z|lZ1mTSa^=_{9hh{HzP_J6Aq*Tw zxslEkrjod(=QQO^EP-N$-?Kb*Wpx2k;eB%iA*lJ9@;X9M6P$LGRQUAar93Bc`nH$>~vlt#L+P@ZyWQIya@Zy zcUKam#y(*}H<>4ydnZ}M)#X`Xf?b9O*<@b`pZRp9t36mM_#-qc+fvsww|_m%`fN^| z)4!mU#Ry$t^K^yl@Lv?3B%1?VKBZ?7i16oK3+afJ3jTmK6Knzd`2$6;MN}fZx{3Tj1s`*rXCGS zLSlfZmQRiQE4(NXaeyqO}BIj=Gkl=dG5eXB9 zD<%IM!K}Bbzz<%$6eVTK#|w6kOD_J+oaS+Fk*I^)-OFR4cGeSD9%@49+$kikpRgS) z*QF0@;K;rO|FtY|$TKmhzA3-<`aD`{)F-&QN#To!OU&0~W52m^1 zl4)OR9mc_tCI8D~B+~aDKe~vu{WK$%u{dGz0{xU#7Fb$JOC_5{3bx>HfLlhy zIsNcwFxOxW1lFdPs$JhE(Lv({TG0x;;>pU6gOeNlkOY7-Lv8rLW1214$RTeWOu~qc zt&(8mpg14!8~MM|a8wbvQsuvvz>yVx{kx-N;5q)i1^zQn|NnG?OQfgBoKsxN6PyAY z)|6rH5djD^0%w%hOCF5MAPLUem}jbT`ugPUn45@@U=Ivyj7ch=@^VbD7l>uBdHGK% z6hLC(bVpIq-=lo1q(c3v_ghS$xA#o-`?n4jA7JLyyu|}DZfJna4pd|{Ykzg_U~W5;`t2|uZ0wQ{ z?XqHx{~nV=3SVcfM=8y-mY#sz)|(ma$R!n^U?|DSpTu+@ju4&<1WrijEb7X=H<`WY z-V5J9>2zX*7`N>kt_&FL7M>5DJ$AIDlJCpU9hD!8tgAzp_0pn;uG?te&Cb# zRQsXN;fN-(PG?mwY*nKU_xy+2ukKy7@C+Yeww)<5n%QB1zCzFow-YQ!K5?9ed063d z$$>tR)$^y>#Y7kuU~4no+rS}v&3>Ucl(VUJg@~)o6;is5CQsNE=*{$<3fXq5s1olw z_naevad5|Jn}R?2Pj^uX!Efz9WsA9PLPSor@EK{lq;Il>jO(5Vhqwhn-rh@@huFVk z@|7D8ve`rps7V}=K>Rrs6VsJ4YeY)p zun%xMtuRy_(1-K1LeHPv7>x8X(#Tl(5#&mgwx=yZGH#Oz!fhSOv>uvGMDyMLd^TK{ zZq945_ZzRuO`^z-Y0r(76?2RaNjBY=&R+yH;LZG*m^OWRij6ryM%{dWiC|Y-A|)eg z={kJOe4wvb1?SF|Xv=1?21qilK6c$gVF0))glh2ACo6@&_}@K3N4dWuQ$K({9v-Iu zTtpkBZ*#des<_)(3Cx1i9HqpU7acIXHuIPLGs$$o_lCOhL-|u+$W@0;{+58yhiU2s zQVSq}i^-weAb8C$+n#Uy7D0y(=;CxBQ-KPyF*oynd(H7ntnnN?+MvVvPcA^%D!c&I zkbbtsm`*3|v?PvkF+rD50pAmagcgXUJxd$Ve>Qq{(GO=}Oz(0P;^cpHDd0Q9WR!5_ ztAfoWandrr)Zqhn&u>(P;zz4$+JC-6xsO|h9{rujCf_jNDsJFRnHoRG?~NOy9{D&q zcQ|Glqc)MJ_^R*qNUYcU4MoQWmWz3qjE{mB)2~exAi!3;jrc9U#}7xuk?O_T!>O7m zbfujJ)6Q}`@CSg-!RGoCBLp;vg=+WnJG!%MYdL!KTS3k9A@6=#)V3Zk8ai;YT)K@+ znN4XJrnFWJhRP>ANL!no$@pDud7eTi8Nqvo+k*|aW9;$K*)+rN*)CT^9j`uSC(HpW$lI^Amr^ac>MkR$~VXHQR95yCLEL7QL_ zMgMo^!~9q8Fw5pLx{#nrL$U16SW9HLqdmYamef)UF3Cf%^W;a+y-`(|rcybhFx4Hs z=CB-(oL-a=WSirqn*{;XPKh1vFZd;o8$lR1uVJRMCYWP0nph?bnM^MQ1CA_dmn49| z@;M0Utj(a8u#F^Xu!`eOw>p-$vJ;k*XdpS)>uMN_dh-4Q5!yVV#bGD503H?R8Qtrs*9BMV*fU>R?{q1 zm%IY#|FEHW!j}m2{!?urM$fpn*#Fc9yyQ~y178K3(jp%$NeeJ4G z8`jr6%xEz51O3AiCVCizhtoT%@~aJ5;#Wr|7Wjk7tlpH4+DEmLC&uQ<^iug5ZnY}l z+T};ibKFFxaE1o~)IW>_UB5LFQxtwPhkdsAV zC=v8DF*(Sz*QoKLa32?+m>+)btHj8_Iqb;u!*NgW;)+GLrz>d$^^`gA!7PL`?~izg zs%+gsf~gY&Ik$>k1}GR57{7$y+&+7TpFSBhZIB^pxj|5KhB%?b6c7X!d-s^9VO&+S zHrb9|9zqQXjE@K{W$^0#unXXdtkg0dV*cixs{U{ww8HHAv|zMj{)eUuF|-((Vp`IB zGRLCY=kCj6)6zMF_EYaC5v36-w(?Mv-ZFBW_V0&n=?)xu1NmalaH%$P`~J$#R-0H2 z#nZEBqB*+Jt*$BYg{2tkUBm$nS)He_-h6B1sWO7iXucgNY}BAq?$hx}YjeHoi@+m# z(zZ1Ae8>nIS0$lI8b)0uh7uJ}Ilp+ZRWQT;r><3KFHdLrj@GgL`n;7gXDoKvx&H3% zc@IG{v}fcj;hX9DCDdV^ zs^UIYAzAuQ0RH`}BBV3X&>yifD!(ii@fnCE^PJHDGRq0^BfC7F=uhnP;+?UeAB=J7 z(Qam5$*l|`>o`3L09~~AUOfy2rwj>&!mx@HC!}KyT`8oklc7%pm)#HIVaz54)Cd5G zJP%-Q$Kya?&N?Hi`6YGU&H)K4+ZE2wBTgF^Y2Q@ zmtZ;Hvm-%1nMZ#qrg!P{vqy3*%d*S&TL#?+n1aI%EVcpf{bymt)!5#YkM0HwHFneG ziYBz>k@=1kl(47TyLS41OvHR&ms)R*cq;fD2<${*d#&4vu5m*+oM$ydm^6$psLS*h>=y)a1Krc?v0@oD*O1lPJzFi%k= z0n%=sLZgIzC+o4tUJM52-&{mwfyEdVIMAI#3CVSzO0A5Dd`)Pr4C)}*WC2?% znxr;ptxpo0dh68%?07vCuym2XbeXRa`Ifv~wsZO!v_oyfTf*zeZ^=8eC#WfV1+-L+ zqQbVCPITME}te~ zE`4#pVbE+Fi}~5t=1N|`JlB8lHAvbdZXfBq?s=}0PjNsjtcN;}U*pm|V4aEd>9?F(flIxpHNsk++Cb@=UP@9V!W=byR^CB9DL6qWmp!t1cG-Iy2(b~?$T=5zIbTT^ z3!3ht%t+?JWBu+7D|mUW`%C|orw-li9ieGDl6n1Efxk%0JiiD^hSMdz+_KR{lxh<5 zGgN+INkLeX{RS(J-*OR?&N!|wGbZe-PV=8;xyY&2RjiM`>ISXn^mDcNp*-;~PQtMJ zz4JgGeIB2{TOF=>ZQRpUjM-EPgOz>7o5tI%Hu_dM(2{b*eWW!}0q-N$3#L4joT&YV zVufx8_G#t7B{l1Q*|I&e|NPnyX$rrdpGo>V%Lrq;k89(&3B&^Y|d+gN%R;Z6r2>S%( zb;DY{3fRW|Pr4UJyWw~vI|FJT9S!5+)4#5eofe_ps z0)x9d1b26L3r=vC;O-84^Za(JcB{7D+InmGkC~pC(|ynF?)yET(?|UE#o?N$Pv||s zgotVwP{#F@p=gCyRCwCN08GfA(w1T6d>o_i&-mjm@DbWzlv0+Yy*p7-|Ce|7#x2Jl zGiF;TdcRJO%y-y`{e-PjOhms#Zy)jfc<^*Fd>wV0tj#FW36wqrm-Q}jM3GfL;inB& zIWQaWTfW28&-In z{OO}o@EoRNI*OugxFeuZ$#ocMN2-=g#>1halgNXgU8G(s45Q zsnhkD>KAubW&AhamInno>*OBGFsn}o{Mw5KQ+5cEx_2sBu6@p#O)bVmZ&Z)0(EdG5 zb|2MkZuge*{xFqQpZw-`5kkwi%||?S@b0Vbf7j&3pIE-r&`V8kor~^a23@9h!HS3A z>2923jY9G5iT9K!`BxTIZZDhhqs(zIeA`~QvbE%5aDrQ9%H{n*6-P3ao^jgJQzfJvY@T+M}3%NN{ zBBWZ46^yWe9xDFJq9Laf-qPR5R6W;X|NB-rP<3)+%$om|byx1~{g1SJ`ivA1*tps3 zVE~(4EN;gAiDr+9Ob+8QSzk~TK8^YOfJ-I%1WWJugyVdX6BFJnGf3&9adxkp84@02 zU#5`rmgi`Lea97-vUMSaD7hbnHmaAXv8FbTz6`m%S1^B+zU}YFf!(UgQS$tqkaKo| z2;!HQ3!^03KcfvK)u#J_fmOpCeXe?l2Ut`>BYu&QB!tP_kh}#2)=IL34`i700b=Qe z8L(Hc+GvZ`6%_b{+@G+n?2n^#LLqsDqc|-H1Gu~9mZ}2{u$OXFD^fca+K$PP((C}7 z)a3X=T@h51nt4+#WvaO#O((fzW81Sej*58k^`B6LY1l+FOhFIK0$sl~hI|;4T~+dU zSy|bKT#ii+qRx|vef)WEBV_n)O?!OMvO1J)KciB@wzV;iyPxh9bz z6pI-d!r!p7nM1Be2SoxuMK5dKIYlro4$0z0Xmg5hvol+nSpT@)&1bo@u4$Hqa@Ll} zx0E%RL7y`{_(?t;q=YdNl5)pS$i?_9@+COJ;Oz4cK?b9D3yoeuRPxU6ZpH=^tQH@# zhm=tE#)8dzmCI?l++mp)CJri^4_8gz|wzZt>|C0<~2%%&ct=B^}T?M7=WBRpt6q55)?>F1@I;hF!Eqc z=iPF<6JhB)bt<(8>rhGH`{>_iTGmzD57Y`2)%txqnqfzzl>;%PK4I;Q_(M)~bU@kh1Y?@fa=WKQ__>Z_ZTcf|$qn z*@p+*vPFxl|2HFDYq}pNmvG)>L$jSh&?Rc6xPQq7krIdm6^PiXWot2vR!XXw!jXjH z4EW-Olr0JffMe#h-A1kY?mXUpom-Uu8r}cwp)DU9k!rDAtk34Ptl89X9H~EWk|PF) z*KC|Liv~9q+aYODnEQ!W&<3SSYpQV;@u?D-OoMkma-u_1q?^kg;9WE=)apphze2H? zd~R&S?U*1W!H(m&vrcmVvqk!nWg5!onrtiOJTzACUW4>@H%tZzs~&`TJCPtUgnvh* z`*FsNm4X=}923U`Tna?~BD^9K(bB%PSCH|X*MORjRq8!0-;yH+9~FS~#12q{UY2}N z4@S&*ylXkOdPGr1rL}u^hjw zJO5QIw@pmV4A(LVG~LX}X0z%E44NL_8?D;Fv+qC&V?`l$z^;Cyz2b=Qmj_G z6ke;O`_cUs9ZNK5L$ya=6U>gCyZTD)C_8LN^K%@>>=Ug%H}7-5~<6Q zpaqpp@P|Vxq|me#%dz<4Eh3I?W%{rE442Iq7r2px#K-$JUG!nlw8PxpG5^Zd(K6W= z0I1Ny)-kABkxi!^C0_!!e3BpohFtJa;>3Yb_aw$lm(9YDUTrM0fbfPoryww_Ey$bJ zN$t0lK!263s_*81UM)Y}DYCB%iY6w6L}1J6 zGGMkzTw@)UV3Gl^g<1B0XMA=5dxs-;X{^#151;E7dBKtAY5gv#9l2KQ7!(@bvpYT% z20r5q{0#kKk^47-sPoE*-gAv-sArxn4cEJ5nG+5K^4U!F4Bdq%;4$x>I`Jec3hz=5 zuuXl<)(X6zr;)I5OOQK{vaw`VwQbh4c!&MhFgKPu`b7iz^W=e>R0Q6~+{tWJ2Gts+ z`h^(Vhg*V^D!XTre@98F?)124&PwqN&vz9#`gNrp32+RnKeoW#bLTYaYWWoYQ|P7o zPiM(3!O>?$OEO1I2>PYox1GiQ-fWh614pI%#?OCDkbtY*m1sGWZIY{Ocf#Qfip;%u zYNqmEG6RT2`tpyhqnA3qz#_XKLK&tWn$FD&m=eNFJ7|Bx!LI*tuSCA3{!#O}qp1wz}WAhYk1YtY%7jqH5(1=au+YEEhx|* z(zp*}QDVAc%%kQ*%cch>qV@YtMlv-g(&Lk7ZEwQ{d>_SWix7E9eZI`= zY8xt!LsI}mp))XLChj7T#YHxphUsr_L+r;f^NbDy=o zH1joA752&TqN9SSoW?eG7CJ;Ou)FVVA}|2BHI zuUmQEQM8s7`uUA~lC4f`BFf6Z@M|E+YwJZg3MDnh6PUSw$oXdS;^+n;V}jWX&e-_t z$I%wW!q2~ihn`f+V^P!<2fgu^9vB!G__17m$3ajf8H!4jinwKlowHQxC(QLLp3=hi zXIuK_%Yhg%hK&Th9gwg%*P^Erw}9_P$o^*B8P%LgSv zV1tMFR~_Udua?bL%3^?CK_xj&86Ar=t;)R$Wi{y+*3lVYxubib7Q^*IZ`{*9b-QcE zaJ}Ehm1s0_A;Y=iTkPH2aTV}taiVo6!{*eP6=b2i4Adm9uz=;TLJ>bM{v_xIr|IS}wp`IH5-2ylJ(WwPN@E>+BcS!WF z&nK}cbMb<&wpErvU-)S^fwfE|DEC4YMNAx+3#go*ksB^9qGF)%k52umdwo7QJtIG? z`c&!iGF50>R!azeP$EBH8_(=|>uI{BF;Kj=YQx<;XS@{yg;ZQo_!d~--<;Eg8?0|! zGo5y<1uL}+uzK9kJMQg{burDh>Cau#&W~usyvIULuSZa6CWu_HzoPmYMI8B!p~uz& zBTEXVGUT5$OngoobvkfNCe;Mid&?8KX6arDl&OUoB9vrZIbKY^htXlLu^B}&_e4Y` z2aRyst@Ro^eUVKkbys~?7SpKa&i~1?b=aK(Je$Gcen@_|;EDt78I$YH5i;_{Wq(%G z>(;z~{{0#CA+*@glQI)9z({*NgeVp>e| zho{Bu1NYAM3$OdFDG6y*Uv}UN+s{_YC9Ole+MLi+q*^?E>%Q$mwDaQbj}X`=ka=nAUWc=r=;S*z;sz`)8rCVDb?~-0qgw z{Gg)-Jyf~is7GXVH%!>po_&dQ*#oBwkT^9WkBr2O>labhe-f1h@Hkso7&nEwGl@i@ zH>Hdflnc}&4YD{S54wuM%-$WJ`@sui!~3fR-m+_Z*T(2D3gwOImNNs(IXIs~ugI{V z4kq{JvS;q^Gu(AW&zcD2)FlnWa3gc0Bv5a6l+MrXF(9-b*i1nZ=u9T-Z_f62+;&yw zKrqhKDl40&y4%3Un|;LwgelV=jN6-sA>;&pmG00G3oJ12RxmNimN~U6+nA1$(RyQk zRx1KlvAsKSvStI?IG3C*jJLJ@lQ$49acjVOq1(@d08;{>iZ}pw*v#?)+aqA-FHP#> zpZ~PCeLQ9ktZY&xYjiVT{dqBVOelG_JQySIarK=uh7uyxj$dgok+a&xd5Qg4I-=-y zv1!E@>LT%*nJ@wyJa42`@0Id)zKFb%p>B@q7b>6@mGPJV`R|Z*y8nx;lS50WiCrJ? z@!L)bdEH;0O;5WybaRhk`})TtG1RNGsvQ!zwXH5e25BRm1-5z-&QEv(x`ACnqj=I$ zHVDL#1H*2j{_fQaQ)XUZx6O?mFHHeP7%YK;&fi5P6i32M9(2)Zkw_$P{y&6yGNKWi zMNm^Xi8XSZH)@v>@moq*py8IXzg2$FT+8^vLk~c)X7OCa2MbN+gje$@-(Ljoe-XIB zk<|EHC5jCt%pW!WK3C;Z%uY?pwpJYu>+{PWY??P29^5#oj!sDp_r>s33p&(B-w}F$ z)gW%WD$)9Qwd&gI^YH#@D_^c5HsA9V)N@-dp&d1A4!&RaT#jnWw`D<kMu)TP~%v z6h!L|Z2Dk6+D{OLQQn%J8;2r-^#^AiuUarX;fdYrQW;W#;%7JikM;P8D0wm{J#9t& zfvGQs%Gw44MT==D$Lbm343RVOTl+{nsb46;DUvqxN0{^{hiH zwF;p&Kei*mUQ(5)?q5x|ijdTeCI6A+W(Uk`F?SsstRo2~2CLPYKOCB@rigyM+eq!; z?_)qJ;Gab8oUTX}wtkQxCc8vnI@!XLsgB{OVl+?<|uDGfO!*33IZFMQT?_o6i=7k8uf; z*J_LAAr9i%jX$f^k%s2g>{(Hg%d;@1^< z+?XK{Ox-o5Hq~i(CfRECBL?b$6i|D<^#6Y>;A)t^J6IMmUkd3yH zrC_6ig-)=;;WVPT$acpPfhbchS^k-ge#qA5_zIdfjk*}SeLwVToc=|AHvMspj;m;F z;3`R3Zj~L*W5Pyqf242UI$1HS;s>9bKEvhg$ z`SYv&@}`@6xQ^V`z7mhznRrn3s%N}!r8jY~hvM0W#@X9Xmu0B@PU+=gEVkAR>H^6N z#lqIAJB*N`2Am>kLnv0;&9imC^I_JaRXjuKStpzKQ=VKUV-muay;2Q#!byoAS?{j0 zEOfa1el=?svYCz~@vuKpAYY>dpccvy1A5Taoo9qPu2q%!J|UpfeBO^<0|ofK_*Brh z8)3<@Etln_D%)~5s-2K}sC;3o2a=M=wo<>;WcOs+Cc0U5g&YMnV||uq+Weeksk&`) zbh6xdf3skJe|9O{d7EW?vtQ0d59FbVS=y5;3zj(KbYt2L2Uz%gm?9I?iQJ&ExCL5QMTYbse-Lo`}rh@{FXdP~K}f5s>`wjZDmm-fY7L0MhbHfd}o< zTzNc3;$|q?O48z(_d!5jn-h`n_3Q7}b3Gw^jZ^);!>MJ~^IpS$H5)EeYLE=6R;$tC z&}T7yA+2ppRB^78h@J*nNq>S1wVkE9>ErL)4Br>?JrRr)97WX*V~tVEC&SiCq3P$;Gzp6={I< ze(Dq)u(|gGIn}jH%~@d2e$urxUYpPfH0m#sS92az2aLlh^fJSTs%p$JTej8YuIcRr zqxK$v2E%li4=y6^h`#K&UP(BBf@y@BQN$ITvmY#bb)7jU$EG@M-#I(z05f+63iBf618HSwr{Czhr@BG;+p0Jv%tkW9-C^@I#8>F}N224D2t*`nOcPfCBU*m$8+bP!kZH@Y3R`2`$qw z!6Bd}H{`@;?td~YTPSSL5ocygX>H?y@%<%IJI<>0C(ij@wfzi@K&)Wj-0#j^52mw( zCh&$wuKBozu$nS(*a}ma{MN1hQZFr0@4QkDpcrV0IRGCl(*2DVoj>%;McJgdqOU%m zFQ8On>ZggQnwRqAjtAZ(Fmcq^SM4E_LuO2_OJ7h!`9IXC&VE?hV7gLNiH&HRrG@>b zjS0xzkU|18e5Uwh_wt3foX`fn=q0QroHjX(@-f8i#a;b68S zuxiFM^Kdp7sZmmvjtR|_LgQAcoKNPz#1xTo`qZvOZ@wUD%vN`$0Vvjf`w-(5tAoWS z2kboJc9mq4cRe&9YPGCnd`=Nyikh75pykta$!=`|u047x-MU3)wiu(ELT;SEs78P#m5n;cbhysNO z*KTLxIl?2J)pNCk-)xpU%TyQmM&~Hd>$VpX4}$=if85wKXCc z_cydY4gA4zMy)NML!WYUlpsA)Rub27a&iFo zqmFj3YWkCR!0~%)>Q3KkCzWJ+lFhyghNe4c^Q}oZ-KA7E`2GJTmb>%KIA=&~`h>IV zCtKN=Qw%^=HgzpZCPug9)3AfLKRg4T@v{*E1yNqdP667HsN59fotgJ$=qa>Q2`UK1Nj z<=2Uc3Ej^%1&-wc*1!sO=~19khA(mkTva3cr(d(*c|1g}NDHpyVPPk(l)PzGfM-Y(6JQuGr0SHHc>^CVMfvrl(At zPOszH?FfS%j|h6F7I7oxMn9iQe#@c;rZa59w^o;vt`FWhZ}aEO(dI=`OSe6zJBw07 zMMw^_(w`H*70P0g62Ny6WF?BJP>rlT=|Vg$$G5b z!BrNy7$trDWZKADH_MZ5kmU4E0ee9mIBz8R$S}r+U*SsB?wiE-8}^#gC&Rs#*eoH8 zeN%n~;$HeQJIw%TR|*g;rHPULBs4N2eUm8jf~Pe`8xiv7_&NveO;$2w+w@Rcvmn3a z@86&WGHC6WFV@=dj^I)*M)a0Yf}%uIjEQ~mS|3Gp28`)s`C#C`k}f=)HgyrBIs15X zGZHC2kK>ph5f8{KE5J zt;TAzc68VBbVU1d_S#{c@lGXtx{2LNX(%zJ_=>V%oGF#HxGezZlJ_yCPRrBwdfHyD zZd*LL<(?m=xzevhf!@uW5tT|IFk)AR(n=bru-&}B!3M&)=S@00z{Xl3Jm4&2{b&MC zl%!{c$O5OiJSr#${-G>xN^T{t?l`^^fVg^vT9SA4>Xc={8S=pqfD330v?1Y7neE23 zw46FGyX0I>usvQ6j)vreKMm2uC|>q6a0t8Ys)vTxQ1i1WU6{R>81s^z4dW+1*%5ou zvR8hz`4&VhC#cl%{1O@*ASLgS=0K34UNMOjQ6&Fx_{3wjx1CI653#v}f50!qBJ6Oe7CJ(bI3C|Q zomj(&A^;(QtYrPjg2yF-vKG;ivbH5Ur;a^1@f69zk8A~OuMgej;xG3!&Qyfi#eXEH z3s~~G5?`Xj`?q@97C52^EE?{~DQ>K|o)T1K`wXV^UFgtauzHhuS8Mwo=U}dsnZS+1 zxxk+-g|X}+Z$!3tEFrI$UKtw8g>K+&uZ_lPcYiN{Zs*#INp?et+Z~ngb0v#iqV8D`%=}YU75Y#Jhm%s3w4~URrLD{!X{m zu>CpDfZqmBHB*JGW1sNyEM9a?67nuYyGgbJBh6008~cuZq`iHE1Wfv>um6sOdt@Ij zemjZLn?}5u9oxJ23qOCid_BA=7)olS*|#*6!1dY`k#coQ@vRs;xFAjIg5!-~FfrlabPu)6{I7S+nANaMX1Mm7)!!W@|LRvMG+Rv8^}> zcN!jH%QFn#r_a6`C+4aY7!s?U^Iv$$dwcFs_#f9a+w2HjRFfTGnRF9McW6aF=a+8@ z&T>~Ga9}XwPu0SiqRp&CX;BU|6$`E;Y%NF035+3PnVo-A$Y6T6oc)^IUxFYx#9Lnn z5?VJFBw6VZb{1ptARYJ0oCrjC{D%~!@{9_>4lTfem4&nCcPcHX<|ZcB#OJ&(hLDx& z+tLDT0v*OE;WYt>OuVU~QK`2djgp$Y%o0<5P2z~jlzJ6Oza>T?{k`9Disa&?T zjFR4{EK+M%LH|M9>f21+m@b3B)QJ>2}npRH(m()n2~ zR=g|k8@-pO8QBjt`{Sen6<|Ia-TC(H!p)mx39RdyDSXYr!Q9S~rRRXcs8hTo1j*J- zOKy_sA1cjRYtt9_&HFqq)ch|w(n{Mr*2s6ozgnOpc1aYfwJ13eq?No^9-bM{vO7`- z-xCH@!Y}aV?Nw(RLE1xF<_6k4vm6MNt{&zZj&J-@7~4GtfV>A zJ~zvJLWjBFo9hnBy|1j*w|H{=cC9{`;WR7 ze+w-3E#9gM8>yGO!Gb@b%(}nrV3+K5?->2<#0J-CR?Ca^cKD{Q|Nx-Ao|E)#(ckF?Z`b6`k8{>~qrNFd zkRjxZ?Vnr6d)K6VMvO@&&)wZF<+zKe&{sbRR3Yo^5hlU=hg;Sl`p*2+!%!LX7JL45 zrKmkADnZt*(Z93Niy`rnha%SgJN)yZe)XIWw+sIBDXW6z0w3CLM`;b>PX~l=vOZls z;sj2l1%)yQ$klG&5gHkjRwbz(+J4lEIUtn=GJhJHc9Lo*yDrn>s<|WW&unjQDrJc# zSi#;#&OC;nc(tVchU8{m{=J85=3S&n)E| zV_aJALKpIHvJM#R)cG+WP<9RSa+0;M$vHF;-IRQ8@+QhtHa^X7Gh`RVwXcRpsJ)ZT zE^cp0(XbVwQs<5LCi);MPRndl38{d{s|TrKL!oWZ_F*6L%?Fq!sV>ErAuwiR5Z>u4E& zu-w7ChvXaVHDuVJaQJI#;L z_sJzg4yuKID)S*~8woT$LWT<|$eg*8q8c4hF`MHZFoa|gK9{$JHFded!1?$q`)>qu z>qx$@MnD#>xW}TUA7xbrD`$NT?GYDxwMGFQ7EKQ$hP)G;*hmV-rD5uBK-z)h!1j|r z^5zb~*{#6#vvnyGIh#7>kNM|flDghQzmD@${D<`%1U;$y*2vyzEBd^%)gai~J<8ay z-{;+8c%Y$P%KJMuFXFu1c>}LYsJ?)JZr*6T(4#TX$$0pXrjRaxQ z*EI9^7z+|P?qGxj;Ba__=L%cGZB_ONl5d`&{2uJMzPiSQq$4A0^se%z*@zM+I_Ze< zEr+7$AiT7_iyj@!r?7u`Z=QU)g!qsWc7Mrjxvk~M5XTra(bfArhVa#lICQ~v=_=`}B1`a;M`h^G$aoVDvE=A9 zN-!RnG;Ap$R6ca6T?J(EpKVf-nd0}%P3pJz&z2`CRrura)9DMWUgRHV3wGK(EJ8)!?;G&&hXtPWmlI@FM&E{Q$kAbx$lp35{ z&xgISLD7pu(UyUo7TL80MSZ|jwxzSg@5D}wFn+(#m0yL+@v(Ck(Oqhf>nz6ZLlH(p zY-2U(AH`Oh>0E}#MYKjyXD>wx6++Vj|fTb&4if}>@14&5$$V= zVDM#K_hOzSIqFwJK_q9RnJ=)RS9h%TRUsPfT4B~+X{vZY5BeNF8EuQBhZvqgH}N@B ziZ;A^Y1`Wt(PAmnX@xCG-W)#s{T3_)HAJx#KXhOd5iA{%+V3qf|uga$Xx}s zh0Cqj8TcGcbYHj8uT+YQH@NXRf?LUeyNnO++APJDDOz)G{TP_n{r2(OFlF_DsSLr- z)rq`X82xYg(=-ty|G$lc!2x}6|Gs2}7=rS*4cQVlCJ1N(BV6}^3Jm;@;AgHP1ytV! zUYaV1UijMtAxHjOBe7U%Ny)`ge(de9Ofo)3G?M4FHSxc#)t2Nf<5%eVi(H_W7(D#; zCy|Ny^4i0IZN`*^quXuWF2UyqWez3=24e{uG(C$*_piOn|AoEr%oCY{UT^~6&+PxT@eXX2o)Us3iS;7?jyL99Z= H!0&$m&0sTt diff --git a/doc/data_flow_simple.svg b/doc/data_flow_simple.svg deleted file mode 100644 index 012a06c2977..00000000000 --- a/doc/data_flow_simple.svg +++ /dev/null @@ -1,92 +0,0 @@ - - - - - -DruidDataFlow - -1 - -REST query - -10 - -Broker.query_cache - -1->10 - - - query - -10->1 - - - results - -5 - -Realtime.working_set - -10->5 - - - query - -6 - -Compute.working_set - -10->6 - - - query - -2 - -realtime_data_src - -2->5 - - -Realtime.Firehose - -3 - -Compute.disk_cache - -3->6 - - -Compute.map - -4 - -indexed segments -blob_store (S3, HDFS) - -4->3 - - -Compute.load - -5->10 - - - results - -5->4 - - -Realtime.segmentPusher - -6->10 - - - results - - - diff --git a/doc/publications/vldb/README.md b/doc/publications/vldb/README.md deleted file mode 100644 index 44025ab1dbf..00000000000 --- a/doc/publications/vldb/README.md +++ /dev/null @@ -1,4 +0,0 @@ -Download [MacTeX](http://tug.org/mactex/) -```bash -make -``` diff --git a/doc/publications/vldb/druid.pdf b/doc/publications/vldb/druid.pdf deleted file mode 100644 index 3553c0c4b5be9973e2bd7e1821d8e06e4f9072f2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 520154 zcma&NLy#tbvaMUTZQE9tZQFm@wrzG-myN${qsz8!+veTp#fuv^Voy$vG9pJgTluYA zl*;0gjI2zYaFnx)L#uGC>|7))B#tIFaQysm%<|?AmTp!goa{UtB>!K*F-uz8xtY6= zFiYA2+|0$zO&!h5;RFTYT-{vE0rqfS>&^Olj{5D%0WYFr=9>6MXo+2&Z!Z&in6do)rFzky#+4rg4boV#8b$`IelHUVH{IStN}vC_ zeV;b;%02t*^NZO#J9{q0{RR$xL}@KVS9!0~oozf{057GnSu;jNf#0nYE8~%|14p)J zpF@erFE@d|hxr8pyl9k=h;Md`;9Tz#OGiBK)B-$Qf82}&|M|Z^zU|%5c=jZURq`|$ z_u=ey6qHy<#^f}ohI3SMJYJRTVZRUuzTJ!@hR+5X!sX;~m;^)8S6^E>5AD-c)lb40 zNc+Fsy8Y~ZKO9}~jP)4G2Hf2~+7`S`ohTW7dz4QOIgYog*ptx8zn{TSu zXj{v=zTH`BAD%=;mdYMprHz!19X%6LkCYcbSz~zlYIL zc4WlUoXfMP`=d6pv@~)iD~k-#Qgs4*1G8K}%{dBt{V#_A_N3MpW;tY|$n(ZM$RvND znj3$o=v#nVw|CfBfR;>m;ODEgtDJhB{77^6Vrp4_2Ig@|i#X&+TZ@sqTuQ4-jC>De zkGN)T556dn9L*q-==PdE%?C|S=;IDaznPD`xn$EBsa8u5j;eZU< z`!y490%xSD+V_i<{f>8hoIQ7%Wh?rpP08TL&|N2gs9Bih)&bZbj-sN@rREO=PISvCIN&Oe@vVt5XhXd&ZpKdvxSX#R8 z!htR2Lzs%=;_RpL+_b3n)mX`TY`G?u08LRNdqxQ>!k3u@+r3t2s`1N<)9y*#9Z8?N z5@%}#ty&46nTJW^5^wwJW?s>Y4k!bkGrh|_8)mI-&6?GFiih4_m^TF*PegydfVX~` zzS^?lev3P+Lf!^SPKbT){5Ap9>SEY%+09hYb2S5CW@=u9l0WA&Ylp)xyOsI?2S&5C zZP}}q1VBxg+zxVyzeNqC=8}it(>`G(9Xh9Kt2@7zU@V0+-IGc$lhVeqeL2JQ7tb^T zL}D$gY9x#z2b=23(Xl}!WhKZ0eIDH$18O&l($#~@Q7_vZuw={SjL^XNAUPXLtC$4*jcZYr5>&v_@K&_{4AbfYs*H$V5YvX zok8AGLqBR$pU_Gc%e2l>ey5C2Mk(cTT3joo4E+k_0v~I0RVk0tFgw44`enZBq#4c3 zh$PNdb7#GzP>xLGq?CYX7nrfas1r(!pFmeab;0-JCTwXNmi4FiJulzqgHkl;Qt|k1d z_C+FC0?M>81%zE)(IYmI>_a6K&H-07aE!5TTu-~|(8F&CU~Hx(nrlxWnjKV^84Lu= zbo69ibmftc6hV``UP>Z;liO*67a*mR69$h((;9D#%*Q!gWy-O2%9G!JU zl$3dKL(IAE25Fp#eXs}tWp_eZ)HVmW<4rbKV65-0MI0g|_=otrZbM+8TQ1rx(Q-{~+9pWhSnFIsa zmPTSV0_20V4Chil;qH#e#IkD=t_@b1g~-Y17-)%8!<_YfHM+i9VI2R;_IRE!L1s?~AgaMLh_2UbW zs6DG=%^OG;!jxR+h!^d-c<|>Gh?L@*=Y&Zu>b8U;p6;wv+J;qF&tG9nKF%-NKyA$^ zGdHVJBGZ)wO{VQMj`5^dKv@GT$lu{U)vKq3R7h}ldw^w|p=)$hx)Lul>FyNUV(~XeA zM083F-<0`;#uW>6#RSW9VjPd`SDta16tf~DqN3q_@zg_J zF1|-oc(d6TvL`IP;7W}Jnocz^g%+?Y{H1nnU;D9bz}RB6-07h%nZYdy>MvYBUsur? zdK^7?T(={eS!Aw5Rcd-^ce%d;iVjeCqQ|UPDzCZzFo9|2MpK_(CzlB(0~!EJ&J@b;DMmc*p60F`(~U z5magUWgUUsr87=YVUXUp(+f&6vg4VO3vM1?PZeDm*et^jitJE>iJ)+fNaweiCx3Q~O>Yr$8unxH}D{JSoDYQa8gjoWcSO7UOl}ZK9c;1RE5! zkDISm$1O%rv*tL0p2o+0zzb;=7lw~Ec%h0~2)>XKF8P+%2qQmd?N!;It)J~V1HhFz-|Yu*qND`xKw zcc{e-bP8}vh$mj43snd}RJ8FS--4!g(| zRl8|)vckzCSx%g8J~qecuIt%bQR}C?BfTszjN!*Ea!VRc3qb?zH-Y4SCB+1_*#lZs zVJU*|wGl(Y?+D%%Etcd%z|%>@A;6zchDg*&?hY=%QJ!eE;mWb1HiO-tuhr8^_&8F__cfxMrlmyn-N zaaBDYR*R?{oMKI$w@WYN=X-HRq#+XbOVHZ1fQ*MI({Za)1meh?x`ljwtq9nz`&?MS zN`J!BqW%6|pna(UoNq*+N2$yWptMp@b%KOf5_Zalq&IumTR^kQk4@}s$~_v=Gxc4; z@>{aHO5rn>6jC>=lwZ^T9ZUaQ)&Q*Zh&}hc|7jzpu$u0^dFobjjWkLEpg<&sQ`}ZQ ze!dzzo2rXgQ_cd5<`E~msxz9i96^g}1%spN7G#QM=&_uwuQKhT{kkX~F~*dk=2^_ux__k)M_W`ZG?-i!U^H9j`RSV*9n)#x(ky1ZZhBIi!Cy8Qi;m zz!;FK${$P_ae)TESL%JX!@#7zEp4#R3flk!+GsQ#EedHxscXGwJ(lC5c8(ytf&n~g z)Gl=>xSBV~)$VTv86IyPNNVxmXdgb))lXb{NH{8tniR`SSMTo?2?g;!;6{)wX2Zk@K**WIMZ;BGo(yXrUt;0DQbBu(a-iypIW&cHQ zc5P!B!F_p3NDA9;&vHvUal)M(+A`mLBWl>g-uZHRI7ZHLcMB3ukXN@bJoPIWJb4KibMf_=fsh?b|=ExE7w96s4hfR5m*`tfX$nP*U}O(Pcc zhDp``Dl*pgw~%jYx(vbcQ;n1zASk(|Pw-+ac!rWXwB<&w3&&;g-;bSW-ox9_Mzc}f z5Y15slW}571cMOQqF0L4?LAsbAzc=@7lUmljA*_^yl-g>n2WUC%W~1u##*cu7j9OY zo((IYD>+m+ya($~aixVgY-N4fWD?GhvQ+BOHQKlFkif*(PUUG1Aam6t75WAL>E<6H zJRDGddjc$_13##t>Ol=Mp%YrF7jsoGIXS($b@U=QjlNAg!p*h0(+-AOAZzQO&eca(uI4GIK`~M$7|YmU5R%eP$Cb( zF4Jlj87j1r>Z8QWdt9%w1HSwmpmeXieW3I7=K|dPQzApIps$|s(Qs;c#@mV#zI)=% zVM2uNS?yzpdz@Vf9(t+jv`OXE66Q+ldVwoOd--^G84J;|bwO-{eAOcs-_(O9)^#;8 zzm}7h<99~tKYa}E^=ZK^l*Fb)FZnydVLlEZwc z<&^Wpmw057MHXjI`*L=M5IXT~(v;~()tj%v)2Z5vdOey10ys<~#ow@z7{1^wM_vxp zsKN#}Rh1I;L^+{+YCnsqg=bK4j2XW@CNXJ5X6t&Lb8O!tLk6t_;caIl&0ZUz4ArJn zj56|DiIdQ>+hx;a^XKt}%$2R?bt7aQh0m;@CyOQR=D=&GwWjEgo#|+K8UuWsCjs`a z%48h1%c0Aa${qijG5FmarJfEuR1NrRTriJ8ibe@xtOP`%u%E>cmxH@;CpS~Pv)O6i zRgY_id$ZY~BWBJWR_L&X$wcH_WAw%=mpUCYkkNSbF_sx>LR(5>Z9Z_7h60vk&;?mm zR3&2mxSZ>gRL|_0%^Ik1^T~_UVi{gsJ))}?lpOfbShEFi>b7DDvg@;!_P1RFS5sHW zWZFM0j*^bcp{$Gjpe|9FSk}iIMD=8c4YP>!Z{CIc;Z?;UY(og>2W2{?s)}(92PxIwqmFW>a-pLsw+*ui;i(qq=OwDXj4b0)oLCd6UdfpGBlaM z-43&WdJ&2sfz2=qx6V#7zd~%NUf!cZ7kPYkD%B`(r5lu2r+tp&#vpgWoGLFNT`l5i zIDKQePKI}@f@EO>p&RaM*=m$R>WPm9YTI!kYaN)(ZNxq{%&=hy7_F`49Ez}c3(%6% z1Q_>jIF0jL47w8?b+DOM4!B;QdwRhRR|>P7R>F$UEt_7$!5+zS#7`JE{j@{O*gSq6 z8ZI^3f7hv5zS-yp6I-^_eS!tpZ~J$Lfd=TcI+F&-%u`hMK=?9oyRTsQi9m$Bz6E&ZWK4D@l~vY)%|`F=ZVCH}T& z3z5xQvLwHZUpfwPAkW^VuVfqor}@X52HWB0R|C&KyZb!!4k8D>7$>(VV=ovPDO&cT zae@PdV!r{tI;96W_m9iz3nur@qj6gRga06L$_0)8fKKEUVVC0)G#wtL8ZIk<1?mbZ zlbF@Y5D%wg?i!Bbr1g+U&cj0pIE>(CiG;`QZi-yTPtdXfQPq@~b!pl+?$FqWWW2@4 zy~?Vd2t^P*%eBOjGyqev!J6*wLsNF$ySwcgBEo}o{rWWlR_vnsWGX^Ot$|Fv#BFz^ z=y-GpL}hT)FNu8#;WYoPd*Qg(NIGIK_28R~K8ahTQ88K?f&cG7azhNvzX6mYImGhQ z6MyVs=E`#-ur+T*{;gm_q-JtOt?jP*y73|Bshz>9H}L*iySBzmS20o7^)j+QfkChL zdw!SY>4om4766g@rHqqo>55Zr!#BC{`3X`Rz`)R31<6o;E<0SE@nE_Ka?LP3>r7<0 zhfvjiBle#w<`?gb^^^lKJoxPftRUXjc^a=`&`dXRE^QjewjX+t0$?P;o2Cenme}>1 z{PmJt3*0Z2`Q&EUf*6*sP6zb8)n!d81V%qK)QkI%KMhU%7@YF;UOh5o(cS9UU3)Y6 zo|5(@xdK)4edPEX=bU}+XuE#CX*LWb{9DDfO)h>B$(U=%Uyv7jull68<4iFcd0CZ)|31VOw zH-V-4@?44Sfhf{XC)-XQtyZEN_5Wu<&0oPsM`0_17*M;->^OLhh~Yg|8CRWq87{Jp zJ@IQ$t<1Z9lU&6L)(^;lmPCCdHoflL9{NUqiqBCy-p`#$J7QRJHa(<+t;y$tI*>-p zy3GMQVmN3PJ?8{4K-*o|njjjMSqm6eE7wZ$o*)L2aT|Je)5DV2WaeHselBe*!rZ^m zy#|_HKkJXVMCgdC8o!b2UHCx-<^#6!@cZo0Hta!S1VR6+yR3JhD-HIpn}}`HJ;ru^ z)Wt8XME<52_f0UVSU(J;`uYKL7nWat+bD&dZF6w>z+IX@)piOSC;wCQCAFXcg=z_?-~Rgbzowynhj~IimY)y- zGvneJ+$aQjit7#`tzUn2{tO|#U}BV+loURv%wC(!{M|8E859sAzeS;N4iDv`DT59s zvVmdJYo_tB_jc!bN63A4b%Wob>#*5a2vz^+kQ5>PO`hPEAt^oRqnF70{>cO=L>#pZ zdNg{g!6#-)L^4n;$v&|5qZ0NH7M$Il^J+ww^9qKDSVeBkKB1Bd-32O9yy|d0EJ%HA z@}32J3hFwN$oquML+;Nr$(iK4MS4+p?IDyvW@C%i8X2?VSsh0N7Vs^nrVJP9oIrZ0 zD3@Eh8WO#T@0~`ihz{m|qylvw*DR!CJ>=HO=RqctiC38{7i~*G5w~~KT3lO#fFFXM zf%kJOYP}Oi36KLbV0(X+aIte~DGLrwi{9OnaE_-Ao#-#2A2&AQt$EPA3v{8J)J2{t z4AhK7bIsxYwnqL2BI(LPtwVre<#+Y?+otH}8h2t_m0C*udyb&SJ%$E^p9p)(0yc3i znGaFgHzjkRQ zFw!(8v_QrvRb~_oiywIU06L*Fs-g=yqk%Rx%jiIt2@s_gyj>GAP>06OH^8u<3I6YzDP zb#npZs#muL&U7WevznXdX~kF>rWjiRhC&;2L?sqGJ%b6SfMk7IQDxrW^AtRgEHk+b zPv0BrL^Cq>Y>*QVgD##tj^xrb%A!j~XL<-Ho-D$r(vs&yR6c|o;%Cj8x^|lm$5Z;&#WT{^{Mbkl>=9>E ziBGjEb9j|YNS~DQ!`E=$B2^ z%AIV?*^+NqM8%*U5G0?*I&S10<}aCY%=ot3If%PU-Kc2Haq)3IrE8mrQFxn=vB%K& zql^oY+o83X$l~3w=17B?67jA34c;e(s<8?t+d5L=IprR!Nxl)Cmvpt~iR+92pZf_PdUGuaI z3U!yUgky@p`bAkcG?=72{(bq_%`(?8>jBG-z#1hS{ptSgMOoPvv1`~!ML8>xp;(|j zYoXHBB^^c~>mJzoFeY7)%K3NZsFwEN`U>omGbqbS@6h(4WzjZG>)imAd8pL#C*m8% zon+Z@*{WW?oD#_B(TbkQnVj+d>tOTMFrCk7fjTO%2N$-d{o zeZFD}r$IP?ZZwuwa0PcX6lk@*hJ8}Z*3x3Yk_P#5@BDL?9>WM(|M#-q2a6>gOGyB4 zi+sO4&usZ5rL}-NqWotGAHA<&J5O3@nl`jfP9nlLd{eei7n|F*6Z%F2#tN&PA*4&N z9(^YVOS8%qUV-tH1j97ed0|OaFLq3IgP)pV1TN4ybY)@5?FMc^(2}OHfa3TT<}BcQ zX_qf^?4L`1Xb0NUDzSv}6?uxeF}js=xQNR`JK0^?8R~gLCTtvs2K8+B->m5srHqMv znyk9^Yo~Mlu-i)II5<}`T4rg5|vy(2Yc(JQ^X4r z%JZh3hLuGSF`yFv2sfwr7U5EtBwkJ%jqOA`OZu7C`RF|lVt=Q+2_>Xqs8aL{$aCM^5VvZbOHE@kWY8A%#p|!@IpUwSK&E}%^Ss%v4g|_ zfwKcES}6GlVKfNYxjH$QL5yN{z(IntJgf7xinGb(MbWLb9hljN^3I%uHuH|)K4vfI zMm%lMR0aJpJUhzpl*6-3_A>jA*raR8bf{m98#SqCifsIN{NtLvJHck(JdReg_2Yk-sgFG|N&*;|8 zLGLCs_>KufO8q=L)d?*QOCI5gj#|RK7G{vF&oMEPgHR>CD@*V-!I?xk_84c|8unm= za7wGE@uQuha_6s7@Z)0>*A_#q)evF+xumXMde{Q}W%?Wn;Doqzi(UPCNDTZsI_dd* z9TEC``!M}|;~@0&xbyQdf&BEJ$(xeU=Tok6?wrlVXL?! z@%yWKF}3>^mR=C-96{^t#^O%#V|!it++Wo!QU@Qp7x~|@?2EicH1q=^M`uQ%@^S;| zcByOWR$6Ac=E8_Fi1(8abTbxNR;kZU;WAbh!-t({6i+G;t6NY1fwGqZG6|HW=A*V1 z{^j-xmzT@Ag0l+`WWsqZ_fN$|z z7qfGdW&^#Eo{*@5uDRU0DyE=Wp%7eZornkO3{8O_ss*>bRUSJsT4^ri?(Fs#mzb`k z7IA_Hx1Rmqy_Ll&K3Zm@753dI@tNvXDRYL}LRdUfmVwAazfjw*5_ZvissW!~?pCIP zMrxLd>}GFGI={tiUO_X@SV9NY6XuiJJ}RB^n0@JUjsP_N0uqS$$>ezJ&*C7Lh(s8a z2DbbcAuTr{BgvCG#L`ez_vBsd5FKsi0vdd7k26K=pzPatQ;Sx5WUwhCCb~N-)Az_k z8hMJRklRV0LV2*B_84=LC2Ju7b%1VV&{FM8 zgA}HO$F*bRx<+7_Jk#nJ>^ZMZas%Jxv=dI8c1?|~7nRpu=7&!RT7qX<1C3C&xk4VC zS~jA3T1v$Is+`{{G9H&rX=OgzNbE zXc@apuAo>tT-LdRudJ;(N~-Hpv3+@7`^aex;~XzC7pYgRTy8z;H09zYHHIP1tFpBj zhi3SaXVPR=^tsAT_L!c4Pp2i!L;ySmA_jWorF^oY(=r?Z^X#=MvSI>M=*4QX)ezQt z-F+ep?BG9BL5u6UfHIK?I{0nH}%H!(n;2BxDU~Ru& z01uadT=ReQV2X?hRo=;x!gI@W37y}^+xZ=1k)PMNO*BH@pJJdw9rhMh2YdEW)x^Dw z>bJAxBPRrnvULAy<>JJ=lCLgUs2>$GNpJ=yI2P8p3A!geZ5K%_@4d5a<%w|HCIEBF<4sb)|tSibqP z*if4K_hkNFtyUm(@t^EDEdoa##iuiP9rS4~ZHj4x5~MaTbsgz4)dj3rJDovn=IXy= zM^i6el4An3u@WS}`sjQ+I=}PMRF@xO9I>>RPyz)xZ385c5cs)Zr>Qih7)H&@M zZuez?54D1Dd+QjZdrvQQSy59(yIj}GKBa1?A9?-j;C_mQWL>?6<)IApqXxr61C@_Z z2{rI&zklu*VXqX7&B?vq1%DogPGb=q_MI6jeuMsHSx6BKs}=K!*|Cn}JtF0{lHb8F zBpW2J&1JlEu4dBI)RP`D^`#?-@o#DuLyeiM=fD_AYnmITHV-}wG4YDMeIRs96^lec z_+cJUn^BN_%-%eHJqT|qL1G&n)vJ7)jIQX-m{SUQX|9TFk^G`CQqji;U1ZMGMZ-5@ zHl1AD&7959?yxc3t?%<_C>^ra(szldE#*TkO~=q^2E-Xq}v+cBRrL_)+Cmp){Vx1P?#(+}b@=U{pl5m&3GW$#&)NW>E=BRmP}lwhYf ziIGfH@IqVbW3@$Px+LRS^eoc|n9;P6v0rvfL)25!nK5f_R!okTKxjLUZ%({zab%=% zWRyf4XZU8R%u#YjG3=vSmU;t(Nr$~e>}JTC&=q&kcBe(90B!0}=&Mlr5Y%6reGT#= zP@((^hR1*zOTmBA-rP_Dktk` zNgaxRver`M=zNl5CPVtl%`J$Fe##)NP4A}cS&NfdiQ);cfAK<(&5U5o(!LiftnN|$ zh!S~7C|3G?2N*XuYzI-e$(q<;8qL+0&tlgutY$qqxOmwPd^tc4d^4=PnO^wSTjA2@ zSYy=Yi^rorVO%EL&?WJd&e$D@+WwWg8Xj||Mz^EWv1J@X5!?4i-Gop}Rg}`(F{*>y z1Oeix%Q6j(tId7hs=&$(r&UZUd$_}vuUK82TBmGIvb`(1SW0vQCvDoJt~HEu&1>}kBysKuotIeKitZ{f`Rwi-d z4u7O**P5Fbc_D*aOd&&!$PMze# z`mM8hqZZ0$xY(-D$ZwQyj(k@&ZY~dZk4K}+b+BKP?w1^R$BD?~|rLI+3J_NNMT>v6%Rd;YUhW@SQ@ce7~-SYbfxHqTeJC4hPBU zRSkUKlWj=mEz2yj7!%S2Y>5X3lG9fBr{00p(SE0_nJA%+JmwF=)$#uW$Wo~ZgCOky zL_9i8(JtfXuW62j7HKL^85EDDccdw|uGHH1f%Arp?Btl)Jok1#2tG0^o-90LC4Wkx zGd^^l?nyT6xE5ST(m<_}i7~EkVHY*EtrHojP-6d8MOFlM&BCz#EyIA_7oGwlfiP(o--|+_qqhFU1*q3~Kx$^%g`ioc?_Q zr3mfd%LX~cn!n#f(N1>qb?M~U&Z{085fpyg8GOj->v{z5LT)LsVlAeE$tB`f4n=y+ zKY8;g6pxH0UL^KauuTEqr;r0*9|Cc10^jaRJO7;Yb8*Pg?e`d_CYHSNgztSE>~>EgM7 zJ`G{^;N=iotb6*<)6p!#+IQk-B}ztTNO0`voYRm7mMhL!A*y|Qx+J7*{N9t9YR@yy z#{iDvd)LZ$`rOsnxp>}~3tQ;C0wQAJ5U{Lp*eBvK+8yofJe&`AE4e%ew-nk31*Ol$ z)P$s$PNj8R@^5jq1t+{y0-8#a9j4$&A%#tONXWjw)>tkA)*E^ zNqcUc98*nSq~4!&gl1N$8Xl#+M}i-}!dpzh9u#4(e1@A3N8edIMEI>`kE{6Z)mt;N z(Dx(#UNuzCyp~C(p=NmCTK}`O)SCGMhk^E+_u$>Yt-9Hio$nLzx77nRB38n!(KP&D zC&%nHbr!Kgs;NAamDcsj*pb3N+f8`Be0_GY&Zv_u-eo0gQaTp5ru5HCQ0LrcMLRq% z#In{a9%1!9@UqCDaRE!8OHAZhS0$s101wRQXK@%FpT^o}_4IQV)}cA~ zvVo;7$#vu%4A_fUR6sX6Ra4QI;FD(zzabMpH_E(|JteH5r>iZCG22+>$|ahr{2|F@ zYI?f&V|i;U8KQZ{ttZ_&TMG{p+e<#$Vm@M&*i=8ym8xek;m|zQ#%Iw0 z=uUQ-k%hE@ga7IX>;C%#pEPXOpl5aF5dBisHVQwwHEAx7X2p04kQ;pGhm~~S5IMR+ z+T)m^h*(Qh1N?^3K|@o`F4|`s@}>!C$wp30{3;0lQPEhW-|8e^m2;{c$ix1)pi%`F9q_U7+U`vV^Qyjc+nvP z=Z$YM+9I3;$!YE8R3n>*bl*Oghcfzn@8l!DthvlKI|yW*C0q zE+Tg7OE-{Ogz<*i1HXswbYQ;~E(tMdZ+50%1D!VlWT&)M<#->qz=AJ!?(7eszV|i?G(HPCrde zYsgK3E)H@remvQn-~Z!Ofit%p75}$&_x~Dq>fTP~B+N=CHfnD6aLkG%tSl`5lZkL~ zbtB>C<@mpe2{u-Cw*RfM*Q~SQupy1+zoNBhdS7R@!whZ;LcL}OCXb~<9s1AgQ&`SE zHc1>0wwRCVI``T$y!XBPWRS*p zt4VVxML~Jot3*UTF{_ct7|oQE(T0;Dg;8ISTwpDbth9ArQ8=MgUBBCU5#m-Q!kj6# z!D;QnQIC`T$SJCT{Rp~FfLB;`ItD68w~A4?y9J-wDEI+7Ryp4;Zo8LWS=KE8`|m9j z?ToNot0{TebUg%kA62+zQ%70>;pkgf1{zHLBkc%6tVs|!q<#Coe|QrIuuK?^;(bz* zLm7j^&E}Sk_%90req1onuXA_kUN)-)T*ermbevt~0yiMJ&7u#-y9&Q9VG|N&?MNnb zlP=$b5JkyM3d39##82x-C^sPcNC<>Q3jPK&?>b>J>cQ>pt}=N+o@ERo?%tz!-A92$ z=f#I!q0oViQ?mRTQ;AOsVlebhpoIq{kPje(Lp_HQHys zxNl(G^L~9kT&`^ZZ`Ow#yq-VH>ek_VD&AwZn`4^Rl&Vraos%n`^4@*eJ4UyPyYjFNI%T)Y z>mCKF6m(Y<_{?P^UK%o)FOH94wW;7+?OV&TV)Eu4XSOx$9(Qu{VipLT3@?{U$1Ib2 znwCNg)h{YCgc3NF##nw@69<~PH^GB+S|Q1!@WZ$xa?|C9u(FSDm^{9lv-^EKa>BwIQ4`2Y1Dce6l#?# zJ)i@R?6a9sNV(NH;X9oUl-cE_SjXzw+TD}i4Pim_QuG73UmV%v_n#3aqxH}B>f@l)+`C@bXy z9>n#t)HMzvOl5usFKG<^9$fUiAmx6E+8KXWm1bs>_6g;t$77r9Jqeo-Apu^GOpNE} z5+8pBVfVK|G=wT9aMg{sh{6YNj;6DK<{D+ieu+nd&aCDj`f@KtQ$-U+Gp3`an~AcA zD#v&EQYRk+=}#3^TIT=gE;Mv%$9>$FD;o`Sp3eVuPe5bgwXo9*UDmwseW!KkOpRkp z1IqCkNSP+3}lz;3Q3jZd8Wvukp4Q1%jk(z(iZ7N1gtoIku z&jy+9a+SgzO$N=&1`;f3RPY6<=Bcfaqbfu~$E%Rqb%9e<(}f`@k*|}?ASHPy_#2cC zh1Iw=L|MfmC8fZJ_sw~&OKYq&Z)n1zVp|0s^W<6C0tb#Dowhy{y%X;sPNve99O^gS zo8#tMo%T-n{4>z7Bb&NMFoV&{Ph|9CA4{%%gH&5Jw5nH5F{$|j%ce+}#tWiC2(3~i zABV?H(vV35nK9?3tgosqR|F z;X)O#8TMrm5*mSq(idl(BA>1{Kc!fz-~s?OGLuyt9mj)u)4DuQGHq#H|>cr0>-Ul8CgLwZOa*nlGRz^gJ(l6&@D|E*L`Z`~ot z1RgKkfC4OdeNfGSbg^-YH@f&9uL-DJm;UlTekMs?l_!m_i<|%KPu|Yx6U96)*vo*DyxCBFrmS=hunzZS?jQqyYiA}eD zvpz=JQlWiMKa;m9s!$LHx+z{L>pLF@L`gI>;C>4e|B zx!@Vn3_G)XMq*yp#{0YwIsKSQH`xl#tK9FK*aGWNQt04 zZjH`oJONfvI6_TcQ$(;Wc_AShuc4V27vjX?$7nGpvSC-4Bs7;_7mUN{pg*#oT$$!m zrV!5NEp{X4G7#k7hm!MuOBhxjp8x(nV&`RL{ojg|9D<1i^7bo1qrHR40$XtvrCzJv zXL;x`QsJO>NM6b;f{-DytRHAq%4m!yI^PC(#|@1XaTUKmy6$f)5(eM5GVXzZpW2RI z$hhs0Cv4lR2q>n&pQnXDLHi)XNC?typcY=vxELxR_Cx6L`hqwxe4=$n@XP!s%_6Yq zY-jNEg*dM}0KyYi&kbqdXhet^qPBU%jRiZ_B6gQqDB$1cQ$qw(`h>kGFy|L%(s*0ZPPRZrEn&&jL8snMQQPLgy0|-t!0gy?QOm-VqKwIJ z#2T0Q1B62_gIpr^EZ?`9^Kzou<4}^TR<<3Np)kTIZmt|nCDa@460&3pLUyc2eX1F; zzc*BkVkXg%X1~QTXp+EL;pHf~h8-c_h2MO=@y|0qXiT&)AUZ53`652v_a4^(_^rHx zcI3!PM>&M9=xW4f>nDL?zDbxWDBq;gpW1W%0(u832?vt15+v+@O|}bY8QckqTAp=T zXkK;r{tBa56Dc7KDCKAO1&97kX?EYjW#6sIfnY~@X%ERB8_`R1DPIBaZqB;?R}brBL@vwvk^gd%j{M34Z2yT) z*y%=Ms;YL-Dex>SD7NVJm3agWZe_7_ZOC8v_ROXK+- zLpd(((={O9b$)X$qg*_7m0tY>%cR0U71&y#zy`=!BzsZx4xFt@Z9RsV2i_{O6}BV^ zsS7gO{vM>TGF*U1Jja(0nKcEg?1(MqcAv|@EgENFXM@@-Y>*$u7Plz6&z=TKB}OQMr+kU${gB zJhWw`9CH6lIV?2;jp>lpzTFPPn1*T7+irtaRZS|TxmhSDasP9_5<_N968mUDg8EB< zL(LVz+W})AFMJBq6lXa)9~|P0JsU%ONpA=2;T%vg`&JB}WA!2}TZCbOyC$qJ!PUF( zhQc+F+-dEwqydVxBq&O$XJlDeovM-NO!i2IRL!y-27M8mp!FO*KCTQ)u7Qw(U^Sj? zQ-d47y-1X*yh!Q|nd9aXTa!;i^6%Fp(Yh_RgDEPd8KrP4r+{}5!nS8W!TY3eZFMH= zVa%b?n-lJU;leTMp;{X#{s=kldxQgC{^Z*o8F=z#A@wuCBJZ~0H z7kmgu+0+uwC@k8ab8el+g0X8?(pn((F-2d9MNLFQn~8@jAwyY1o@`_=EVne%0uYZ@rukgj*b(ZTG z!*Bkg-XiUZhNToGQJL23NhB^BmDS1_)g&r4{@82$(+K%2Mj*m3{k$nh+@xz<&Y;_A zm#GnRGSUy-PdkA^mOJx3+CM%r{-c4Jv=%F&)6@<2kpmo8M&i{xhV$EWLs-%$#Dg+g z&g6lIEci8af0!YRSq6Qkn_>ZX8x1XDORytph_0FK-mWe(C32im{P)g)&Vp`rw;!wS z7~hxvJW_aKYhVG8D z2cPHt-uHapxz4%HT>LQ?_l~vKUh!LNuYHd0T*}#Y_p>3e_(q{m zU;3@|)?u6n*P(nhk)p-1c$01w`^(bX4*2NKg7nWmt!CW{(uD~Zr`g|Qu2J~r%2Fsx`gfe3tX6pO%@5+wW2g$QX1d3q_8eW6ZhI?R%N9~haUSI z-(>ED%Qsu|v-O~Y;_|-3_{uVcjaFuI66gJYA4E{ar;HEZvn}vlAK^(!*_g3?Np14t zKvgZ6=zAdEU6fEd>Bq}e?=EA_(}f!Es*4mix4KIgJo{v=L^DE=^byVV?2EvLM(%sI zP4`_zO^weaQZXJyBr72Kl*R1a=;v zP6{6J%A@oTtRi<=g4U?otF=mte@E^xX%%P*)Ip-^WDQ~D-mlWA@O$hB;yx)GISn|0 zm^J3`&4LZKWoW2kcjnu2OMNPBkhw1@Dg}!9HgeHDfa1Cl%F-u|v<<14RneDx*>lYG zabmGL^N_KI>AK_6)zuh*x3%APbYGdr=wDpZ|6E9Dnv%I`KYiDK4{|~7F4cMkD_*Sj zixDC{bW_#!8jd$+XE*8Cu4_>`?2p)7k>SA z0{&TZ`5hEnWWUnhXx9Yn#B514u@6% z(w^Xh*-Clrp$Y|)Ao*3gPBgDap+>|fJ<>h&eb3R3!`rYj|FLF)ujIcSA3S$4X5R`@ zkh(j2bE$Ax*(FAz%v%4NVBvGVDxtge$2}#bRE;TC_$Z@C4n_};E+b22S|%hHZZ(vz zgs1m@l5b@YZ~hLQN?(Y2-1?FLC31&1RQdF?F458*_mB037qzDUstLN?uU9~{(Ie7C zha!jX^4IQpZjnl)|9Gf?l$_pn34%PCeZnulesq3u+gz3UlDb`i@%0~@T(EH(ZM^s84PqM27kCnPw6D!^ z-sG-lHDP%CE9JW^S%?)0UHPEVAw1y@huQ}(redWX3>ouPi@aH~toh1m-XLV^+7WSf zb_n6fV|M&^Y_5?Jf5nO0=)`{IZr^B8vA2 z)8KE#t#%SsoQUloATpCuyNF4Mt+}s_7S2`OJHnR;Ubf6s)jTp!&bZb0x^;A6!A?(r zOJ+`ljO&Nd1_xKP#e^d!ZmPMK<8ek}Pl6O0?W0tcx59&8WjWTe-oBG=!EsUh&d|r@ zUpqH4!*)1LmQ#$E5nf~?@JIcx9EK}-@1u~Y$?brbLLV7RBd@&2Tv{BL?0%DmZ9^%} zvy~(L2q~*l`EHwa6^FLJ{%ZYVIcr}AVl68}bYwtp$Kl%QMKMedA*Yl2v@I#L7A0~` zMyYy!bothE$wk4UmFp7$zeiuh_3|rNR&mgd4){GI((}0Xnwssf7A-Z(L%DBU!ejkB ztum4(@e%t4zOq3E=|E`5vGv_V0|ly4AA!aYT-H-#PeFiCN znzHL+p?-EOo8;0v%2Ifp*zanH@s}bL9<&@BxKF&{^nFEz^}BJX0B z5R8`&4==2iq*#~j{umUP`*r{4T>D8_*N>^#N{KV?+OyNk4f7MHCzt!?4SdcV*|rb5 z>P}{2&wP-j93*y(-HQm0ZCR$3l8^}@D?-vB-lPzB%opqXKQa|2+CP?7r$lpdDYL*r zi0)oDkg?Ko?XAfiGT*Q(K8h+u>|Pt@yyVs!ta<-^N4t`~tko5P4fc|?I-$mXEgv2| zx7!G#UmBUFTcnKfYv^!Iv}%u3=*=|Cv^giwGHl#?DYZ13h?c2JodlsO`nXgJt2mL| znsNn&%ekn`hiPU}#rX?y5BK?^KGVVZAnvmD1tG>RX`Oo(mlSibhB|={7ym@?4-F3R}X)z#|a*`-Avn4o1=?xrRif^ z-P^btu=&HvzCp^pKW%A?btdnJg8cAJ#6r^+b?=io%U%-qghykD@!*si=Xh=sC+iK} z96BX#`b<{Ux7$*Zdd8~0u|AK)n|u;^&`gosvtn`Wf>nE}uRGxBl#C-x*MBMzFYcL&Sfi>$<>$#E-hV}h>@2! z*eq5kWZKmk|9uvQ_o6TJGCg%pabYyXtlqDs&*$c?ccVWHSrXX2RQ0@{IEkBTYiL|` zLpQgLrjqWh#!pXfPcMHG7U?-CQGfd=XPaYv*Z4R2ZPr(o@9NI>x5cfOv?>3GxSWsY z-{SH=hYMQZbivF7dW}=n*%*3$+#q9TWMOuVQ`5>6yo`yPhYuV%NSj$%SVHe}@$hh8 z<9u!fb$VsyDB)oD+QHt;9(rHk8mEMVt%IZLYaiQ^Cl|=02Ao z7noM%U&j?(kNNnx_;I;_|N9G`mdDPV^k$Qo4^N3Rt!-AvAPs|~2zVqd_toZ5cIQe= zg{@SK8R5my4Db zBlo2k2CbN3q@o!UX29N>TfKeEpwmu!QtRXdnu4+}*Dv(=nD*@`5tZU}xIMGAyCw1r z@Bi}y{f5yz^;BA==&2y$>0)L3GmP2;*v0d2ZwDe#7tX&uT|#32Jp8tUf&Tf{?cpf& z=bNRdAoZVbWyy&*{(RF>QpErBEuZ;+7|!{r;vTbsm!M!|SVcx@X#o*v`*pw6f>!}G zTmhH6SkwzzR8FyquOMgX?V0EFs6d1Jm%E=RfMGYnQRn1*BT;I&;L42mAuLhxe70ye zk{dsSFFf_aK+kE*`W5;5=h#)eX?ocCN}e(%zrVPuh4g2zL=xX`5M)xWdtTyFa38Cl)MBXjvgFw9a1CX zLOW_$sjN6uhT{Qd+pCo?woTmz0!MMsfHepu|2+e9ZDN5iKkdQe%1>)2&(7O3!sb!% z($wJw{5V0~jEu~#A8w$>g4_Q$(Nwf;t{sWN6?Se<&-s(|^AA5X=*}0D2mLd05y?3_ z4Kud>a3fm!yftgmA5!yPoHJ7kD{_@N4=KZo<7iCEIsY^eyY3{lPAoqx@?5BB&m^Z( zm?b6ID^0r4s@Y-R5e_WYHAvlx-EohmpJTLlPvTj6!ohx7D*Zj+@Vv}E0wX2I^KCJo z|DZleSKQ_Ggzv49ChPX-F6Je%N9gM`DqKR#fZ?eOu&$RQ9!|5Qx&3h{#?k||UEBBh z%>46dm+|ln;bmQs>+LF}MotJqU4yk^hfrGA=k(|?CyOjnE|J&yX+1_@UINC=C0Q$^+3#ZK&I@@+xtBv&97ul|uE)u35uKt=xE z2h`o*5Mf^}o$+_I#NJ-pQw?V|?kqpt=?2T^H{{K^AHmYo*MNL|{97QMtw$0IR7QCO z-Nk4Q)iNYehrReoLvwa)EVi40EIglRk_jqoYYNv(_O{n@x8t|9zF?E2VfG3x-bL$e zCl}GI+^o~M#Y6QKV~=}Tvs-+XGk}z+hMmyqB_P{zI%(K63zg&}_Q|(Z z+&E*w%FUqryfo7m&mfi1gqP#RDmby1ou7sBo**d-Y*Lse05wn6v_p3CX?;3HC5keYQ9N5MY|0=vM1U6J=u(!~>3;_V6drpXjvuWhOh zuf$e=&i64)kdi}J|GBuvWn)8gXMeVh*T3orZ zs>{~UJ!v8ScoUPf?C>t#7vOWzpDYaqhCnJ!vY_rz`0?V1JnuXsq9 zPd@S!tSG5r;P7aTEpfW;Uf7)6-3hayA)IhX{;l#t&hmX-N+;x|g z0}pO*&;BqcTer#dRO@oHwRQ-Z_%M-c*f@36i(YT+#e6uM<`z)E=l|G1#iS@yja{Cp zG$bFI&ALTV-k6SYW@cUr$)X!FRDM~Z^>LND991xmnK~pcN2KjwJ7=HaQ;qcnA37me zB<`!~iW{uCMDYKXA0=DNBL7rvb>yF++x~r*tfta)zI4YoO>J|1xvyjAwzWxD5^tqT z|89Dt9E_M_8s;j zn@e6~r{@Z9T_8AF|WeC@5-WI(_DWtL+oiQb8u%;kY;AOboP3#JIsEItw7Ys0e zPsR4}i$^4_eBf34iXD9lB&Bu=hj7bW@d&XO_oz#J@?MFpbS2}-JJu_!Vn5b`MLivL zE5}tUUZhaM%g5>)-s7mXb>zN7%X#XTH{xTwq4#x3BE^Aot@VGHuEnB6j$4-?O$V*@@rBt> z78fD9v_IC(VjYBE%_}C18fFC36;dA>9(3%Et8#x|eagU`;Yydq9p!)?(0#5D`Bg7I zZNTZHoXtqWz3k1yf>h0=tJb-<-6tlC?ykOnb8JgoLpqk*gOkO zGshk%k~<*U5z)%o-tYAv_5@h3gpaD9r@g*gdTB&UvvR?oNN8)-|M@1fh8krMsl zZjL|P8oK=b!#nh~vK9j!OKtSO26ROIMrqC6Z=8ox#jIN1=TSMbl*?SIw%fO)maN6* zoJ!o%?-HGPCTcYtxPI}$Cz78Z1tER0v(~i$>HM&Nn(K(@m%%ygr_9=Rf@34iAD<_t z_gmPr-Hn2tS-xQ+Ik@5gc}Luh=5!ACPxSwoYmTB8z8RuUQ0QY&`)$G?nElkA@?~( zBh4J*Yq{gHVJw?OS3>5XOFX{H-bS@q%ZUWwO#4b{+j2O-0cpbn;+Xv8~D*Xuq;-_DXxS6pxGPQaa)ym?V&&0>b4f{=TEv-6=NLhBq0Eih2Wpdfs=)RgYY zYBJjJ!9IETKWm)9#?^w%|S(7^GPI1^5LyZYqAr{ zb54xy^9l-cP+)n zOm_gMC3oGR_-pST8X$8;j*bY)FP;zXVgPQ%9wiQRDgxb2Y~^;6_VC(;E5k>_|)U9nZ9d=!9YO@pfsBf9deQjXLu9iDqK5Eo&(A z$IVaU80mivoc7o1q76iGucG){@&#?&CSrw_Jus`Blr`}uR)dc#2HNo|k`)AoZ^-oo zwR4$atdx7MC?3*A-mod^MrX|bxTyd*|!P(zD zu|^^yLWo3I;h8knreO|B#pG|=Q{u`$u7z$Ydh&!#xU8r+(cZfY#^Hqyk&w#KyTj{* zXlMH9geY-l73z^h>khHE{13W82~3o99AhI3k%CT2tqJx^7S-at40Ra{&Ij(I!;ADP z<}Fknk_bQ^DjKSB3q&RuqF6rbnlndk9KHv z4{v1D+YXLd@{=pEFQtXKgKTOu9(LtyZp^&Wd8b74kY~vp**a&}3!g0It4Ri+DJvxzN^UqEL=!_ZVVB?;oA4r(sg#w~nF}L)=zF){ZuR#kSX;E|?ZMS=gJP*>OT? z2<|V98M`>W-fNLFNE|&i>|u2^Edcn^R%XI$XZ38quTRy?fKNmxF+HhWtB~_(x_4*J zgY`3Y)hle9e+_b}R(#hG%jzk+q~TuD((H(EW(>DvazELMjGQ-k?M4S!ro7vBykEQ7 zIK;PdqA-Nkv=+Wb;Y&ROHoH@E1)xj~M0;o#%+Z40p2y;@Ktux-YUrVEn9MU^AGDCe zd@!55mDbT$^Q(wJ{rsC--iDTwkw=9h#TN4tCntNR9?TIt8G^U$i=Hd)z0^TZ@K?C* z^{qeGayKa1zI=;aLc>GIqso4wI%_o2`pUIgh@GNRfreFK*v|U>fUK=5m*Zc_$EPR3YDM22r?I zSx|U#oU0FRg>a#VFe*_#FV8H|%)L{VW&NCa3T3<3kg;q9t6M%kU4xFvp%|i4)qa(R zG{(g?1{4Qdc8ZeF#5z`FD!=g0vcj;04E$# z9|cl{gj%t{SiBixkB!ch~YTdDQ$j;`5Ru9p%L z7Kiw_=JY%)nj+GmTy)-2dkme`#J_TCyfDgFIDQmR#bj>GE@u?BMt#sTIZ07re^M&H z_JZjoyg?j=)b&8o4iE4T$Y(ma(G7C#2S6^S+^gNp^>kGl6NtzccV@5tJ5;3--0P~T z5d%9)z5Ddsp>-#VTy>D>kVi_lRM}!*{R(33y1fVx1>3p1e(Jf&P)&ryW?`rspY_Ok z?c`lu01tB$&+|0Hk<3$e(uz{r!AQfIUb5LM-1K1JrwZpd!cr;uedPnev7873zk=0- zD~DoF)Sa{II~vbMYo2^(EjyChg9~546XtYanIK01NcqYpb`||Z!KQ1Dg9xufS6vXok`U&_$?*WtWlZ;X zvJ}wyPXZQ~yFI4^Jl9z(nc2!79r`fxJ8o+vj2YGrE48qx=)L)8S$KFqS#C)w#1R)H)@CeBKrYIr(@*HXK`}XVSo!-(pBIYt^4y<;pzn5%L z(R%kcT{(a7(;fGqWNSx*NjSv=11tEGwSGI>D=eCj*eNx_QM-pvbT2ruz+H3*{wk>K zvCQ0dTtU#B*RAvZcsgux31K2%JdWm-fQtZw)oh`Zm|%mmt58aP9xU9UgPDl*pLLFnL@KJ;yop7#kMVe zOOXzy*yMf;}o1uynhIKJY_y;3}i^LAM z<+NeR8A*Z*NS&~Hsk0IVMy?!WqFk&v7Qc;V4ZkBMj-5>q9Gj|Sc&T;;mzzhFaCl4} zIPcZmwA1Ap+nkzO9o1vukL?Vs;!_#&ViDdez-k+?0*bZX3^3e)-07tRlXd^)D8wRj z&->fauLiHLSiM%8`>{$2UC!d0C3k_a{cLk1_~LdnoZ*k{B+jl9c=Oa7r&X}@OkBQR z$%QYeWAA;n&NT;_P7J^I2D3G8msavD%@7$YMSwfMUMNbPjPFkFMFMumc@6R|dzjBf z4jr2jpXw2TrGFb^APFu3gID@G?8@Q2b!)|1VoDA`k~c1(f8n`8u(sP9p#-HAfzqjW zn*BGy;KMKa>!F`;`gxO3kh;U|NTTq2CeUFoA&g{1=WG?lUsdzx4P%r$K*N>;DiCz2&27?cj?~Y4sF145tU2-}P!OjNXrV(S^T~0F!@c9De*@ z46eTT+vh6YO-4cL{BUUo1k3^j_Y$JA{{pc#BC0wF39HQs1dVT|@;;D2G^sBhd+Xr6 zB|u^=i+{x+F~K}o+gb}I+TC8+`A!DflelvVwgv(dw>LW;I0r@Fz`M`lC=?@SyqJyI z9)jA7k!{lHeSPk9D36wKI^VSr?Svn@5igrjII*~y0c4QlXfs=lkJe5uxmCDnWi2Vi z2k|;ng=Z^UZ{d~;ax{)8j9?j%bU(2?nBL6fM>*lj`tYw*mh_d=hi{}WT)?C25fg;$ zZ4-vm%CIokYs(Cb?LFrP0@<;HX}Dg-Hz1>1Dh z*>*fP)Z}Xyj*zx}^%C#x99XFYp+BqAAIo4N;>wqlB&XO_H?uEyPJZ<5kZ584x2uP| z!6#-0)pMhSvVncn%71sYh!s1wW&ze@PJ+WQR>_QlTUd?}w@KlkImMiPfN$NyH$OsM zA+A$uaRIK3B)Ydr@xIA83TeD2^5sovOO}~$(8nW;brteFj6g=7&scvM$SqNN6R3N7~REU;z=l-YAEM}KKcUJ(+D6*zp0 z+oB?CNaBxOy>1s%h3UgU{t^MYwNVyl{OH?w-*{TF1Q|AS@*W;yNkmkG9h_yP79_#% z&wCB$Dcp0Hn?^|UIlq}6mX#a6fLAmPROcyO_<|5uY$urx&@}RkRods$4W?-W?0aeK z_CuIIHaSktqLCf`wQWvy2=1!Tm4kDXpotI?`{Bj(!bRjV7MQ3cLic*iCepU{C_MID zoq*X;$3R~!O?&!wMC2O(=Y$0;sXPH#Zkz3vjV>^B|ceUe$xnDv=kSoWWE(E zy1%s_Y6skbz_at|er`sfZdGDp^<_HP@?KI{%l>xa?1aJQ`9#*O%UZU;sVKbYq&@Vk zFB@ki6!fA7U1g@%fiVx*bs)tjyM$*L{RP#-?clo*-3JwmPEZP}PV0qx?FI8t(YT34 zG)kLIeEqVO6xJz(a|LWw;?m1H2|rFFCMxNJe-)Mdp%mx-#T8~dzgsl08=@lWoqgS&9bF`4jOs*FZll4hL<(? z7ab4!w)MdK;a|`xdmqg+*RZu6@p-RWd?W@Y+wmYh;DNJLX8=@cfT=+BCf-~1=ZaEP zvJ50+s8Xz+SW*(9=L&!V03DPsn^)1~{3r*ffSYnmKLNppLpY~dr=+*%Kq4Id7oOn` zU~?ylE>k3P=XVP7qV}Mg8z?vrQY!Xygq&hZ_Gt`oIN#;~#Xy;CTCrNUNK};=oeqG7 ziB2Gn8vwv71?yos6`v=VWX=-nyOiFQ&3E1!=Xo?@Tzw9)m|!n!bFt|5Afn|(gh#29 z!`ZQn^B>qU%>*+E&~cq0%wE>>Ft25%*twjT&c`dIkm@q2lK@ZAT+PU}8S^VT+;%aH1b=|xWz=mvJWJnT_DB$5U%Dvi~0~SQfaJWJ@Q+vL3f^ zVG&@*6T^-K?U4lc(S`*$u(I<^WqO-+0C}8DQEiNdr&T_r$HTiFrM<%jT*55C2TQA- z&%B49u?UYWSVJvThrB$@Iif`)MU?krd4Q}sqvm}wzvISb3%|qCHb$OanRZqpFlldx zWbLZi*Dw?L0YQG#v1%A0Y>mNpNUms133Yz!xzL&!Lj&XAJQ)1J%uE#ut!1&l_6r(KJUjG zs!E(7VYO+WLDTMfAhnAIaU9{&S$FX8-m5G~x+($04-n}^5O%1-OMxk4!OaZCNcanB zX6r`&0B>T!18)>_MF+#xfE!67tP30)0<0*A<2hec5{5l^X;C|_(-5Hk55F1+s-zTw z-EZs)a_5WgfR$`P!a5gIE&>6!(9-pMJoZ5SFR)S09f>k2Uzg&piZ&E5S3Z7Hxb!RW zC$7jwql~v)Gt;w*1uVo1p?EBkBs>y{vn0T_>S`UY${V|BQ34lY$e2?iJJuAu)Cc&e zlMcLEB+A00gy?fo``&L%6-|CFILzPrh^s5J1k{Q<0U)qPznlZ}34lQ7^UmqK_2VJeu>*-_HV`K0^F5z8 z?;(6CodNgsT{(*6%(tjYO0h!Z)xZJkmjmj+=UXddBy;3^OxjW$GANO_wDXVlm0PTNsr)~@LD@8Qd zDn8RHUZW=DWT;d~5ro5eRcirI4!^_#W>-FUy24}-O$Z7mfi{^|Ai1$1ja-f+c#hyt zM--6au)y07>~qlM>I--tgm~RXX_WG=KD+IsL#F5R-ego7$HS|+jJF`-;JJPz3Wv}6 zft}~CTkl%-vSykP6|DkwYwB8H@4hfQ;JqFH3lEx%3BKu%?G1=YT0(YaWX?wnd7(Gp z7|l+Gz@TVgnv7?WRCwPif!Se-HY}I5V4Z_vm-TYG%{Oscw+h|tNr8B+5vzezhfY@{ zKi*S^3m>{V@Be^>^7*W$ys*@~`MvI-Fyn&4+|pU~3g)bOu{#8PG_t#W{6_LRy*syFeNdN|QV-ay zm`{I}e=~*23Bh0T9c7`$ooRhuv=kHm&m(Zy^W1Z9B0jk|Fd5w*<$-ppJJ^Sxd2Ycg z_z4b^ubwB${zavdvZCfAvUo2Z$>CkSCOwCx%4DnAaG1a067v2Q=4|0$QJj3&XEL^{ z+&#PM6U**kwu6Sx#_OB5zV4}gnrF*Y)T%(>-;%Zby=tW-K}h&Y9ec{a%3)BQ%}}^2 zKC9K}P|UX<#$8u9`#}MfKTcYUNd8%`6&sAOlW0yh=F{TpHJ`qCdZmJ_sGOzch;-`1 zjndqFoW`*JYjNNRX>!!Eus6$o3tfwfr#H}kq`tWwkuJL4zLr;k!(w?a{)pLAuXrT( z8V6yU{9AY5FNO0pCeL1*++21S{V`o`xM2I(&11@-d1>6aM?PZhV91qfe;@o>PouN_ zvgJ(HQ>$mYR|qp6_DxylQxH?=AB-4I&ITaJhWqP?#70L8OhhS;{3bt6aM;4T3w%WP z+jBU@4$Y~r1u5^pJu)Cpb}R)fQAhs6l1M?wvrYJPmVL{FCu0ERb6f1BuAReS z&pcslZ}_aBETGTkS^pk~6)`?%^PB5L599Yr=8IL!HBtzTbDs8$5ZCU`G@XgP+BYyb zi_MUrPsxK`AGyZqbBvCbt=!wL>pRe<6QPI}(@bsihP7+jE>-HE_#j=bIZJcoH?CQ* zit?_7f_=&j`A16ToDh}KH?nhYJ~z`RzV&!VM8hYkYWiocH7oTH1(NH@x9y@zpQX4g zKC+y@BpnPyW@=_5@_rkfDGXhd7S(LIAhLTB|ID;(K5cn^nDP?#G2`&DZ1?dz*e#*p zf1ATkVdDBCWh*(0VNMjOWP%E>%XK0*Dfrdr{Jtx+cL}`H@_*K&ee@o2vL-ONsNb;} zc>0}e1^#e&I2BzxLfTh(!f!H*adNKlL5m+J9RVDlNAu72(jl=3U)_7J?;VVV$r)%w zeK)O5o^ZD|nk}S%R&=y;|D*#EgRpTNXps(0tyd9;?6z7s5W|x!S&I&G4`kQallB}< zWTf|2VZQE92wryO&&9AD)Tx_44LR1kZI^$Bsbi8GAK$|iqGmIfMD~ZnDx1@0@4wODAGC@4dv8P3+gmg3X7 zhfN5~Bxa`|Yd(iCrS}ttCS(t(n<@@+&-Ul)s@+ftoC(+cs%&cB6LPwnyq?(AnOPn`**=DikJ?U&A}u}|_)f-7yjj9Y+&`vAi9vAed5m%* zEDHX}j?j-7J*a?8EAdbzYtMz}{(G{bRiE ziz&}S$C~7VdB56`5p}Ruy%`ppOTB;vZzs*FnU6hlNJ{wl7*-`H06Bq3)(=#j z?N+IC1VT4hobv@J%*4^2`Uf%){DZmoF?*UUJJ@gL*FhCCl;NKVif5s|sKJfxOV5%< zf;nNiXLs(ZKGf7is@Tg7o50K$2Nl%>$G{jyD@&%>ws@Kn@lm_(QD)rjeO6M~w$gu2~5-ighM zG^`MQV3Xb%_`v3LKfa9Y|GbMZxLWgRu1hkjOWmtU%q7QT z^J^3irC9&%$$l|sCKhyB&duGwbJZ_(;)*}a|6Q#A7|j1PR#$Y*jHl&h@YM4i7A1de zGnF>r1U$yo%Y<|XHO{iF=Jehl>M;tA$>Gi&zOgBdzcqUzayV@X*EkWp!Zs{O7FFW| z&iWpOj{nsy2K0)9(C<~x@7ZTXC(tkY2#od+xq%`u3NI`l6ZDeZn5>vQx~1{$9VleM zR)Qc+x-$TGQuD!|+2N<7wwqUvw`z{w(eDuc9Ae)55g$7BQtHjGA=$MCQ zd_qN2pOp(fByH2%Xj@Bu=x*j}D+s}~+>BIU{uH)VGb=XcI161n9a-Z$qofz7z_IoI z4P{Yx*5mM*lbL!qxQioq486FJwvX$~%#>LBpw>{xaRZOO9?Kh>uEc`2vE}USZ1T0J zMGTa+0L-cgW))?8*RIa#1KVyXd)Pj5T<)8-T}q{@NYTRe3uDK~$S%xa2X{nt7#tnV z32TO@fP^)`TtQjM)%n7=)!ei^P@ac{Av3LL=ds@7k{2;84H+fX>UNE|+1R41A7pOB zdKH843!IbGauX8mGj_<|_Gcd?+Rx0?aa@b`p%KyAj-O^*E2dRHWdFbk`!sWB8nv_I z3tzXaJne9eFZ@ZRdm{zqjGwU;&$pvW(*pm3t7&@kVG*h8ST-HG@->;qAST}ObR*Ad z?{#*T^@(fgbE?+QWsWdM#J`pas<5yVpAb?!!oHCsmaDh1Sd+K7YFhFEYa$C>j}7W*j(wjfpm2ZeIlD^2XAVl^7-$299*ZM_Wd)S_AVJjvK+2%} z6@T$-*K0NckCaEGy>_R=6PmxxK1!KHcbQ%J_&hG>f~=L}w(V=za@&W2CIss9sq;P? zFTA~rRu6C7x4jGikZfalfYxqw%9F3Crq>*}!^%I?;*KyQQ4f?=`dvWv`ATh%H@J?G zH2&Aqi-Kd_PG>9G011C5^EnFgHyp|SHosvI%LFAvHj#Fz!)ut8@EHqzrw$19Uk!p+ zcYRxq5vNIHtLh+PeAGWzC~B~RFB)iK`K{Ai>2%M(5MN2%ZD_Uki~R!;Z;ps-PvTHD z`J)7Fzezbe)ZDzM_I&o1o^5Uz(0Z-3unSNCSrHtw_KX2>daW5aGH`|_#G&q;4tCL09)XBwuCZLGRU~b zW-S0?9aKpTj4PZk6r%cksXiI){|;3@5XtLu3n%*Se<-|%fsT>gIO5?S|52hi+k-TO zZZ}_U_T*S=Cv^K2>q*=yH4pikaN@)UMoa zoED?|`ufh`E2G+O7P|$wLEqLV%8I{$USF>A3m9TsI=c^of-3eOG@ues24|ucbO)X( zqPbYe!0Fw{A~>1l7bL;W+>iWQr4UL338)<(-nt#^S@F?qfcxDPYpClkwJ$QPI=>$}(eYAqJoozIY^EseG!YL`_C*#$q-*V#OF zzenIV?nvsa-)&x*beuw;kDz((T@bMWooZ%Z-LInc0K4~-g#l(0m}ozQ#A>|(b#|;} z;C4t$<$rC7W!&DC`fi(gvck)nXM6%Wf;&~*1TIMsw;|qV>xTJuH&S%N?G|i{#0hXA zz|;K4T%jgeEgkg@LiU=+d~SQ$0nVcpVkdL@wy^ z0pDG@7lto^!Stkka=W!EJSRi{=#%QKC~Cl=V%!T6Qfy;ylb6&0GDQPVSZ zvl3Es)RhwNk+U6=4?;*Pgx3Vp$pA?l%dX=)F{pq~v+&g*Qp=MN@2v$ca(8e~RZm8d z>7&V%&P>>zauYaT zaa9DBVxBCVTon%59MM~WQvU3^XR0L|TW+@K3aaNTZ`w_9wLcUM;pBDVY}@>sYDe z^xi%)A?9YVzAM<^^%QSIQjz=)-f(VGUcf=&}~|j-SBWyr-jEujPMgDjI|nGwHMmVvOE6H zL|GmDd^r>QD!4Y5oB`sP+SvdMG`!_fg2$Tmv)2{Xf)HcwK{fh(f}UL>zEk=fUnJ%&W#d<&a z9dREQ8Yy-c%O4K7I-I-!0Use+UdY)dr@`0-L{>FOV*90|LBq2o&FelcYss9%*GE%i z$MmIMxD~x~*l6p(j`L}l=>MXQ^ABGuQ?%O`6)mi)r9xMX>rxQgemfg);7zeL-D!U5 zh`_w=XcQg*6|wvM4@aAM;*|QUS7=A5g@$Z53S~b1o3=g>gs5@V2eK2ZxtDpaOqxdT zjf;>~yJDJFM$VjEHyfR=SU4nIyMixXX|WeOZYk^(&CGMJ#$&H<%|N?1@QS#wu(Q*O zj>3R`krntgIp$^`eJeuE&|?$slVD4WS9y+C9ESHWfh^h*hjU{mt{j&xTukXQU6pnj z>rEJ)h?mPS|K(&PdggeX5M9B;N4`9^SE*Y$5$Um2l6`tB$Lk9_mUx-^7&A6Qwsuh% z=4YCpQ9^Tr7C&z^kt3^udzflv=IF-DS`^u#sQ1jEb(2yYWA$I%a?>R$R#6Rw@z;*rza)m)BkQ}PiTNIr zd{=n}ZdKAfa@z?$J?urNBqrur2#>t9#t`tY5pn4-AG3v~AK{A+{A>wZSApk+);|+I z8mC>n`%sVXqNdMLe9cU%n6v}B_{2XFT$>w{^H|f1HnE>fs7(WM$|<>{8dsfq8hJ-u z5uH4N;t$PV^s`x`7SQN=2`*i-Qz|KHWp9% zS*Jv?R-g*Zx%}iLZmGe}j(g$b!!d$r)XGX-#?-YJ&e?er2eutE z)o<21HXzIa;AKqque*P>&fj(OjDBX?&wL}KJf|l|oO_9~kV|1Y^!I72D}tmB_AEc! zBkX!eU{xS%kTJmH>aw7e*JG)owzp7abG;2zh?H02;kf}o0h}#`^)ujzLvPsDd5#Pq z2}TC;eFvzUW`8ixLQ(Em#wn$ zYO4rh+1Q=iLEN9s38V4`K!fSiO=O0O)8n!5RQeTPlot5avMlqv%Wr}B#fK(UeJ%WC z0%KkB)Ulg1D%nUdU3+m8X)u_jf;f8FI3|v^5Ysg>ovkO^wjr_kPea4Nl=Xzt5L{qY zkFf)HJRfGVqv40@qJm?M$_!#X;NP)4BLkj>lA})s)LU<9*ePmmt-4B5lQQ$Apvt%Q zpp7wIl0Q1f0>-u9&<)(woIzZpEqeFtL-hY*??2<3%DOjT7>5y51ZGrJx`P7JkxpnT zN+(FK0hHbmLT@UHN|i1pAW}l_H8er#B@lX7S_lC`4?WKbmMQoD{k>nFkI(raGic6U zd+k-Ob*;U3;P`yJ=lHjgTf_g3;;LZXr)#zSV>u7(z!oavkRh6CI_8YB?9RkTZqk1h zB@8f5_;E*ViUi~g9)vR-?n}iNy-AW=D63R*MZEBEIDQiS_-V8_L?S@#vUZjaWlcLI z!N+L;XmoaF+HuR~UuhfhQmp!1q?rHjOd9q6D=PBC=8YVzfh(t3Cq_hvL6YVPTNURH zE;rfm*w!UGLGb$Oxr#-;*kyI_R*QXt3=>fBX4>}nd+~?E3g6&tT&ibRpWYBe&6fzL z4rudES)1)I9&9uAaHB`48re#A|CKZpp1fFQZj>1ae%ST|IGv+AYM(YGB8fVBT!h=# z5)9POU4-^f-3;E<$Pw@|CG9fhNzX_wOkR@}UVE;)V?w_V*8PHq`YrO!N>l=ruu>*g z)tG+uuhne682i`1+^$bCNZy2jXCyIM{Pz!(tH~ESr#Fav5!FGQDzLNk(q%11r&6)R z#r1ghwcs7C*2n>E)h4-{icP!|TN;9S-rC~LX0_B0^`_bQV(D}hHosm#*tBCf^;7rl zcWQ~pe}LeA+z5}oynK$$#a;lOr-2BPpVI+l9H2KLe|vx5oeTB&wF}3)$Vt!i0}@8d zsa+-N$8yIDzi;y-4ZcWr)SWqOYYTk)HXyi|Mx;8eneJ{^)BsKX%)#BBi%)$PM@_;5 z1hX`0<-xiZ{dvGKDWTU=e+t!X4`s>e{!TG*?6|0rvw4SNb<=@dr%zQDth+F0pYKhc zy;)$x2Dd}wY+?sFqRi}0AhW z>Sx(J%Kh8$xki_mXYdU)(1BuUNClp1J~Uil4po~vJGF_G{bt=<~r-*RQD ztqbsVx8+=WA+*KsYetyk5i6kA0VLmlo!<(5(=`iFTGM`NR2O~yvBNT5+u?A{s;ua$ zjA&-Cn_sc0m$E=GhtFJQ#hy58?FEi{F71Z21WXHFO-|Nou@JKZ7?;qn*v1s8WexO zj2?3{zzWc~Agm9yk}3mOS3XayCYY&BZkhU?1y(38iua+X0at1#;Pgoo8M+K+;P|@7 zV*ylub|cEs%4sw3*pIo^5;quVrj~6>adMjnIIiP3ypvv7Yhtg~&ABR+^?DTM z3XsT4hOKhoYhJ0jd1>U0q<*5XyeV1in+<5!zG0=Er4|}bm^CIc1lz(_kE9A*>~UdFFsNQ&Y+$?4kB~|nd*;vj<4>!c4CG$X zi2fI~Ggfkr4oG&HymzNY)sA_xOmF}Gq6n+Z_}yy=K0iM*kIJiPD7E~Ehrxn4#N^;E zws9|~Al2#h^^bwBh%e?XUXs(2IP&yy;%xI6Cj8P56J>J4k(juJe~$YA<-S{Q5_yx| zNki1&DPEZ=T#__ufTv6LdS6xkeegBA8Pmhq8Jz703RvA^cg{bx?*UQJar*FNPi=vd zR?VZjrO^V5^;?t~pTU=Kkr9qsBiY&6PICI>Rc zc^07RJ$W;ozvACZzwAreT^WU%XzPxo!LZ@UBkPZ&DX^vfpRfOW~^ZUm*I`R=KQnry^gm*#k}gWT-Ta^EafO4*nPXrGj|hi zr$1wg*6JG}WSYO>=%SL-co&;IFeDA|Z*w1$H!nyXJX$o65P7eEg z*oRea;f2=BfFg`Y(pRohdrbaHE9VX$W%kuf4ddb0Dpf351|+7V4-^D^weWCqsvP$u z$=i=a-EL+RoH%Y#SDFHJgIwyzc1&NDeOz+kj$dF?Jsh!|SRg))z z)1;fGUpxZyw=EFrbjr}az_}pMx~c964}h%dJKfx;p-_SAjD8!R)~oOm30ss|p8EwhE^4t{GjVuq)POe5jbUOcxUo;qLLc!6!U(h`Vr5LKLXt*LEf*+M zRcZ#zua$IO+d6vGwtH>=7bKYV$rcdXO!7t{CW0b3JtZV>Tf1U8jg%vg?zsM9=pb5a z{zDT$3S8Zvw*t1Ira93vi%)It=7{W0>SM{57TG9JD%8skX7@5}guJ8sRqRX_I&;wN zx=LVO0J)N5Bm_zbuUduYv^U>iSwI#ph)od{vLSdn?XrYtkLMgQxY@*>v5P$VRtN;Q z3O>tC=i-kK_I~2CSJlk9_PHg~O-ct6HV%blt_ZOdfek{L2cDolh(950iTXNLAy<7Q z(7sM^_}vQln^zAMM3UTqvq-A9Zp=S;5%cVJ@ynN5w>!u-ic8OY;Z*9&z2e{yacyV( z{F9igM`2!#Gh4}19fz?A z&kNOL&{pEv13Bqn@?M@*Sr{({t+fpA8&7&XKs#vNQu%6E*l_)PzCf4^>$)p`<=z&( zMtyeYZjs=E;Y}SJM6my3APfYHOs3{O6A+ychS(L;mY5lKZptTxzu_cbl+D?VuF#%g zt4tx@QM0J=^;1TtH32kaXfpljWv(*Yg~?D5W7G)-?#%TSyE<{(4dgb6`BanXmpxGP zEt%7>gbyT#%o5DD$GI5dV$)D~g#}4K3W{au!$x)Sp7#(D9u8Wr?cU-wQ#csz{nFOg z;AI+7iXK~?hR2r1J+QyZWM(+7{ZWy%abbaU;5h|kYTw}rb_}Q=OnJ}ts^o_19N(Ku z8!Td}IpYTUG^{3*nJML4ph}+!hKGX~#j47w-9VzrnlvxX)(QsBrk3K2Y4v+J@uY)L z#zz;1Fqe`&M~}FjGRJ&K8-pIedauVCMi@MTyKX#9G@B6oVDVY^wvJ3imiqChl|iox z81k+U_P zwQF7QNcqd)>*CIyZ6OhfSIf|ilU!qUx8s-kyJxv|H{8!?4>PU|VlKgux2OFE?P0w= zZ8q-rqFubavl6t>*^G}ssB9P0tVve4INzjEZ{k*`_sY)&qd9XM9d!j8>w@oibUb)& zrw*4z$sB9N(PoZ}m#8m8m$L_+yDA?7(|ZU5dxcN=!g`Nh8E{q&3O-s8kMDdXV&Lda zt6Tw^%2vc>r&x9i>#N=A^CZ1y7-k!)?}m#}$s5SAnx2<~_#`jJ_+_A)T4=Oer3EjC zWrXv+P>*(>!0U;^*)QbpiInWv1KlqE_}|*tw6*+8de&-|6mB(S)OY6(Bpf-ctMI=g z%rpJQ0li3dfDv5}Ea3A<+f>62F-8h_^d4oDm=9qD<|WS3`a3GZdRg5-Te^2cx4G31 z76rVpm62sn++*TmWG38XG>)q`v1xFjC` z?u>n!dsRj@qL}YoUSVG!$CL0(9+kxh_g-Q#fZAL+I6A@=W7R9A>h@+`Pu|Az_!BmL zkzs-Myj5(E^h^Yg{cGEt9dGFv??2%$p2pcR)UkREN#ppGLT(Cjj#NF!$9U3>U`lB$ z6MR!*8n0`*PwpwI?H~V%d{|+hYSk2DYOZM>Jxvp*I(^CHTg&HIL(L^ZOI_zq<%tm{ z%B*i(t?@}f`T_F9QmM6f5M$n`lIxk7?OB1G;NL46TpYf#e-)zX4Ukn+;BYSo*OgpF zB$fZU`ObE}2FvDxq$E*n!p+iBFgi*SN!`{OFZ~tIw?#F4(PZuZ{(X+^k$AEbpYi!a z-Pd?mw$C-0ih~$#aX4q;XlYT+n6rr-7xJZ@lU(m;Z7lO|7Vg)~xvrkEJK7fv`#o#+ zWvA}yO6i!R1XpD~7=94?Y|qo%zUDBqlWFke4T$z-|5!KQsoQ=&@Vc?yQGO1`#o06O3e~e-oDyUSj-&FSTP8=@8mT-n$Z!ZKA zXME+3$VWY9#p){uxy<%P3qfyxi)Ziq^l{oR)?`tLKktK`_GN4ZTji)sfQf5C-0zFP z7y9myAE8Y28s?(>FOJ981GetnRXbQ~ws&5@9eJd}J+r#SJLL6MEB?w50d!PXkPb1} z@pAMJY-J1E38KqelXmiMTLi)AH@T;7UpNs#eubMXZM2SG7pqcP?lue>!N~0r4w|$d zhU}bls;V3x&`FIv&7RimU!kPY<{QePMJMIp9vHYeAPU!&YCsU~e$C|r`LBpoyJfe4 z_*YN7rcKIUlQ-V0>Qw#HqDE8pS}+$C>roWS@Xy|E5JUwz#pmWQSkY$FoOiBm|5Cs( zgeH6;g;0VkTiS=tFx~9b?tM;^C_WOEC1iWJDkl05cQ=dX6t|TjP4xSschu3|u+SPT zrWrW2SmkI<=ld=*|F(G+?I0#}2-A9Nt}0-wO$58zj_kUcYcD0U{%)FPm*7|_+;}!| z(yT91@qNw)^H6NSjOhpN(jxILR9GTxpH>2sDe3xJEzM(X_kf(68C|Miukyl+I&=4B z?{M=<-aySQq!~MIb95gAq#nliUR~>;shTiaETk_$saND3$ zwsomiJfttfJL_qEtKHqYIYP$CbNA><_5pEU9s6%VLeu-sN{=*i906xDI(R*ayKq%0 zY#*~1KO`y&m&`AJ+o7F`wmsq;+2{WfS}>4vZLr&Z5Z0Cxq>0@d+X+2ib&)PAM?2LA zN+2mbyrM3B|8h!bEcr6DK*!iX9Q!r)mTYSZ|i8UzC7GCoyQA08d87A=@?hG75M zPT(p)-XL%<)P{Bhl!p$TASDRrtG&e5%WxeU~qG`$t|a|2PoYsUM1SG z9of2hu9v_d%v-m_j@)Gabkll4QIVU+6>36}zo4?0`|)70wT;}*$HA5R(WbfMN9B$e zC!Bc(3*#VIJ6KB?z!&L-wwnm@Wh8wEmd?x~I+d5xz;(luQNsi7~MN3l!b2cl}5gJ`bwF!rT;#` z#p#4kJLzd4-(Jis_?sFFoLy~E^Xyq|`WPLS*J=-!%07Orzjo|`Gpct(-11Kv_4r2u zvbd&sIhRf?N%*K-VqiT2T@yEqxu~|I30Hm#f<$5pr_?K!UWA2<7y@7aa$D`-&eHlV zm1gns78=)R>8&}t&;@-$jkzk1Fkk;oPa)o|F1xKO_*WZ6H1YHL$8zR_7_llSi5uu} z8G23Xy3kSCM+N?3M7Fhszwzu0;*N8z=Z-u#tll~^!&+Z|J!N)w$e&Pp5@*d=qKp;x z$7mhahrDR%Z*EQ^ZBZW!X8#6b3X?t~@(>H%DEe!mO<`=??cJ)0RohiQW2FmrA`jh% zMB47B73A#6hMB2dW^#~)^}^aP&$xous)TGds0&w8>rGcVo=j&OK;%$~}%1kkrF>W-WhU>gB`o!tbHkA77HrzK^^3tU^%ctoEHG%h3a; z7~9}NeX3|pp`0YNeT&I&aBy2qrfc?qxE6fC4>yA$lhG^hDW_+;<5a)X$oyon>VZZT zn7#0}|ECTROuNx_2`h7NXxE4ByyJF!hZ|@^lak$~Rb;txZu^kgO3SG$2X~DQ-TijV zwd<^tm;5E9!PJ(+lYt-bi%KOoTemdpY?0__d++*(?dZTFrzu6zVHBH7(eknH7;aKNp9KS^VN^{c0LS`_b+o>CcP#Z%y`R0lW<8 zsRBT$#bB$w;iWr)jb}1(Wl+7cF^VJd+TXgq&Ic%sU}k6Dvl=qy4cHu#sT7CeXVx7S z+3V96cBsX7l%Z6qEC#G?RjuJ5*hKii7Q(&-zHqd;KXbxugDmI|x$k}j6#WP;_-}og z=S_7re^u=tLU$aect!TX0vk$LuGHRl7}mw-9Da~d2;bqo%?C+?7I~_plGoig&ig-8 zjb*l>B%ISX@WyAHr*DKPon#w*y8sbP+dtGbE{};VoER~}c30=*FQ@hC#4;BJ`-K$p zI2kie&!7ljshu{8geA3jvwY>_=}a7VUVbV)s~s`L{8onY@lgh$M_Ban?4lVzM>PM< z%Tsa7S)H_TPC`$AyZM%HH2IMc!RXjHQ&^?&U~&#(*%jt>IX3kuP811>^b@H=Zx@MaI72eUp%TVLsf4XttfzoXw8*TsT`NL%SAZ4 zBwSd|v`3-+B<1;;l=LIbX9;kZLzg^wtegRh!3@fE5(Y2VOYOcSnY zP{?a6WNqrga^8QeV)WYX68G&;_2Wn*r|C)8i)?n6*qWriNQzpgk{8=9${zs~4Y}asab!-DEe}^s={9YIlxx>o$CmZNgNg@GsIC1i9g+R#^E&n7 zRVp?)hVZV+Y;e3?OU*swj%Y70OLkp~de^EE8PkrhF+$6n$Jdklcnd%&cJmrbxOmgY zVXb`JwRlsMXpBku^ot3$`CcDsUx#J~puS0WTdOwxb`&Bjp!8X)aL6fpc66-_bEW^1 zS*OG*kgn&&80X=0D65P93WYf`9#XqQn0&f=TtLEcDnjWyFNRCLb|z*F>wZv;UBP4Z!sKTUhNV<hiKFU=Vt?ZWQX+~TPi{4No1kaZs8}PX$?p|^Fkuzn=72Wro_e;2r^THr_!Di5$o=^r_OPuYuVv`PKc1PSsHzm64HR z{WSU2J&Z99`6mF;XU+m^qjUjntN*QiR>t~xzsq%sm$I=xSI1zpTV=|u@3~szJ(+_- zh4Mh5VV(S(0sVuzF1?S2_rMg()fwRZ&A7z83t{C2rl>rfVT@6lCsmz zPDxah{DouH{S*+g>ZW+{$kl_*Yx{o6FIu{Q;6(LGB!ODZ#zML7sx$cMBaP*)VX}~T z-t6%YHY4&+PG)P7KJE67swDv%6$_1}!V?NfbLA}C`ZtP=rP-xwD3B6-2fPYv`+e1g zcXW&p-zLqrRv?cpt}n>$SUv12kUdZm89Q;ipxe~#>{1?^IwKknn!+DSg)Kuh1GmZxAAO!{ZOCJ@4h!C#qN`Zs zIS#+alj^U;>?eMH87{awnq3HE@zEJMzJ$a=#ts+xax@F5=x6#H_QwTO&#SRI6Fy?>tx_9!gR8Gsmqlt(VO3e#Yq?YlFM0!|o@i?;jQnw$w<_s_5S zdsPI{vnLh} z4rwdGSO63CC=>*_Z}SLFk1HxIo+GuOA}tZsA6Muyd(;I__3q&ApW?F`Em5bGXn7&= zCf9gSf9g*T<-UpwEl$xNW>$DbOyI7DDFFtVVn%bmg5u)eCQ*vrz)>rbULXKIy94;& zV>OuL+H#+5G-a1gw*H=^jA-TkOq=PA6V7X)B5nv=rZ)_Xo8q|aE-6!lC@2|j?zZbT zG++ue2$$c9(x1-%JvBk|H<>Em^z$CU+#y`t*IQR!uzkJ;t3gE5{~2BnF6gV>+@h7zB$U2wV215J8Ka0BY+s>u zScc9hAA=z6tc;?X;|e2yx;Nezn`iCbsC+U-OcN<$nvF1f{}z?Fh3S1za41PpL(hp^n)DnM4f?s3~`- zEa=ecZq^*QI45Cv<*v}v!-aean;66t}xcE)pSy|vGqYocH7^#K9_8EUIQXsaInsS-(#nf zk0%C0VZaS)ekq`chYrOV;CSrhIc!ok9E=VEL6(^U7iErxq&a!ps^pH@KEdqR3~JS5 zVINC+5)Uk_MHc|_?C4X2{-U7;K;4W>M)2B*3auM3c4xbD)`TJn1JjD)b5)c%Z1$rlM-MVh{J&a z@7qP7-fx0;3VV2y9KOj#1EySK)|+fz!4rmsvWz_1dhtq_C-NNp&OepjK~y@Y8z{D8 z=mF9)ckN9~f=z4}T8n+SO|oG@(Vo-|1OzB`_`oIxo>v&N3Ms^X?#lvA%?8)#P=+YM zz^oIT4oSJ|T+HFG#DNRec?j*Ji~hz3%Dv|Q)MOM;jIjcA$QOknE5UuqEZz^!TgkTm z{ZfifvKORN!EIBIa)42k*WjszAc)SbIQV!iP;BLi$dB0yz% z;H^E-Be4054Tb1@5P*BPC5tQ4W&>5TZe#!0Wbm7}{aIkV)c29F*P$x|$h$pUdE!k| zd~N$#Ni)`^Sg37DNv#8bf&j~mTGV@Nnl9f)$#3-q$Q1s+O!IrJE(OQ0uf@+D%VklP z1?;{c9q5k>jpDX=_}2h7?Q|3%MAWQahi+Dj3^%2Q2k6%W-~0Blb6!sVDxoR-@S@$t z%5R`4p=(BJ@ylMkJTtJo)xs`OwbYikl;}9`JHKK>>my)QfugxRDfa7He8qzH)ym}+ z3l6Y`7xz@mCvL8Thp;stfCdFB_=!79^YGMf&8i>Kx~;E|38E$acNSnuT%rL`U;33d zo3EbdMnE)k2F@OL6b)p`rzq!zv-?6}+KOK%1=78H{*4qC0nf2onOa6#F29Mv*lO!= z$K|>6smdSw=vln*$a(hmzWr;fl=+ zxkkm7iVN-LFkaZEB;rP{ajtR_yMl`&=K3%D{d9d98V7_lc9%;>K?Ie0O3fO9g}nWAe!2o;7xK%f;dZ8OBGFE&Nrq#)(}gGZWV3YqR2)7lpdE zt`DIzr<1O}W=F??|C0jpmZ3ledaQw)i!Qu<5LSV#6XZ5b^VV2`J)W}GgdZ}wf#L!6 zx?ThwVXQWvTco>S%O&NzN9D@uyJAxVL`NDTB=lKgw+6%CST4|8E)Gr73izyZ0<9x| z&n*2E?Om;L$$L?*%5D7rUQ%tKgD_ibvRrFwTBQXbK*Aw*emBl$eX_h8#VJ}p=(25bO^7344=Ln2x|j|5m_7Gd5Vpzf*Lynf!~YBn zP=^O+DN27o9k|4rPZw~0?wV14K`eBiX&HL(9zZDHu0u}-kWm8i z;Zkmh_sa!*lt-J-T1E0^08mjC|GoJmHy%f$C>90uxiB$SvB5lUpkAJx|17H=7J6sM ziiDDTtIMr>IJc+I+p*7$GVu=Q$N+8i{rTWoa_$5cFb*`OfV}idJy*ob33wfu61}1f z8H})9meHN25hGfoKJ@x|uduHo%*CMGe>lg_fKEyIo}SyQAYM6elBCtc>XdL-<*xs% z>;iFRDHJcspqCD{bBsTH8}PcRYF0|qB?n?sXD?uY{86#25x!jrAm$}!rB8!Lf^MLj zZlG%nfm8tp`r3;Md&(3+qPd%C^T<>i^V^)@>~aV9G5dnZtpas@wKUx}&E-v{6BlEY z%QXW{xT|VP2v_BboukOnZhxoSkkH;$p!@guKha7apo<@PG;X7@)z8y9L$^QOTAd54 z2&zq`2@w`q1NV_E)OL5cz=LS}XM<(E-dbYZv_GJZ8sej;OXghCt^+hkW>^OEseqp1 z8o=6shLo3WTlX!OZRhodDliCVv2WTh=X~E5>DO{HbV(8P*7di2LAncG<^k|RPJ=p* zQboMgt4=Zao_1lfAzRC*ZhDIEPsqW27;r9;KYn&yrJ|!_dFA6_L1Iipx$y>vq*eUp zU07j)qL04XOjxT6;*c`gOjgdd zQmMA+Q&1@&l1_<5xPE^G7RuYPRnvB;hkM}_ixqaXTglf;8J9JMKYappwTuTbdf%UH z;u*U|ji~l4ptbV()h%ipUaR@I`XY;F5xr{9dqZTIu@6{Eg8Qrm_A>zu037 zmdwH8yOVTl8fwHFVh;}V9=33qg$&PQk{k*XF7YPlh0uA_E>nkqW>VB?sbAb)@mEu=#dkKg(J=eLst&0gSStQ@um!xQ= z;ibGHPjTE;zu&0k&`R0xRa58H7^@f4wkO~Akdfdw-Mr6%YR|e0BfNa`24(0aSS*r9 zogIN|C4lGa#YGM7$0jh|4SR7(`1E7}=u2)%i6+)D z+?;3xqgk2A;-i`qRU;u4JtdE<^OSs04Pf}m#hfzTg_tWmToluqMs?#jlepBXTTeS; zL9n+dIhTlg>@vssz{FBjZGi0r!vT=@l_q4_yFS;>hNQoGB}yKg?o#Y|DHh$}lRSCR z4AQ{TzcWZy_`VA$6NZD>F&>4h6Io35tR6<1n!2g?cb-I4|Gt{64&ox~>$EEf|d}A}FdYQ`botzSOYM9tU{X^o)X- z4>**fAsEud=(;%i=VAt0 z{3#5K269L*2awHAhb93(;cdB8IgEKim&+5vwPYuUztS$_??tx!G3P+N3O8f7sLSOKmqweCS=P z&Gd^79PQKtH@AnMy&!$-g+79{VD{FS0AN3YgB!8i)a!dN20&BK!K6uR#}{pJ7`^4* z{bB(d=)FNB<>eJB*GpTHsIo7)JaOb#odtf4y7hfj&>hga)<=pnP?@ti<;$Lhk%uf) zrB@EoHZzX*#6%3x?t!ni_)^LkGr#pK47?cO5|&+%PR3}#FSCXr*2EI40eTVw>)nLB zgRz`SLv<=mNDHF#*KZHQ|~}2+G-|71|_a8X3UEh&3Dw*v1Z}i z#65hDNs~K%8uPm=-I;p#Gr^+5dKN=ErI#UM#ZL~nN%IQMwG?g_iMWLDJ53EW(d5xF ziw?uMlv6jXM?1gF9;kLkPoItU$Z7O*Pna!0YZdl*-L7ATlEQlU8a_vL>2WiNO1M8q z1yZ!_UzB#|^f~GZq>e#@@(N9qhwoTp;i;WzR=d%U_74Q|M1Y>4-gC7XZF_&@3!6+X z5mC{JL2OCn`>H+1h}P!lBDPBi*LP_Y(iR@?qodH{i+W@8JpKV+(yQo}@@hGtd}#h; zL*iWdXzQ{ig>RhCM_Q;UF3y~u4_i&!bOP0x(w z+2|N@;baP0CmR37Q_{JFclZ$=ZMTHtMG zzd;9d%VX%)^))+Ide%r^UjvrYNo?W-534ZqrLQ!#LC$Y!mr zphIP7RCdZk{6*$vH@^uopbPaeud39$3 zKc2ZNpZ+fi`p*CN;r}-kHVqKTKe9CiMTj85nMJmJNpf{t*?z@ zdA;Mv?Ko>6Ukt(Sh&g>jx3lYjz|ChJTh*mx zu3dUxpT~@%l)RL9;}r4v^ei~IJu21^n`A(zV^1MkA(|hGyf0c_?CSE4Gy zPH#b$v=S-xPU`z-Nbbrd7}wTFUuu-RH95%SXj@_CZZL4sT{BPUZsK2Ik!&RWk$w!IE7l?V(ml94~RCyZphC5SOe8MgH#zF6-Y;O4k1Ec)- z(6d*|EQYfKgQ^aWN$ENb)3>Cb^7L5FL`wZbc!h^MRpr7$V>^?qhebj*@vX|&MkCB4 zR-UhXPemjo8ZA-Q%E4ewYS2@jsmG~N|NOLb*+Z_%m+2EIv#f(7eXIjYc$aVNrq2zZ z9}>^^sfzRy2d`z=}oBX9^?5pPtm>RS< zO67Nd%F=PEx2|KkOn0aRfBHdl;G(_L=iQZ%Bm)80Qg749wdgazCy1*5@pocHe~qVf zYz?9O1nv@l|7$!F^s6KR?*;>e8rkWuf4%K%ok~hRlU_Rs@NWS{{#0Xfo3AkK4dQS9 zUZ1V^MQo&}zxicM&P-cF=Fk2uc&}MQ;>CBwskAfw5DN$;y8k0;#J=tS2zkdu^}q5- zF6MRo{};|V0 zCETG4$gImJAU9v=c-7`V&|&3@8A24m0V%)B{kP(xGC#7EnlB6=WW@#1Of;*mQgY1+ zK6h6M>$IKYidW4aWlM?acl^y#d{g zo_DPZ(aFr@z{H3v^iLgA=`ncaq~)+O5LSuzE096CaQE&d! zd4SK_r2g=p8-ha*E5E=mB#moWT&f^^li)$w*UAU|Lnvgz&&5s1I*F?cVyjK@Gb1=1 zRw>VAQtXCNe}DVw^iu?X8k9iWsW;J3fxK={m6*i<7VuqOV6Mp>KbH61`Ktw^m%<&D z8xyoKG0>jz65ErWqy$NQo+EUpfcMjy99B49>(~!zt^nyyXpnn=02B6(}`x7 z9R)C4I4Zu$zx{PDja)g1@dB09m81S|QE9Ia;sfvP$3K3p{JN7R;0>XIFf+$@?nJ?u z+bL4Cb7GE*mcaZybds!XE{4ET>s z?^EBs!6PLtnP+xHO-1&a=^~Z57HHeFqwi#SBEnv8mMg-EMeG3(Uo&0ENgRtmY-9#m z8)m6iDaB{b#a%Qdn?Bo{E75ow2Yz|)yXU(LC#j@O%uL+0sDS+>RqdKD%Dw}vvYE7~ z>o%7T&yqy3Bzs*BvMRDKwY@mu!K8=ayh)~t7;~JOOZe{WKfSn};jr!4N8Yr(DeCd- zxvKo5$lAClX2s7LUa40I&!oo!TkU z#*JOdPLoMaX8N9q#BZ`uo>R%su`3x(DEnPTlFJ-}n+Q^%4D%~&DP zR6s07A8ob^`n0Dcm9a~_0oEA({Y@O-`j9~kbWSr{iz~)KN`OJG$deSSRdb}HFJ>Ob zQDbCwJ&q4NoH7S)PvQF}P#i?_Gs%!J=hH4_XTIXQ_UlcBZnyd8A7P3Ixsed47X-tP zp*p-mvhuEjIH!dE+JgO^l!5xdlaUna2uI&Nm=-ew<^_s!CSwle_sf5OxXRsJu}4RN zR}d|kg-MT@zV^G3mP@#HGS({vmzWl?v!d&q~q?1Yt4hIf*k#~0gcz1 zzAa?+v5ArzizSM-O#C7V5`mt@$NH~-lcgemooN!eHoMDg3C#2G7C6)^q^ zcHjL*!$7{jk%7D84a?Lg7FaBA+02H;qTD;S1!5jtrs)m3TsCSy2 zBpMLbd0R&mxL}GpoX&74L9jlRS@LADmf)M?N4T6MJ-U8a#yTpmP66)KXi(*Jh~{b{C)G#E0+XS58RKL+vLW;&SjA*W6)wk3y2Px)+z zXr{U|%-|!;aBEVVtrM@^Pb$pX=ouX(sO^Dy>CaPd>h8<$^Lah5Q#p+X218_+aA;@3 ze<$?{dWE{9o2aZQc2W{kGS?uHu<4s>xc%}u&)hay#0|!$r}prV7_V?X!LzMv3+Jy$ zM)K+6;zb9>dLRG8;WdwwF|#RFrrvjC%4L}w9sgY!qcvwtacJQ{EY;2L15t%eZ%(*N zZf>XGr)69YLi5-QGrdp>Hn~EhQt79V;B-)8I)7$o;U3A`@a_x=f1tQ5yF9LOWW{s6 z-=nJ0{cEZ`Vk-KKO{kC}9b1p<*3!?&1Drr0E}Nq)Ec_zLGum!iHoNVQb{;3YCHTt9 zfP*Iaws4PBcNciqqIr6$jm6yoAam+M-+`<%=`z%ugrwWFw~sZwYO@g5=(Pv4 z62aN|?UXgHild$`J-{ncu$5S_mU8N+5lj54|TZNG*CA+;7Ev+`}c{z`}+;ofhf=o)R>>} zjKnTm-G|KcmkbCx9oK@gVsM9>lZ*#{v3Jvm>n%z9)1#ai_>? zb4l@EiL$K@peB50&ODR&E?aAkrj<^C504=5b|A0`rsHQT)daj+wGnFHQ6UgY#G|Da z0RrdiA=O=5FJI4L`MDryC?P`jIkWrkK}-xXK-MajZAyFnLIz|cu>FYO!hf3}u;Pq) z1o+2B<1WGH=xh`7Pks8b&JkpFZs)lfs^ts-h>;&tkB=~4OiqJN&iA=7;`IF|!n+xy zFxAi>C>My5u53O(H6rTc>#5VW$INh%O6od;OEz?=DJt!l<`0S7`x~)sqJJ{55J6VG z_kCFS%9&?gw4Us|d20<7#}yar-zGqb==;0&y1FHd1pwW(GXTnXRKkZRFq}JCH|=0_Qf03bdK>bnM@e!qi$tKBT@%QPU-SP%47G#reT4%tX+@-W6Y z0=ZRwSHs$CK0(SH*)tZ+_~4hsydFzdvQ%MJxC9e-KH1C-)Y-$<*FVK930T1$?BJ%A z;A72)Ox}hd3w$-BnCHiR(b@qwz+TuGZ~2(>af}@veSJz^$}SPIf?=%<0#Im%d4v~0 z4D2F<)abG^VE9Bd&M^?UL+W0M6X4ks>a*eXzXNt zz(>p2xw`CdOeWyH3$icrG>~#aqQkqmR<92MNq@q4AG(7W(eMMNb%RkoQ5#qU#&c9A@*AVRz`XM1V}bw6#;O9Eq)r+Y~(9eht2SBoM zX({;)sbrYL1PWo&g*`kt*bUfS2q{QC0#MSFFVJ z(Z%DZ$9#qg#-f94IFON2@xKhvG*4>oWYw9CMyUa!6r|L~L;LAs-C?G-N8K$S5`oEA z$ORk!RU9*eaAmHbmA8zH=TVObVPE6$1jrY~-|$W|CsCd{5s6Xf2hgebirRWqun>K;H zgHqdVcq%68$3tx3)HSJtmK-&SaNJ>Ai3gs^Yi?Cmf-qh(7ZzR#Bd@IAOyJ%m-_JWr zKZ>urYs|(MSJh5mU>gXP2MJqS@zB;URIR>?qi6mI04ooMfa9LPq$>ZTbxogPCtpFt zE)6z?c{@1&kq#G7$F>K7Q}0L@qf{~Z75~5wKNC?@cDkp6yaEa z>BCJYpYXbq#U|n$%GYMLXXTOhz-6;r%#tOTK>5@uuKy!r*Cym(C=^CA0(ZlHW$x0S zX-^?or4IXb-{VHPGrv!5IJ6)o^=HN5VM%>ASjkhnS*p>`4lhAiDOm$9&H9Tn&Qf{q zvU}fnf34dw)(MAEX-rmtXV|&_jSGIg-Ff;O>M<>Zi-Xbfm|DrniSVR<2qUCq!kxkm z>=;}_GChk)-(Y&uOYd)lpr_q6rB2@MUJQ$UF_Qq<_lEh#)iE|2&CO;EjoVD5ooNz5D#>WvRq?gDOri)30P96cWlF)*27bj|z zp{u+G~?TPzW?L&Q+O;B*cy{Yv7 zCz8DMFC?iyYTqWuhj2%mA=mu2oCxLSdikuXQ9+7Z+u%>9+48hB9t6$}DwqLif%b*L zTUMelPD4QLG(DZt*tiQw7VvcL%je4hv%!D9-@E(+Cl8<~zIzFYIX4CS3>n1HxZclg z%MQ!z>pJ%tcl52JQ3kMNnd61eJ%%zTU0td6HdBoi$D03JS#(aUI;L%u&4ASAxEb?PD&5l_aa>pU4+aue>s zV5JL>>;2$9-Y2`Zcr)yLnEc`urWEHKSqSilJy?V@DUM7*5L#YySpB_5BN2$65_DNJ ziIVo_!j0E#eDg?|+3crxVjd@q#j~%}BqjU+<__g+2j_l4)CdscvokkCm^+YpwD#v) zp6G{C69o|k1o^h)*1PUozs1+8Gvt9lZy!3Pc<*-D+p@+*OqVPOB7L_DwI#BP(B@8UZE*bB+CH|jsLBbnA1@Df^e#S3pqx^h z9ylIF?6CxRS_b|fk$0i{)L&kS%l@-~0BV-wItvFg`_1{bz~>cn#GQxVxau1j^yZPT zP7anz zMIlxbZJ}I$Lt_;HXbu*Z&%%D25lF8PuV_K*>lxJLXUztPy=wiCpF7Z@ymjTiC$McS z)YbN}(BL%7*+cpgGy1t-Z0Iw2%e;;v)?L!(~`SPhj8F)SAPY@ z95LcTjgOv;KkrT!j0sO-Km=LK{g>=)tIG`bZJ9mi1`-PfF^(qLyah47`zt%P+I$-X z50f5t?%7ymor)-u8QO&lGS?sWY?v+Li+*s|ot+k!OVhULiCmM$cr-HLoeeM{i>KxW+fT=wEMIHgky#y{Qy)fqe*}hC_o04jHJ9T6UtnJ(AEx^lJ&tt znzCB|iuFAy<;HQ(E?w%vqvbSVb|zyWLH~6!5?0k7&_OY8vH8R+q)vT0r9U00&hG{< zVX;0>G|5!ym8>$m;x0nO#^cgN141hQMnquC2(PXXtHz=}*CJ&TM>oZOlwDVnY+3>J zU|V96S)_u!znF;Ka8fk84AaGYzI9>m@&cmL9|B5O1q$p8URze6<_1Q4MqmD;2r|44 z6hQ`J*2Jz#dmV=zrt}|Wp{oFpWm80V7~h5s8>J-aAf9NN$F~a0vy|j$JB9HWN8@TD zL5$EeF!?SUP<3lutGEO>FTjC(Vx?k`>_1AH!hjy%1{h2OY3mTD@$uqPp7g@ok-YyP zo~?i>mARB(!AOcuiut`tC&;I*i%8qTxq$g0h80xhH9Th67xc zDxCWcH;S|R8jqBm0@j|_xoJ_B(ayq0Z*O{&mgjrH#N}xC} ziK>A*RogYrnsO@o%A%P>_W8Sdfw};&qjnB!CA|zTuF}2xvB8h*?Kcvi@9z>WJc56| z0r=zFPW_@FUW43RW7BsIV^1%L*6pBm4NCpL&%HVFY?BIai~>M4FvYT!#^(dR?<58H zsjA<^EV~tD9@wWr?-9#9pD(|7d_O*Rn^-C(QJ|fnJa<}9VY-aXG=03S)IbWmM(^t{ zrdBaPP$|U z|EtLJMB`p-pHx*<;R-w?aD?ND95KV&MB0@g-RO;{)GL3YEQa9ue@Of4xG1-%-BFGQ z5fv2`0fD2Gw30HUpfp2Cm(q=NhlzrM(lvy{&`3832uKYvGz`ta&@l86cMp2vyZ8I= zp1(Z5d578W+H0@%tY^ihVa_g-9sHN|-ZkOkndc~qy|MO}n`=d;BQ2s_=E9;g3piqW zdp^Y9JtpG!3Y|Rd-bKcw&!6%unsxdStr1LE(O+?VKy$>3(D;JPUZJr*!eIQ^QchS( znuZg?aZ-!qWQup-(m~r$*D`rP0r+a0*JujUU%$Sf`tZ5 zfR}pV;FFzRfc!!M-_tck($p6O8e#^W9D^9X!lJ%rpz3Xb&WGM3S`)YE*C(z@tmTrl zAKlyXF$obeID6mn8Iq&Y)x0H{ymEYKaFE&F!$q?trE9rnW?WxhQxGb94^T60>1k(r zFIs-gYi`#~Z70eu^m?;&SC^0gRP&A3rcHVm<7%L zcPZKUOfND+7g?$6a(R`Hq1IE~G=^Tr>FTkm?IvB9K&Y{ZL^g2L*|q=}aLb?z`@e=a z1BHgiQU9O|8ySA{@=OrjEDuKr@%IJMbMf((@IKaU?9tfOf6~he_?sgmZGu;mZB5_} z{dJn6%bq$|`n(Flj!>~ZEP4WJ=jB!8Tq-5F`u!ND@WA+Z9~}h2BLppv^}e;;#-^9d z3=EEry5Ji+SbAlF@%z14&S<#TP8Z$IR(yfg9HMKBb_KfKYHoOJ+x2B|LU^bY_dRmp zD9+$;A*K_b;C=j>z+x^Gd6|9@`)N2wH&-`%Zj;-ip!&)9J$nHj5%)6yl&`S;OSPQEY16>+bNi(%$ zu|_A;)=XoS6&|0JdO!<{`WvIBCN~e9pP|koeJ`kk*mMaZbM?}n)R3Up_p4gVEa+P_2 zjuYtv66a{ll9?22ce+F^w} z$kWai-WiSuvH8E~!4$Tm>NCzQy=_$KC7J2$Rrz7A%4a-nF^bqgFYX>o)``pB{+69C zp)R{TNRpzuJXKI*(lEuF?kWA3d%2)2vhI;_ON*!ECv~?(yelYku57L!*}fhr8j8rp zlLt}gL{t`hA(R(wG-T9HZc~`}^po-D^_En;QL5}KVHl&=SWy)4OHGyQRho!fQf7f3~R7at3%L7mPazxgN#LQ35cs~hr;uQ?mDy+TFJ-CaNAq4xpFFmX64R;85S zI{HvPEcJP>uw`ly3(x44*0kK_PM;RT82MuTWs^VcWlaQqe?DKX}_if^{8MMt1LT&!O$;ocS# z=Om^x(30&EJJo{k{ci(nhM3k|`5-7-%i~e-m)eW9!l^~G;om@SIQy>=5H*#~`unNG zuj{>VIBMQLbMm#_FGV>yPUr{b!8~55PC6Dln2wbXz;<~~L6`LvoenBPto`gKun7ru zH6uquP>-1}f^P$u^%j*KOu)4*|0I%oFsn10%}#@6u3Joo1GwEmzS`Q|ZUnP{h&wy? z(u@5$PS!3!AplBe`xxi+3kKz85t2_D`>hSMVI>tf&h+g45xCPk3ikhn+^DqO@AT8o z&C%n(Lmk@LD#c+boUEAppR+>Jg5eOkVTJ1zP_NwetOQk^VHQXa35O%6ZLf%d`xT2$ zA#O1=6w&2H1PSs*aAl0~cFpI)ue@)Tx2*R%Jf2GcEso|ouWb34%bk)Y$g^TN3s;fg#Q{IYapMyfg&^}AY-Ad%$khmM@5 zI72a-t_Bct2(iPs+JeQEh_ZNZJLAE{{ou$LBnN+P0@o?)eP#KS^;QlgAlSxJY^;rOb}uQa+7x*Y}0$ z%<-f(V4l}qodKRa*7{j*TMu>5V#Sl1E0QR=D5^1`@i|uO1ge~Y!cam%a;g~|W-0?Z z6)d|4m_`PjbnpI&Uue%w(4eyXVBwQrc1Rz=`ScGLLt`PispNpS2BfQn7SG|xlK<%T zBev@;kzK;Lo+=I7um2zP5g_h!DQj5hVYYtbKd!)`N5^r@Cx-#~{Nug-yz>@{vh#eA zB0-9$1#5($f5S+vi=|O9!cNYdgBFx2ez(#V+$9SU3q!iYS6fJ7&~8{9)3mC^ooq{f zK%)FE3bX`fQAepq^Z11-EJlar4jA5GZ~H|nukA<4{Ga+WlOv_r*cC3aKFV>SMK{rS z{q+2sk(yT&L!kQFJ!iLXWKFOEDEYRZQsE+Z90%38;I$%r7mxc>;AC zd|%1)XI`!o#t4eJ7?TgX`(IZNKv%TH;2wLHN$Z2%^SMF#I{B`jT8p2lQxz-XZ2Y$2 z$=?){E3>+Pp1-c($p^fEfW~-+<-oXXF(8Vf3oUdp-F2}@g%YieNd()1~ac}oB> znhn_nwHa(e6>}jzh)T6=!`^3>rM#lgiF0XckXeUnxm{_7uvE2PzA6H<_N~#c<6y6l zil)D^{iZK&CYBso{7Fy}ex0e;CB=h zH=1r+D%M9^JJ2!Ca-PjFYvw`1J;RhXqBT7O1gXflR<;A&I2^|V8sNgDwL$N;hQ8U- zF%%w0Hd?Ht0|q{Reo#Cv>d^K+PaQ`{>yhSn6~5+GqN1#OkB#@@fog+kM!^H&fYELG zPx1odx;J$xAYgFEptSo51?qwMPj8oNUqt?{=Va-;1e)K?dRCG)Z)E~7WusHnEG<`2 zVXI??r{nf%uZk~l)7?w0bJEXq=aCW^#BTMAfEc~~OQdk8qRjn01~54ccDCY*3<>V7{1mbGJU5;4W>* z|3d|KMgsTpi_woU7!viIi*n1&JocOq@~JT&OCsL?r3S{I zekZnvP)T?8e=397xT?>ATXkLATY4Uy4Q@Ki^@}*7(Cov$(qI|w#Zr!`RoeNY$-7ia zZ;e{%Eub(KO@yxAgF$hDqLmygOE=kjk#|7IXw&t=~X*fSIuzx&^6 zpei(nLc@Z-%LkBx@yeRZ97hthj%=TnHAzy@E6Ne-z90(Jc`ugQuabtnm~uL$G$#W~ zuQtuz1=Zt(V7sPoZm8Nccyr7CyBNy0@hjAjrZBaa*Q!@Yj>vJlJcebY^bJQ&!u|G8 zn$$zyyzW}FBKpq~E>OSP4dVbXyepAJXD1#1?;F|VN0j@@9xS*R2w}9`7dP;BqWg7Z zyN5?5c6-~2f)ml_K5_Axj29<6YeQkz z!-7D8{3$`GO3{U&4sQ!f^`^DXJErcx#j_*KB2Z@Q-)2)c9-zyTD>jI{Dw7OF*Cfle zlgTbRyDls4{;Mc}x1U*vF!VL?og{GTM^;ms)SAv>E)%U3bb=*E`w8e=#DpKNgW1bd z4F7F+lDP&>JRNj6RYdlG5gCLuJFln>80rf2Px501mU6b$nKW$YdQX#pt*zE@q)J^s zmp)T~o%r7_3ov%*$y|7`Zf42bamq*=iEV=dt}ANzXUuwB<6vM2k(y z(0SBw0mYc#P`XdA5zxnvk8c0JB%Tcst2j+6Yi?jfGVQyZg3gKd3qZnm^xsIQCXl3ErY-*;+swGe){9 zW@8M}kEx`8v)M94^j5GCE%(xefouckcGh915LRSJqo}DC+ z%`$;#6&iLj-!U2dToaA~N>~bl(utW$DBLlvxlPt(Zp75W(8-pybY(R0s!bm16Z`S& zTzW8sX%-^)npu5$6CWpB;j-UGmab5ost!jU1=s?w;P2cc*dEvV_LPZD>3zHt8X*Fi zr}8~AkLpi2`BoA8hd@E~ay%4XEs#C!5Vd966UZ^OQ0g3ie~d|=KHE;}pMCpL;L#rc zifN8yzq=O0u6aqmkh$M_HJXR{#wh=HFc+_ru$Fab{)LtchuMmKm<$x-%C}D{2%^uN zwFfI%3V9KyT(k@Hl#6~iXv&A+tPC-=e^u7Eli;(lehz^iNMY=We8FpLMmm}%1R~#m zwoUE@@SCZdwi7w56IvrWO|siyG-2*YU|VWsn}2zQEm5czB91Dj(CMfTdtEUHv}(3< zWObWrfL86lr7-F-xgLI%Qim;uTcXn5Z0mB=<@RML`u^|Le6Hl;6gm-&Y@}c+Pq6#Z zz%!w-wOHHQmi5BCZqeS-+QLe{2NVO9@)}f%i>h7y8?si!1IQ|t8<^SAs#Nr~+UQP% z>~=s78;gzsLf&)qPlf`Rn|Ver5=>g8)&o;5A~hDcvjI`k$Pe71I2G3Sptd6?LZ(IW zG8)vfQ|cqwDCJIfh~lBHTM!p%T?K>lNyTIgM%i+)iO99Z3*L!tyjG$+rdboQb!mvaz6bJ`mX*p zbsH(;kMp1l{+C)A9C&(F*%?L)g2m-H4~HfTUGM0YUre1b;i>?!c=I5 zQ?pV&T93}s*2W4nN(IE}gzX9)=KfC2KhEGj^Ur?13ByIEJVk6-x9dAEwVNnqWvkx< z5aNt%4oaK*Ea*7Rn+p7j@3+?zEV~=}f-&KW3sm!VDS!=-UX@+)C9A2s)$P-R{HNP8~tihn`=7C$DZI$ zKmm5jX!)8~SC=7v;usZXXK&kD!pIo;hje}1I4%C#@lvUamVOMB#@!yt4XESQ;79WS^ z>U2#LVFc+q$R0G*xahyAA*a&C$9 zc_J@X&gglKtAK`edP+RYX6=g9`^&!^XD(gsk3m=_un&Uk^G=(3EO>9wmW>VZ8ik>3$yRq|v}uLIElVH%d{!ic za~dh8A0{*O+8@G12q8xI_QhDOXTkkTy9K*7@ddsVV)MOC8k&Oe-HB34a6#|(Dg~;{ z?s)u1+EJXfDC2P5u<%u!bDMsF+kgy(kfu*xNxGe6WgZ`fBo@(HGGAg%mSKGC4!v)* zNB&MMT|OCgozNi~(>Cv{Z$Bs}k4}jsW=TZp?JZR+4sA6dx%yWNOxqGs!=2>|Ircwk z1@(`wX(Oj^9c_GI*7xLFV*N)524jcWC(m$xes{L8JobjZ8$l(}q0s@hwADUzVMkbs zsd*va6(da={(%@~(*&x)gs6Oxq~pCA>?L|BJJOHmtw8&#zd}Smrcy(ymP3T+l{nYRpOIe+-1i~VNDrn>-x2c=2oomaVfYL z@=OK=-OAx|?wSw%TF8f>6K~wEDqn$jw8v*max8KDZfXa)fM4xx^_roI#c3qeGKWX$ zhcHcEjZVdmzvnfrE4`*pm2d8V7?B+aCgZStu9?5THf=Q~kH{e7qvDbw7Uv`vJICM- zi?y!yK3!0{;=rKNqo5QHuk{cuBvKCMHh-bmeYv(_qWV%e$^&m32d-pR<|^E8KphXC z?Dz>CdN^z(B7E1YG(&ypR)!~&-z!DI+m8}^W7CQ5QmMb-_+pK9Bt&)*hv@C7-yU`M1gsZhQ`Porl5Z zXWauANWJ!CnsfVEbCDTo!{mbSf0m5enhJV}r8|D!(}fEf#%?Jcn=OXaInBhK94z)E zD;c4qe2r6dTi+8bL9CLsXhj97lQd>EQ1mH|w==9$#G^^>2(_{cU(<6*_Ue?@ACJ3M zWc|tG8D{DzY&59bVZ~z#yhQkQ=t@YC_}x)iiuXZ$8>D@wCy4cv?npjYZLlONFNCfd8e8ru9E+AB$M~6@taoa5(B1dz zIDC$~gCXh+h`JWUE|QQLM=%f->p5(OI48xrEnFTkr*r;6hL_6?{BmK{s(~%-JCFOA zI}R&+4JE{VcQe;I%o7eDxPWaGhm|yQ2#6k=b6oyfsXuDswk!1FUQyc0kHfI}PHbU| zrQjx=wY5jydm@X==wiW%m-SuGV>iHyBc`pr{CJI*t<*8#(gFN ztEV?wI0SSQRmyGJ`8!op@e4xyN@a@A-m`KMLPlY5XaQyu@sq#Sm^h8vOZYTPm zHNRHR!O7-$Vd{=$ed1F*?qaZ7@XW#Bonj?VKYSK0WWuii_Yw!s<`S+wkl2k4($D5{ z_2D1&6DZSP;65?w)cb0FydS&lhdkQXdu>pvg7kjBFOHiUX7)Jr#gSwuv^lI0ZqWs{ zlr8I_9gFEECvj<#2Nl(OvF!?Pf?HO!X%g(npME%JCzyui8GEaB$#nkmDvB6?Vs6 zU+(&1?Qb3OdkS~!YS;QIJp?j;2m*V}>1M+3U8MH*d{FtY^g(b5V$tlpdDV`Dth zY#CbS^L+Hq%1^8*9qk}D99=gNzQ$WM;u*x2L>!W9B&a_gS$U4KbA2b6%Wc&?H)W-N zAd{PJj%_Nqk67k)n;o)~U>sEx%iXq8lS0q6185qdnX)k8V1v>JjOA>3NAa$*4Bd zQBX^L-0zI)&LqsYxiI&|pL+T?l~<)^lD57#9Hn_s7lm5jM& zee6Cux$@d$aSeWWSVj|XUlgDDAs+eq*G@}*(QXYjK$z22ZEo@mo^)y=jvG(-0gTnr zt9cqM$(K0b?dU{-3GP702)WEnR{UAMQ@-OLxY4qeCWw9!TL=V|^j88^*;qt}Fh*)J zwC*i}<(luor|HXutu+I6U%H>WkuH2O&h`e|3M^b2d`)IG^Cvc5tThrb!B&dtS$JjL z(Hd%t?Ak35SsQ!|n=kpbarF`9&xcK$j-OzwHzUmRgPH8San@zy#{z18n`q$ork1Px z5{_FZ7EZRvrW^gPQyk0c9eB9%oXj=)@t$-a8e+(z#mtVs?O(yY#58s?KS*qwdSz(B zNZ4)vsQq^+c}M8U`o&*D6@F+}QN@Mobu3pPK?j%aY0e7B3pc=P2IB%_bk|V1+4QbV zG3?UY)h}%LdG1DXA$Uqw4-Kx)kM>XAk3&udMK&%3=nwxH@+I5*dka?Zd} zpQa5R;k}pXh7)~ZICv&Ny5A=?$x@JJ&UvmXOJwuQ^afil=b9U&Em==ncE#o&RZK>` z{DD|Potu^jWoRQ{#aHgGV~&%`W?u6Ga<{7tzCA<9n5?B%Ntc(cwM9OAE~01K*@$=b zU7=BAbKB1Hvfh1fX=#e~Dp!DBTyMkAL{6EQ4Wj%;23(6PSX6#? zL>AcA&x32eW`@4Fwc~|6n8-tU^!4rSNN6o>McuvDI^Cw5(zSG#a7xa#14V9|G%Ikq z6QBCV77e>Q`;ACs>$9b)v?xV28|hKfu2st1I<=iwaIdc762>@+A4~P=kH7Jam&~*` z)=a!`?qTAw6{@XWr*z~RjxqVN_R8Gzq|~S71F(_0m<0ZZCk7UQw%fDY6F=>BSKT0b zk2D(izL&I=oc4_JrDB>n!dgGFpO~zTuZfk+yCr%~$BB6Pt&u7nttp8luXr6)`6T;s zwV&+k2Dq!Af$;g&5h>p;y!OjDg7=4H-1!K4CCewyWKh$s=%I$RTdDr{4_)%@7IIR( z_y!?^w1cN@(S28%L09A08l>+NPDq2#QNt)f^w+1#Uk$N`LVgN=Tze<>d~e$F0zp3t zjli^QLs%OZ z?t!i#7=hkgs)}j`1StdXeGrJU)ZZ^Fc2a|V85AP$SkEXS-J`I2`t-=@uEy@FBACFF z_vrq9x3vERZm&as2X5A#ggq#?aM>dF6Q=GzCw0+JB8q{N0KdYo&VVLEuvd#iU2caX zO?gloJ8q>mVLk)HQ-`GBE4YUZ=5o(0j5fg0gi@B}1-==YK> zu26{8S=mi}qq23-0a2XCh3a6!q6rTv3)})dlP~Isx|NY@<@4V2SyZu~yM>?P{5u;l zB{Kvg4W9Cc{w=EJt^bLtBftGd)#Riv9SB=u_gMl#RG{rATw}L)p`Y+wxBFnA+N3XW zQa#-IAT7~N8aIw%lx~5Uj;FgS z{Ob3v{uEpZLy8?+Q&eggF6g*8%rBCn=XBNB@w3DF&bi!f<{*aQ+ny&Zx~-m$oO*TE z8FTa21C2MnkF~){CeQsAi2o;)DE4S28!6^{(u5>Ny5-fb?}5K%&FPw3JG4CBC(dr*JDC|M&kWKkKY01$YdmsnpxI89 z5XHHl`Mr-|4~%Q~z(Yshbr{QBPv;lKZCKcj5#!Ib61Iy+%36a>P1{hv3?7jfH@+|MeG?A`4?-Z_TEDj+@^t9+Yt17PG+kG-Q(4&?hr~@aR-*3}C5_-4 zQ%bjv%e91rpF9FzQ4ad&E6wu_^F9T|Vs}J8&9BwyIm4|eh4@XnGBIcefBsuSlbngd zb0JH|kB{9~T`3gCk}Mz>Hos3wlhzSkk-4=rQ+AJ3xy5Ya7B_E`aw|C7+kgL*Kij~Q zyxfo*SGwj(x1H-D_u|F)MNVECAEW5v(*6>si4%EXVNv*ilCLa|8Gr9QBq`yyW<%~Z z>1oQ3Bwl&j>{PMhv=JA6Rop=oHygtEfg7@IVArg;qlJC=o+4*npOoWazg4+?*J%p%L8VY&rvsv^c?sd2dkqRqx4i z`Se(}orgdoqXj7qdS%_J5@8W6mwzp+?`wdocbTb^O-1LSX+#Mn!5%4}3uV}sN+)Os zmq~B?n>!`UnY8}#{UQ`T8^!h}uARqDPl+tFP${oT2UB&6GdsLUelKGo8I~Q-sQ@De z7REmd_rz{b+SyQIkKBU-K^{)}9g#>&kE+O-IE)fshVq@E>fYKAl`J_A-dip7nj{^> z|De8{Q8@@$V8SZMH7@IotjASOa3bQ;JTi%0Trvha zVlRlEd!irV#dx;RQuMh25rYzNu`m7gx$up9hejSK{rs<$8Xv zWVREl6N}gH^8bkq1Y9R!M@9;L-52wku+d0h4NUA3mYtyJIRwstweQj|F~W}H?VN0O zcjj4()EuE+AXEp=HAC>_UT8dw;O%(JB1yIBd3dM%hxY0WjCyio~(~dEs@4z zFJ=KtKwp(Grn&AMMH-cXx>ZSRHEJmvD@b`W^Nu7bDc{|~+`__SPRoP{G2vTZH8Q+D z>fUX!gp~>IG$Csp*TUnGw1PAMYe1_1`h2Bgf#Zt6>Kgw0@%+)osMuP_11#eOdZrr> zSg3U#E4(w8CW{YpQwyV-rb{4Uxaj9f&PL1$z~NxXHdU;uvy6a%CtdnS@!Zc0mR5QL z{oj&VE15`fEBUc?gRPgZdBiLxU%hYl;r6^ByyB?oi?a5A6hwXZ+3Y}r2YZnUS|(wu zQy@O!L^e_Q$r+U^`k&Zo%UXk1&MOY^J;|8bDnI-nQK?GehkyGTvhA zR^-cH3tj+`_!=s@DkNljKh=Kxd_y?nRtuy;Pnwn@ti|rx&!b~ z)bQ_f(hm?-)U5OIn{R$U!Ijd)*P7qGsW&WF`C}>xCy}9&{N&Sz_j&K^6P8mf3?>b` z-Mxe|L%#G8eOg-FFo*N@JX|OQ!C=eUt)U6O8 z1x~i(g*I%5t>NIy&6R)Uo@J*$JJLWKvpnP_4Bu6GS^krH{!F`E2!&`~w$u~;>aa_E zcU7li`ou#&mrpyAkNd`AThdubyQA|M+`wfbrgq>JvP{fuWM(mW&Mcw2tL>U^oGtz$ zyk|FX(#M-~=IzD*D)aiQ@9z_W77ytqtduZ5@3|6o7|rg{1k$U@VP`6bNqFX?DygH) zTN>G2H0^5lI;NX-B^P}I3L%#kHm5!--=GNC$Y*dMS%{@c%@C}| zlUZk54iVFle-Ewu8!^vD3Z;DYc-NLg#|04<=h({RhebMd$(5p47^q|K@-kMv31mV@ zbA9xt_pyL$t;6!qs)!s9y>{=ZOuvbNNEJ*M5S4mSxDDFeTm{)r4N`EBgb^FcSq8VkI6WzJI&`n3o27JQG+AGbaqVYUG-vA-jkTYk3uF_14?be5o=NwTc|Vr&cGVs*X%ev zaodlqf}{Z6b^JC?oZrvt%=jXnln>Xm8$&J&($v>e9iEnRci$cYa zjo<;RVit4EPg>eOboUue~t=-DL)HBI#NxVMFoaw$JP#DLB_O;#vZCo zXG?2@(1*GCE?sy=+OGNh7?i8EKxZ+xx07&fnL;O*?G%+~reKvT7p2YDo*hOX8q)H^STg47uE3ikH z_a5*wVFKKvdwYq)%=BDu8uHePmFwwaIyo7gN$<+I`7Vu)9Yy8c4k=`=9o{UtCyKBh z*(Q%_+ty=I9~nBxVheZ#BPIZguf)Xp2k%_`^b9Eyld0DKl62xj#Ql}Z_C4!M`%9ra zXgFw*V21?qq^(e7$~YvBLsJg?iBnrlC7q_^)yzj213xtX0QwUD*o+#dFr@WDX4Vc? zmL`Cjb_Kn3C49RPTH~EcHM;4>PK=1r!#=|b_gjVw_qd10hPF#}Tf~1YB9J&D?)V}7 z`g-wlRGEJ4k!psDtdU*@1Q?x$Q?U0)8*+WVnBm4rKdDZ#^+`YC{PvwHi0OyA5X-A= z@8323S^*ht8xPm?jIBuCT3@oE8&z*BjV;&mBD8k?1EApcKOt$z>jxTPw6|;!VX{lyxoD|^1`2sd+4T_6qx>wJ3BE}|%-w9XNw!3!e6TnUn zKX&A3N>$bPnA!vL(aE;9>4Dk#5iB_&wq=}c?hXj+&fNUs@aLCmt2fkii*ApK)h-)1 zwkk-^3-CwrAd0#SMq}w^1xuERBpoqVs~8F<>4*D{BNdhR(%`ajEzKnzJT-n*#w|ZE znAbQmN@}WU#Y+TzD1O`Nj2n$zS8qF;FGxN9^nlexVy=DkiI3b5QAHe)L!;ZIq}AD^ zhZ03p%wYQk>QJ#7*1(Jis4F}SLa+S z)<&+(8vkrOzBGz&0s}R{>IoMmoT=!u^hOSdj{u;vkP6F$ z52+$pG()Rcj2#X)C=K;?8G92wHd~$&i93W2gc)$^d94B`7Ofmfvi$S_YxPd_YHO(; zhxfVudpLuEVzSVcN~Wp@M1uAvyifw96XqV>bWE_kMrS`de*m+!8QHGwtamOI(Nw14 zXXH`jO#7T3U}Ii=dr1Ae?a#3#vnA!6@_1yrvyz>gPT408X?831?`~gdMUAxYL1zdM zi`Cwxt4U8@|F5dezKeV>1x$(E`&Cn*KHi_@&WfuqkVl3YAcJYk@$+J^cR#$xQjkp| zu?pQY4)RnBH`91nM($W^c`hSIzx635_`FUQJ6)xkvdPZR;I$!Qsmnz-+#I2CmejhQ zf%o1XJ^hTYykL=8)bWkKrM6q8qrlcuPH{?4)~) z<8cIkp?=5K(>hPvZ0BWtNXDPl3JT%EQmIm(Hr8WYr996h^=>(2bE(Yk8{q&{BrzeWK^=Ou=ZtC|nG4~?m9vE2|hDw@72XVs&P-QuNN@g6lo z_R{`ZA7q)NA1)Y`E^V1ZINZ6a>myx~9-sMvdt$eqWlEJu zZhhZFdslx4d2diM-LW|DG-^jo&K#M58fMQQC@RVie>Egr)TqA5U*d;ua#_7O(!G!F zb*ca;?loV#iNzlX7oz19FnA(IG#eJLH85;;N7bA z`Z@Sd*N0S>C2xPCwp(eft<752krANm@4u&hj{Bpo(aXDci=<1Bf_VeU zk*zvj5pbAc5{r>U?Zc5Uv4Z~a_5jJgn?C^@KNlF(vDxW6tQHK%XemZ&b$<+s^po#2 ziDg&f0Dy?CEAuV;>u-6b_2S_5Sw_R11-_&SB3lXz_3r@9Rbu zX=*Fn6A0d(I7TZ_ckIejtT|8egY&1RXHr_h)wLXDxO^B2%;E5m;}5Gkt~Z|Y`K`?y z(m{66(IM9??-mVDn7Ml8GhEb|NNNlGZRidv!?pC%`sH&3^!7|i0Ymqww3GW zHMyLsRZ<1bA&K#G7{#pz3?DK%AmKrsyKKEVnsc^IeBB%g7Emiul8CKZGO~Bh0k+Or zcU9$uo^O*hnH6q0ZP>A1>;Och;1g*x>d~k8j8(4SH+mD6W}LyE9gYac58wcM?Pb<+ zd2Wj=&8_G~|19`Hun42C@n{i-fNb=zD;ILtZ0BV4i*dD5JunFntr(G;NL-2S%Fg@l zTymm@;f1f=;aCAG)4zJ?48u|TIj_?mOR4%5byO3u`_u51BLBCdpB{@3{u<6Hiv9UWkO* zQl(`XxQ-n1$&ymXhW+Hrvi^b^7(VD2>gKM`leCPpn)B}S!Gjk@Dxpe%Nl*^Tf_K;;qfqF) z=fEclA-!$_&z$t|xO~sqpz2Klr}u~0Bs+z=9_xL^lMg@9Vu>0z8ZC|WXJ<^{0#UN) zwY_T%i9db0|F|~IqjVh2c0oKQwS^t;yj5^#afaq#FG=WkcuXs*) z*4NhdhGdeD3`=x06G= z@#qG;kXH8t!3Jl=p<-x@IN_Zp(Gy}D9ODfH!vo9FGy_Ms_7AnuWO8)8Jt99;-gfV1-CtoE?_>Mk*`f~@U((#`zITQN=QIE5} z)eZtmarH;7qvuCExo4+3YXnk?HeHN(Vbf=W{y{>yt}qIai_ID+=yw2_FV@BNN~n=} zsP`HL*xbKD@OaT3=9}}*ja#VXadtb@XBuQKuS@a76uXjFCbUuoZrtV zcw+B3@fbwj$x@^58*s|seuElTY%S?#{c@qpZA!&<(vGl+z`hk1Y^&oxT<8L@ zyF@MS8}_a8P;TMSth0Ta!MJyFDLKc zCzk->a%>V(PxN-1i`7Er+1ld7a0Y!Bex^*;i@ayV9WHVh^`|<;zB3K!X`#=PcELVy zgpvs+j-?6NrUV7{GC?R}g?*|+Vu{=ch9+;;S(UqJm|qHOvEpEj;D3%PGaWo`TlR!S zh3-cBK6L?982}?8e05KM&dZva*;4+RS1yQq)9HTRv|CvUlr$tG~k!8YC6u~M}RQ?II8 zM5A=_Ei(h2fZEF?1A7Ls(4(WhNEs{4qcS@|$_c;q;vsQ*MU-<}RZ+3*imac+ET-ZT zL-hPj@rcg&k6OO_kBo=8Lt<%ZoNXPTl|406#SD~L`)1+r!>p9-3qiQErUZ2!Yr}Gv z4iAhmCCbXBtragWq59{lqx>8gR_lYD+$P>)CfVLfxm2`YJPn44DkCpO>|K;lcf^^G zl>K>&{Hnbx@!mTW>0k<%Z!IW*FIr+uaV2>kenAYA{h#;P%!AxwaO7R^QEW*rlZwCAU| zf5wb#&HB_urbUa6Gsp9vzbByNY1euKgkYhl{cDy68W438L2Q~F4`{1)zt13-0CAEV zRbtGRSGWh8uY4K-!0FSA84v<21qbC|yXB-qRMO*4pYAk_Ze)X|oju3tdqk{EOo#Ck zjw&~vKT$A92g_h<0 zaVm&QFtHo_@t+hHW0&DviSmG}CmlfGn$B@C`m=H~<%r76?MjF)s*!hi#W&iqS2ERX z$9<%F5MLsBL6OC0^9AG_NPh_C%{#}1f5GoKY3 zW7%3&fIZtMk0jA_^0U=13$gR)6wSh60=-U=lofKW@lK08?jA_12s;j`PEh&@ugEVP z@8$gHRf1%-Sq(f+zBgEe9bq2>7=T+@^Z13k&An_v4s% zM~*JDSPqbz*#(0VRNciKqQ?k9H2tV=%i=n%{ZeK{_8=xA>%)y*LZqTs;JWAeSM9SK zS#My9{*EByGP&-Bl~A+0y@W7=JmW6@rtUoh$G&tFIU_Em z#csvK3tG3Q=His)NU#{qJE&@p5VKX7!spjA46tM_qSiZaA%$DXGE6_nXlOq*mO3{Pfzes)a2f~69AMw668wN0 zvRis*uW~cVdn?@Lp%v$67j+M=*%TuP)j)2zsbL6c$`tmi6XG+ma0EIh+FYi=`Z(jc z`5ibr@4jAj9N486!eXL;9>lfFjb(QBMd2$w+nqah!VJcJ_wsgaVe6n?w1tTgr%~u1 zzdD5&&y6!h+g#H+03 z=yGT{KpqrD@j;?(2~J_!_#)aSR9@~-KCU;4nyi<_-)knEash+{lXQoHk^KH}s&3ny z%Ji7;;|&)qvT8j9OT#XbgkjSPMm|65ZfrAq9Y!VOxzmCEY+^x>1R;d{Xt)bWDy~=0! z;;UUyEp=fM#6xeu#qG{FPCskLXH)gh4)+kjIyx_w!~VGwP(=vN7}(i)j-Ilke$jVm zN`R-Ny4`g4VY|6Dv(H^UA23{sO^Lo5*sm)k&0gZHospQr$9!0pRi6~IxE){ zK|SIF6~3Jfq}8U!hLqHBckVz(ca{luvD_#$(079C-XCS_UHUWg$2F-lvDlq708t76 zGIsm=)n|pO^73I|*;}XenqL4xCpS5BOV|EVb3~8=pyNssYhyu7c>2uNPwDcu#h>we zyPY1iqW-4^g792O=bkJMeQiBKW(C*9CGg$3y5i-R#hcvWR{GEX47jg&?QAxz>7-^5u%T(9aFKzdorbddBV6ra!hb zm6Tk^5gotKcXx)`o}8vTFvy+g^&_9n3F@SLB)&(G05WXlFtZ#=3{O!@)L%mbS!YZT zfdP0wI>ki0(Fb{UVc!RAuTPExkfy%dH@cdLTMo@^s!5TPe=`wRxQ}pn1AwgZGyQ$4 zoV1L*JOPSGAyhqnb3Uin=EJ*-U+@2-zHjY`U}-eGu1>f|i0J;t{schZ6CH*mNxnP> z0I;$WD>)`%q5!FgMrLkljIbdagHOpA*KS$JI#J9s0A5Q7zG-7rR=syGT4!kUKwrWS zc!BUhBHs6i0L&)WNdSeE*HRPqiKqh^a+XbDo2g>mr8-X13lPsYoc<_uPHx}=0#%XR z?R1$KTC_WmSOyemG!;F#pa9r*bOMg-Jg*L4>q8E;}2bx+x#o6&0+`iJmZgw zSnk1L{o<##lH2t*p<9>(+0Br5VJufM3L|PZrWP%FSNTB5VTy0AyDy7C+C@s-OGD~n zOo&gfsD%FXJ3guRXyX(Bt~E`7Ki^@TT71E$bC%Rt2XBFB);~huHnFiGTWC14*|*e5 z8G?$WSj``tSOoo2WlxM(9f+^^NB=0R$j3$luCFI$vzY0djG5Or6mna2$?*=iNzqrj zv(nYAZXDre=Rd@|P*~^!;!m~#MDdouuf2EeUNMCo|F}pXmWI&U1JMn|>W*@0J|Dd% zzFt~P_q;lH2uEW*~##c_AWG$#)}o( z*-6K!C$qPgWtPoIH4&OI@NgTpq{ZqsxQiK#%vjvC6q*?muI7IemTEpXb9ued&{FMP zC^&Zod>=p4)uzyq1wM85YelH7%5HYsB4OR|hmk6kJLuQ?Tj2Y6p9&|qWGWm1jH$c< z$h!eGV6}jLI-ny`A5dgYJm111XzkI>Y!l4w3Sjj6#=-h5TO-7=^y8aWju}aau!%jX z`m6zqdZX91_A_-)AR>_{q>7vo!K`n9VnqL6by*K&y^kCXD8{!<1 zpU1HAHi?SDEJc+0>@!}yuy!ljCGR${^SzzPNLTdPnBHG#;eA2+9Aw=aTe0*N0BW)^ z%pMY%>ciaHnop68x7{D8BQs`;|e*g2MTT9?3rClQNi{!oA& z`@0Pwao2qHu90+zpFx;UI>rYMkbxWPf?W}QmetBC`b1&jVL7DW<(T?4-iM4EBKG2u zm}lev3Nt~VlD%cisF8Hv7*UZ?=`N7RV1M9UD zYXmq?pu)v1c@P^LziT~C65wL31^|OvoXiI&;r~ki{GGJYe&7Y8`^`P+JgMtY9U@N# zm=#qF*e`+|^{=(_X1l`i4E*GeQN;O`zMCNFkwmYc@LvMx#I29>`vZCJrNV5O3%Ckk1g9eklzDJ z?y>YbsQIV$n4F~PWNS$b4TBPUUO&7LQ@KdC`Av7=&J^19@wzC3pS;sEZq$E`EOfG8 z^E&L^;bR_@tuJDkLoIdt&A(10vQ`1yxa6`%Bp~w6l+>S52+pdlxKwTa$1gX z5|n~#AoS$|(*fS4TA5l6!;<75hZD!UST^Z9dr(AnGoC#G9>;*6sfWbG?ov}`X!Ja5=BfQ&6T(_k`e?x?gV5wt_rC_F_ThI$ z`Vc`z0kb%%7T389MXTk)fQrgFas|@9be>rH5z{%2qOcUSk8ewo#VLGc7LBo}^W9Q4u30g@ z3V5$HDx>)hG3_O6VyTFq7YCbK=7wuLeEzR}6+vD5)hKwc0O@4>195{Ste}9JqM>9p ziymYGNSRV1LW|pOR9@3cRr(56J*ybYO#d{n-@?mf1eu|uI+mL^EuQLuIF;%HD?28ZF=3L08r#huXwi7HbTVmlo{Q0l5iL;~< zicyeMcyTc-@^??2pu~T0(=aHNVUtEHR_eM3B3IGx6DnyXaZ5-l$hYx^UJ7c&KK}mV zf81@TJfRYhlKTlm{1(yF#ge&pV?`+edY1Bywir+609v;Eu?&JSDw`jVS|;}UZXTCm zBvKXawVFWicyz~Ov#;GrV9-4+|BvrW0s;v!W7}7K_P0NP+6aa&onL4MR<-Se5<%Ej zS6mKeDkLas3D);yjc^R0MI4P*9VSK zr|`8w`PYGTLb~MAjlN|`NqFK&j4+^sO8b<09;o!j=t((PaTo&PZ0J$;frLQ*JcjxvkD%^BK zQI2(Cv=uXV%gP^XTBy$W zcGQ>3S<1$0{t^NDZ|Go!?R6Hd!U#-%u74wqz&RD)o=#6_I;OakzS15}O43z!p}Z$I0XkW~TQ)n}CQbQGeN3 z-j+4PIi-!L?fA#yc!49T*wQPZB~-}|al}7aNTabhLsgBpR7HU- zSLrpTW-g9A_p@yAa71+#@B`)q#SKI~8DBIRZ01frA3k4cj8-f|_(jS-C>j#KB$f=U z@q~8fIng!(tjot<&SoQMms2c_OcC}KD-wukxczv@QQty47YhX7RIdQZHV9O4)Z?=F zkz@Yj*%{6GU+Ty^#4z6!E8kH>CbvYhi^qPdj`HuW(HC&s9jXieEYNqgjle zIy^E0pK47zjmag*unly*!kP_ofxbJ?XLsgYKE7fpkzl_+1sNoMQ5L3`meu66`v-Rd zv=t>LTJw>iPkgRuzp`V|2cpcM-X@m-R(YsL2`YQVfGBC$Q890WaU#ni>^V!aOnCltGI0M8d7DV!zS1id@7T|v zlS{H;JAfqPX%8vvp$gop)&7lBo+kJBrvw*$Iem-SLKkN;sYrBToFXk5OM)_WU`-?I z*8)AcKtSYv|68kvZhR9vKyJ2(iApN?3$BF0SX0lOI-`j!p!GmU_C6wy;6maB*in4? zmCKz3*K9p-P8_G=>lc-GB?Sfg^m1v>`3Ya@j4xTcNRWQ}&yv*w#S#tp{fEbs#Ptum z;6O3{{uQ9x;q%EKRrITX{FKq5PZR4-RRyhUL+nQ;)}TOh+*1;Yg*7&-E)22lO?H_v z61{oHhWGbG+ia8TFoKF_oe_qZEx<%;|0lWDAA53745@%#nK;C96|@B?nseTf4jp&B zp!EkLE3+7FH`REL)c#i=szv$msi=c0y}Wz7L1h#|GV%F^7fAhGJR?$5@NS5lFOgZV z&ezOM|9q@n>2}Dht^Z+{88^OnZ0~{fS{$E*OSZmgS9Pp`_XP0f9-)X+8Ab1=>YGfl&B&n zjA$e~po=>ksG?ctZ@IL?3N9w!j!u0C%K0P2b>Dr<`|qX1*Dh8}Qqt}1AD8C)y-Lg1 z1BRHpM#+}g%i;0IZ^`i<62ZoQtORMO=>Nf? zdq~ZI*XjlgAdoO?G`p=66exi%r8_URyw*Ufh7Syuh*MgabP3|~ErEdAflv{Z z2+IU;vWYk;7af%bx`qKDv=pW7uebrv%FuJKajQNjShn>5I>kG=!*e{gMb&RTQ9!RJ z6aR>~FE?IUo1|gw3<85#4?#b_h!V6RY4UW5rfCEG-kwR>D~o4CMTzw<@lSg;3z_=n zT#T`$;a{zQ7P0|Ww*6DPivk*szPzM-s1LuH<*ChB-+iWhy#XLVN%i#k=6E2RssZOl zEoova>vB&>`?Fl=r_Z*sTV-->8E7ug()0F=w<8pJgxeu(C6I_ zE|x(Cl<*V%8LX#ydk%WEH4JckrejovrXrPNKrs{YwCh&)g~bg+yyUvbYj0r0BESW} z?7$xo0fW@joGuRRDscyMPO2sq>`FmXxFAr}lgw=XidL{S=GI643#!Yc7;FrA^yUV&(r z29L>NCi4?y`lxF4l84T^t~~r1Nc@9N=FNmr9;F0S&oslczj+V%^2ju#Ux|+YCGobTP^YA;-Xtit&_HFC%J{5AnA?pgoq_@Bg8w z(7Xb)%517{lAZ1I)Wo_}G{hsY8GyCq0;3u#SrYSt97b8JPt`>gW|LTCV#7m%bE6d_ zw&LzlVkjy|d=v=GR)f*@zFbOKHURzk>;}?yMe65+JaqmPaGUKEw+vDz_)*%Hkm>=O zI}}&cD2sDd7X|1r6Y+ARF*yPRH~qp>I0sP->@l}S#$~+mDXs*6qy(% z?SGgU=d+|IKy>`c5Dnf#&jQ#=4%22iFuQ)Y!Q+r~qsf$Um(Y`QvYj9Z%mm zwl7)!=*B71V7JuWm%v5~5V#<=*G9W3>FU`MI#YiXv<|xl% z;EvDlvwhKr%i=d+(ihbg4B(D5AQLjQS3j0-?dmri8sO(OD>`K{3dkIgi%QYV?l4zM zNO}4-6a}P=x$1JiC=kf{UhR-0KxQr?!}RDkrN(w~ii36NO0+LyI=D|UA=;i$Mh0m7 z=1wTqEFMP#k~7@rAb(EE!WAIK0b>%|>6i^Aq1luV{saQgHz32tGkND^cG{T@5qg#3 zGLS0jD$cJPF~nRcDr-XzAU!F?9v2iUyx2t|2O(MQUB*@#PxNs zFF#OF@`k@UN)yXbjf1+gF0iN-IvZ`_41Rj57B>-ISL-iu5He|Z2YXZ#es8Gb689Zd zJC{@{wLM@UQ|*ZW>Aih|wqnH+hrR;(=*t}9_?w`R%TwoT>;6F*{PU}Ev_3ceFo3!=wR?L%1)ClTxFaw@ z(Z^TD)`y(zJBIVTkcq`mcbn9rM+aCG#sQ~L9lDl5>4vPv59(t$5H$w+3`Qd!XiD51 zI$$9WoCGUSfD4T017qh+Us+s5Ckt)fBddbp*gERBZ?^#lew?02Gf+3tjkKsXEjoo5 zSHGgRg0ff{SQH1}F#0w^k*V$6!12Fo=>P6d!uZY5WsEi;x9%z^e%EpN$LM`gbuQts znl`z4yGZtq%l5k|Lj1xSy4v7-nlYIsugMNLj9{lH5E%T?9>4<4A(y|`_pZ9y~`mgm{J_05tp73ma zTy?t~2U-vDl4BNQ12Rs~HCW~=ZaTc#6mO*DAT)DsqxO@IYUiUVVh|`({69tr%6{`c zHr}rKW4&I?Nb(!Zk6EIkW+$y_5Yynxxm^uL+eI0AI&-K$7lIc`35LKwJ zKn8FAy+Tf&Iyem9yvK?9uQ7g%8DS;}&f|A)axwXNTl!eSe7$?{VwDxrZcMB{{%S+^ zAxJG~HI9tbrL?PsVROrK-$*2Nn7nLbxv3@6z2L)Py&@61c9shU#mUTVkO zNP4c#VU_iYbozLl$^iMDo;$04l1UPHhDFOMVW3=8&?UX;%zAPAvbeMOtK_tv$VgOWZ)1M_u?sD%L1T(B!INzTEuz4by< z)uNl&ixviNDC7i?Kw;@1<}=Xirh=QcaX}X)_U3O2+6dk$CbbNfrtO6LuXUM{#hz`S zw%ry__6lTpUg{l1xWRzb{P56=T|yD#J;LT5Zf&9PiBE(t&drP1gG#o*}|V=yaAhPU}Uo9c*Tk3 zZR2KK>D{6+ftr3(9caSB3JbfgWLxIy^t!{#KEpZZjIj@%ik1+R3pVrdvr`OUOe zT?7T@No#uAPqW0neIPgl(mr*%f(sCTBv!lu;C#mO7vSdMh3Q1x1KcCUjJwLa{V4aA zS4I2LL&9&4xEt3S*FQmjO^g2YCIsp6L7In`{J$rMF8~a7mw)q^X(FuTauamIF)#ad zH?1q6Al=HLb05`ggVt^OT_FQ7)vX77RKvX-nK%;-KZq&7R29edJ^;JQxe=B=Wx7;{mx=!DmOC` z9?KnoCJ{+v9~X-wBl}B1#@VtD#*2)&J-TI;HJSWWm(RBijwCK6Av{o*QYTVQ6b?qbZr@aBp}=n~~o(-9&3`j94saw9q*yoo3zgoO}t& zUPyGm9%a1QdF^5nZ`E)11?PhNcG+;uyAo41;G|`$H&5%R%#*^$qaN!c+ryvj#VsGk z&EAaoJHB3g*9l^)$oJ-U%0sSAa}}E_Mz9gCygiO}p}xuJZ()6Yit!l493aPg)7S4p z^AbN!>K*4=caMPQ)M3t8F`*xDYSL-M+|NJ_tTOpurK9=sdIiI&Q8TG%5`o|=&>3g@ zQuB_3>HM0-__BAw1Z6}WZUc~ zr!8Ug@VMFXq3op3_@wX)PT^U!A4%8`u@REn}DB!qid`H#EKX=(LKM6l9cBCZB~<7tn>QmAXtv z4f0xsl3yK)@Z3(W&fHzrILzIA+?ts+d4rjap@hmHnSgRN{Juc=0RMkK#z8t)Or8*A zRqvEh0u-^Mz~E-&jC%6Kt>~f853&=JNW2q+uY2OqcU4-Q^=&Sp43+rBaDw5|3mQHG zkkKc-L_wz1r(q$I1&h6wN(N`2D(6a%2~c}H_mURxrjzz%cD23DPhoF@lQASwsU>4S z_5D$^pp47#5wa1rwjp~$n7R9KZs5r$k4X_~rKuHG*%ooF7vfj5+)b803S@x;F3((F za)^gxc+maWwVcY*a8hYqq>qsuYl&A9v?dgs;!a1Y=XnHTE>rIo_%Nj1drVBYTb$&3 z*VB+e;-!Aml7ILo3d?sRZgD4x&%WQ#%X2D?2#2aMT2GOcF)0zHmwmj48OP_&qiH~V z4XJhHvybB3T?6s=CZv3^pJJdSt@c7M0n@jjzr$?4ehjTJA@4Yc7( z$Y!U_Vrw+CJ6q>hwK>LOTf*!!(DxAyDTkyAE+_m;L`1mT#Ds&#UA3R*=|N*gi5l7M zMWydCjNIoor*gM-BSv2Hu4&;WCv0k=+Cy|Sx?w*#~-k$CT4UwrpO?X+;9T8vbuuk<2MRn04JN5{(wYnGmJOfM?1Dw1%%ASr=Ja#0i- zNtobO1y&iD>p%Sp9Yf95NPYes1r&k379h34CC!ET8$yn7`E!vqx&5*uKWD@6a_!UI z#Tof^Pn^=tW>ux{8Qkw~7p?8jsVlzbXGrGzr+6=f>~u?8jbADOC__L z^m*}LPVTJG7Knt*6&~xJSF&Ml{dYIIOZ&AF;?1}UpI`~ zep@j1y_PBem_^V)Uu$n83>P%Qzsmgi#cuszmhbOhrw{Lr_RCw-&7N^$rzl2qy)E-> zJ@Xe>L5Tir!2l_!F@%1YRqo?BS!vpw2<$zOf1i>9teWaobkboTks(pyfysX2>wtE` zk%?FrsE^;6_nhg7h~NoPte7&g3W{qNsE! zP;N^BcodDy_HHA~$92fNgD3DGA8b(m-ZsqD5Sj=_J$qmVxw;1L0C7ZVxMa) z%-A_;2$L}?@&12XgyJadSCji=G}ip>l9ljpZ7@+EZ=l<#m<%}HyCF#E0h{hNNxt8( zvQ^j??nUe)5I2rMm`2EHT|<}s)t@GEU!UPKOkv04bGZnT`v3799TCaS)D3T}WUl}C zaG90`f2K2AZ(~S0zzziaQ&7!G=oG7Ua;Ja7Sd#cLdVF}|5cti&6un-e7p~&=rfQRa zMe8!UE@c)CG@`f4tiwGswd*5S?{Skkyl`0TfR}YSDt5j6r=~}-Rrn6O|G$nwN$^bj zzq^PPTyJMORfcTxvipE}@vFmqkBaWMLFA7km7bR(dygwg56Z3z?AWmJ-@ZlODo&|* zlh|$bxH(XnJjqw}o@v=aeoHKL9;~&w^d$~}qFb8d45Q?}dt|`G=mM)Pu$mJE`4>=w zu?s8>MwCDD6#jtUIspqWs|8kNK3uIq0 zux&E;MT~|#>d9Xa7+f^UvF>FG+iM*qbv~LKz)tF&A&~Eb_f^$4$!onJ1p*C8omz$s z;VEc${)I$E$|)nAh>s1YpyPM%S)1``{EqZIChb{$)jG>#@UhNmVMdrch zxaSKG*UK7*PirQpGc%4v$a;g8zWn$IBlLu>>KwXQ=6e&O8+&E4#?UGS^*UbSc^hd! z7j_+T10F5u+$Czl2cFOtNM8!wv^^qWsvRBSE+5-f-4z;PH3QsWjtAubn*#i%whw;5 z08$>G178ua?x&=tr*7vA+;!DoqP@w)WPz}mQZjQ6ZF)0^73|Y~L#{$dBh8CWeh!A? z&W5~dUwXTbUnv@-_$F#-*N|F<&oY4LjhNa9UfZP}{5yKa94E*>oAQ6%!=Yt$$ZHj; zvWf&9y5!*QleJQ8bTb9vz=NJVp(<5TGhqY{n;n%grkE2>GtlXlyLivt5^ps*T{t$@ zLG-LJ*$EBr4kB^6xhu0r01^kq)KJW4+x8{hxX9x3N4gDHR*!--Tz#fA zrVjhzgdc*%BZTHT-?e*unU$L7ndsj{5=JD9DAwHxla~r<}715$C$p0rbc(m8@Zg2C*#tUDHYlzeF|8t{T3<#%Y zGd&6UBTvD%Tfh!^3f2|Qmogc-PK20aob6JWSh7S)(+NIPV4Fv|(@WUH2q^#dUG05i zXbUS#i<#X9DhjCfkrV!N%=6s4<0Z>(ZMvaUdL^`4D85&{->ABhf`iwEj0g=D6R-%u zGtNYpEcxTYO1KqHL3+T5b+XOI#h>o0y`zGqvDdyDaF1H)rdg><%sVX&cQ;sdp* z`Bz36K9~K|gXX-Mz%&Sh{d17G4Z?R3?}MUN?{V6nsrNmNKr(mv%p8{h+&8y*{nLnL z0^lCy9){|%_G_W#HMjN6&bA$a5SuCdN^Q71Y-WDDlDDbe_r`JiK+p$k_-gR^0K8AR z5_-9!Y2Z0L!_zcX(YSG0y5A;uXDvRYkrD^Z_1A7qEd1Kmj z3i6@5(buj@BxS=Okic$!B_tCr0}TOy4eBuYP>35wkK4vPgp+&qH z_R4F6w+Zu_iHK$Nllee(jtp*kGu`&kLE-}!gE$)T&iI@+7E-tHJN`dI6=LY_9_F7p z6()V+r+k{54u^lQXlvf4y%Jj%!4Y$PiRcG{9yOg>d(o})vd!Kr-8!$cW3J}RL*2{; zr(MHgVzDuH8b!cB1xA?uDo?lYG$%;NgtV#mB{ zZthMn7Xx_ktItSa@s+8Zg|B9W=lia#)y4!0s@hm+8g{GZ)5hD6^N=6#u-z$T^$MzEA=W)E*jL)_}_lcZ%DqzuN6yYvZ(Eyqa~xxMHy46ztm z&hk5)C_r0u+Z`4=iP?5*rIr7b0N^?^>oU*DW_~eOn#@wwK-O(KsGN^gZpi05TR#Mb zXEaO6e#@%jgxZWvk3L_1nv+GB@ciLPTY6&kPMN?AQeP6nsbOlW(=nvMv(2LEkS6-; zC+5krtJ1JX$pG7YdkY4+TupwTTE?|e$n7%@l&#m5RtXwTiT%DxxFr<9>EEO|0JYzm z&sl>N&)9YWS)|KEj!&9?-w2Y?+~tsS?=eW6|Lu1iBODNg z7M$Qol}&r-`#u|)cwB4Rmkq5A2VjjpxPEgvwDwQxfK6dQ!A1$IA!5FD2?a8HR?)au zwFdV+E15_ma;q=VuGEXso~&oehsJ^k?hZP8|He$NICO9}`1vju)UezV%`vEVMn5HVL;Pb`$m{D~Px=AMl1G+^})0q*C5N6{)IICl0PiDbxjG zRF`eF8ds)9q^<+Bbc<)8u%+8(zwI)!d@F!Saih&CPWP(Lf&FYlsm%#aYPz3X%)jU0 zDgu(+eZQ!d?4d0x=GH^1t*_}w7*Chyd#=ANRbWXsoRV<)2cEcHcA`XJER z?zV6iH*G@7x6PtC)d)YrWcoVSPRcHd$}eR6n%5C;-R#X>_Ac}b&wbyCKRQLUUSn;| zf&pRLNy|W~s+BIppv~vTPhG#aba%Bq10kkFKLzQjUHwh6{9AqGtlnLe{lt79YH}oz zW!J5(MCR8h%6F8axtCH;lPWx(Rxt!{6yin#U9~nuvij@KtZuPMQ`<~nwbJl`7X`64 zb7=*G&@~}A=GpE?`3XPv7oYQJKp;^A0ms!ZSgrv)#*_f7!pqUj8OS3N=6g9n{<`({ zYm(o|*`n{q4QameCRdq{?XSPYQ1<=wy*jMKqo-0$3_)X8>PdCT>q4rAXxpn13=4qs zGndi`e_>C9uLcUt%~%`x>%=OLL3TQQy6azc9P}nnjT71%z6#a&W$t}j6H%xC@KkET z5-e|}Fmh;0py@DK{~es>IEe#MIGay7NmFxG&kz;CjUcjas2aUzdi;qOT3{R#GqC3t zig>DTFz7f{9;ATibyA|!q7dPMxn-$ARQN?ZXN=^Xv|Lu2B+c^PbpU1dg zqu0euMFDH4)2O0_<E%c!GIK5Y&uUIr&%Wa)_ zs$xOsh()IrTDof2r$#-z6Ia-@Io4Df`$cq@u&|_m@5J5hvgLfRRDPdcd$9(_zb(hA zI59_KDOC+6-byUx<1sCo=PN^W14Sf@aaI8$fh_r%I!odn?$}!lg(X~7gWVTFWiaDZ zD|0lNMbT)tFnXi`j9z_UlevTAVh9Bkpn6!bCYhh&-G8$^+!{kXJ+g&(mhxE>E|&5` zzAa`nlVl<_Q|ij^dOCerl@W`a|G0xq>?Y(F8|2Ro`65tAgxVO$1;Fm2Rr{W<)jByf z2I#$K<=e>V=cZ>wevT_Nh4FEmrxtYK@s@HB+_B^)2s+^#*wP@lP534-5i5v@IMeXMGM3`08p5{ zk`yoFmwa-8KsDd)z$3?gcd_S#{9Xw(!$iQel$?Pg!M{ zxBhe+ppkT^-LHT`&}p#{y6@R}J|dan)Qh!Rco=r5d-%XJrCDo%$|1gCn4VTss(=nB z-byip=blpXg*tmjUb?r1gG>GT33xtJ`wyQMCPcX!R+}{Wm6(?_*p`bC@Koc}xoU`mII+XX!rT8mupr39fH#2EcX%d5NrS zjJJTG3F2^EY!gQyC>iw2<=AbFD(OwGS~eOT(S|tZy7r+qhEmH!$O4O#^F>1zgs!ek z3haJ9nO8o!Sv>|E+$0DRpH>iFy649Z{8D!ROV(XZ*s*j{cEr8uS2q87%N5JOC+f@% z`ANL*HH#y5ZE`-DATL|M3u*lu@Cdu0vRj^l0@L3xe__{xuN+$Tq~SsajEUt6KmPP9 z4VUj(=}2~ManH49c-ra^6*)t$j!~m(tZTJ+W)NiR3^kXTfGLD}Lc=u96d+Sua`lhg ziaxX^QPakhSCol~-7p%XJQ(0m&1~||Ktv6Z{(d7!RL3lmW?&KM?ttDyVz@KEe6BIp znCNS!*%>TJXa)Rlc4fg}RcVb|1Qo+#tj135^Zub!oQfzb9+YUJiJ*su#JSTD9x}4N0Q`}Q!qk~6vZ_T>vU7kKhCj-5BtV({{cDFLChzVQHs!8nc z5EDJmQ1Y!YT=>D@hg{Qlu$GUNV5Lcl3+vA}2h~_lyuPOyx7z@kkuDN;G0^(`&Qy|` zUe}Sb@4NXQ&c}BG)$6(S>n-UG7ulL;10Iu#@_h&vSiwYM@+xWeaA{`vn0VJE3^=PUp6qGU1@^*Y3MD%_y=1|+S0q5y$ztS}W}yoJK87;* z?t?Bit?7wP?a+zkv^s^7QKw1ncc?l}1syGHea~0o za7l-%wq7j_6=r!iTM&0pME^DDP;%~j4_%Nh^S10()2FiO;jHY2?z|l`NEhvD;C?{M zKYYpt-6}@P#3^mn+-4zOtRkB+O>;Rm_yfr%y&00m__lSF3PVYQ@Nn6uaYdSx2j4WgVZ&Gmh7rOy+ z^dEF3-(xN%t*mQ{Hj=}K7sc1#t`nQZ4d~bs8+wVic~Da#afC8)W@}KWF7Mj{ff3xC7NUA zW8(XXh5$HOLKZ{6=|mV$Jl!A}{>q}~?ZZhYW z`tmU+^3*DIW2Q!@b!p2Zc8E`jZ^X;l<#M8x6(|EPL-c30S3&-kB`I3S=DU(gzq=aA zeEe-u`iF@CD1MkAPy(nu?}hW-K5d%u+>zr*TV8$ zGGgW)ci5Tz{K(RU?{;~huN(4e>)V*mglT?)Tb&!JMq~EXMvZMv*Qt#+W!nO)*?9jZ zr+v))CI}Q$kZhn0ui1QvReyJ+)GD|EsV0&Pr2i71)9?Qzw;X2Ypzh(g5RRiW8mOoaG40=3%e4l-sZN2)m)Yr{564y#~El=GU?K~Vu zQxI7tjOC9-j*=aSHtmuYL|EPyf6G?UX^t!b}N z!9^dMl29lf0fghJ|A*Y`jT2nS)L`RVfZW5ZcUxP3Zye+oJ7iTyqwi^yrV^-Fz?)2n zkgJpfiI2m0hpXw%HX3yLr0mdH>ALBqk2tHw^WORp)HpsyT0-W;Tk`p zB1X8TfQD_QHq8Bl`ez5JAQKd$heNz8Weytei&{>g8`j-9D3Y{F1lR{Vy<_%5i7ZwF zWE4c=F4GXwisIN3xZPjID%g1Fa6y4>SEHg^aI~rGj?Sq}1M1G$A_G&)f2)`iKE&(l zDPltm^1oQs1HiP~Y|!G(pB3*`_unD8J%-k+-`AX+J~Y;1hzE``rcfUOFQ};8Q>uzI z2&YkC4LNg2usYNW*B8&kL@x&TVDOBeLY7}rdsu~6Ny*U?`UTNO^Y@aLJZXH}rJJ7X??$ekg6E}@4A&#p@N*-(9Gf(o1vpw9AJ5~+}$Z%XYe%TQ3 zlq3UIWearBPb*jXUwREkEQsY3Z&B}o!d7*kdkh%XF-#_Oad<8Lnm*=VWc;?}RF+yv zIx~`ooTO@R>D;56B{5^3%=v%;3$3oFA51YN+sKFzgnyLxnlong;T5tOCERs@liUqeB|4T& zbDqiNGZh!dS0`zbwes}EqJoUZBsR<|Hbu_@&)94hUCr%{I!OJR)-t8TTZh{m{>YJE zZ<_ku98}Jo0k3s6n!HzOjbN=|_3P(;%OPTedA|n;NKY{$yYcpI@r^+Rj7SnjxOU z;ku6|&umS(0MJr`+o0z?bxctC<1nX0)1b~v)s#;3T_&s7OT^*h3A>^TFqb%P!rs2i z#ItP@Rk3<+9PBTG+y+pb=L5K)C8A&{c@n!JlZ5xS2Q_p*AGOp_#+3vDG9%6YEYEbyvI z!0yg}uiGaaOx+A_S!IVcz?2yh^;XW<;@IZT+ij({x%>q)VSGxO)B;yo-folE$@uob z1Ae1eIPxOi@`yZiqu5ISremCuQWbW2%Nr4#LpNNOXkNQ6vTP`$SuF7Umbsg6N_9SB zh;T%^8#pF|@}B-T1lsujya4#)U!KLL2ezJ8^~XInnY2X73CiQ}e4nin){IH!(Ic(V zgF2x-(S>m>m=f-yhx<>RxDVmKG`DtGF0IO)s~e}Xcq(7(ot^`RY$dXfHOoXt%i0v07m1`y-aaAks>Rl|*X~>!I zB16x9m}BfObQ7V1exeD4p`!Vi3x1<@<#e1B_JmRdn@hiJoVxwx7uEFsPQ>qpxIWI2 zi$kSmrhS*&-EGj+97nzNiG3DVnO=^={P`z=!Ece^*8fA-TL(1Ve*eQWQBgoqkQM}y zmWGjvg3_G=N_Te~h)4-II#jx)hA?0%T|*cljDa*GB}T)ZYj}U|Z~dPA4OgD)oYy($ zbZdiGr|rx4Vw4^BOr2f>oZ1rB#bJ{H{=!bRIs@cVF3_k-vCqchgPI3oy^ncil2*aHVE1K*e z+yin4wR5lV#kXqtG9=8;r2C*I$xr7;3{)Pict<<_`r^DYIyJOTaH@yzP#rOC&z}jL z3;mhEgs{bYS{e$RTC9E~K4j|XREyIv4(_jDbpA6E@NS@D=-2)gQ$qyY`DoXcU^d<6 zjjiQVrzCEmofD+Is!=}C1t%%5B$SAhsH=Sm%W7-`2Z1%+9Cs?S?{RRRIW5TO79<*iw%zETNhOb(NN##d~+l_I``W94!v88ndY-dhWe~Ec(D> z?pz8Ie6nf@`4&kwk>bHUs8YOqtjh9+)1;Qi|0AsPp4#&0YG$Pu_jK|=#Vxf@6(t#( zWlF|Qdvbds3A})s9^}2X2O?&llfS$V@7hYewmr)bT*XPN<0+t$%x#|`%tT?$j->0& zh|P=8PzxLRJtCa~wC0YR*3ZIAvJGN@j&u=ezj&~_{oS6EE!l*JB|^f`l8SF;c=)bd zEwd$r2SLWW;YPQ<}afEmXNa{JD&CI($^jy87yrL%Q+`eexaR zjCaFVHqE zoKo}N+~!}MNv^*7($J5{9p_Zx+pz_S#cYu%trqQPZEHaC4WAS%Zw>~;Jp$%&Y;31s zHJ^>!qVxXS!*>kUC6~;}m6JaxM+t){Z!QQ3mKQTDRZP^+KKlH^VDdkke#Pd7O1`lm z!hh;sa7u11><84iz;j$SUd}Y1$h8T3Pnkb0%dI8w22dI36G3HkhE*N z+C|b3KaSKlW@n_@224Lovj0#V@&Yf}Pf7Xvc3F1iWdz?xm zalk$k$@mD~dJ?7Ee%3POdnBQAi%u6eUu@T-nnWZ_DZc=lBzX{NePF|~;W5?~m0dkM zZ9RvZGS~ngxWu)L^F+gRJBJM9eV@@?jkq&xAFD5~#c7 z8c4x~CgjC^DcWkf{0vnpaF1rZg(3I{$X_>gJaJs= zbxSL18lym~i0Dgh{k{y<1%+3n2YR#p1qzKL!A%U#j(CSq<@La!d6 zVeIk&5*6gfEs)XKev_bh6g{bL%R+S(mKNJ4@N0r<0eJy%&GxJj^jhSjG9ltR%Hnp2 zzFP1oQqPT#_%!k*I;hh=w*ONr?_mSu9x?yBhxrPQs1;{vLM$1L;je`=A@!e2%8N*FWzWT!= zwScTLHu!qnNk~JOosBYaxMrU{zop9C<9N~%SE-lN+sKW7vtEIyoJAIS?7`f1qlPCs zoIA*MX<*qVvm`K+6lZ$Ei~8To9X0J3F7_vxJnWZMbOEw$6rT8bprmvM^|-&boIa56 zcGL+|5%O%39%NFI3hB&Y2L%fkD<{XvdbKsm1-~=}0mpECVys8N1b%veSM&UI(mP9! zYrcgBry%+iMNy)qID;*;>9H)nj2@&;){sZPQ87-2-b-=6XE;vCLcg3%kE^F1T67Ay zx=|6|_#sb!FjKEM%y`kwsUD$xBKd$V3CM;Z?IDr>zNGT5g*p~V8Jq+s9c+lU4 z7$0b74gHr(WO1*Ow!Z#RuB0pH5CjNUOv)}YSQ#ktZw+IMC`BpCt^{)fi6N9%LHm@iO!t2u$TXG&4&NBAPq>~$V9tx9?6jSoak~fzf=(zo zn60Yzjo8X(K$uQ-U-E^9_}At%^zG^dD9X(14_Wa;E))l*Sm|*FYF}FN$di?5RHv!) zMhq*gUB3C9zS42X#@m(}dRkzh|0w+(qOYSzXA%4TwWTmi;(ZqX6nQj2Z3Qp_4c+6&>qTBJaU<`2R z880|RXh0wjto)dJTM^^i&E&|Njme!LKa8I5q9q=XhC=xAQj9F6fr8PF82?2|pr&`+ z>

;_uM($j2P1XH6C*7H<4qjUATB>{}R>03!3xM3M?JvFMnySk!S|( zu}uvbKZ*3`po8pPf=Y$RJm0~OJy=~1?+=n!w=S@T>>WUW#mIC-0*{)_aAN)*&4c& z7!*9#LAbJRpl;nL03UsNLD)PDH(%k2SPS~P2_oP%SgPG%Uz+R_W2BqRU>^cZp+bYY z8nqu~@3y#@*bmo-@d;yz3%o6LT^rU^6N?Z-)Uim9h-aBuk)S)Y&iFB9x$X>1+pYVV zaD!w9as=NK-0?7_5t(W{pyTaB-<>v!Ms0wSTn_DwMK#7H0%y=pQ-Sm0?>=+(V5X|@ z_V*Ug1SDx@itn`i=W024TFk)BZbOJSOG{4=#0jFJ_e3g*6Mw(X5l1A(>XV9>+as3B z89oV8d~9AK-NQ#4WSv{;(iI%eC=NixeL}quxKEWGbB{y@ojrFzz zj{_(6Ymf)nzWG#PpOb(FGsuGWIPw;s8j`eU#8+3@i;bJNA(U9bypWJk7l)lY3XYXL z!~IJB(6{BHrsM>^FM9f!Q}13;G?CaK z`sLHtwdk^d9j)G@jROU>`YH-J7`n4#_L>`WZQ9c+-dEOFz5wPs1aiw}ylRC41WSWg zLk!hqe4}hBN)vwjcv@c81o}qYOi-%=E^6-pKvrl~f8cW~2d*P!JDMU;wV7MgWlo@} z*%|dy!<6W50Dqr67$*man!ojsJIO~fLCX__>4Gz1LAw*ky*)zxZo~w=_!c6o_x}=X zHO6^C6okX6{t@al(9Z!}pIa_~9Qi|UTU$Nbgy0XX;dCJvXld?UUwQ+)?nZmAtE3fr zn@@%}c6ZOejI%ANGRCW1!t^ZliUV+P@q-~v`tzL}eK!c?Z1ZWh(}oMrPBv@vnXT33 zb6eCrrvkV8yAU9iD)BP330|0YWojNwG4pBRGKy1J%hMNb*_}k?N~_3+pS6mhA98<5 zdoapX;IXwWd+2_>O@_n4D(!nN8#3)h)s*q?q}Euem10o{zVCxs4KJu1^^#;L3uR7&G@3!s(SNZZ}iNo*jYB zC2&5ugcT9o+9$o!cDsia$@o@S7srl6)LrW7IIRb_uhQ;XppVc7b$>79P%O%sb5|N7 zoVd9gw=>)geii#}cz|3uuKj_*F1q5pB= zR!a1|vu>|$FtRh; z4>mRIQWBO;)3~{7%e^m>rV6WNl(IF~b?K1nnG-Q9#fl8ATOPsYt+lbr^MigidCv@9 z)eJJ(s#>d2osOa8H|Dt4Tw?T<=-UM;zQPkTxrrb#bBwtKeN{K~N2QH?7UQ5)GF>!w zl?u@ZrhHnoRhgCcYgBy6j?2r+1|T9znCY}Ls$R0%z(8`o(H5ih`W%F1 zsdzw8i||z6g1%^v%5mgOxL}-zQ?Yf@RVz3Aa;H(3H!oJ z0%c$n(&d5?xYi}yFIzIm+!B%3YDeDP{W!K>-d&85hdw=5#KA+kNv6T54q5qWnq#rF z;MbDCk!GbR9@LOkR#+?%3~3q|5CCnhy9B({Y4P|lP(CMzoXc8Xp7EXrwJ0@#LBkjN zAR#|}(D1}-1~r^&iS3%;GnAemZBvYtiD3p+4R*6D+Yf>o=JU|?3E8Ev(mvkmpI=$A zpccT6Cf!nOSsr|s%f8M6oms4`!0G4;R~8o{wDC72MaMyh0^SY>WM|%UzyOG+wf{-O z`V@ruYU&357!ZQzuhBsA6E&T~iV|=x%14&=3|H-M<{{BV&sMbg2h*67IS7FhY3JdH zssrd?;wp&U=mSVlRr=Q%zabBtaR(deuQU;V56q`W!JxRPj%~Dmnd8T4NQskcTX|tR z?oN#UP_(rw1q&oUqN(Oi!#)@4OO0MI`&IkP*6KSpZ$-zy^`qjvhx(9@gDn-P z$R~9jg?0J~CT-y3i>Q!EL(ah`b@n+swuij~KcF9+H@T^}gqYX0d2@-68s_x-vBzM^ zjh(n`V4k#fm&NRLZXTQ>A{^#}F9H7ufm@Hl(_(}qzQa*+zP10l0XS$0Q<U2?I&RB#x6>H3vc^70)vj<4~Qy z63UTL4gwI!q=&&g-GfU1L_l1lZ0yaq;%8v9mxv7e>UCZD7+JEiBB*{ zhCm*E``5=V4LR|#nTs=3d(5DUi~5@vW;b8!d$DHlv~4keJ_T;ecKE$w9Ixt<5=8$d zZ0IHD#aPABb{~#RRx`ZkEgE+zkv#XFHzIEI6NoRZ384zkrhm@;PU7h0{WwUf8O)5# zU^@#^$)GyOUT1L`=GSGxYmqPTjK=G&%=MPPGXneiWy-*aO>M^zjxsR2OeqA(FQfLa z5z95(i@U$487O{aTV_MP<0a&9^FXL)y*D};6bty?Z*u{)41P4}TFT`<5XjqN7>=Oz z#6;Wg%=vr48ld=LZ7NzO{D<`?{S&po_)};Fyshvvkh48|#@6c? zAkjPrdtc5{;@+^p1})~rAjh)F2!EH&a^9Nh4-mqj_KNwT*r@5+^iWHt++KnUd>rTP zY%1Y*KdMq!cY}Jqc&OLbFynLo)EfOsV)T0YZg1hqJCpp(J!Gsp(<)lo1)blfXS)TV zpm(iK$5}bNN^TdAh?;JufGpZ9jIX>qrxIQLdr(K?3sPlMSylK8F2`u#DB|wQ)M+Y> z#`mUc`ZU!7(2Eh@D^5Wmg8gQrEtpa_EvLcPAwbG1QeIlt`~34b$6Ymge#mC( z>S<(a%YSvm?UZY1PGPD%TiAKX{oSRf0@6rUEKRki!2!tUhjT&(lz_j!gftrY@S_fJ zHN$_6iJ3&n(+zI*!!=8T+q$M6u;I3C4K1S#Kt!YV)xNZ0$!P#=I_Qbd^~`6E*doA# z>dYvf-mCVEp|pS`z0!wEmW@4y$7MuVr{`d|`%C-dR^tPh&&UP>uA+&3Z$-xsXv0_; zW%)=UYhitg2^HN{wY`DE%5pE)PyL~{Lm8qA>gyIjUj+!oJ$GK9d&u5?+(gJv)fNa- zt2Xq?&@b^`wT2{NP3Pkdrbb5aHOhbsJrwTU6|8_#VnFXI;2}RLRQXrPIA^rDn|N~f z+%j%p{Pn)ZRR?}5ldktqT|!WQGUpv2G3+3uyfG{5YO1Ml1UE1HyZfo~Fv6lK!`*o~ zP%79vaP4zcppwY8mL|zVV{Qpe4n#gZDz0>X$$ukU@;A%TPBYDwT(%t^9F(yusu|I> zp2pAxEl-PYk0N;259G%U-&`XR^JP_ap3~X;J`*t;wc#Zmo~EdJH;)Z&6Y@2fKb?Dc zy@e(omP``_Owrq$5TU+5tB%oTx|a9c3zJyP-q8!LRIg7wFx}rREjfmHL|f8#mlhp( zs%-(XxmNGaiYq8gG;y5FDoT5^}0=BS86pYp%Dq zsQ=kCIYDOroUp63DuGPa`eYuGmJApS;&S<`Vx7(+EI`5WjHGL{)3iI>*m1_My(3-}QGMvskye zql%$ths53WM$S1%hV@H5IDO8uTu6exYczbde{cVy2<%Js}Y5oS+vpJHiPdjW5f;%ON*n!|b_HzlP62vbNRAg{G@ zB5sYWcdPeMw^ihHaAwQ&rm^}ZA@%nke$J~PA3j%`J3AM!eW7STs^gCU`))Q+N*}Xo z*co7jb?Yg`nlBiWGL}y@+2iuzZmu3*K1l6q4 z;KNs+bNj#j)W*HA{dz9lMMcln4;Nck(bN6IxTZDFiUw}b%Zfb~aje%X)I@007-bMY ztPMHs+*Dd;IRvg9-w>?$_>}e&i*&)lFu$jS42N(IKx8u$54&jw2XPBE2k~J}nz}^? zKZ(x+$Iu9x-JbpbgE+Cov=tRCi^pQa^wVy4| z0<~LPZBu*384CyiKB-Upwt+i)MvQ5m?zU>52tErGr8*d z!&lc8{Y2;A@ICR~Z?6@&Pk2)fY^ZYd%J|IXLQk=(t+;BbsYy<1Er6m+_UB999C+=r z<7O(0hFe{tpFS?9D@;zXY-#{H9Hd=UF>sj-o|cMP_tBVv09S5N_b1xidUxWIR(qjA z)H0P;g2iFVT6enG$%u?FLx#d-1uGi7WBD&PhqAlLb-H;0BiX>sBs3?R59PJ5gW8EU zZLTc30S3XAqm$Fmk9Z9tuYJ?4{p@eabD_v3m=7xIL7bCY89~Aq{2QwqS$~lVGMMhfh9+!-9oE(qeI>wB9-mW9? z^##fV52?(BaP>GRf#*Dw^Js9PP~e?Tr1B`jlcFBL#MkPD5Q6{D?(PdDq7swKb{@?72WijYide^C)dB^%ZE77Mv|%QujZ^?pTDsPsvsHA>fPT^g3RDE_8~)Va)h`JRxC~%TPnlO4Ovu0S3QhLZ|9ij zaAW28%zJf`72C|~Sq#h)OOb`69#?(gzz1TfRP!g7k}5rEd(-q36J2QZej-1L@5y}o z^?^8G9f0c1z=Iwh6da=4ydaF`xR8@gCYzp)aTm&Pw=z}JzT=_cb-!xUgZe%AQ2VV> zGOOX9&!>whMj4Y#XoT7d)%px-Wmb?pAC22VqZ4AmDpv`d4%Eb|1AhlX@kn%=r?3;< zur%%t5>n82o-iv7d2jg)RFtI&TS)99KpX)_mu8k{NZuXkNOORxdan|k{iR7T&}#4# zZq{XjjF{|P+^f}H87Ty@S{(toAuC;=>12gk;tbsuk39k--d+sx69@L-*CG6-(hPkG zj~Z)MIACXhf!~74K(S6-My#2C(%fqLygv+rXPKJUIFownccVkQB3R#9MDZ6c0)Y^y*km3oL zRkB6k%hm}#du=rpZH?~QlI#t*JE42~Kzulgpt$2*S9nDUuenB+0KzSxHzO+M>gYy! znpBNRq9AObwtSWOGwuBuoIa4yXNADY3yyCs@ULNLkQ!OsED_4tN_pmLqQbYCGI!lR zyZ_DF1ayf&-Y+z)$g#sQNX}|rY)2ebHZVOAn$Rwe@6ly-TlGyPzlOwIQP6wdM z66D5#33=GN8#Al75JV&O&+g=bv&U(<(?;cD*y+d?P(oKOk2Zqj+Ri6~+l*;_kN_h*uqv73i4)GzN z{MuW&<&r`ZjdHmMQT`tnEg{pCR^q3y4OZIM-9vs`4 zuf;8`NhH2M4WSMBD`f+~S^e;?mNV272P)fjlN0r-KOXMWpBc>bQfZQbJ)Z(dg5c)tPa}Qe3iQTGPd_wF$%9;o>$kFfH4hakA>8Q-T{6{TXBlcftQSEgOSq zSFAaFq^{#Kwi#2l#0DZ2HWYw|24?+;!mW9b-0-_+NHeTF`^tEHmddkiv(A z7(FZOh^tE(sl(Ag-+}mX6p=*+X0hTbvn?&qdpBoC9YEv0#9aFm#QnBMcX~K-t?90` zAzObAfc+P`apq~p=6%R^S0c4gQB%=lNBABeLjAUu!D?Cht_0ja-VFBH9v59uc?g#A z+tTj|_Jh#a6rDdcR{u}HsTL22I<=wO1n_RnjAtJXscKh{!^WB)Wc#REOH}|2z{^hU zSi5{4$f=k9um}Wl#l=Dmk(E&BuF~-IvU+?_V;B0ufmGQVO6Z zZ7WAEtWMbQjl@U$l%cItOqhla{AilfEy0NJVH>Rii5A*W=dcaO?T}JhDfF{)AULw2 zYERO8?}Izc5jGUbi4RZkf&AMl~&6BUzHup;;)M_N)Mj;j>FoQyBq*Izmr?pSNc&m~konW)Sx zwjSVVQ?(*@Ri(e9*>eA%b4vNayXta-bg6p_6p2SMO?C}I%dSFSDULLK@~;_AGVE*j zui4Qqlk!l(t~y$4fMa?_W~HO408T0Ytv3L*?xp+A_(9=o@4p2tKM=7Q*2w z*OcyX1MlWrT0!CfQNoWCtkl>lymmJp68Q=gEcD-36HnoMI!4b!ZWYrD&Sv)d@km&G zwUYCH<~LYTAk-k4WUQBU9o27&e_YBiQ9Y=EiBdJ2`T}~^0Oq+$d3LlJ6!Hheuq4MF zP2ZC`|F}Ef30NR1j%0e-ap%hCC=g#sEfK}Kf)`!SBCFcix zC~G6sDJ-GESH=BMFeX=etj4@`XEZQf(V$C_CVf0f9ZYoVlD6XF-P|(}zTlyCgbynG z!QA6qj(A7EZ<)}0ej~=rV`wklRq#SNQyy7lU zqx``eVBMkZ(3?#_gi7^njlP%RW_@mCsMV=tHl~k(fHx^e4r@B>FD>v9xZjY_-&!Xk zF30*_i}f}f2*6~MYq*$=O+9Xzb*Jn_ir%6IDCZbJ^<#G0p__enC|R*!H)<| zm0Km}a1dMrA}t88n}!h%g0rUd6o zpkCD#ECL&B1UOEcqf1AeG4sw|+@Rx2UXmGT(+BzZJooQQF}ZeC)k1lBW{}tnsb=?U zzh`=19}^?(3Zd>`^)q4_F8fgV4KcUb@y)`Av+{k1YGe986Ms9&a1}s7pCQ0p>e+OI z35dohU7nDU8ypATknzM7JUv=0z#`AZbosGKxRs@jDA(7M{ zadziyS|10`2b9i^Z1tynN?Lv9l|G#*CxqAA%!~H`j5)4emg(zPq`kq!LM;O)D()Le zZR8#KgAD|hJy--(e!!Od=E-8Dx4ZN4Jl^y3V?D1{NByz0HN9Nd{|zz`t1hfyNvVpU zbnmii6;X&~zn-zx%Q4Qj6?^5r{9+IE*Rl7r8Y?s@1{67?C+t}a;|-2eEwe8i6=~|$ z_Yr_H%0xuEDV7y~)rkUc7OqcKX5siyGz(wXtnRy|hhz%e9?qKBE9-NXl&D)H$;v!N zEwDFA?f*Vjyr1^}SHi=Y)Mo`GFC?xStAiQo;m4aJM0>Um;oj&jT&2ByQ~}Tc*3_JczJjf9Riu?mFK|h3^v;(o!d638GhHhWpfi>yp~hcJntZ8Sb-+WF zy;a;r_})7HsQJ{%*xFTu&%c9k6Y^n-JPBXoPj!JBvXUDoV_%)WE-~o$%*sJ)Vj@@j zlZeUgT&xyXHtOrX?5K>!!extA(_BQZ9ex6+iGX6T=nK*Ky~+!xA-lH|uYR*Znp@}8 zuNuiPFmRASpa18Ll+c}wt>LxNNTyItVr6CFTsU{p5skpjep`Y$d1;izvIwC;Xy$Ho zkFY1={T%wlt^PRszUzWp)x@!M&5`+P6G77NPYZ-?`JXmLe99*WO`&pPAqOTCD&Gz);uJbTu6|!K?gDl zG4#$hqpASjz?8Zm7B@9cBswIY1Ml_K&i|WuLeSQBGDc}GFvWx{1BJd~kb1;7lT;^| z52s7F6+>^=pZEa~p4U(1Bjo)DNybW;yk!2I+c8(Giy>HSecQ|`ODmMyP*r95FNP4s z-^HH`b5?;Sl#W_g1i$$K_5TK1uHcS4mrOfM%idoBA(6^R08VxXPF8MXJ0tMp`bvb8 zJ0B8*{Wae>f7@w}c_%Q_ldvDLdSvSAkDQ}oDjS&U_Aj`=h_fjOkt%TMIl zTGtQP;AX!s*U=vI=#&<@*oucIcV_z9o=o=hUArZP<7btsQ;=`kdG9#dGDt~E|p+y}+I&Z83%0|t}z-~W(>DfX9W>gUCk0L;VY3S2tBhTNyiRU})6 zgTB>#7SsX&v;~0R0{Fc?Uwau$6At7$NuxGNv(S~UTP$}Ky`{->MbBPAvg#kg7u=%7Phk{rVF&vnhsy0$y9_nbuZ5NCT`da9mAC!L4{g%c5Kw!g=T ziv1=hY}yFP@laYjhWQgM)79CRt10^dg09lmU$!)zNnB6bEZ@4s2+-rxH_FI$e&lpt zdy7GC_h|GFB2;1lRWN#sFK|aGz?wk?myPl;xgZ4|ec|s^SIfwv;DL8PhhHXEkFWE4 z%PV*X@jh@(=;-`lfYHzgI`ZK=EoH>3IyFTUKN_`FC$7ZA$d^c0A(C0{pKkff>RtZH z2~r48?Atz+;O}fsy}3=2;l@`lJaaVxki)?6tu~)6lG1yaqlKkFKQEPJT z?W9Ult$5dDw{^V5m`|=VOI99{0_pR2OqHw46gOX&hR8@s`W=n`l-CEE+rkmO4yb-E zm^@qHKZ2kg?14DftiGQK`c6;_!#&Op2`=*;Pu&J%lez02KR|2=D6;0GiK8s)@oVAS zZ$NyJ?)?`GYN2U@(HQ?i{gHm|?fAHYa(GrfaEQKXgFMMk&_a^<^YI6U+tNE!GL{*X zdY|Xnkjv?dk)AXe-%l;*xNgBBz%X=T!%qGf(%sodnN<@YPz8=MMfpqkB$hDtws&n5 zwU8JrRM%nG@v9T)%VS}WW3;tn%@(+ue}10p4X?7+c;9eFxW?l;L03C{nhy3xWB;4I zNXh-!U=Jg+CFlt;>#4h@fGeoYk|-ux&cDCzA)R1za+uX|Ngi1wc-lOfE$;zxIzvQG zbTp1My_kvl<3I21bZ7Sx6=aUmHIDA@OkvJk*F33&-cHRbq>LrDkxa`ocicSs&zfZ4 z8~>cPVp{LYIGk$fLlDLtYa1H=tEuD_6unC1O1}j5OmA-~q?3jLRuiy+Q=D)0+^MA= z@BjJi|J7x|>@T+ZyX=sW6rYU=fRAFf9Idws&Sb3}RY@efT%7w`hkhigNGULzYY}}X z+I!gpxwky#eEVIC>tN6;HvRr@NsRhu}l=KgQB*1vrWEGy#K!FwShIpS}i z4>Sm%r;nA|Jr7~A6d+X(c#11%6ol?K$z>gxdQTX8Bx0%F9)|wYE0QC@ufrlWNjybt zyni~#Cez>TtCK`T)CYxN{KkSHL27?jO`Zu9F1|Y8%EU>Tx)D`&v;uY~aHs*V1(OT9 zB?B}4FcC-H-Zj&*LFJoOA1KXH%L#bjzl~6bU-bW?S>n1pALhP!CCL!-HYw%$33d{( z&5D!Se`OjSv=!b-S2jNQ)hhCY9;VI9bLRc?{1}5fcby&l=4>{;vaAKIv}H{i=>gwp zbZf8TuTM%Gb@8pkIsH8d%V$m%mp$xiyiBV3Xud^^w71VzxwU+WbFGBTyc46x@c0rh zGSI}9^xwDuk&ITUXrjypD4pk7-G%U9KlKJ~$Ud&{1pN=Ahn4@q;eS^SKV2LExyo;T zp84qG!REvOkE4(_x6YKtGLpm9SY$8K+I^$}=LD--0Ui+`~@@BC|cdvMsx8++r` z>+Z{MXr2iqN5)@tO?v(M!idJLJFg+nAJ3s~*>qnRD7bYmgym}W?|$C8(awZw!)$nI z&byM59DUoPQpLeRmQfbuwQB;&weHCII~4Q7fvfIofvamf2l#rQ`)+HSX`8k4m*+Ej z)7)vRPb7%-G5j<@W1(vyp$iEBWY4Yk!BaguV=&3VlGO%|RT|%d>hppxAL{ z5WT~=eLLMvaB1OWa3w(Nf}gGmxOJZ&%{9N=#e09hGB{1juQ)fJ_LI-2_n-1VeZwb- z8*Ui-I{Gac<0gE?j#}8QGXo3k*2!bY&RuA^5PI4sv$oqsmnQ`+xfl2CsYXIrYPy7E?Pyq*FRpL ztXf!h?(Y5~f+KLG4P6t!e6)v5KRU?lK3-HekbdieaoPOy_Rq&ZU;q4kc&W@$)DBY` zX^F`&aqa0F8t^z+=%&d+mf9L|HY|6}A4%q&?yf%tktc<~n2lZ}kPC%ty75X)fv#Fa7@~jG?PE&EiW=}T$@x61K zLv(0~Rh5a|y{)w%o}&es?x3~g5$bp`eV#GaG0w4jV7X=lfTx~H?qwx4@ zZhXX0YysTCi8u94M5&eiVp8Uec4V?B-?CqhOK9*#0?qNgLmi*{J_bInK1x33K8WsE$#gjOvTq&Q zTV_~9U{FK^oxu4qdL;`T6sh&ZY8(Q|Iv3LUzC^Z*B;|E?tW5;mMz1S%TM}vJP{<=3 z@7*`n|DACvLW+|bOdA33+X}|+QOYzG+!J($g!G-3 zwQSe@ID1Cr0dH1ZpOf>ddJX75hi+IEebYFck;0L~QyoLUbtbJ-@PA6#i~uLi3jEqN z$}29Ri}sie5BBORi&6R92;0ZzxS;`Me>=YsSnmOiYG|_W8Av1*gj4SCt6d7tE1;zL zv=58)TKZMgL%*UPc=C+;beGdPJUK9$c83OzGETCB8a z$b|qY%+df1@94XMN%*gVI(XkXx@V|eepR;rHEVC9kUy&Nj*7+WcVUZqV!yj~q|vYQ z20}T*-q~ZKuyCdI+!Z9ZlisiGzN-Ioh;!@y!Qz;-+DP^F!n2244Sc-%lmiQG{W)^5 z>J=*+OpXnWU%sP9?oYgjb!4ld+Hl2W-*%rlkjEPZ1qlhvGTR&}|KnD2D+uH%ys=jc zpSFY`*E@rw;^S;yvtk#N^NZnFNk_y(#{%<}EWlM|*UN_I^>>AB=7kuENJ72KO_mmv zl_E0tuXp^HVCx9+GoLqjm{tRIc1PIbpC? zdkmXl`Oe_Nxx74bwd&mHlE*4X2c(Tm^KZb6q-sj^;)`w4E&Tsnwfu@Tmk52k@~V-K zl&-@Y`;~`tFG9T!fr8uq1%^rE=G`XSne@FqLk-!Y{O#}E3-p}}2DJ9rfj z?n$1e6ng?H@Qt48El*IIeuZ<)cyLp`Cy}B}5X}=WpDEKVF@wYM1 zm5}3gQ>%#}g{C_K4+bBcv%k0O=pQ3*!M3<6RHx41XoTVILn414)Na3VRO`9!Q3Q0r zIdA>OoyTdSyiC41o*vMr&Z-Sl1hS-mWV~Rz@`iOOqbcV|)%-|!Pw03dTRhHiN*BQu zpJ!A(H#b-w?=eYB8<|SBw3^DjfGXpgKpdvVAH3VWFh#B0MPfI$)xEqH7`R;`Dxytw zTbo^*Q@bSLHqvaZXr&+--UpQm<4u*isPhxf6@bTTlQ<&8czAds5`S23dYI!9swqRq z*ade2>b8Bl)>Sj(rJR|nb?*FgWbXRz)56S1E?($dWmgUi(o{dbch-W+6V`kw2TP_EOCO@G9#;akx?Xa-ZyM%Kz z|LDb>0p|?frxQ1s-UVl(!j+>dCp$CqjQ%vQaxsk%MxOFD21r(VVXh#uUICt_@YxLy*>tEm_tAlAFOqRSdEQ#)cpg_LHqP}ImN@4; zXz9%l6Bg)#OJSjIRe>ILJi=Nn=F1f%y*c1BKM$Tih^=yTM8q90p@QwzRC*0s#I;u0 z;dxeb^Efl9a?6#6OAF=d^+U1dNd7G;LWHg4f9ys6k|g%gnRBd>C`O;%89D@45)eVb z%8$R)R**t#hz+XzO7Hb#4+K+;>-@4mby(Fw=UeMe^r@)H$nvd`H5I~<+LTlJPWNVI z=HA@$pS=E}aXq=c2=yfOZ1pdhNQxSkPo1BpJQKTs_FLF$2s7GY1Pja*>n>H6u*mr8 z?(@(`-RG%~jZYQ4#8Izu-Pvc$VwKzfp|O&0aMp5_Jz0iwL|CwatIlyw)~AQ=mHs7J zg2?KFi-bhmd;fh8i>qBz6M2I3G>)@9=#sQ}rpv>7M;Ilf^^S{`2G4f0a)0{IavSpQ zi1S{phZjKKU8tv-bgi}IQG@j*iCH@gSW!kW}Cs9%-ij6P<;!`?2}(<3)vzQgz8aa~ z?ay=I6+nqxHqGg0Hyumq*;gT=kx;rJV_j7YsK{=}2A9FU{Y^4(ZPHD~egvMUx14TA zmkhPsiJ7Vo@?BhbsV4Ygh+s+kFMwE-P^+<fG{TICZ96?)1!|A^;}N6GrKY5jRC473Xfk z9wqMY;?vP(LAAmiX%>}!(6!2rT)^k@N_Y5KVk^7zK zD_bjzGHYGyPiQVr|T{RG<3Z=k~K^LZrz z;Q}|>+W{O+6WlV~p_lfh7+?1u;jGL}_RaFTe1*{TM^n!rc#=x&dDDyTI0l!1!zbUr z+nl<^frn?TrWc$^UOinCAnCs3?(Xe<*f4PTdh}2dYlJ$8I5*!I7`*XPY<79HVhzfC_6S>2#dk@16aX zW~X0D4{&JqfL=zaGrncy{)3(R629IQELC7wtW2FctrhPZqG-#TrSF}~&?Wz1HhD(_ z27P!QtS(nG#6}ZaM_D_$j*DOAf4pFgA@9Xv&@z*@# z>xt*WkJqfcI;UVhKKgAL|NXN_gXA8}I_R!EY6-)5bQMT3p35(T&^(B5-15vs@!n6# zD0r1Ef7jx!-CdWvUTLGYXGY<0l^fN0R3&*4{0-D5Xlb|NK#yEUi+!0jztY9(3$wNi z{?#}}= z|Lv}VwQAngziV>W;y$eZFj)>O-C13@`UC^ZiK~OH5As+0}&)Z$}I3DEM!EdvQtTgUyBq(dm?xn!^c1=;o&% zZnBM!3wv!?z6wm_7Dkzx+4R-Az(#V;1-LFWx~*UFZ*$LGtnk+>rHc@+KT#r$);Ac& zpY{Q!7p>fV^f7xGY&f@O9Cn-@hBggw+a5_QOc=iKKRjWgy3#V{N#99t+*!c+=Z(Ac zs7=>5;uqyerI+9X9)tFMA_D0u!qGji7lrWc z_s<47fvp%RaQP-={R|Uny{Vn69XapXhQRDmL z!wJ;!k(yPoXFm5x)PJeq6A$}}4Q5mXZ`mTOKpil&k|n6ar?b@FH>`IK#jSMoS>x3N zSUZ8;e3}H9o~b9lx)g zsjVCev_tV(I`*!Qv%R6>iSR0n95r-59*QNRnca7N;)IrnVE5%u>qh4Cw}(YwIZj8t}+iq!dqDZ8mfjjOi zx3x{xBYu&=gcMu4RPn%F^(`*K5-KxoEXw`+kGk0=J`wqwv$%w` z57IS+TuMFD2_Mt7W8tES$~ybv-`d7IIs3K&8y-EweREz3D>r*o zEs4n8#D6#;Z;tWo9{{yhU4u}cTIi^-gjblk5?2ZL5qEHpR>Pq;15^$Ys>7MS0BnFp1uL{kES<#!w9X#TN-f!j4y;jjBcy;I%;!i`oc%j z{Lha6(8daviO7OwQ<)}So|ByZopO|X1+?(;!ylOCE|!k|f9$;nG+f>LFgl4O-Vh`N z2|+>-J<&!PYp$sn(*9A0j>B???XNR-s|6+j%9}o#(^cbYMnz0 zTU9lyA=Aeh$>L4No=jbA1J z`nSLS>Jdm69|Fi?@sX5-pRr4L{ex%fV$N>3-)}@oLL0Fk#mC`Q&m&?@qg&>0vZFmb zj~I@N5~P+x+~HSSzZ9O%V#Db@SR?DQj?0=U^BJOB=v1_2P0FI6e(wUpYrwutM_Wtc zj19%I?UbHl+}^izeuq^YEN>OaY{P{&h6C02oow1B;XRnOlUBBJ4DqA(JrSybLiimz8_;6MIaR|vl+->k5s39V0YW-U176l+FN z6>HgI{makVKw1dbbMl$RwAAqek995?>KCO3`SWouBHHOsLeg36I5F)xUC-00!1V8FU3NG&K*NHwqHAx&Qi|L-_>H(0J}`7m!s@a z`t2w6+eaivp~d$J2FDBYj;Xe3sB(416KsCk8@vqQ8eC%m9MO6pbP=upEslE&6I$)T82d(9E)X=O(P8hjCl*aLvD2P?DhG1jVc{XiUX6 zSUkf)%DAPN@FW&O1w?;IfBpumAP6Qe3WFuXDjJQ-6$rhpf9GY(Jn7*I3^T;3B#S~? zzVuoxFr+S8;H%O>rZ6bra)kX32c+#4YrgLGXd&-A4|)7ouLyE!PmG{2>f-Dko#!{=IaFF@D!{qP0rc6>nY zQ8|z;L>QIv4Gukd{?Rn&sP8qW4tHK*UPcfHb<$dAv3D9E;tWu(v&fQ9yp0UL$+5`= z!gheTd*4tTbkCY4sWELtB29bALO>ZfBB$k8+>}g`;I=G0cSu0%z&S7?oh9gE>>yah z%3vZ-X#-6oX;^xk`@~oZORrKt%T-oqn2vF?W^U4b(Q2 z;3^d^ow>QTh{N{e$H9Rg85?e7b6|DN{7ghQlxTMRr^r^Lr~b$7t1~OV>fwc84@0MH>ru&kTLo zRk-V(%$~&rBr6-A4&e}~3oDd5(*hA45xrjb>DRhoavD#)&l}Z_Z9nx9+TPmGuA@j) zSD~e`?z+?ZFV~}-X$1z2go?J_cbD~TH;p_!X*9-8*635YeK~4G*MB)4rQ$~J3uWNVELwaS z)F~8kK>ffyJhWNS%(_#pn_4Y2xQAXH;oBIA(s3QPKJvB@3kRg1q`cA0V&uJwc?x5) zko2~hQ@Q5tBaTm+r^P~?#L^nOBhKR1X~o1F|sE%gs01)@BX)?;<}>mF0A)41BU z)T!oDagcjoY09oT>c)v_@wDS$qsg!X!iptdDdrn;6{!-LhIzxKWMpLAv--wNw43yD zPGVbo=?mdDr{K);!ClcxBc5{EG#2Mb3&1sut!WvZ=h4y8k!AmNXbgZ1e>BCm>%G&F za6TsFihynp;O+2$8L3kSYx;ss)zGMWBGN!eP58xDXtI{sgX_*ktCSq8Lx)O^qf=1& z+uO-fvx`|H%CuIao3evT=tkC_Vff@G6}r>YU31$}b&)4fu)CseTwitn`L`y-q~8za z4hVZF02gThE(LE^#ig>B{hS*p=bI?Biip^RQjUJ?D;{qp9;<#mtyu;Wq4H0rMDCyt z667d_k!~#ylwD491_d2F8W)yPmF5$r^(P!v8@|?}`c(Ihjn^ip)}V83UlU@EMoi+f zdA-|SzzH=B8d2&a%3Ri|PHt-I0=I=#>gGR`5Sa9=t){=cwAKIg zq<`UA?$XIqy9aLb6Y$oC^m}X%H-mV&_d6GY9B8=;W(K6_Aw_)mnN`AFe#>wCbg>p{HaSF(|~a|+?2NbC^aNupDHdl zO1BRPBv>zLSuSbRH8BNs7JW?v)UU$ON$O$NH;C?H*ujwILcBhjJ-Qoch|9EUIWf9C^{G)Ii!nNVnU7;?^gt-MsV$zp#Mv~h0e0$JqdOa1{+Yi}p^mDqO=jKdHFCv;aoIS<_4Jraqsre>R>1H8Lf zvE_chAV!{7of&miKQ{fxtv2`WZAm3ti<*4<<^E7UO|UABjibTYZ-|H<5DaAX7E^aF^&X4JLafp-hM_T9F~el?G*}YuC6u zeEjC^l!NBMRG2G1lfH!DP@4b{GFj;PG0I<(C?gyjy!YDH`pKH^kR z-__gJqFg?BO56B3Jq)09F<{n1U~n{azx+qSwt8;kJ+1f>j>3u!aPHWl@m>@BcXCnkB)SNZV>dzlsI8oQ7kQZY{$8EM(P5NaMg_F5#-DclNYM0NsxKc6hI%EF6aPJ4%+Tivp!IJy~&UCR`1zllEG)P zl!zN?6WFr)rC}TTXb7U4vm1WEw%x)Rk+Xv(=~$~}3J<)Av9irmr)A;3K|$2uHhiNCzLt-ITXwCEyDSa}(#TA)j>@1-d9C+# zyvK1-!`n4H*$8}fS4Hpf8Wfz;!Bc9q==219LnLDsXD0Fbf1<-})Q=sHYZ08Qw60T~ zleZ>+pR}7yp6s3qE+Ob&ye~*&NtK}aGkT<{fUTT|r|>h=DCfdigno4zQ%x zPX8fY!DGOOb(47z^ZFEioX*=&Gr6%zHwYc6SVTer)YKjZ@9kA_bW*wlQw^h+WIB5R zotu~1`MUU%_?%b@`9i{euFy~9`YPr6P{k~o*QN@ISdRoYVy{f!`1tajwq&{B@Ti{i za1BJ84n9~awCVRl<&Dtn7ga&7O|n#7mlT6HI5dmPmHCw$2}uD`5d)G4{?FbIjJ>-w zK6D8NksH|>aK1HjLPtXq0H^$*S*>BP-0Y{e{w&RC%whnAn~9&Oy8dx(DsGX7&#&>V z_tZcz#$b7&yClTOI<+?^GiIPWX0Ll1)6R~NEH&-=<;Diy)Ov5odgM2Rl>VwvQ^pf_ zH-->=y%Y}|UjB6C{cv|=EPuI8t|x1piSS7*9S|Ggan6}tl#Xc^F|M zU~O-1)L;7;J-;qxO;N@#cwJQ zo}9y$g9PMIh2!J{BfT)8+2VrPwx_#JM8Q@^&7tdKLLrMZW!|boTaLxOmI8w)@+Vw- zxu0GjMZJ%~8<>|Thrzi^UTDk8IXx#;uHX=0hpA9n?OVMu+KF*Ux`sW0oaRF|4C9Y7 z%PwvATuOc~QEIjXkYxDml>U_wWxO>tXA2cEmoa@_xhNZg~#&BOa;Wr3LS zOZ%Mha^vZvz-;#&-Zv*g>JZ*rmiWL%`!g+u4gB*$@>iy27HOGPTcZv?jMxaHTdSb! zIt~s>BiU*N?^Kdf*>b%kb_5W==A(^6+j{$7DZp~9LQHe39c&fi7`+#qL3Ja?IK;o7Q^+M&hyh=GSYaUHSo3G)JG0p0*@-|z| z?!x6;`4tnRWxh)qLU8v+6Sk7it&T;F)m9y+5J7tQpo`QLQBD(__}jt*PK}#(9>!_t$FBj zmhG%ttc^qsvA+b{47e_P#aipYU$T0ljgm#7v$4yP@53X9~KkjkNtW?ahRA zK}Pf~-)waP5y5=^O^w~0nr}V|yy(-nG@~Vj*R&-~1cpPI3JX(&TQB#pv;Zbrak?>W zkH(PElR&1CShjI#Rt0sg>bZ6{g*iTw4c)sy$TUl$gm%Epw1dq7gFe~&go2Gtb@djCX!xja?f%K@#^|$$_nym8<7geEl4(G=}%L4FIqf%a+h$ z%3Qkd)+zM=x(SZ4c*XgK+^e{9{>0MV+2Q~M`q@6SNL=n#nOkMOUqXybo9*}x(b)36 zdTai<^>|sKtV<9qQtTG2Vl7#n_B#)6QDrzqPWy{g+5x3m)K^;<+N0MDzn(D&Km{Tmx(`ttduKBmm)U-E< z?crfwV_ET&9GpGbvP=98IJ_X5o)ceri5N~IpcXvOjM){vejK{=x)+6iLjMbhVnsGX z}Xla?eV7!eFzRwab=P6V6o%QEiPe z+}w6dDDQm)OrS*YsEKoEw?)`}@E(0$t}XdaPzx2VDKRe8?YT-hE$S~)wC66pHoZ#x zt1UkqqD!zB+x#_-)>3R>^d_#Ox4Fs#}E^t0>aXlR}$KV zmc2}9&BH%vSdY7!+o4xnN1|jjGgW-alqQ2B!6sx*H??b=wWZHUQ*4SUq#z}y?W4aDskf)c2eXyc8L(MfLsR7emW_^7SkS?oL=E>^D!*PeSV_eoob~XxA?GIe3rl#j3v~b9+Ube~Kc!O1 zZ>)G9)?PBg0r+Dp1iRV`$de0AXK~~8GJ`lc4{8AWYulT?wC}2AjPc~-ns2eISD=w3 zrSb*l*m6ei78flA$hi|+QQpcAO_#|^ZM1TN1>e!OPi zzm{K|16MnTys*Jr>n)BtuCi8fOxe-Dus{&cxy5{ms#L4If2JR zT1_tQJeHx72MX%GL1f7Ogu>nOg*$7*RV;OW1NI6`E<+SN3gnKYs#SHTAx#VnA4y_8 zfN(V55MGf7kgo^;q{p;sn{{BJhO;x=o^;!BkZ4t&IqTMtUsdsk4^MZBll9@bA={3> zOLXC;pqV|K4IPxu5cQ(Og>dbWX&d4QqWiOOSE5m9O_y@qDC0BMYY+MlN$b+GD5q@y zW@n=ToI?4g#RQfG*vU1>1(WL~1h5J`$D-G6nr&jQRE_Tdp<2e>pIU27w@fOJ_U{In zSRQ=(oQ^NR$T{zbEd9W^ht_#9O7e*eq#^q!S%yG#;mSpZteM${aQ5Qe*e@wsX^^q8 zdj00o6pNXJH|h^n0|Q>&n_??%=XQa!j}( z9bcg0$Cio3-b1Cp4;r3E5Y-&Akr`OU!hq8;g_l*z*BvYtGJ8EavVU39R(Z~IRJcP~ zxm8^XKHX&lDRRe90JBtQysxWNxY;eU)VU5U1{t9gP$;e3A$>P_HQ|7v2ihD(#P&pu zre-P2qF4hEb~rF+B&5_udl8EXrL}@{FF0)QJx*O;z-nE+~U7SGs%_y#m=X7P1t zJes^a@pM;IP}&2_Cm(cv(f8{qsK-?^)O^xTTnTV-9FrSyreMQ11d<9#m$6mj-+G${ zAsL(P{2mVw;wXPZAjx&}HnFu?`IdyAE`S*yn5Bjo-rhfzY3t*a&;6AJ92h_%u=P;6 zfNE7FV&e?`A}_BUWyND_%XY;N@!uzV?8@QdB6 zl4*$~7f2YiqiqfF)W~&1GT3A*IS^_}w#}4TvS++$1nBj=!SF?+ ztUwYpAiaSriM_d(zMM|K#;A}M1!UZeP1Mf{KFtDDw$0mowFjiPe1o=Y;@2N$wh=?A zC*5qeUbmJ4^i__^6H)#FC@TQxG@RrFE3Emx2@zaJcY!d#o9eXNmV2gW<_ork(V4~2 z*f&C4TR)4cuBYM|ztl=0+@oMOiE5UeQS&+x%8%^1P41hUQT<%up&C0&!Six;D^(M_ z&ycapEQ8)@vakw(DfCAT3;2?A2m^VU}W1ijDU+czk z_}7S<6g*~edCqz`qPz1fIpS8}upwfTiT<+buKe>RWEtC~)KKygR}3;i(b;7V}SOJ87k%RKbEW!IOn_8xF0t;{S+C%#$L!#u6gTodY&w+gC} zYAkq6HMTCd$6boGElI*F4N>V0zlkACZw&5zQYY0aUMQ9UYz=kJ#rq1SUx3-sHhYcFVU*7#n6DKG9SY`qk$!me68k z)2*mFKa!T-eju#jkZJYGd%gK$(L|I-Tdrr!8cJJcwBeW8St{QLU~}7V>f=kRPui3x zS=-HzQpi5GCxovpm)qE$bex=k2>|m8vHoMtL&|TEsAX8sECc}MycjBL6$aR8S0k+Z zBN?Z?7pisXpsVF;qo;A5t4B|vUG8cwNWqW@=`F|ZI{r#vS^V${6JtCOy%qD;Fd2E- zG*T0<0?#FL>xhA=Q#pV7wmgO@aUywxQ<|fVPVUb}#g)d7fY;i&Szu?WiH~q$74{5^ z?13WXx(BSX3ZqxV>HwFgGT@QI>suf4ht1HvxOi=XJ64`0kX11_`(5Zz2$KXn_nLdF zR6@vzJ((kRLijs@I-v)99YnX^y(8ucArafc()v)Ox#2l8D-v!wnpCh;j4CSv2FE*1 z=lcW9UtZ@DoQS7Uhkkl``b`c!`!0=ofFO!apH3kvx)ap`#8)kH(%ZA`i8C`SOY|J3 zZ2af35mz8mO@*n(PrZ8Hdr3gvPYpg@Zzdrj0U`qdn$fwC`6r~dp9rcE3x*G=eXAIa zfe?;rcj*`m4|ZN2pL>vr~l5PA*-I{|G|UKIKjb}eey5@WsUmZzi{ z9q3Y1mR6cq%j5@D1;t`@ph1Y*$-b+bQvj?j_35+&i9$MQy^@T#z;5 zGt2ldE1n;^<&JRMZtn$(sJS^8Jkb$`Ya%yKPiwT*T!K5guC83SbGPhrf@-@=kAP8y zE=@Ej@a9|FMJ5v-?qd!d9Gc}-n7+&4!m&Qv9nEowf*buHJfuoBEsN&og>g!pFgrSfcvMj7pV}y zebUxN$_{X^2s>Y73*3S-Am?kIvG*$Q^Hc@wJ#*K2mI3xY;$j&kaQi>$^9j)~2W&1_ z+iXD1$5`w@?`yuI(#00o1MzWe!Tvm~1u=Q+I_2Yvya>l4?*ZG1d zMg`BsAmw0;!%*9#$U0i z=L~G}u4WmtUy0bl4=&XP~LeM}j>mfwv;RcpL)K=S7*_9;#%T3s}K-e(`T zo$MB6y{c5J`MVKEu{vaUG+FmMr&!N^&UR;1Z+AaKkZ zz;SHunRgY?7UuGCzT@9}kme@s_iyDF6ML z=f}%_s9&GkxFdDSZZWg`y>+K9)D}&wG4XzJ=7b>XUe|9R*GKuI9K%DDjKw?qt?7!8l^tIuNig)vV?0s{cPNIq3k+F-*q9De^PH+JM}?-h(xY=A25t!#osRma=~81+$S9499H+1@x2&6;Enh?uv!P~+q7RghS5f|h&+I}uBG9pO|(F~#<;7sI$RBrr>Re2Bz^gqVYa(qOOcJ$2#A z*$D|POT#rZT~6}jWXWWMWLGc2ivPLd^2EQ^gzv$f)o~Y83wDS{OH-HD_w*(*gG z9GJ;pn3L30?~_S7Zvp3hB0oY;i`6=JwMgz+_pa6p>3o*q`bk0;IYI=8epZx%zJAH7Gtxm@vh=GvJ^f4*9))bedqPi z&eUG6N8M0Q<3mRMIH6-`1grB|ev{*)r;na972o_PU50ZLRZSf3^ZSWlI_bP?pi3h( zf_wEG9mgXg9n`S=9pd+Ac|-3<-pBb#yv3#0_(KoDtc(s^2^4?N&!R*JtZvCc2>>3k&*Vc01ZMEqassHJi!*OqyxaefCc@%M8u?TAp5L z(LZq${l6qr*C$RaW8J9tPkH_Yx76{foS1oE(cKGqULG@juA|r%MMXsRySZW=srOS& z-~LOj1Yq$p7mrIwdcp0c7rdK7r(d2s#NaC$@rR1IUhnEy~0 z=W3_#btbYBJ)xTp-ha5Eo9<^3@HeBNFFeNk#7~sS#_YceEDCNxE}>mCn&x<)pMdB3 z@n)`KrXtVZg_;(x`^rW)vV8vMy7S-(#i#LALelH~xBh0QWKcBPrPoe%qC^u0`%GBF2BgCedoGU`-^J3rpgEX!IApp6nH z-%plI&Ydp_5hv$S;ZoU~>G-2-|Dl;}Qym4tt^SijWyC|L?Uk7g84Pr8-rIe}Pe0ae ziA!0*V+!De(Go5?E*&YJkH5`9SyLV?&uEB5P+Lg{Q$e0_| z`j^c^#u3ElsNa=O>-FACSuj=_#(kp=@Z>U*N{u(C@#DE5N7{xqLb zh^Pg2YRi=sye#&9vFvITE3PQ+E52J0`1h0ZlKPcvny{IW>Ryr4?V*^(VZ@6+vyJ@E z>Gsbr8+PC%R;>vwH>c7mk00q5O284t0q1)?ob6EYFV26tUCW55jHEeiKR|e`zBd~8 z=JQ;b?`QUKJriS(WFINnIbG20pOWD=KUS2v4o>>9!*=t>;fGCKoyW%`g?@=F5-;kl z>pSY{zQ6fC{auuVsp=0h{zFv(srVCLgJcDEj^?ROQ?0Q$=|_x1b{p2^PkL#v1kx*V zblA>;5*?Qyun#IR4ZfC)zaQVL)yhfT1T{s*h{}lICwrxxu5Tbio9RE;I%x#z%# zFKCkW-0tR?>Q-s1YcauHHyafic-GGl_3oe2E|wgsJ@6SH{0zujY&iE!G|xqj#Snfe z+-;-ty=Ix+zr332{r(vy4pjb%K?3`D^nY`Ug5e4KIpj9_dxj_Rfwu!F@a_e${mr!d z34$Y?fHpV;`Cv4bultie_v^OrSZF;J4M?$ANiXU6R z?`bc%pzvI{a|ipq*U~%uSwPM>zO%L3rdKoF7sc4-tk*MBV7t9SXVWFj4&jDq{mq2+ zA)%(QfUONE!;!1NaZ`Z%*aEKRD_?08i~^QHEXg#P`s-I1ox8j|m3>nUU+nf2%2LhRp zmVm!@a%$hOg7zLt7+6ePMq4inj?q5E^-|n%q#L4g{WWTnLda53-O&+V~a_f9V zznA!M<_Up7w;!m(<&AwX$ZB`ZAd>efk6JMpN7K*D>hyLlI~5X_5j@u(*o>s=o&#P{ z;@kxX`9KmbP`~APft2RnbB`HxU*Z2e^nXC`zkv$QY4CsF?}^M@k-KNR4FY^oK72=I zdZ+y?wgK|#%Jq7`H-0zv`g!X#IwZ2|^>;chYN5I;ccv>}{!xpm7PA2)6OM5@f7PidoI{msnvUVb!HqSb^?Yw66d!gXi%xeTX!N%&gul4fTR5b`xng>}PP3f)1 zqjnZF9uYVnSS=eQt#|iYali-X?)BW?7dGQ=8@{Q&Ok6RMf==;yh;;pS~?s`2jS}*t%b=s1U z>uHC%dj2+yXw6Z4ThrYpDR%LqkEU9`Y#b{@OUCwlB`>rNWey(Li6iWum#=Z7?EpLu z@#>GRKOkIptnXS_bnHj+oqsWGu3l~spL7JlQ@-Nim9^%`5qzQljt#;R*_MSo84MO3 z#YIXnXKN}zK5Xxm$7=oSIo{rLKebuaVD8i{TU8wg&b7r}_g}SaTh=GW9|_D_DB9tF z!tZS7%3I|^emKWht?3|bzj~%dm0fYn%WEaNo9^cWzNtvH=!V!@3N`_@S$$*+ zx_R<)MQbG;i%qg!LJZ{%D{bI_LhMymOrkXASI@kbV)rhl65mQY`Mliq{gMH>q$Jo{ zRlHDj;p|b+T4$>E`CVQhBKiEwwd@xV0jKxv`_!eNHak$9No)Cg!GQj`yE%adF~tJTrH6P&Ba@g<9J}ZA@$&@AKhth(fKP z_R6*|jeswTnK+vpnMl~bgyC_>n%I~EU*X~8*7k*RkpPo$bpj=%6)r&<3gt$_4m#AbAUi4ufdm4Z^4&9e7g*W zwGqJgfF_cP=cGYrr9ZVC=+9cxvdp~#LIEdGS`z47$9BLh$Cu@jD+#dIFOMxf2>Si3 zsGQgbJShikC_R9Es2$~anBCtXi*oE%zu|H0Ne36rxa2cHQAV8Chf*n8kQL~ek%LzE z*Y0-QGI>S-y!HRk0UsH8k8oh?QfxOmB0#pk{c(5Rp>kr-pB=)E0?yDa`LI3W3}ADG z#XoSd)|`6vK)@obqVEnhj~F(3g8ayk8r{?3oYyYb+N*+er8va2{}~N z;y!t>>#X_>oj?F9uIM!3ufNj|MeBufxmgb@mF2BfF5&51_jy_+Wl-tX4hA6L8YTh7 zqrX;c>}(;GyF=a6f|Qw-JI4diVDhgb%a)~qMd0gL{N>rAZcFcGw|qRUSIW_1`)Y!l zzi$z?pjEEKl(s_rBFsDLg=bVNl)5DRQzl{TBb8RzWS3RGPG%%m&x|@te~QK0)#dg{ z;PaP2-^#%64pV_;l#D~U_U}Ol>cGC)5&qrnvP*Si)$@+A#z7keH7M#fnF8V5_P_@% z=3G?CDZyk~OLuu3QgKWUzyBiSZvzH^#Tn{01%VwNrI^K2l_`xh1oH__RD%WBVo}bz zpPY^}^_n;a#%v6+c6xjOLs~yk!Bns_+kNlQa;plnuGwV3qJJZ8vuXO2s@{TiYPd2B z;dWWMGu!31WLA~*q+FU*y8WS0tN55P9_h3a=QN2}#oR!$Q6hiUbu7zNnTsLgGQRZs zsR=7D+@YCZLN2vPH%(f_);F11zVDjtTD&&g$3Bm@{KEy~mcF{ga_FTr5Lj+F!e?ul zXF^x1W)V}}LOYH{AF!`-uTK%*mvq{lMU@?8woK#_rg)Kl%!B*3w6n2jheU&fVZ>uE z6F4%HLwW42nQLlp(x$mXh+sB8w@98nLzsIkJNvaUg&rYvbOF zT|s_WaC3Ca%Fi8S!3oT9&D`Idj3s+<(1`Xp>DV>Q5d$3<=7hZcFq zqgQuIReE4y;&U4pK)Us^?tZJlr2k&ctxB&MebtYmC4!Mg*(B4$<%X+rCu?bCzrTA= zCI6knShTj~BDz-Epj(e&9L}SLM@Li*{wldl({%&ISPJ(58WTcaA8asS>ag#} zw#N~_Z;5C%^3L1O;&^O0_@cpv?BUCst}j;Q?6xT$Ti1PI=G}GIc%1syoYif3CkNti z0p_r}I5+lCp1$bf{F|B{Jn@oJNk;j2y1~C(OTJ|F#1+i(bBe2HX>{ccts7B9cgHR& zrhqj~2%(p;5);bls@z4KRX|aM^lMZnhGtPRMsSBw2z|@Wtje^=trf>)J`yZ898xM) z%en6Vn1 zv2>xhE3tG7G_8og-fLo1pAcA2dtxl;@6(4H98*j6* zfQY>FwyDFRVqoW7T<1lj`V!x`d#(`a7E?MUxG9gUc~(5Qo^L6!BzNdA$6B4a?lTs> zFW+cGLERYtFKNKMO&E$B2>0-h9^ACqI0~$AMKZ95fA)x zu9a+Fku&do7-#vtR4oP&erKe5Exf_Mh;GSOh)h4`Y8vv6&aidLy>@pd`5PvCX=!EB z@Xk1;AIYdEc73i5CqBUFu_7~KJMEiSaaiP{-N0{w2w4(Evc8enLW2zkyPHabWu4OB zi#h`iHVw4q<*Z+n{m_+`8XIxOHfewtpFW4i5nPm5SK6XuzYdy^BZe-C{G#}1gGWH4 zA%)AJ>7n{$Nlx8Gk%e>WkkNeFY%A1mvhuyTu6`ky!1N7;Bhx$a2+9CuDG?c8Re(`| zH8~Nx4EfrRx*zYhl=fTBW+9i5KgKBt#Io>2(_|$w0~3k#<^n(Lg>rid$?Ls97j>-2 zvDxuvd|dmL?8_8k*gX@;XS*TUuPyt03tL_My{reUeo zk*YXS45}X&+bUKRSl?t~ej+PnH@0l6y>s-g;`$iuyEJquvAnSV_XrC@;s{EQD6Sh5 z8iN2k12QjKRNgH;Ha)uJbRQhPLK_M40c*O3#k{vkJyuL(J}@lKpEjpkZuF4r zZAaAICF!;fcZf067SY{*^sZPZ^NA}TIQpbgm6_C!{miEdRV% z+%sJx;-x8VE2H9;8zNatVDjf8Ed!djxU=t4!@oiHIiK`pJ-44QTkhQY1XwG860<<{ z^(p-9$(p^*B{X-n-hwQpM4#Xa1IjV4NIN%KMw}}XBhb&|ru>M1O>F6QgLPZQQ0%oE z_0;~-z&ST#02#kcRcQsztWBj!ot@-tKQKSnXuCIw{u`G0d=anz%HmXr^$PGeQesqwt|FvmE$@6n6%g7*k? zRJHp&)bBd>Yzsj1LJW(54tsGYpVdpX#5Lw1x922odIHtaY+Wa~JQAIKaW=dw8y!g4 zqbTg^=vU1waTY=FOfc2%_uw(3U-L4Eu{u($6Imto-Hzj^z51}4P#4|0s>2NeVTQbD z3_*FnP(@N%TjipMRO{7;qYnoOK((gq714J%B_oKDP8NFzBfe8HTpvRZI zQ3!d&!?Oo`_^MnR5d=$)maE3O^dKLJk`vp<5-5s%kcI$sc&52fq^c`Pel>w;3OE?e zb&|9mIP22zQr<@Z98>g(5Cg?@j;h!p2@pt!4y=QupxP%P(CbbMP!b{^`n@zc`NM1k z1$=N7gI0={!3RxzENsLp5{G~yN8)%}Y9`6v2&`Ji7k|nIsz4{uu5{v?J|^fiIcnIX z3&B#KCuaS5c`XRU8=r{~L`sT*)N12+4{}ORnv>*30K(yl;RE^5CxZLpO>YtVL~Nuf zKHB=y=4F!s@^alwEpG*Zq~*cvZ=AT0;7e)^alHO;y8{jar&^cOpu4u0K(~*h%ujyK z-@5G+v6&`kh{PaV?ppXlPdh}*2LQ^VhU&R^qF=tfT1yU7veZ|}80 zHjjk|ao+L#cW6(?>aX9Z&4TqZWVX)V2=gCz-z}G77Xa0M4&L!n_1d3&BciY#?0jf+ z87J>GfSyAh@ZBC*(gCi7!`LNIiR=DAaGOFrgU8m}99$O>$m^e=Z(qRHTNB$gHj+zY zm2kNP;51G=-DLAb&#K8nU`t2IBm8GL_h%c|l}Ypo0a^$LkG+yyO`!3(*0p}gpdZ#; z4TiHH>`$s6?>~Bt^O!$y3840+oN@5Oee!iL>9;a?pzL;*#7yEWKazVs5qkin0lUvTg{bZSJ9rlB7j537=THr1HOXCR3o*YosV_6(Xn1Q@s$&#jlC2{~0rd3euC=coz& z0*X8lWPAarcXQZ;Nsi2#%pE}Z@G#aaJ+BvF#T9Yt%n$g4kv;0fd2|Vs_vN8?9?$T$ zH}$n z_P3JjgKOL%jX=mCWF|ETIJ?+2j55XaxR--q=&*A<$J%rIup3w3kecxfo+TfImi5;; zXIA16-ewDN(;~=b1W9tkDL;`uFSsTT>$MJD;Vpe} z5BpIl=a!hJ7v{va6mns?rykx6sx=H#Wuw|%aQKZ7t0>?r|IkEarS>3#z45P!@_kp8u`xlFz%ev{m<}AQUnPf+CTC_E5ZjFDHeIWFqxbBDL5VWE8G@?^5&9FBJ3Dg3z^h|C{s zF+4`&HR-{6h5YDiPn_3ox(+kY{;5SadnG!+5AzD4bkuZ&T&^9!i2QB5w*iz~ina{ha-6;Z(jU|y`;Wip0_tKt?gzRN+ zr@hor9SV>Jr=iGW_g>}S@N1uK!s>;p>{b)gOd}#CboRG;X~b|zK&&i?$L78B*YRo% z?PeS(TqxG2MF){?xW1P_8*vLRA5lB9uP^7>Hgp~u47zf7=O0pDxdy7$*R#s7KuSx3 z$U^HAH)}AwS-nyEQS^^NAYtNias4jgsLvo)=I-a^Zkl5NN-S>s{-v zV3o*?ycFZGvpI?#|=FJY20#)!5hZk4bmW{cAB&XK36xg1@XrPMSt#XDqi9R zlm_H@juQGLt9NzXW9uc*a|2>w!EZ|X;;f*^z%YI9!WRi%Y8c&R+%4p$*%WZ5<=p0* zrJZyMVtk*k)5(aV0-%2L)KtTf9k~`2Om6{f0w6qukii5<0=!D(wn|QTw{I~DbL=kh zsN}R@FT3d$Fu6#dzMIu2NGx{QKod5RjTi?MC<5Ujufw1jr$2UZg%)ULrd~V3>=Pe| zl?&}Y>pf)VV~EGL-$H`a6ew?I6df$=@H0vNDA%n!zrwoa z=aK6=i9Z?0CxFO&So_1}T0Gn1Hep!s?+I%M4Kl$ro|xJ?$&6|RS~U;|kJ^4??TGlQ zPyfU(a&JfE?+JunhoqUOQ3z_GdUw2#%OJ%KxXg!3wM(B}>-^R^GkU8NbK=VmI(Acu z*4S?Xa)4iu`sodjTEM;oW_IwJ)A0UiVj_dAM&7!MKlY%hr@?ga-DG`=J}GclZw=Rn zTR>T@k9F5^EmdsjN!R^nz~3M`Lhs>9?_A<^_s)kvGCc@%*-vOzx0Iz}ku?R&*hzF- z4KiMjNYZhT3Rc+q8*gdQoub5cZiFFNN%DA$Q!}`?uJAf`^$MT{rJMMuJg7r1X^@>N z;b_PXlzYBJ@vZq(<)Mx)8{z;I1!b3yN%vGxPKb#h<{ZQMNe!A@hL{ zltTofCpQo?L^{&G9;4CrZVX$Ka0Qupf*zHF==2OUveu3UtCst_T$=%+@fpAc-tMKP z@PA8An+zOH+1mx(uU&>0IzNmeJgcm>n>_rk@}9eId= zx@{VT-KGF((SW4a0(zbh{#yJtpGL0$JBev?pvpLc@XMKBO4_}YG?U9~J!CLHpmL}Z z`@T@EoC5T88Ruz>E*~^DpuUI)@HNjm2-t>NEDI#$NAsC~#2(MaP-AePF8s0y>d2^IfGXQ^EYcJOQq|DFjVovNK#{Vm;Xs zjE8M6TqimW{EisMu61}lVb6*$0wpSd{7+RD_?@9Abm`inTz`%b3)VF_1S#5+kHkI8 ztl=|h=pr66ELUP%7G4&4ksU+GJf>!kiaoCU*y!T_>}x!^l2(>#HvLe$w6l40ICI3* zKL5)q#ms?#CwN!RF#<$QGDHNb_>Z}?%%sxNS(nFaWwfm=Q_%79VePlrg)CsUd9o6N zw<1?t=N~Jky>yMYb>ce|>W<2o+)2C*IRi0!*-l#B&Gb6S)s&Y(ucI@B={VI!n5!~# z4!_&>NsC0_j4Up!*!VndDHkAkGyUZTW5#uFiAN-$B4JRtj_tJE8}o!7rHH$3fy1`# zBe9Z%KNQ6i+?C5S5^3x5#AG#`etST0GX0R?Xz@7v_ZZL_-d?jnmlU2W68~S!y=7Qi zOZNv_Zz)x@c#8ywQY1JPC`AH=;98tuDNfK9Yk>mAB}iK!6n9Uc#e=&O9E!W!-J$26 z^FIIQx!>={?46levu4ejCBHS>Dy2xTjdb)!+neN{tfl@` zQC-~N?!ts{RQajpK|Z1MX&;gizDO7idEFx$Sjs>@_IPG6p;xkzu{j5X6zhFd-s zkB<{IIB9Y3^Ss}K`r`nE6ZwSBSeV@tixuns2x^IEYpa)+ESgQ>vghQ>2`;4Ox-*XC zp$7x|4nfW`t$Uo=owsmHW6@@FG5Rop0C0d2nxT>vWl_m9zQm`pbARTwhqm9@gf5!k zudwH0ShFb;A_wj+lqSJgtucCXhGXMU+j)rJCbP*ny=ymaa1Zu~Ivq7?%@>j-v4&Ro z{tqhvqqKMt4iF8Z|FWVWw1JA>kWrUNLCEn(vav40wFVdL(E7hYzyHd115(EYI^+lT zv*wq4bWN{#7i4?3kB~DH+c;6=W_Yg;p#koNh)a{_04cEqq;L0{xAknb~Zn!e)GT`fK zB_A?n>$4ed{+xucFGCz{msX~ZK;y4V48dLzuG;Jajx|rwuFwX7yBE6&?{6QhLJSJd z%{HiW29gIyN01z)C8?}VkA!wn(I*;YFuL z|7oAf&%A1*jWg$3sY_pE9y_IwR(~yQ8v^4+6L@c;!a<@pmc?GO`%xnz`b#@{r+y*n zmzZ2~dhoF#TyX-Z*=??0wmaWHA zGYil94`9v=u%=i>9rE!TAO`q{QrD+cutb>!#h&=OJAY&#{LH?LKdL$wlTq_ZH;O@u z(t8(G(|74zWwv6*?r$X7Z{z?FSc?V+bXE}s|CT#BN}@^3{UR%Q-f7&7E%o0XAf9() zqL;{QL*5`&X?Hii7!(M{6ldd%3&?TT&Rn5l(SO1^^6pRET>MfQaMYXZR61`b%YSoX z{>DUu<9e)r>&U?rQ&WK}u8kYpoBJ~RPp z7Gmm4POR(`sx{3xHY_PQhKXk9pLQCR?HIj?2Bvg>V( zWu8$Y1!8;cy>mT03yW!ESAbpA8xBkdNMWdz-Mfj8I@!%ZXn+$G*pc`jvyj5eCBTLQ z0&T5xkf}FuMiW4XGmQXc!DDb6p`4#_kzJi z7_gLG-3P~IODAz--Dl=rAB3qdMRy^yO&;q+w|$QZPZ;rCd0!C?V~tN`kP7H;zv^qn z3Bs%yZz9dB4MtE=Tf@kf+Ap3``?y(A8-_yQChrkVW?bYm-ygJPf6NM5DgMrbeF1_H zVX~WnTwior^~L;9F}|8*%>jHVJzWlbgWXDiC@AeZ#to{P5d3t$2RT-6{TywC6SrA_ zgrN>ONc$qB-mg0hP^RAqwkC%gm|RL#6UQx$UMH zsSPJ0i|@)KHJyQQ%?zac9IAd>xPd|=BZ`4d1Ppu$-lH^d0;CsArJ7`wmf|hpE#?t! z5vB8aU_W1JvxiN%Spgeu5RMC*}V4Cxa;haqu{0NNO17PYxdy1&n8(ki9`*u4Uk62%I9#w zBOxv>TXD+H1nZ3C8^Ox_w3)rRWUae4gW^*Jr{PO=;Lh3rotzZ);NMS1(z-`sJO+~r zBl>V6>9U0cM8t9VVk(Mv`~jW7Uen=d*D6AF&-}NWu%mKv;r}#tqnnAJ;v(4OIJEi z$wuS)!Z6Z9D%WeJu-{HEFTKAMs}IjS_yb{Gyb4}^#Xg{kWv6@Fpb9eB0L4AXB;pDQ zG%#xiam4*FXXrm0V}1~sFx7@+;~GdP+CwEQRfJS4GG*_84#$Q9eF?`QOmKtT zj3iUY3gk!cuvoN{tG>TG0htloe7>FO^F7wCXg zS#TOyH9KbI^OqjdnA;z=cD8&6SJZCVa*4cfi8zr->_ejKts&xTVoVI`vN z&c5-w^CLK*wJ|jtjfIX04}^B5;z&yWh-Wr)W&gv(AldrUFAPsTGSym3>^q0upki^# z;VYF1`-Py~jj){+h)nx@$!_G_^Tc8ixGZcd8$J;o*IPjs+x(Q3B{!4*{ZshIw_GR^ln$od8B5IW~*uyah?-t0G zwLGxFRBt-EP;W!s(QhtD1phC z-XrK~ei>e|Rt!V zc{?h-2g%fWiLoi|-UAe9ST0&70q45$7WB{;T*BmI2igabm%Vez!Ii{ZpFqnzqU(#R zN4D8!jjYVXfQ7gzT!`O(5@ARw-xuC}X(%w991Z3i5zPEjqz0Nv)oesYcOV%d7Gjj& zz$T|Vpe7R*O=lKDPejDihX?Ouw+S$;%MNrCXsj*v7xIJfn6Al_6m1<-xK{DPM_9XS zHF@g@9bC&5P6T}Os6mbz0mTUzP)J~~2J*y^g~G`ga4<@WQlZ*-5yCaCRzUG*J{c^; zc|nrm8U!Upzlo~Y0JYLSL&H=ttR(BPYZFfONN`&E667BF<&283iy3K*m+pedZL@|L zyQJZE8qe@rcE171$nH1;;oS#iF*~~Swzx)=f5=89P&bExNl~K{v*zv*)vI|ig&8Ws z(0gM@U*{oLViir-h@f&vFESzO4qm~ZZi4!?3tf6SFfbaxG)Og;4=wzmhsJs)gyi97 zQ}2}ciK@Of;)ecqi*T9$Q$GXXl4ci$5|0;6SLJTt4;OJn#I89d8YZC5<^#Z&>?4_~ zDDd1$Qirt3t8lnO- z+9=YON^s^RH{WeIlVf>ra|!Z2WahGhYMmi7bPqHY_Cqbkdj5$G4B6HS&kxy_M&|xq z4Psa5&$T}#&S=^QBfydYw2G`)7Ak*3he6R%ljZz0ovb zgE2z^ZbY-0#!sQOANKke%uMb>Hr9Lx9a{T4*r%K39rdB#*l8<&r|rbqyc3;cF7vQZ-Z zwo5OyYlK=OARYWlKp)PV#T;~3Eeo3*h+Uw+orqr7t)bQuA~O}o;PBgkfX9^2aq6K1 z*wW9r4mKL_v?_{Eh?u(5E|>aJXGM`6;VJ$(eM(D!>}5kC;3$SNJ5>int1P_ zBGbT#&)SlhXR#i4# zGm=S~P@vxb@*jafo^;$STMbs(sNA+nshznl$g>mDj~p=0vt=cFu3eUNB|F9Q4jT2S zQS2j_l!oh1UzqcPFLI<0jV;f9?>Jq~uL8DH+TU(|?-N{=QF*T(3jq`yfJ*8{wge_Z z@7B@(vfWioaIf1*$cVb#g6zv`GG6I4I-&nAhk#~({;|AwzhkVaMObkz!IYn^K6kIP zCg7u0FSE#ynXx6dx&ONPsPrhg+1&MH_;OV4a zNQd=|IN@JCs|MXxGNn#5E9LsZWJ{IdN zXZvZI?LAu&xLwo)x;?5Exze&SFmLVQW%jz{m$BiP{u9#l&8X!+vN8EXZ#pF4Jz}Ss zpDTi6DregLB0_Iyn~JDqQ35%83U>I`1Ms+VLxU%-P7|sML;of+q`xyy+|joyQqBr! z_DKk_8??TXp!JP*6-qS?hT>hvMSi6=dXV{EdP|p)Pcu6~W!W~1RVK&c++3TK%u!KO zq0*yc1gSGu_I(N>?$WCUW4V_M5A;a2btLb6_uNjya}<7-OR=Y0AE>)4I}za$@sm_r ziZS`KF94AA#AJ<%f(9rmH24ZkkA$FvW5{~e)iz{BDAJ%=C0#Eo6yEh(6XCPyY1Svb zOe5Jl7Y)0qj@Q3vm@>RDsY@Ww{G&E-rWPLRARzxm&P06;d*k zYN*84gdF78UfR?U)_~vE#+%o1{_+97{W#+l{m(RS4f!He%hTBzk!lJxb* zu*09_VZ{`2kt)6a(GbO)698n+LemDnEc9!P;-i)Dkw+<5WM{bZpWr<{D1| zwhCk(VvzNbb^3FgYsOn5uX|*i=nQCH^zFG)Rkzho0QkbuV|EPL^DwAME7K-?&A7l$ zEr+@9eobUajUJLR$}nT3P~|HKYPC<`!)XYObPXH*Qqc5s z59S?^Nixc2n}2ctu1{LJ&n{>)uF#k_#RdSik7eL>|1Q;-5>+X!1ru^%M%GmsS7bj>%fiKo}6ZNV~S@xe`m z@xl^ohI9E^c`PF3DkWYz)VH5e-sB2-X{}juHPN8v9eUbjC^O->hpSz> z?dBm|_+!VVmk}3Py7X?XS?{xD=DiYCKSwzwseawNzmPk@ile%!Hm=zkLGgVlFclyF zXL#C*waAY=YC|l}ZLg1nuvLDS@2xGH)eT{46&Td@RON1v-VST1UO+8D8bYe}8KI?} zT1C3BbeTR3iO`57Pggp-LRc6dC<6za!ABjYPjSXda6sBuY$){wsshAk&7R$paMlJ6 zvR_EF8?@FU{43g>_HzI2dk=Gr)aM$gK^6t`V|w4hDwW{fTEH*0(eXC8 zm3455HzW2lE($|Fiihq zd+bogLdpQur=k)k_0h3P)%FZ`7F}HYL`{mO^(Y6NYySw7&ukj{MUD$51Eid6(Z35ff`Pud20lDFF?hECC4Fa18N5gezRT`C?d3lr| z-5VV-U{w_lUgW2K!en>dYhH#-#Pki3UUu*nO$bq9410FVwOtx1|JH?PHKa&qw++w- zCL6CyK*Y}9`$4-(1MPNhYD{^`R@fJ0cSn>IU~U{`g}l78bGXT@_UwslYr1ZX8??!; zBRa&vx3|)6ETUn~K2HB4@}waTU&_tP8$>oL`98Cj%^dB`C7i=@e*mjS`5Sl^GLjM` z8#0mph~J?R&j2&!R3d%ly2JdRtc)SY=F2kmp5B080S>?UA!KT{c4_RqK>3_;Th?aV z_A=z|+!Lm_m_T2}f^fpH9lV#wHXxTMY0OYMkgZ;jsj){V)D9o$?OVJEc|8rqMHcjh z}cnE29W_Pjl0rSfGgVJBQu5`e;dPb zsV?0DR^3u>3y?uh-_fio;>M$KZw^S={T}o)MMyQHS*=cE64i8t=qZM=)uFn=k=~@9UVD=|_ zp^iMk(6;7884TZIS7Upk#j6UITQEYk8Iev5j$&2M0e8+Z zpA$D*i=;=jxox9LVNap`rgSxEtl-UKQyF<6WbH!y3)tTc94yx*Fsz!+|OnP}3K^1N-i&DQY=z$rb|S>FavJnwMUT#w+S@E0kx7_lQEp1Ug(WuMAxB6SJ4?ylW~k(oZ0QhbEH5@ zD4Z}-I}gXFck*(?TurAFIz*;b*J9oUx41qJx1(d<$1(bYg*OjCMPP)h!h(1WMQ_B| zpO+K%#p2J8ujW6)F?PJ7Q3JqoZG{L(kXc-q}v!pNItVVCKcf4yJU~`EW)j({a-pY_bhfSUAZ{H#xTY=ad zAwlhMg?&lqyBC`Si162w8WFaLQF9qgD9L{-zYWCx)Np7SY6-Gv>*%7mW9>&ev;rpS zQ?*D?CtOXejqdNJ>O;O{&SqpNOB=CIB`uOU3hw#1ODaU&O;cD>B0{iv3MrrByb z-P)bMu~}KkL**8x$S@`6IVv7<8-QS6wDkuEC@%E`>hduY29&7Q1*z8pk=2$eKy6?$ zti;w>K+B;$9OQe!4rwu+F(;tEOzpm4YQmRq%Rit`vfFXUk)myodOnLj@6 zb|QtNJHMc2AU1~lVc;|l469N9luDRh%4gv@nIIFWdo%anvA==+B7^*btLQ6BgN>wd zy$@|HKDzZ*&fQ96MMNMqAk&b>l8CIp)^`erBj(BBYuo-7^J|9O;M-z$F%$g5gr20h z^s4~R$-0m|E!bPa1NJMve)qMNCA5gO4z%9lRPw3&d70pL#90BrS}sB;AiKX*Tob7Ze*fvanSJMM!Lz<}DimLH2Z%n*B^0>W44jxVjg-ObS!gpyPB zhR?$E)yfk5$$;1|bU3)91F2JQ@eJOg0qoPZs+8rKhq;OhVZ)q(rz-wMoJLGk1dy*2 zInZKz1B3u&kbCDv?M-$PI6(K&QTHffQy|Oi6U2Z|*M@?!H@5u!Yc4WKb`BN1SL->j zuPpc;jZ!C7gDXDw8v{GG$qL6oKM+(Wo~%)jmBBA1VVnug>u&<}v#@{SslcvJe#kMj zdv8K?cUOK_6adeDI0PI+yJz0sdLETIu9*HXrDXrJTo9!GM`O3#xe;qgGa-r%NtE-`eWhdHK?>Le~gE z&adA^=*?g+A0nhbGK69gxjXi$Fwd+)l76idBtCKKXDv zk5$8L5aHR6>(j)!0mb>zgUE1TrC)|czanpz6KMW)r!(pq5I?P0FcSFN99rLlbkH-Y zrhd{Srhuqg5$=FXmnb;~tP24^WIZ?$NQ$rZWoefA?w!t^@W$1p(^BZl`^;<85Y*yh zvspa(VPJwJ4acxRR~3)?@WabmwJ)4#=F3>f)13_CE_%5F&-_{XK&zC0du?86BFC-; zwVTtxB&Ng^QJU(3eOGR}0D(n;Wr(_zLvOGRw+qkmsR^iO8ds=7i-tLjR_vD|4Wp&M zEX7n?-nA;U62XQSoiuKt6!J}7D=zw<6Bbp?-0#oq)#5gZS`S%pg~RcNQMJ;V#|Vq0 z50GKgwsbBD8L(EhFYx=Un(x~?H*d-&96$3pl|4sqy?fp<@Os#U2C5jhv@_qc(zmU& z2237Ej7%m0kQ^Z3|9^hsALG*|_{YH8|BrwD|D@5~ODG_`0VI_!<8iO0D-;V!&WpL0 zHX|Nx&g*Bv!KZFjW$V?8@wzcd&TACI*0TYk0M6)_shsf{6=y(S~uLyvV*qd2J^2Gk^{hs}3>70bNLI}I8c0ii} zWc($T;eu_`JfBJ%T!oKl7=WbupgOgaB>~opugR@I)q$-X0`-jwo;--nio*L(#tvT3z z1OTr8@b-`IxFA5tP`03&g!sS^z+_<-9P^e z74b0-U1$8~Ro9h|7i0s+anob`TwRDmUf}z?1pEYS!AqA`c@P)ad-G5%zGMMlci06{ zKyBh9Jkx+gF3$xkaK1)g@eD9A(cb%F-&y4_$PZqv##A|+8CIH`_Ei&kTV6#$jG;Ob7-|gLADF#x^LgM@TCiu^y2#q<9 zOPSH|o&zLlt>V}6TJ6Qcl*)*{U+sukr~uk{+|#StjwEddNdIuYGK44^G~Jv}6qzpq z-r>pp<)u(#VHECfJCoVFkoWJy4l6~RKxt<}-d^lqew+*KzV^~;qEbvu{`XOU?)$F1 z`OV4e1&4TU;5mKk!z%b-_&F}$gLcH|g2PvQDiG>%5qk%tFFn^&hs&B~u}Na=k-$tQ zz61cT5{aFT$0gUq4F-T|;Ki%DZVa6-Q+F;<$g}*PjWe*( zdk}|%#%=E(c_m>M0CAd~Z3k{3fIkIuw`HzQ%O?P(8Xp?GD-a*|VtMgJzaj5z^uCGB z%L8lwfdR$;>DeSCAprj-e-dcZMQe|*bNfZn)=pU5gl~N5{WbgTE2}(acp*vnJ%)bn z))t4a>oEK8@9%IU(%*F6gF5gf%Bu{HBy-+>M|*z!hDNI5VmRnG8J~VXl>c5>?eo7+ zh%Z5yRvG+0+P%2wG4sYoQ9RR);!qdj|F?N0^N2J$;NSPZkmc`Ph%ZPCaJq+gx-Y^1 z&Jgd={z3{8NO|*Tb0q?O4k?L}(cZM+)W8@98>sYc{pjfUigW z&wB%fVZ`c-vi^F!8WHWpk-YnF2$<@HCQ)r@7jbS&2VX}r$0p?6RN|->A0>K|)J(ie zJqc;;TvWG~SJt-DaXdOfK)Vp$*o+%W9GOL&9|2M5?zoIAUP%~ELf4un3+dxX+>_qf zbxbUKWemAhugE)JQ@XVN_(Cv>?E#DpRXXmEVxsL##`X-Px0(3B_LX9mjt*IU02iPP z12zG>Cx^0t@ZOQC2{rn$03D7mQS>)G0y_2ed_hPB=m%JPG^}CB(1Aef`=dzChsddQa&r$Br$l-jK z9NdDWuj?$@j-D2RP~3e43@%BK3Jivp`DUJ+KeFWlbvXboGFKK&XHB`B(qD^Y`AC~P zuro06L~#~YxvG=7{MbMhfXCpgT7O&k%UXK$EOV22^CzU3s15z9Kx-8jmL0M7v zrzH`9PBUGFaNY%`g)a^RnjCqSP@W~HzesBC{eodNba_3f%biMG;PNaYkFN@tn=g>- zaUo{vTJ9G%D4R$h_P7G5xuRXvPrZ0q+tadPt($lj{Q`OmAo{atkq8`ZMegg2_<>d+ zmG*fu?QH7$0`T~$(X-7RtpY8$BUK=-W;pE*o&*%>RLEnl0h+)F>zd!4MPvDifVxgc z-2RzJ%xtZt4v)^|>=(WET|RkbXCakR{Ah)xI$5{nf^w~gT!hnxx+r1o6>lHU#F@$K zJX<6hNnAVJisT(KNmpPC^)Te^0VFCyZWdCMVm$-d9_i}s!^4bv-U`m1mk1kodndT2 zNGz0ZssPlE&0)`-xIhU)Wxgd>|Cx2$nMLAo(#gRpFK|4>pB|u4GdMig{xaJYqm`~- z!h`({&J_meP6vRKI|tcOY1uRv1Om7$@!WxsM6jFK4xk0TGH>ybk z*u<=LcWA~ZCy|-kAW7kgBH^;8S`*C&(acTp)C@2JENH*vAEZ1!VLd zPp;YHv!eKQ8O(BQ(s)-Yf1#%E14-aiXcCL~Wv*7MQSE6vBGV#kjwNs0IS(O= zmtFo$N_iS{QWb32)`;WA?2zKzi}$pkb1d;-?zP@$ln^ZhmUj;r>3v?sZ=^Sh zQnEoQ(VB)IBl=M{)NX#!YZDyQ28)&~=7^d~Ff^ETJjY;Fe~HcA zfX821Sl_L|U8vSxBog>i?-iy`$y*6E71Qj_*8XcIgriQJR&|kXc0CO9Kq4A78w+!!Yj^ta;HsiyaIE-aNR8>oB~Eyt{Hwh5PT^$J7riQ_Kn`ti5Qebi^@ zi-IlC_)BbrYsB&gH|$+pu?~OJvm}BW`_A)=d5F&4ny-GVa#Ib~RqyX?@ipNhcle%b zbO(aj8kAMRZRyKR;wjM*CSPxCdQ^TnkSbEn=BpHUfvP)t^jN@``m)kqYVL~iq@ZU3 zsM@Ufq#htFfN1u5Z@9HMKT3Y>CZE&{eSj+leYt9GP#nojQ6!#4S(Mkm;gmUi}&-EfVJa_k@E^k?O^ zk6^^D@|ZDTGXp?R7D>GwU07dk8s7Y4eE_G-@{5t-Zi2u?GXIdW%2$UjQfR;)Kli>F z-2S3WO&&%MryPY0MK=CKB-_ZGPibt+b1RRwxn=Bt?q9H(0MJZipX%7lqJ{8{xBy(RSu>Npcb70Ow7NG zpno_4i?^IMD#lxW*JmKBGX8$rccc}XkG(`!iPi2VM;oWu9A!Su058(M6HwiYF6g04 zX?58JabMn#qyp3pKu+KcSGgQ{Q852sdiFr~+0*8-X}4$RIwno%bX&O-$wXkJWqroD z$nb@VX~viQs%A+*pp{6J?ysikJt;ixar2G!_>4D7(}h;xly^&6wLP_C9RRV5sW~jk zrG}J9nGgZjFle!sDinl{>(bEP4oi3>LcK-gfswm)b*Se?2u`w<|oHd!duvYDRSh zP`Ow3QMM@~?>%+`Exv3`lF`x#iw}#xq$efj%hfb9yaI1wm+B3JrF!gw6#t_HTt{`Z zc3g3!yTE)n{Q3hhmtPpx9~D$n^sAZ5jHRpyn^_aO?{n&CF8wm`OlNqe-#iv6dcCvh zue9(@=m>otlIjqEA9&?Z7TvoHsH_)mz2!-km15;N?O*hawUf>14Dx=^v!;ZJ@pxe} zh~t^zTV((8(cRut;$4j>Zg`$~W$##d6|nrD0K8y*58o26hVhM{vW6a;h!+D4p~$o5 zcx9{7iHnQ_a=ZWAgOkL&1)=dXz@*a4SWo0TeuUTAH9%C$jok9&&eLaDaRJVn9&!Lc zguZ_N;+$jxZHG^g+U!g3Oz%ddP_I{?-jLw#0}Qn%jDS!2k4{oZQ@E`NM`^_*tFS)zD{c4VMkcsdtv2(clfk7f%!i z5S8!swj8E(=CQ%xid^Iha+r8Z>A-MTAv|8MpFQ*_k^+paNn8%#*id{gDH(Z$yLejwT%kQ#L4X0PC=sLT%kWQ zs`IN|BeEv(+Jidl(H?`k4g)-d4~T%h1ROZue>)>;D$E-hB36nX%+y+L zB~L9c0~38ZHrE#81V1O#)eMu_ z)AT(2yPDkBZ)78!;u^w&);;^?`_g4xQ~`|*0t}=`5(6{2drDP+_T*w#!zL= zWArMiIYMz&$3jOWO_xU0Y(cxb`(s_i%1kYivSkw5YbGrM0i3FC@^4f#jnbRGIVzpkK5%Hc%CFg^#Icp7jr- zct349Y!`>0U&Dm9YGD1+SNYE)Mn*_B6gtF3g*_L%ZCq*&p8wdHeuhkN-7a@N-rLck4jszG30o zLvEmbZ+T%ZwmR4$eGU@!4zC~q*_V0zrvkBw`gZ7)N4REwTGAca91iKN}!^i-0| zu}~}z#en&9*oNM#6AmG3Z-8iiceDqAuebYYUp^$Uy%d`P4xmw3D(Pe_ z=BUcxTZA&;2ZiG34{ME@e_^nF_al{qsV%k3ZUvIK5YPgunF1f1K(m^cVitaVS4{hS z^_LZf6{Myl1zbi@JNtqn>nt=7-LqX7ngf>Vi%4QFnFO?L_C!`DgQE%qmJ>S3gjWHK zPk!R3Feya;;?5<&!T27oA)S*|CfBzc!R6F)M07zA7ehu0D!I^3-lj=?m(9`2;e07=dFqT~ zyp#Bl;SRNDPIUOm(kl|mA8fVXLI;mte|H9;Rkf{U`ZH<3^vU14SH%^8G?KT;V-Y_` z4~9T{X9muG``TR7O_FZ<>$z7$0DEl~_E9K{+?)A*&84!30x z=3p~7UV0)Rrl8$~^ad{`R}m}>1eXuea058tu^WH+aK&DM`bmv84R5#r-&?Pk{47yDoA$}R>QY+{*qW$w@a|fl zb@U3E8j1E{27P=yr_lS+x?%y`E zTwOE0Bswec891*5ZvfFoG?sA6MOdB1<}5~Ej`1i3yq-TSb`$G&#wCC?fo z5cMmze~!=+=el3Z!^QmX&#f1NuaiVMMWrzqd;7TKt3V?b#cwH7HHLi6z0=!aIXV)_ z|2%zc2SjK{jrIvo)Cy{>V7oWI*(9t2iip zN1De*G)k27u5?lE?MtiFZW;BuP8t1Z23?~hceB5pM{s-{N(dkQ~Z z&TSDV+)Dq`i=g}EOI;n8=U{1qc(>SP3GWt1q&Jw27U?+=IRoKqpwC~VO$m>!j$Q&iy{yV8g#XU&^2I(H<#2 z*SPYRBf#$`5)(V9b1CXe<_jNy_s2kHjsz+$ouPfoFn`(=vudZCiOSSiNmo@3VmSPzBf5-ina4+y^{BKb+t0b9kh30s3;8cMg`xoeD1w`zKd zN6&BCZX~uP8ElR#O5H66uCSXUxUz7zV10-1s3_p|yRkyjcPP`HM4H5c{UV#Y+3!Ti zf$~kE5f1aj-$39hYO>{p(1bHd!L2|S?pg_k2jbhm)^B&lbv~xecU{#nb-3!AXrWqm7`n-4##?twUiz)DH)WYHM)Ht?Dpcf+p#MkvJCUTDU zUr4-q^DYg8U6P@}Ng}RDAhH`@F=_}ZjI}%LKNn$k?G*+dd$VqyqiiKEA7J;izx0;c zP`^+C+Q_s{LL!-C?|D_Jw$aH_zq5VgQ71Za%5W=4f_ihq>@;t`RL$|%482=~KS06b z$MoFgJN!){=WRC8TW}&$4eGURN9<9c{Br@;Phtm)d9F=n!WhEpTn%QtMo0I+X{$fHhL6KedPKo1(1wwLtar#Ht8eH-ZvOCOU9NICi%pW&p1y5(y zv%ReLFF%RrPD#{$4)0OMFOTXo6qHQeu^1g4*#EQPi zms+xGO>RZ5^`RJFwg1!f>j=cmBav z_A8K<#xubmGhga#x~yC{R>yw0LT_HoB0=YjW_&^#uDx=SQ25>E^4jTM6BnC=)6&)C zvmNpg$o=%x!ww$(xB5#>4`@G~H0I@$z{pp}YA+z~EQ*)*P9nzF7cA4oKd{>|#x!^l zfwgpI$sf|CquaJ@gvwirDmXL2j?yvQKsBWly+MrX+&=}C!sJ=`%X~^nmsYT2MmZ2? z(XWp zD2#rSi=#+DRhurn`VVUk_keam6zWlhHb}c#1UP+n2`Gu}7a`YSLgvgI9aQVFWuc=8XevxxPRh|A&8I}s zLi|A=J^T`BBW6L#s&O1azpbO-^Qn5RHPDmDaK_t9Wt>Ywh!Ud z`yeOqcurAGgo_Zk9Jc3xa$eU3+EOQ~C)Kd$7w!k18&6100}5-8qvs(pB+5Vv$EwxQ|E;o#*YNe!KiSf8HJxl$HfaV(C6h&EA*+6_8jXZ+J$;HS5#Tz!Lr z<2FSLP`k;y0QM@;oISJ4@Dk55eO_(Kw*GF%>w~V03=sG_ZFA4i=SgNw9DI<}DMA9& z+7j&6ai@V7OEQ#Lqcb_HuCF_Ej>bnBrgfL>qxh8~W&M^*s1)HdXUPu3>h&8q{Jvt94r3Y!N z8hDOX4&j~AO?S9IY0@!0H4*08yAE00(dK(# z=c)gv#%_1tt!vPkaSXXhwA*YgU32_LxT1A;p;`4@6v6G0;?HB^Ct>KrrULGxeGlz( zoR~h$eVX*RjMkxT&c2f2Ou>HJ4hUF2!bOtMRtRmP&|9@lsy9HQY3vkhSkwq|YfF5p z8|i+OBmJts2++w-_D_NPkHm~BwjUik~LU_WM`Cj;Ycz=@=OLkf;GRKEBrLou*mvkPacU&73};IHW9xgsM|u^ z7Fm~k=$)2){eS}o&EFhU;yUlK#cr(TZAwdvS8aZ%=&saL??qP+&>_YzH^`B!rx!yOuyu$~|8_p-*4DlBRM0xZQkb#}RAEI;iO`_iBcHOfj>kd5y+iHQ1SeV}cb0#K7sul?%zn zkuOU&=j2CSpgoyYGJoC^Su3LGmA#+gk#Fq0?+8D#@r4WL^R=ajGNLi25|a>SMv0&fTM&x@w=;PG5tXb1C~M zwpZ%V(4qzQ_hw^UptYimw|~mkXig8wNwS$XKn7|p@g?{o(83*oX8h~4`N4{G-1bKxm@a$ny3<N*FtRE> z6|3#6S=r4{ZQVJR&bwjVE=la^b5QHJ&mqydb01?1B`MN;KROwBR_D&R-~s~#Ft}KJ zrGNTm_04n7)RW;V*JO8o`QKW%;Y2asSsGRXOA5-EQZk-Swy*fQ(KW~nERl`>s_J3= zb*0nkTA>zV`JjEP=yEz7yx|$fsHVEsrO$f$?!m{DQxRx(_4zN-wg=*bv_j~l(Bw~j zC#65KGMb?CAoKeJ?L9!j9?5QulRMjWO8_z2_elSbuWM1DMHKN9vey|{PFxAqA7?^*5Sb@Zm=%O$m}5yGlH&WfJ2Xy8#k|_Y!3< zy4L65I-G)NrXL^7L&6i6lmN@TJpiLRtzQ(L`0$fMCpGO*55}!Mp_#Cw?o-T`^ZXP3 zIUlm%-jz?TRo-?bl7Kj6qRVr@bJhR=bR?aq7)$|*3gcvyM_f5a z9Gd!_#r3dMTrVYuh1 z^02^q%iG*&EYkO5zqR)N0YL+XQf!3YLA|QG6e`oBfiHO9sD=UO5mPTlBk10kd`gyP zvFC36o`M@0u_L;~oLM@VzWx4OY;V*sJlCwCQFQ|}MI(@7z5ww%+yU9b46F^*^?oDY zJrUe0{S+r5zd{7BhMy|lZwmG_^u)RAo6uTlk1HZ?j%S(^&i_ATy>~dAU(_}#5s^rc zL<@o>g6N4hI*Ewr(K`vz8NIh4dJxeKBYN))V=&P>(Ty5qj5>N9Gjk@t_x-+euIv2q z{I~aB`&qlKb+3D`{dZ;Nb5*O}lO?YkofzFtP|wH&Xxg>oP}R!m=p=IlTQ8bkcnFTx z1QXr->%CRPf1T`nOvt&jOy#WRG^+4$nf`dD6IF4n6}p5vc6H(!H5GurEJV@X4yyVN z=JWVIx z-{ljhxv;MF7W^Hh(E)F*VxhJJpRND+)l~7I)_&C>Erjl$^8T&c;uQoVt+4lV5WjTo z1tOi#3s1GAa$98*{pqnJ}dysQIrwxxo0@ALTI2Wk8qe_V&%fo&&uj{>EXQ+PDia0<vhV}@+9T+NqAler;HE7LxUHUjm)XyjHWLFKxlG7 z<61j^XUCJVTufgp9WZC5H&ET&$Sna5DZOUI7zOtM0*3Qj+L>=#~YhE+gqiYs# zLmjVeL`_nmSy9uVR5li^D|A}?kl-Rr8KYDOzvAT z4WtU?{`f_5XkFX3&4+0bh`_`0SiRdE>lBF z>E;V;?_&Geq7t3Q!9=M@d|;VqdTkH#d#fLF-&-A@UiqV2kFc$^?*nMN=KlDNwPhXq zhD+=v2RD>xOC_upnAp4)%LWa6cl}3YSDor*mCP@Aj^_UC5?%}Trsi>qW%)pp4T|nc zj#QRzuVp`_YmT|2cdG%AmwtBZ%gAf)?v*+?I_m)N$7so0gxlOS2}9ai`!U${X3u&1*=wr@eTJ}JtNY3qNT8^r*s!BiY zDyNuwv^fLIN{j<#s_G%aEBD4oAi8u4Am;>EkB?mWQa9-917a?W0xd=_&Lc*)3!nUKohA40TV+7F_pV;$Ae*r;OoD0BiLrWB(8o=m#YKKDBT@GUy zcj$9`{YM8CMY~kK9+fF5H%m0yMMw{fmTjDE6?*V8@EJ5|A?by}PqlwOvhx-go{YC+& z79u;@@y6hmt*)ukc{$Q?{XW&#&pH?K2XyR`eU%A4ss9Wl`(`+dEEH*_oSFcCN)*Hi zS-x!e(M;7w8<=L2cwGFj>MH8`_re>neinA7Cu6vRo+61dh9h-KSct|ovG}>-!Aqx^ zT?YYC>ggAzvN4jrm3;jtDRXSylrmM*6lyEa=cpWxV2%ZGOV&+Ja?`D1CEvV<1J%dFJy>*wNNeKVc`QGL1V4~!q`ZhnMKQ5ha zy6^(4ACsOv#_<~oh^9(#Y$9YE@!?Aj{bcK?JP&D^3@9=R2ur6mF`Qx8lCpEEtUV)s zSz^1r_0i5gOINE>qDuCS!EpUFq|Z3RQ8n3zAPx0^-MYRVw;vOPkQDgJaQI6w4`w|` z!K|Mia#^!+6zIaP_hp&Qd?LX9*50?{CpPM@w5ffHH)FNm`CduC1P4fqePcKTbFAA| zOI7pbaG1v;2UX(u?wh{wO!gf9w0eKU?U&TE^a+s<<1KHupP6t(jmj;$>hRljS&$(W zUeEY^vPA1{z!qde%W4LdOJ8#)*FXBN3iMeK^9V7GL-n!o;S-^3KkC*q(hy0VakX@_ zRL0~>zTPhaJ5>)syXDUr?xMv0fQim;InDA&lm2~S*QLcAi%!!MfOA=Mv(}9EFBm$U z7pG;kiG{Iu_)yey{5B!I){~@?n$2c7j0^D?TXePU(L-*k)`+<3+@4t#tf^_Mrc(Xy zXTQ#0m9Rg!K9IHR5>)%iKe3mwC5%u{-*>g!Fkg4-LXb|I$rGrJG$XU4#}Z(jq>as!;f#Cv!9{@7}q-Y-Z^^G7fqAtn9f z!}Y$v&dp;j-eljeklQ3=%-ALNI&ygWPZ2lJqJNVcVpiGh$rcwlGn3k~WZ8V$?;cSg zcPxU+}-d{ZQyI1zAI~lti3h-(tqq$JSQUw?Z|v zwF{B`bw5SEcuJw88=^gAhK3mKXRtwrb%s*f|Ecr^fYr6{iJE-=UaBvE@R%A-6~1LI zelD>4*vBl%+FiI(4@h4;FvRc#6ZRN{d;P8Oe`=k(ucuvFa6NZ~xy!&TLUnKQ8odB# z|Ih^_wi7AgJ9{#$ciJ(pzY&Kl&oI}y`Be)kOU;}`MB4l-}tzN z@bZ?&4($r{DcogJ=XO|I96Dl05K1efi#8t|LUw5E$JA_{O8swTC|um^M?SgrM@q<_ zps!9y*FzrRR!Wa#XxH=ou!r=VuelqPWFOh!sZ%eRx zRQuLq!sLg-Yw2dnrNO`AyH`HtoYzuR$0lfqtZ*u}y`Jp9u`?AslvvSjSb59!LNMg$ z(CxGVhgZt@YuEWbtB$6rIjP$A-D z6ABz`mI7{7&-C9!Pffxe73sd8L@kc7WmdO|$B_+vUGAqV5`f%(WBI-+nLcg*=pxzN z!8hY#B zGx@JB{LPpAp=I6x>pzLBw%jd%hRxbgWV6guEqmuI!VdSklFqb7#UUsrZ9nDTr1@$e zej-UWwb~eX+3LJ%7$iNl$p?iM@M>{8hZBmcFYv);XjUx>C7eE=-T0@M_RNfQ_M*9tB900_wJH%NqtDW&fpF*FV`<4R|bQgr_?h>ixI-Eu2* zk9W9ucKV}Iu<^rq@IVXMPGHO3f?ray0(T{Zo?T`hqU z`>l3eC{T2+M-NpY8bdO6&9{%{yfv=LHeO3kf7$v8EYQ@q_;-9??eH(GG9^V8coTZ} zisI_cHFs}$ay|)har*Z;l;h#~!G-5!|)~o0H?%CiYH( zh^W)8yLU3!WCXK62jBlPKd$zmH`CoDTLCM+OX_Ng7bOleD^tmIF9#vfm(6LT%PjJO zL_}+yg)f-i^3JYxp#?{Sc6Uk{k6OHy{5-tuW({T(?0pFco`1Opd$#E4R5AZZ9T&ES zvVw@{y9uWvyoh%9@ZI`?#oCZwdl?{l&BMOfTp=uvW^p}mOo8jz#B{^FMh;A)H9^<= z=Qh#Pbdq^o%LdsGDR6XV!25YQmhAO5d-4M)WzqZygo zWB=+fHA9$Qspm60xY(T|P$qu8Q)k#ZZ#&F4`_d8HNZlj584iX;WZf(?cyFnC7bn@BN8ADJU(O zgHqGw_wiZ732!JQG+1?+Jdq`QUBhh?$9Fv|EX@UW(Hse&1n=hfsILvFsIne~Nn(Z@m@El*3jh7=$ zJU4%`CI;fmW>^ysHQatU&{e3{q-8!PXD(i`R zi&b4PVL3SPoxkQbEK+aM-}v2V0^WD+GAd0RYuoi~vK-pw<0Ln!>-`vU@Qty7VKc}{ z=KJejtHlW^k)bJIcHK<0$v1dz=Wpiu!8+My(IpN%8&aXM^&XOhpSm~M0V@>>TuE{H z%;~t|GRZ3WS9R|UdLUHVQ*PmO9W_{#T^qLA=If$x7R(gLZr04@TG9764?W^#Ag)Zm30Ix zl1y<_CRg|}(};w*a>&bI-~00=z>&siaB4w?sPVoj7s6rdlO0XE8zK3m>P}eqWCT>} z&xw<+>i%9Z4Z@*jiRthEf~^Ry`XrYC zD#}LCA7i&cEyyd-dF9SuuDB%RBArwfXR(}*e&Um#pd2)SE+6tjte*I%wl~MF3W>sZ zhSA~nTx_J;npJ&CA$`eU^&+%_j(#=!Q6!a9`JhI2)Wo3*QY|q_il(AV3{4G?-6HZG z-s*>Y1}I9d)w>J?gRI08Kvw^%Q?c2eVig1?+cWyg6U(G(1wz5vs`q-eGBqci0_w>Y z={ZvH{O236%Q)Lp)Yc0rK|Bpv88NM5CDZRQ^~&^M79OG9SZzjBdRaPp2f z1%zF$D>|fMEE*$&8k{roX+mSwQUuF&B!~U|@A&;I$hT$UcDe#K>kn9>rR0ux>IDi` z8CHs*ih>pmd~|1IGBfv&P8!VJyqjbs{--=<8?wB$2R0iHk%{b0R*(PGo6fF2eBQjq zMt-OF?9UFxh?kg6$}BC;>k;fRK|&=|=typ{7sKt_F|)cR1pFi$qu8nkJM~{&a>QXj`Z+chVsq6rn;M|pamrF`oLs#|W*9#*%nLeD*uHbFz{ufIjifY#>> z08$yXpW}NeZbR>W1@XmSe>^|!EWKUzDEn_S`o1TaJ}7#62BbzOq(cu4`ehOJJ25I#JeV7AQiSME$#xdXgPOEbn6<-wmLt zC}w2heNQl5)hTFdu*=<8+^Ogp@4wdGI@gN!Sq^hfMh3GYeO=uPqq0(Z&wA@9PKlV| znvyk=CLNU|(7QxKVxQ8zfB5{^$~_&9)E(F4xW}ov0W;nmnyTxN=#s)M@*`?qal-v~ z^ZNy@hD}=BE&Y0XDl21|_PA5|z!LU7>xveG$8{0_G1#6g1){-HZeAVRdKZg(&TJ_y z`6H{8^}?q<4(M6^W68}Nw#sUK6-=Q`f|TzhG@HQiUOiLVT0T`ncFXX!^;OrH?Tb5A z8|(lGQy)`KU*6KSwB&mIRL8?l$28B*$_nL>Tvs`vh}|l}$?YeqAzM`@9+tIu$%rPA z6(H_W&w{r*s32YiZC_BMgbvMtuJ-f63of$jD1pI!P>p@r#ZC}3iK$G!t>rW!)N-M8 zXerEqcB1kHjLk zSr*Cg1;LkGbWwjMv+GSdki2HhF?`GNem4mPPun`Z3rp3LLFurB*LARQu|lt5=C$R8 z^)m?S6EK&;wUHu18gAQ7Vo*aF&8%b6PA^~fomEBv7-qqC=h#I2@M)@xS&ZKWqJ3BM zv2d#MRXUr4?PigeYa(WTM2uxJhd--9_hQPa`NfCWVt?w2^er`-gHWerZksdy*5$?3 zHvuNR64bZf^VV`S@YEiVrV-NHvqQ4nq>JT(xQt%nw#^n27#>mOr@&lv;%^$1;%xkZ6hTMe}B9kqxC>#ZlF4F$FnDp!`2ZNKe@h(5~pgqvNdFopKwF z!sM{d@=o1cxMj$wSGrehlLY`EFm>zxeFGb7R|CX6NB zI@`YGSYhgv>&#rWdj4sx7D{3CoFluOnMJ_ZLkERmvdO+-Ikepx$J45ozCVtBW64qV z=zL=$bT;{w_SQS^k`o2szyic=L_uWc^XNtPUY)Pk`Xb-R<=K>r(4XI?u8yJtS&RKj z0r;UMwsW|iWJr}%s#Meq!jZ;gb;Sy>1pZHl4_MTew2<@)0qk*vG=VT%4uzdnkR&N> z;O3RSi8{omztrF_>hOg0MZ-wcZuJNYl+*2>su0Ez82`k*HPG|hCE*bfZaBr{qZc=V z+&>}w2nGZWF->ybW||rBX~JVjnuxzBbqK}I{JcjjTipGf;v$XTMu&6>l~aeI3}++b zCfZeA)Opyo7osmL<7DqNO-Vr8@r^RQ^glp-s+q?na-vpMfbM17Zf0B7S*Z+R{YkGf2u_5 zQ)Z})M8?K_S1M?fo3zKB2&UtLSLf$F6mc))fMz&Cf%EatLOU zl?lvZ1hW6{guGD~?N&|ZnbPZ{TJ6)$K~S#gm-)BHLA)b|&4(GEh@JuxK%%I!DX;y^ z_uGFS>M3LdQpglm$O2d zI^?-(m~GYD^)21gb#_v>33=#3m->aF=pFaPv{d_f8UE(2pgOM_7Jq00o|&{ zR3+?4Og41k9q{N-3~o|nI05cSU0DCM2$LY0-c6fu^zS=1_X}`5UzodMTHi4hKGWUi zs9^|buY74JuCSoN9k+h)ZxH%tP}i@Z{Si=3=Sd@wsm#k>MxTR$*j3`xk)e`UypMYw z;H1&51dFFC8^zBy{7ux0=}m@{uW$Z0f&}qz-|ynVvtL-~f5?HP0nKAUjcz$U11(9} z_iL;Zh*@a8#u|$zZ2rxROKj#cgI?C&I=gG4(+M_j)^na>IGvnjfqrqLj&QA6pmQ96 zJmWbmrdg%%b+OLiA&g34xde*tWYSJ+u1ppfa!>6M>erI1W7QMHuHqQ)(idx%dB*|P z#Go&OH}3^j%>QA`gn5<9TUfbPLIvV5aBq-Ud%XbMA!cR9Ed6+QnI&KoM?VqhXVGctL3f?M zk-rwd6iWyzi2uyTu|k>@p$NW1vFzsk9%LanK1ypVxwmlPySkqE{-h^d6UKX8-{&-A zfH5}!S#h1n0U4H#zmie>@-4D;`T1*DFTB>zK4zW!6WwgXJHn9@H1}%R^3Y=S*%e63 z=GRleB_DtUUxLR*MAz1)7aUdm1T&QAw9Xb+RyL_IW4)%JB^(J+CQ_=a=h_i=tzNmx z+hp;b2@zg*B zrucOg%qTIh0VE@c=Ze{>E<{r_%{5w?pI(>2M8V&INMle+YLR4ZAusmFgmtMX=J(a0 z4d}oL*gP5)vVW`yB+D$vB2zv4r>#;tQgZ_o!Wi0L8XIW{^BP(sHz)!m$ znvUw@k){LXQ9FIF{q_1(}Z1)l@%;^Ji&a+^G*{a)keu9IdbFZGYnjH^8*tzdvvtL*e6c~A zSJ#wofK&RY^Z^k3(U?@Vz;3AF5T*nqV{h|^VY3#o~-OEr=LAe!X9*7i9A_J=-TiS@chZxGa<#<6()|{P6=F}YWn;c zkOs8Yb{Vm|*GEVcyj=eJEldR0u_#od+<#%h258{mY0~wW9Hlb+iIb?g$onqkL`Cbc z8lw|8^Gu+^A!pmiM#6GUz;<){rUL68V$ z6EN$&hcm@(jh=IV%hA*ISFvQV8p~3Sg$CX^A6s}wE*ABHcB+~4zf7dmN_C)1h4{E9 zLEkb(G&c!bOZR(J9fv^Wj8o$s_>s<(K~vO`{%{54{)prSM&D7m5L*1Eu8VaSua+y^ z!47brWnmIm8Q#yhTu6dC#@~rD>0^XoD=D7?IFnh~@}G*4mVm!A>q=W*19Q_cezcR&@p>)GdaV2c)Mdk374+=-D>3OjnztNgT`P=1%P7$)4h2u z=!CabMvVB`y_!`#Arf&1qh|BnX<(A!zM~$Fb{jY9x6bZ?|Gr&`a~xRfh!hWb4Ch7_ z&ULkKNG(&ls#x$m*n(62$p~)Y^#qw7@lRj!0OXkq4jB|wc5at8OD!>_TY^&gN4o-` ze(vW6Vz805hl%-=xI86n! za*Mxg8E!FkV&!{`j|()rMz*TKNT8g`u3CTjk5bzRzeINekf%AB9H?(f-FZ|L#iR2x z;Sg2>ntfD>D_M%ZC+0D|s&2vqn6fkh`o6Bc+ewgNez9XN;q4jWFel~w=z?oYK9<@O?amGnwMt3sz>06YH&@SZ9$ii zrTKfcmaw?QM%(y$I$?SGMrRxGK{2nai!zLx3f4jOxg z-EkKl`ye9V0?kM~o1#}mr3wMk@wqInRO5o1iifWrE=eeh?BF(UwlVQ~{qcI_1(O8=8uO(;ZkgG1AJL^7u-9G$&;BQ49$anaxuV~3z z4V?Z73rSRGSMl4KU~<7lW?rXZ%5mdAZ1kELOZk0NAsW$b+K+l{TeX+{ zFrL;a3FBu^kqUUxYVZfc8#mQw#5w8(?JYAO=BxNsLC{zOL1QPnTP;ng%sZyz6=G>1 z-`kx10q+h&T!M=XJ=U@gVQ=;Ol{Y8Ic5uVxcs2LR;AwrhPOF`5%4WD=>5YEqXGxA*u&iSHD!-Foo64ts)4C8nKL0LbT$i ze}Z(sDxLf^o+2&*GH)H_{X0!(cbWI@K%ypAq@w#;F@0ZPY6U4#+D zQn!qxLOJmjU6)Ene%5+|g6Gg5z=rpJ9=V~-M)r24pxpgCdg9-gbi$H1{~i}MjU+q1 zE~>0;Zj-+*#z_n;q^M3M$7)^|m%K1L@dsXMeA(l0{a$a@E@eJ}JGrwA?2BJ_e4IR@ z8CXVGPWsU?#W=(AF%Hs+o*!9I+d?Xm(GyD`*=~R~VU;EZ!Py5@U4=A@Z|gp8DWl-F zJan_y-ISu1$1_!kwRgAE(?|!63sH2vcimiX?J~_ zO!=dpEH{Jweir61teUJwZM-$QkhE}MmN_vxGdxs?E1W%1t8I#Ta>cLzIf`1;Ub4qxp4=$970VPRX#emHDAv$b4Z_XFz`rLHCajAaL zP5xZal)_bWPZpD9bIs{6mc5~mi8(A=om#j>N>VlZ@+={9uh%gWJN%<3(T!4j1J)u| zK(cPqC|}$+40v#wFbuUl(1gX9h|dzFX#}pPuLX_Z7QSdIS~^-fdKj2;?K3ck43^s4 z+jA-1Q)#8z5#cH>FRQ5Fs%iQ_Sk3czsl%qjJ|ethJMYn*icbSEj|r&xrL&SUry0-m zs$ZUsUf=ct^ZGxfCXS#ytBktmP-PuDv#Q%9+Ofap3(hvpOn=?@Di=Yz*{)i0)U zQ-toHoH5HOw%V5uDdK^962>IPie zmChl!D6jxWj{J_)^4j~{CdTWh=o62qPO%X}Vl>lQO z*=1JUAh~}4iOWBLgg8~lu|Y=Mp+YGj;7H*g(i*nnT~tZqR;d0YD?4Bwxb!Tc7WW=F zz&80=SK@;8Q1mF`?PyiCf5C3BcAIgoq$aR(O77BjIa*lQUSf`7LFTca%XjtvH{^-< z|Aah^{%^?B`WylpE5ke#HX{d0Y!>DeNseKz7U|-zcT6Vz_^i zsIR(bJ?1P6G@n=H3~(8wOB6*Vf_})47_u%sKMPs{w_ufgmh!1_aeGs_LtfCjhAQSZ zW64>ZJ)I$WAB6+V;7zu$V4$68d1ps9He>ABl18EL!DNhtkB^C2OeNAx^P=*|xNVA+ zC*U<6u=*e_R`1aG=?|t^&yh>#8j2X0wSQZ3>FIY)f9}ZjRL!Bi@v>|OLsncogZ%d;>_I1 zXAI0bv$+~CGOdt)?J?c33K+VtsNw7GZ%7gF3KD{%_J;mXb(Z-0S*l_qO8SdpZy_JA zd<8wXKzEV~^>Eh~fwc}c-8MaOh4M6{!npQ7(G$+H`Rmz<1YbWz1q~E2@~%y7ygrd| zxj)(sHtx+@cX^+Z{p0zNm+jWW@uJc~&OrC4d$+!jLZp})5`OMKBcMw>oZDb8oMGv- zML3*~`6GC-J+#c*#U0t!C_XxgCU`Q#wC_DA{q@U#I;j^W{MX@LQmpEM`NV(>ww74* zj<4F1_93LEN>Wr~e(s7fsg+7j(=28BHBtX+`dKZQS1_SK^mo(yYCz60F))LPHLtoE;$3WBfzeKHaXg4nG z?^QIwK90P?@&;7Q2-fgk)lZ%@zHj3tT#WUb5dA0P2I}B~FS~iuZ>6{v_g1}PK4xqx z!*S1Eb>LV-Wz>tl#XXO(s2s0-?hu+sKe(zL+gs1LfM1qSWzX)&th!j9ecCTaJoVuU zb|c8pfGQu{sar;zvei72F3C6e_wy2KeO`@xVn3#1|G)sHIBe*p5j^u1KjgJOTO#A=q8YAdQ$)cQZ<^y0Qnh||2;Et+4SLwU8;v= z(sTgaC6g`1Qvm(F@PhR9LTVFQgG96O7PrOjpY_jPTx0q^bY4)=tg*li`JWhz+qG6J z$wXDEqMjZvBlfs0E1ldd_%SL}RhpIXZ-qpo`lxsA`qrJ%sy|*7Nn}dG_H5@HFitR? z>wuIUGCiO@>8gdFAO9Y+NylG983e&#@=w>vycEu0GD!`}zz>`anj0Pbgc2}b0_=dm zFU5ot?Wb+M^Q_Lvd-)h*Ef%=o$@twDi_xQ}10Ppz0TVc)tM2<*4NFiJKQ`k`d&c}p zTI%`4&P0%$yUx&P?4R?g*(K!TIjDzER83%mvTi zwB37)yhlo09s6;3!_1D&!ut`Cf=D4P^E1X)Fhq12S*?4Rc0~XPZg?x#VaIaBunp1f z0I?;XwP8onT^HqiMQK_2izPxRF?7ftX69qV{LaZURkQVMmf1 z9#(LdP{N=muuDtZq#UeHMAb)`9*`d$bCX$Qj+4*F^*8e1TAo zD-LPl1oAUKb;)r_fu|PSuBHC1&K!wmX~jdbIu@&Iy-{wuio9~Orw6&u?(Kc2xVLA6 zqB1AGc|HVn^y)w+OfkwEjk3WUQ;G&x6pInRoD#juZ6$=Xb`EXa$u~gP*Di!z0B>#1 zy-(d(n-_7e>Gb*}P{w0nt!!oj zqu|nY{F2Yh>?s6+XYRsAr&67%2RKP%$lZzQ zwrjgLEmz~PbvuolsH!MhM?1snHYIDEgdz%-S?v3@ljR-)aS)nAuB@p_q0A6IcMi}I zC8l!r`!avSuZ7xcup>cR(R1_M+65lCd6q55(Z{xLwc6R5q&7P0sd$9g%YM|jQ$~fD zIEekvhEOEf5VO9WBezah8+X&?=Fu4+%^3VdB-nFG+FoTC_EQv9(8kN^MHus52iriM z(k>5L2iUp;5|K5@<`!8d9l-Fa|1}(-pF2ZJ)*Qi#FCLF9P%R6z?o~p|O?<;lYkAK0 zyUv+(lupa^USq507T&^-{ z!Yu-ub%)ZWUEJHSmgI)1%^&pGFQk@iBHme4uGZ^uxptUSxqJA>cpcm+Aj4`7e?`SLZ#ic^L z4iF1lXlhWS)%v@^cZ;SZjUo-cj% zPmNVl+2eOkJ?ek2CuV%tM90Av4?_J~O6PtO&o+LQe^UWXveM)~6EDOFd?~|W6f%{V zF7;>^`zNyKy)8YUpc2P(N%awiq^Zq z6XOZMqLP9fG5b<P4Y%2uIpnS#kmtXywg-~cSyYi(qJIb zAI)sprW`7KH5RejJj!q3bh9MwM_PvO&j|%WQ(L5aKPNqckM_}+nx*s0JF4K!P-Ew| z3+={OxckZIoO)jOg84PHf57>aWhe0E{`eE4Z`E6pU|a@(=sc-zOt+@KF7r#B)gsGFVgbn4j*Xs5)KN}98AtVI9)c8P#&e< z!>@2jYUo}4^rG{DzLnh_rfaD1e+;rNmiB|NJZUPeJ|?wA>I234Z?^AQu5w%#otu}e zJ^gjBvjPI|`#t!&yn6z95x{?6K9b_&MFdM@0l~lfk~qS-kaAn*W#HQ>9r2EYx91N4@N8?`NI;K^=EZ^w6sfAqIzXPwhtfu){kwTd# z{F{;hn9TyMC0>QC*KXDZSs$Fw+_Wri9Pv+$2zXwJl0+Mm{GeI)G0boD7YTzLc$0#7 zEGaB5-r`Ow-omgQXN+$Z24bX*&slTe9H9N$^I-ymA5>{|c7q6!!|C}2v{6Oll;SX> z;`UWWJAjjd65i0*s*Lw=Z|6R(8`)1Dbea>4-mloE_Oq5bqyJR2gj#TxZ1_PRX7*CS z%xu^~n^Wx9&)YYGW^uMPD{UIUosQ)gKHVVGM*F0={Je%G%IONa(8c|xotxDv zNQ)y>iuKu-UP$&S*h^1DEp!P6{$-?#>Ca#DXH2~&_9ef6VHxzp8qp_v2dV;FK=1D!$5MK_ZO&BHIm@Ts|OCj z2WFk0>)~vm%4>T!D5J}!0@)?VsuxKTvh_%4R%!+3smwdQ{0JndWl@~{p<)aGg{b`_ zwNAoZhHZPbjO2+8UQ^(q3(|1&`pkdhY%s`nest%2G8A_ETAmxwl7dP-erxK z{PQm<{z%M(%i|=Pt4W2I_XN!d5Shz|DQTw?@Xb1Yi952MeK6Y?HRoB^$I!b63D_mA z;Wr|$7LnwMbRN0WFv7L4OA#t|B1+TRH!VO>-t1Q5rxN#rRAt2K$qCFW>G5;vDawmK zLUdm8?r-dcwMM_|zpKV^_ z4>Fx!AMyu*unTZnV>RGSf6HsGn!$^3Q(ImK4R^7iDc`#GPi=!4Q>*G*D>ruO2!Z$; z%=HsbYH0XBpzcl;93`c2*tht^5$qX+i<9GdWE0skctq`n1D(Sg&;tdJRtK*0TEVppn9-8P2_isPYJ zEt3)MBcA0br#-j`~7D^jFgsw%(;YK03YqyLY%hD-|8D>RBF8LfXl$5xNPyDP()hoXWZl=ryBQe z!NU5q2r+vmo2l5mdswpAjc9-Z+!K6tT1ViJjB1$fxkV8C&T~4g)33*s>Y^oo7T4`r z88$o&W4S;m;e6NST5H$DnECr@8W33r2Y&Py9H_yScv0>2XpCTcpAMAC`0Md(Ecl2)?*9 zJ&3Awmu89w6!8)HRe4oBa}#(}=CX1ksO+5y|HmV{pC){uQIpjQ%P!&1VjWEv)Az2q zkU|Ex*ASe#T3zGo>?^I6ulAKiFwT454Zo<{83+x?Ksl)h6#*_;e}+VG14DY?=s@4N z1H<6)nv?cloA#mE4?qX_9xa4y;eP@PLO8QLyHiFuN&ZD@5mvuLR;tR_@6JBkqPvpk z7)#{SG`%-WU?x7H4cKeq?$8J7v+-rME#J2K{Spo-;&%Sg#3wZPB$N5512p7Uqovzm%AWF^K= z^O)2Kb9qdKL!e8=2~{}cDEK4f+IH1qz3mUkp+Oe>bE|gq9*m1}59@dCpE@tY2AQo= z%eM-?@AY3#cQX4P?6hPN0@}l}N~K*<#E_+Fqhq%O?k&wq}lQ!}!)?N_3!(pFckR=v~$HZJ+O#TJZOO zQr;1;?Hd#&M*K@euQ?BiTj%qfOldBy4-RLG^ioJ}lJaQ{i$Wx5$iB4vQvrc;bk}(( zlFi%=r1J!i)4!|$tGE}r-u2P`HAB97asjH^k@vWnzE9{#6am(~nW^QRDk^zB!Me7D z(>jR-u;QEFDk}EgI`h_XC}DXbYjTr$(>X(q=^vBkYa#(IuyHnlYYZD8>~&5~?O}M2 z@JBTxTBAg4ixa8Qw_3hN$^PSd{Wg#TYe}(MHXTeqzMN`>VAoy}2HW1#`~CGuszr(X zf0V8tR3WKAY7Y2C)X@mbHw7W|zwphzcTK#KZwTql8$|y=vdo%gzfu(D{`a;gKRK+M zUq6_Y_j&9*h+Qq3nbr&;Qp=Eq9sQ8*8l0slmCag_kfBjSk{Hdh=7V@1re2;ku(Gef|7@v!ysAGkaHA4 zK|pdGlH@2EB&%c)kemika?Uxv9`L+-?>T4P|E=}b|K6MBlG%IDuCA`Gs;>H~cW;8c zNlK1dPUD+*BhSxq*5wlQX$1P#z6kV_K+w`@g{3NY!<@zIr1yIaw&|AX3JYTG+N>ly=M#5y6 zoAHHncED88PyC>p`o@Lfo;GrSOb=gYO^obBnfh*EpGWk%KZsYK)~D1P=w3r0Huy9&Xa$G7-Z%M_j{y|qI^;1= zAz0LB175~6^KCF3NgHc)la=c_B}RE7K1pTUUV~Xus%YO&aI5%TbAE6~#LM~|q|V5W zsE$V^go$|NRe2(?12)#*2bwyosPfXt>Wz*YZz^B26r;(8Cr)klo!6!t>yqO9iN^kA z^Br!m+y+z%MwK~1IFZAc7p222)N#W#G@k}=i??dmT)UnfZI4l}O zE=uqWsw;oK(!@fXJnLQC3OH?r(W|n+}|G{G# zTc5wT@eu>`hIhS$nvCl9z<*{`J{|vt>JcL-A#qN&Fc}9>x>w`;2Mc(dV>*bEp$hee zELr7%k>S&%`GeW3rCNo=BQ7@KZ~m71z#7HE25%oj_n-g4kJ9`6!ixUchMM|LjjeM^ zMkA4Z&%}F^%wMgJcxwMa;|K+oP5@Mm(eBSZLGzLfsdCz(#X+fF8bdwvxQf*Q^%oc2 z5I_Yue-7!n^87xVM-rDbLV*sApoN0v(I-Wnam{X|nq59g35t|;yM3*>E!L>3Zz`=U zS%?T;&6r&OoaiXclunPbeQjx$&4f)AB|m_g*!D29D4N5X>3QD<+XRcWGaFx}o|JTI zyl+V?W#d^((0p2O25MVE9loxvMNA}4c#|fJ&NLBj53DPKhH@sEOq6TK;ORwIgGzO6 z6jQ*Y-hmywJ;Y^3YZ^*kjJMyM9jc0ht3%@4!E|i9cLi{RU}-nb_;COg4jV!B<2goU zsn(7~SP_}9NUFVc879bG7D46Qsn8=HIiIq?%!HpLemu3`Qc3rd@j z`ryI!xoMliT;q_B-zIJoj)R(i`TlG7k_LDVEX-6L z0BA?HNc@B{f#S4x(>j7vKdvdCdqCOduNspBHEXd|tezlkvr)9~Zi$l=xk07eY5jzj zB`UogOW3(-peNT`0W4EI&M6-ye^RcJUig%PM1v_9cir4#c&vNu;ks9M6|uc?aqz2* zX4H=njO5uLn<&c2EFG2z>A8$V*{1id(HvOsBE5SIm0I9*L3Ii;h^-gzwz|}ku1se# z55F$=yMvwaiTwAuh00iwYs5)Oo4~H5qjkZ@f*I|N4;@Dr!Yky5a^~n1u*V|X`-`&q z80q@%a7baUx8Hb=_RwbkT7%;sW2AMG3mc?7W-uy<3@cS|^U=22E8ij}wm}a@As<7RQ>Dj8{Y6_-P4fmXTGTHb?j znjog`?kE9C>x!V43VgoWW~*5HU6<#fU=-a` zmsI#8;5r()G;Pd}4OQy5wp+1KT^EvSMiC&SoL;+ykDO%`p6y6A@o&#l*1+?R@MBOf z-@it}Mz>XRC+`!gzwK*`-m5yD(C0aZlxn4gA~j5(VyOOkMjLu%JZRMhVLEV2s0Uv- z!Q*RO6zjZK;l}VN=*vCx67@lv!Fq>W%2Ro_A3bl)D)M_+xR1FDIOrRTC}X|fC`X1M ze@CMTnEmRLRd6dCn2G#dY$8po+S?btX7DD+q!06mL**hbX-$j9yM`=Hve*^$INlHt zL>ziOw=J~VX0_`_*1986*9&Gc$`gx5^V@UtLm3IoTAls3y*{Mc5)+7JdP<&%d=>8{ zWrI_9l}aFc&$A_*d{D!#^uPU`t9tGG8m9&`yb?R{{FkLeIo>ULo@2C^kGHkqht6ha zW(I0SW8a8;uSGOD`LME|C>Dc<>xGe7=%DOpwcKA^y6?vIQ0CssQ)^UV|?Y@ zWQ_hxAzu8P_R4CBY#}#X{_`=7f$V&S8X;lKQA}SM;19=EQMN!Hd7|e!v&Z`cdZY1C z%J7rV^Z8-Ji@I?+J&WB&xDJi8}C1k!OW>Q65Tm*Vq(<&Xy<9klH}gbu`jHt zX#cE|?(;`)BLS85jU_MuEhwcYpH7(94vPZ*WXZTcDUf_vrIhwCZprRP=v;x%1wHol zgH~Pv#2?ZYiCzY(SboF<`hiKp-L*S@`*sM0oUD0s6-qU+==0Q zvOMxE;X4#Qx#!Vha`qGUr4t+ImVsURCsZTP2NYQR-ySiMvb8|Pr;NzCC*}-o67}uC ziok4MZbGGm}=KWfG7dBTX2`?9U9a%RdS>@SZyRHyM|NAZtt$@NxCX##Y=9nZkC)t*033S0@7H z+v_I-5u-U2{0YdJW7l_01G)pu*-xO^&F$gf8HNxfSZT;_l2thmuZ$8BMkpjUDUb-# zYdwSAcT&%IM9sVqYgiJ5nz3L{XuZciVkDTwZ5?vZJ;fB-VbhN7*gjT%I1lfFOWnYsbhZ!4=+$U2x$5RH6#DyQ&b=QD%yXn>i#hao=>He%pFj;cFz z`<)~Rzo^DmL|nznr#WYR!z1&nq21(_68FXLorl=iD1vn(ws$M(B2oS9ulia?hjsL= zE9nSee9oe))#3|}EET&Wv)>S|cTzx|RZ2kI7HhGWTEkKWHOSfe@jJhLuePRE7Oc(7 zc)eM`WfR^U*mTKWNhKZc9|hNy=KimMQAv|^XD99OiLKFy&r+Pc6qr~_{IfIE-6S>r zYqyH*SjSeRFgzI~&IvF+dOkM)Lqp*nhP>)uk{*Ig`Pg=;JTnb%4I@|Y)}XB~7VUK9OzFPT1^PD`;_D#l=PNQi$7rn;ZOQ<)NZsjWQH4Tp zjJCpfD;YF8#~nHPP2L;n9TBi=_-N z>}2R(BTHU7{|FQhCO&!!EsN`heV6-1d;5LhWcd@Q@77jcrr+z{fseUQ4P}WkLXpDM z0m|+THi%*+iWeQZXbX&xM1*~S5iQJQT%{|r|A736heNnEHsJp~o4c-o-kg;_lq=pj zm1qQPs2HS&ENK!7dhy!w&aMa|r_Y0ifa;vDAm>%DdyUZ>gVcxhqnf6Y3hgkPx4g!C za<3Fijq_?v4CDk|)YQxp=<(Lx-O7wW{v3RN@IgY^R#wBz@(VL-Eb`}opIcbANtED2 z#Fqj_2UK}WsuDD!$RR6dSgOnNfW4X_0n)?idV4JXRfdWA8WOy6 zvQ@S+TTSq7|AX66UwUO)Uk%zQ>aTzotCad^;9+M;)tyush}7&}4Yh(W7sep-`bCv= zHK*pW!)K`bID28PH_O>*JTTB);5Na6f`aG5jtK{?p(v$JJ z7+_(i#E&c6#zZ+vH6F2ykBk;V$Kti`PSBdKWZbCdFR!2P5~!-oRyclZu6Gr4c`{x{ zuW(4SbDr98bjzk>oJt}ZwA%#}4wQ-?nRpCU&kOQ1P3Tq~SeVa{WH2n1oT2_&NUV9g z8JC)CCdDV3zW+)+)h0`p&vIwy>?XtOhS@ixh1fR(ld4|78#yV|{ocQMZFtVSWIMVj z$o+6gY=3uVL<|aDMDbBmhEShM1hwCBbVrV$l=L_fvm_l4VWJ(DVZE(jis)5uwL=tZ zkTz>(R$q-WrIcd`MgFd>Rm&fMW4D`Sl$Dm3`XY1e9w{qpjrWt>t>9`oBYDG^e&<-} zY{9>8K9BX`abKBU*N%Vc_Q)J#zp=G}E#I7$qtjbs&q|ldZ1{WxsnlaCYm>)$qy%xx zIc7_mcgQz05|9zdFj9O~EjMqtB1}0b=0lEsv}Ng_k&R;L*#OHp%Ak_#sx>i6V<~4j zZY+F`T8dNQv4hxv@<4w_#Xu-Bq^&k)trpX;9Vb0I#fxiYl2Ts4p?Bs%$s9Ahytr0h zUv;SY(_Ul;QFj8T{#IJi?4h~spo?1Apn&6Uu#4+abZ$5@;ydSygzG78PqlQk$NHJ$ zPknWT6!gt!Y@Kt@d?`9n1RO5rpYdLg&a6k2FNuu{!&5| z!%3%xDF!;NI7KVlG=dh!Gh$R1TRCRXHjz>Kpok3GB1YL4meZ-QxQx)|LMxy9UCB~Y_|KFuMol+dwNoO zTe9-%C0%!LOLe`bqhOWU`Csbliq!4HJp!61-E*#|@J!nK_DDxYj%qXdS7vL}cStc( z2wdef*wCl&)r{KGe_zo-Pt5 z6^B&F4l;S)?K70P->Vk{LecNF!!c)BR->$5w`Vg{65g`BXfQ6#=9S_w*2wO!NhAO} z69&!qF%RMpt;UTrOAL)lBJdJVpT1}91hb39^`M&$i!;=@oC*c!HTS!CT7SWBOF9H% zW!|~gDvmIfh(p>@SAAjks1g&m9acE8bqu05C+HBrO_AR`ooSajUmk`mE$BCy%TO}a zPy4=u?NZrh8WfQ!Pabc}ea4+uh+d*;T=B@O-d1tnpUWTykXyS$3V)5i7Ff{R36oHi zQS(%?O)#1huNoW#8!R#yrdp$A7Z+Hc8jH*u6gMgUFl60z1pqD>2eTUOFw6?)Y&^fN z-7jgF>5^vVA-qsus_E)l@0&$AGPd!ua+1^K20=poX`As3`KSm5i_!-=B7Nn>2Aj0$=DQ?Q&JQmz z6dw_MA@lPlk(KCahh_JRvbH&v+Jx_tm=KVkDy+0bVTWDQ#CiO#QesULE6AL4$p(u$ z_Ift`Ncg01n7pS%EVp3T#G+PgL)MpPGPMTr$o7>s*i^~yrP~5gk`@t^&U1w-z=kNC zmyEGh4+>_z0jPDY^)&)dX8%=Z*k1k~HJ-S3FHa?yaHy^Z_G=0!7W5FO-QlGqSMU|p9S-St*Hs-(+%=0;QAAI64{f71!f-Pe3Z zd)r|CW%5Zgd|beeC$<-6tDfqVX@xu-c7eb_GO@{`e0Mg(CMbO{*77W)e^Y_ujuHr0#?f znWlFAMu=+KhTlK6OQiV8g8?A8N57-Z=%JXE$pVomr8px5n^RRL6f{%J8_xk%&kEu# z;f!}cU=fs4xzRq{lIpn8_BJ2mcRGqP@Hb{g2r>smF^2i?OT?V%pqLcH6#7MA-$&n3 zt7lMuS)!!)L*Y_tqiB-Mf0#P6#MnK&AdbMg zWve}#&R<9O44Q6}&~@}M%8+T(_``G0QmqRJ@_8guSVKR^{3pok@A*Zq(1%6HCxW@^ zu|JNF1UuCCr$-AbG%~Z${f2d|vMw5xL>cfv=_ukyf6{~)IQC#yiW^UKJWpM0r!a+~ zZ0EtS^wXMYWFHnOnNmcaQ7?0*sC*__;0Qw_n^5&Gm^}$Djr!GWakjCtdjUJ5Fhb1C z4ZQ|?M5h(t)tVw}7k#73a_5{aP(puIW-4j%R{(qsK|bt)y=qHrRf*dp1lhBosXhS4?~&;eHbNpv-gdX zv8>su6lQY5ZqFVKT$HUMec6)-0>jY$FOf*bA-_N6pn?yO`rOLwmy&-xcy(K46Ls^j zNay2t2l&r6N0}T7xTBGH3m0TEm~?7o!Kff4&DM7wGg8C8Fau* z_U7xSZ%Y@0rAe|0n|7k;pEsv*dAyKJlCF-8`E7u}yOz<6mX1+vb~S};oyky_*r2IF z+Tf|XJxUe>=l~Frr0=g3lGg@L-$pEclBVx-?GN8m@O2%?v1^FkGZ*=ew{N5PjTM?M zM=Fe?l4}LkwN@iUP8~-w4v1HCFzFiQ;iYQcZ$rc9U>YHzpc#^b>smY8m+)c|s7pU? z|2D^^Tp=2?7Cv7ZO~T={_klax$sOe^pf*SlO&{M0GXy)&`i|an9Zg$Gyn`84(TtOZ zl`c;)PUXYkJ*wHW5)g$A5xQv`UgwgF`SN};1uVRp2P50dJ=|)iTA(~3o)k%fRNfYOe{Iet{NQ1@4+yccRZD8iva`!znb77xAPQ=S4VqEt zwnorqAwh8N_bYW^f-T=8?m%l&16VKw@;nAP>OE+ysHNmLM8b+hD)F;yio`P9#EnB-t_T?r>ZkzvTA>*D|k=CUW_N;v4B7Z7p6h)=4@>%!IyuR~OtB;^x^)B|0j7$A|)Y1(aiqY!zMgL(_fmCl&j@jMSza*5Ib5tB~&cz99}{ z9UKL?tk%<9(4b2{CHzIeN-*#AAyemoc$tG07V%*w+NwfP{f`h|m{@Mzr;hKP>0l06 zH#EuZfoE+hEbkPMI9LiPPWgQv4dWO@CsVft2cA+L#vG3NwJQo9ec_BdG!5@8HpmgZ z3O0{*2b;%wm{vj{24f>MY8qp`2gC@AofbkWWX9X8u)~OKLE0ZK#a+M*<6AAMnkSwAo1=H4Q7ZJ z8mqqnu|mVU5J;B*D9@n(pzr-Qf^m881&IsP`@h}`NTBZpB+&Q%7ZTv!|J3~Dz5mfX zX!pN1e*kzoz^{j+*#-+ZQ)`hKUF%AflNF zusAo3fXU>(B$}Ggm;3D1Qj#K|XmNSj`*QQQivxpd`2b2yMv_+-zvN=L+@VeK%B%tq zp1N79aGNOh2f2&3)8FTZn3uWTzq|yVBx^I0!nkBBD2{53^YW|j#hzt|FVUc3uDHcU zXM5_|ITy65rP|BHs}9Nky%hHFMP(%%Gy+O-*Jr~nCV;o(3KK7ux^S^1p8i6w5DLMqaf1uZKX87qw&DOKE|_|5kW^uL9u#n?k+(E4=t_ zI&{DOmHI{72c4!gGP${Q&>>)mk4eYp8=ZFr+Bn-~8O9EY^k& z%{hlt_ITz%`a8O4biX7UREAwOv;_h$E;4c&d``$5@D-uGxTt~N|A{vjJY;xjgUBEa zgKiW19@_SFbc5pn3Ns$q;J!Gr`5rT(y%?`lZ4~_Z!eCX=rN1S0dx-|Q%OAA+y$3t2 z`0;6y?qyt>G8XihJhVJG|v~(aiszuiNmI^Wz01Sf}&Yv#xV;VjsXbdtn+& z;E~CIQ*FKz(UiRL;9D4(QSScX>rMlYxE_ipP*UZy#m>oB8x|Dr=<1f)S3bFO9 zQD6K#u8CG86K^@v^zcd?J@ZVEuETb89A#vd-bTJ?LAAP#3Hp z`~|ZRkPF)Cz&b*-djP^gN5a6hf&c#L5N*@I4gKf4{^{`llgP&T>i9DoZ#9UQ?lkwb zv=Jkt!iZYBl(C74nHdabxqijwt=2ber>lr>ck;^TZXs@G;at}XYtu8TFSxc%`L0uN z!wIqg*cb<|jk?1@5o$*`~rmjnfT>dK>j~5 zf~J7}?M0)D)DW-kQ}F&lAgurX<8>bX!K42CHwSoeED3Y@$R82BAkSU#(;hzOn~)QE z*|sMUOdgwR%hR}nT|AHtnp(1);+_IB#z7e5n+|8|T~0U!DQ z+?%ChCvP7oUk@pujf<+sOAlIWY=#_!h=OBpq8VTX1_IARGTi*Cfsxyu>N z-wy+K?RJZ`Xuy~(XlPa3dhv^WglU`rBX~v`EY5br$uu8azthY)-p_qdw39qJEF23F zs+fq&)XMxmUNneUiY6kSdOR z%QQFUD(aG$5l8Ql)pRC@7%kD!XGdiaFPYhOpIm1HCd^6aEfc7r;9^GJuHZ_+N5obJ zAtYJRIU?`N9*)$IQ_grPPWiT`d&9at;HJdFsrw!zne_C%>+6E_SkXvCyXvi&_mlTL z3y>NZstt?Zy8){virdr?@byKxD$nSs#HtR}EdfWRUOdhuG>HJ+Z5v>5=aDbQXFg9c zk00)7lblKPD9t0{${7hZOrS7J4a4Mh7kr2nM%2(B4gn=Wb8ctWQE}U^(+`_e>-d2nA&j+mA#0yz3qY&^6e1E}p2+H?ERxfO99u8Z5$ihV?IILZN$LUeT0K&RT zjc|ZNt+TG``>C^4D8m;170AARjhj?2$4KHl)S_LtKdDKc!`1xE`@0?3y>i6hAR%=K z=aI}@?yDNNtz`(r(Zr?FVvmcImmC7=B>F>EZ7}BpfpBQVhTef#r8X@h#(P0z_XavX zv9-hvfqas>LYZCi-#fJb<*VN;4~Wf#Hsi1i5eBw)Hio(u=x>&KrdZFoxE`}UMt^($ z9E(NC*~;)Si<+#SsiD3d7K@_2p54V62@73gLo60m69c<9kDsygaAL8D8=4ruv3m^V zV8db&HnFpnGqe%0w6L;77$WQ*b7QfHSejefC|c?21A;FN9Zd8M6~u(GSfmXR#()MF z2M_DB$LJ3hi>QgYouSQR7EyCuJHwZT`j!TUSONlAfBAG8T^Yx4QfeCgW{v;&dbYxK zTsd6A7dslbO|I$UV!ZSzv{>|5SK$#-@&qDxYKi{{)7pHOOH0QP7Z#_@PQ&dL7cWR# zU)vA*G&3h$Y|d4UEJ2>H$9|wOH!|+-;^B+kAGe)5nLPi$kRJXokKX@(KRgLOJ>Kt5 z67<}hYO=d=(N0zlE`=z{anhYhvpkLA;*^kbsM;kDE)+$VUv@2b7}xn6hNH8J7Qoyb4* zMZ3=uJ9jc}&EGdH4DXN&8ymY!NeTuVulGDJHtcBJW7MfT`S$i&Y;3Gc4QK5buKQCi z1#^o-3~Vf{larHji!mLN0qfz?Q{3L#vU4*VFc+&!UF^#YNa7?3@;lXW z`s0hGdHnkwPH%6obfq%8(a(E=(NR&OV`BjV^{Lsr%R}3r(UCO-51VkbP*>#L%{%R; zV+M?GTX7CkAx2O}dRu{dOC?$pA3e0;>o0!g7V zFMU;1RAgji2+K=M2Xcyqww4R(>s)qC4=0I;hcQS_`>SYBR!gy=#oO@qPMt7&Qm$XPG-XA`RU`T2zq0!dYv4Si`JZEJ4_ z-=lBKzq+TP&UwkOot@oCC03*%A!4EiM9K_H9F`MWJEFe!DtT(w-9gWk__oL}6L{@y ztgNzDTta^Q`0=OVLr2`C+jdY}d3$rS(Lj!}n-yAe5MddauGC-w`jn>3B#)}(dJfL6}p;ziE4gz}e_u47 z29&!s9D`o7`@A`bbZgWNcY3+^wUg7=6ZkP;ip{}n6d(KZ=g&C&J0ScC?Jm^9v4!nwn``h3n`{0y7L~L2_9v{rU4b{q$7; zQ5&rH%(srRpz96lYtiOtw&^X;(>T9P$6;M@B|E4bCTdB5DTdZ#zK| zC|Y8yg^5~M!+-}-su~*fQ`YjH-@R{Cng2Z8o}bnHs!t~_QkJT7OVDvmYh)Doj>x|a zND&ed4JN<1rmZ4(`|jPS(2Uwz_egk(a<0mhI`AlLa5G1Y8o|?@aPxZiZDU z`1lk_TdNh5RSUl-&(A9I>UA4#>X!oQ_2whRSi}lhBl#D`qm~9-v9p8+5IaLnL|qL~ z-d1tj^@&<#mq?%!D$kc=jT4`N7OvMgs#PEon3-H%TqZK3`4HbH7n2!qIEqQ;rci9q*^pCReFvx3S*iLq=X{1tOA?HT6<8CV<#?)t()PbzTS_~-eW z6<>05m-#gX>=r^$X~1_l=3x;N6JPRbJuP~;1c$@hN9fViy=FJ}_M(W#Rm4!7uQLv9H+mE6R6WDX>a?R0v2YGGl~MA3lm@J{LsQxjBocY!&f_tH-O=g}S(pYwUab2=S<8X7Sk&5Jp*kTuEUD7!}V_YYs} zD#2X!5~Hfg0=bXD!O@#5Q%wOuqj6DQj(aO3Tk7N0jx4maS$UgCsAan;h1jP&nMsly zvQyy$S%Jfs{S{?p%Kd+Br7e^~%pfp(Isv1j)fb=KeMi1bBY#?PZ+%i{L<8%+1x)H& zZqFOhAQGx1Znp!gNJ(Y&PE2AuYiUGZnMfhR>q8tUs)EbXt2YyLwNlp^nM<37;^_Pbsg^Jg4*T{>@+zQw+CdDMGnaL99oC$t*qq! z;WnB%KRc# zq3df{re#M(Esw$rrq-2W3qnys+lA8rlD=#%PG=d-T8{86fSW)XM&5-)GJU>sv_d@Nf)t_-fnIr)wPlg;G zUS3n@{J_9KDygX9krCwsO<>siw{(=0ILwDx4?lv51ZFrOU32L_0^~I`Mr&Oio$Dbq zG?@^xmttb7!~MXQ&hnGbDq}jj?I$3S)yCb4G6}qGSP6WNV0s=b)&-*-?WJ1d?u_|< z1I(q9+Ap5dLM2Z4+>h*V1Ea??719+zq_cXkISmxi1o(-UWSf?TNP~;{w>QmyAvP~B z4@-xZgQL`OePWBuezBM8*)#c$G3alVOV8;wNf@>Ouicy|OK zSt%vP-E2N+p3AhXtv6$y>H7skLM~DfIb|@FUAuXkmYh0Ir#`Wy`pO%cn^>2L2k_~x zdn!`iRTUmJh&BwJJM0-K&sORy{PWvXGJzTj1zkJ$`LrQ472iXvXDl zGssDg}Cj8DDm{@Q~F4}g?zf4Y!Tp}4xbdO_`8T>L7qCq5RIU?8Yxmja2R zp`x0toKFl_JOvW~At50yHG~yU^fut++mFVkCjQMve4E7c?-=9}QBh#LSRY+p3-kH# zp~7+!F0)mCvLzKHDgru}^|y1@hg-8M`5J8x&VZk}g%62=T=wiiO-)T4mj!+1en&J5 zkl<^Q3kiDN;C4USLFXIPdxL|6W91eU;2PWIwMsq^+k(}f?*M>p?~#6Ncq1Ksj}b}`O#W=$UnZ3HIMB=E1vECT6X_LgYicrwy0RQ|Xni|V%pA+8((h_Dy#z1fhrh6gup8=BTM;i?|vmAAbnKDI1MSug- zdzU>D{rK_YdmM7+`HgOYV{;oDHZb;aO&6LT6QEgv*j*h{mHF+uvk)2^D-i=pZocHs z=m|*p22^z-OZMyBk0+|^0|4iEmn^nBf^-=%Ue%U;?1xJM=rgiO$8cF_fb`1U&24zv z>%o%?0DxA@+}u1|4*ovc&m6g@KB)_(B8U{~A2}YnQkdFzuAP;`|b09x2Pvwi2 z$%yU`aJi`iOh7@Wcg2Ayr~ucjd3wq(j6C#&Rjj~!gugZRc@)(r1UOg)4BN2T&Hnl3 zi&60Ne7CoLPy56n`26QMaot(Px+-6nXHnn2)Mm-M^Pld-#{QFM^IOkbz9oJ7FkJ0p z`P|iF$ah%e3RLDjRg!fyacIXSM!E0z3@{diO6{C`D$!9_ONHibML#gr8wO?nU?>Y4gAf&pbV z;g3&4!tv%Y^^*Nk{~^fL83m;8bDBmtRe~_TX!#0y=0Za)F2CMzRQj4cRl6nqeDy)u zw0OukqJ9xsLp@x5M|%b}G;A2VoP5W)Wxa}Zx5A5j{AoE{hTDVxR6*rMm`1AJ5QOvD z=zNP#ABh`FFsr&c6oVO1= z;L^e8zpS1yz5Z@gfGMSQlh#T|ZHdjQWT@rSTS4bn>;B@KUG!hu>e#=6fL;S6{d7mb zusx4<7Dw+<4+Gp~h=kvzlsR(1)LueDVpm70RPprKOsO7Mf0C!aas5?&)2o1hfZWPW z-bZ)?JT^J?p~|Z>Aku`_H+sL|LMtb&x`(xcK~Avi3>U_EECcET9jocDRn+_MeD3vj zSenExef*iMzT;R6GzGN$jE8|z2hG~V8O)BPdKuTeYsbc_Pcy1T&F{+cIOfuZk88jH z5Ej0@4Q$56*?FMfD27|w3BY1aW&mGUTGIikor{ah*-oGAHH^D%dn3j`leIHt$m9W4AEM?idfB!zWNiQWj0ht6bI7st?9`bAfBqwrS+)or8J^A_F(uXRD`b=Ql zedsiw;0TsU%hh|hZ^B79O?UbgW!-^NO-xLTCRGFfQ$)q*xYiG6;^0tH!oL&uyEYz# zDjWwh;j&x^lp4IUFXa{XrUTmXOigWUSTQ|21N=|8ZFizGe45eG(T~w*AUVdv!?Q!- zf^pAP$#+>T8#2J|jN@GH%j^bO?z%Vvgh0${g}*I5kC6+;U;^8e`EdI|%Jc`R;GjGY zn&abR%aP(&w{H~GpH(@@%OCcgms$P!QJrMJhN=XCCja@#cFK8E$0i_4FgLAUU6d^h*cBZoIdJKjm*d zMd5!}HPn0)BAT|cvI2sTC{I2csyh-j+2kGvFFK;nlH9iZv-UPy$jv+nX=rHV^TDJj z>dpettfy1V=Wq~eoPN886EVpdnP3wh8Y+KQ`Sdl*aU~4Ng^us$KMhwq76c|6){lcB zH)Oi~kXvr2Byiz;x4^S)B`8m2Pk3q`%oNhcr2Uo@xd2$ol&O`?MF;C{ryyE=#P1Rz z_z^^;Dtq){St7uD{|uGXCW*%V{ixCc*-aS`NJh^<+Vx4$-5zGJbcmAFw>q|b;9g|tgNa7KSOBJ0 z(_WsYS^x(C7Qe@dlbV{E(VUKPQ$$3>^_w@lVxEcIl*6Z%{)IPnx@zg^dc0cEgo8$^ z{5WcBYHVz6&)2)zn_0B0`s+PCAAG8;t|l$OeZtLs;7^kn(lZOPaz0Dqw1?*2PF4FzjsLv3hR|aSk5;+DS!H!3a zvMVhhr}Cc%$g%^7?ezq629^#E4nQ7^&R*fy?WP+2+qvLA10bLa*O@KQC|mj&%MN1M z6w>LbDWmR0ZD6dMxjDPeCmd+R_GgOz5MKxkwS6Laf>ixf=99nw z=3yTQFkX>u8o8gT%E`&8&jMH*olhq=P&oK-N6pyZOA%?5rxe923e^)+>jm?C8AXyD zUhI-B_lB`+MTMF2f!rV8;d=Oys$7b#yomtDP18&~W!(i>@H^Ex7F#l<4h*yd_rO-W z7v)y2^y>ZRlzv5%MUVu}mYx7~gW!sMkY!lg51*Qv=g!_Pc6Oy6Oy(0c>IhTcQ)-dt zhlC}!gt4*c>!#bhu3USW@QIAi)E_^0#P3#xVe;q~Z9d#|6*Xgk4gpL&<&&RVP@G!U zoGUKeWs&L--jyV%GoZcq1DmDw>oLG19Zw{$H?d72ChKYsw3Ny>(UB;SH2rg!A&{pb zF@D*b9=+-2AVQU6FsaNA;2UPsi&&&x;X7==5v-Xf!M1S^198_Io_9ZfD1<^@f9D{v zHy8Z|CQ*j*4?|lUpMM}>+&9k%LZ_Nkqp~ak@3kNn#=M=p*5d13NXt5P^ zV_}hP?6(m<1AC-rCpq$&APyyr(l^)C}Z zeg&@&B8MX4v-T!j*G&qjyCu}B9rAUdy>bb8`g)(y=yVqVv_P2V6)V{mB!FAx%^siY zwkOtlN>GEE1B*`$k8Km6cUarFt43&x<%YIozIo zfIXTU^ng9%hmtHO=!aMWr~!c>LMNIL3xqCm$w(e~U}D6$FE~3sZR@u zm3mEB`W(fM>+sD3P$tRf81oMb%3rzdtVm5n@5+*xn8=nqIGxPzuoB~h02P!{RmI!^ zHb>kC53*aDS#;{Ol$Lu^KkS*J;l!*xFy*5)$i9CuMs2v#HWNh#svHA0g18_rPNrR~ zzizr3C2?o;gR#C2v4d(Hli1p9ptnKgX31+@TR+wIO@Z{aai zE2kLJ5WQ3zEjEK6jEUJqLK-L`lZNQ(U^>`_HDbGZVwG*Caq{_tXJhB{T3AZu7t4 z%+m8rzjd&w?ntrW)aHDmt4wghS-0nzB0Ozl-PL*F^M{w6wXXYd0!4D2 zK^)e4FKB6LL!#{?aLIFTU=wN8x?BW>?Ylctq+NC0K9gC7zQ=_iwRmVQSC{NXCm1`nq?4_;~3*V1occb~}0b_1u~ft|AcKu*&^?+G(= z7YOv4n=^{X4kbXk4{9S3!+~Iuw*(HoxY~&JT;bM(Co(atFIvtP(dCkQ+j$wlX5IOj zt9F!V-mmKQItG!m*(!ffA#z7+g}S2u4T;nqtx6lio|IRhgEs0epLOazG!VZeKqwN( zR6sjDS*P=?0&1=9>7XUKW`9?Z76vjb5Z`TLa$1kfvA5~1$usEPIc#zPT6K21?#bY0 z7DPtPs8%S%Apj8B)q$7dR5?JL%)GoBQk=6iL?B|#oHjgW<0>sG!rdK5p$LHC)VWtX z)d0hBS=r~Yo%5Rq;E&QePADdD@2J)|7-NJZ2mnFdw7{H+7#NluMVM{sV1je7x_eN> zEJEa|Vz=!*VIDm;<^ntNF5?o>cGKV?SFvG(eMrpKtlS zIoVls)w)8HqAG6z>hPnvbb>{Nh8;wkPN2MLx0qB~$}w^Xf

3Ms7a7VTJ841y z_#;~}vvK+k2}f&NTThXvPqnO0bWx!n)Qq+09uW~WbR8hL%Hz9hYim>f0H8V=*BRir zV67Jq+u7N9i<-i*;KmE|AktSqHe8K$jAIG zIp=f>;Gs%g6-Z9Ax#?+rW~4GJ-sEgFm{{hrc?~zGD#AC=49?Z+~Bj zaBAgk{C!ZN|L9O|K7z(8q%)MXYX*+-=QgnD7{PfTGCAf!*bOxJ)KU%=JitJs5-a4CmGEqDTDE z)Z)_85vrX^PBtAN_HQhBjJwTwtCDU`3uCVISo1@*8OutpbRN>-Ma%{~GACx)&&i|pqFq4Hh$VPXRw=j{G(L${lR%JDx2V>`a5 zs^kiupV;#BdCY{0J*A-NT>cG(#}opru*V!kX!hBt0v$p5L-O;34^aWsAZbi=o_hcc z;-wpn>N)_kY(U=TD=RA@5z16^FcolFH$R^J^;sm+LPSd|zNX;qCLT5b_O{Nb!3;-F zN0;4u)NE{9CEWrgMMZR)eGUJEt+$NIYWv+>2I&q7X^<{SB?P1!qy(fJ32EtWkPc~(?&i+-ob&sSJMOq2&vEeXy;saN z=M!@|Z3`!rlA__WQ7|w_nR|e+0ULV*3W(MIlzA_!V_T!1uawF3GhNdaj~<8k;ks=U zw{7u}H|qQQteZn}IzBKUw~W+jM5Q1vFDQ6bowNzO!TkI@ovX97bnw`lz0jeBs>g`a zxMG2pB#($N|xneR^YYZZ6dlQ8mmZ4A>6xb~a52JjQp;I{g(MsHMH zruHQ&yX!qg_gOlbi&#FlGpM_N=c$nk3SxwfPfQ>Reg3PHwA$KQH3}T>Cf|n$P^>Nu z((1a)qe-atHUaA$_X02(L>oELF=8?@F_J|0AuR-!Z4i<i zVy+)9glllJjIDrs(R+6nooEcgC*Ig=T8=2yei!lwV;w-I?OoC4tCwL;B2cU&5X%78sHa2QCE7tQdaw_2 z5&|Jm_QYmD>m-*UR#_SRk?Sylf@3)HgDUUlaW$rt5j1M>t*~Ly=KyV|kfsn_CT$ z(?ikm`or^fAWpj}i%9`gJ_g7%$=|w-`|C^V`DxvgX?b~hp$k<&I`?F7uif^0Uhwkr z8W3w}XuPAw@dg~T{%FF?;HG&T6b$~yKOv7}#Q#pOQs9@RIb5?)2`9VD@wDUg^z`A3 zlNSZf!HigwLrHRSayMfnaN(zc`0V;^h6V-?Ncd_X@E$7(Zur1F;dAE-Krre=mE>y* z)GA)uDAOWJWl2XZ_C1-REk~Mqncdw)#mjq{9`iQ*$KOx4@4+gK-4+!v(e+f%uXDdV z1HACL`B;(VEFX}sId3Nlp8>H2X&X>}h*$}N2Vs7d!{yH~IZvKE0aR%kvPENwXQ~R@ z!s+rp-rgH$Lvidz^9u`Qvtp$@#Ux%{UM+kGVHBt&?l0PV6m$%4hyZjfd5|+UaDCe6 z!Gy>~73~}ohlht@SP()8ZikIW$ziYz0nCZtv!g|9c#)1D#@s7>0SQ-drBaB;`2?pw zNi(^}h2y}cC=Tbf+*Q`jXrJ2>dCAx5?`t zj*DVgZX&pVSQ8>$M;B!eZr9LaFL8D}hu$R{T(&!Zgotk5_5MmDZJ_Hu{;|KUIa!oD z^EwzA?Y=sYuKY)1?`+;)u4N|E?a-?LZvzB@PMEdOIR5z0I;oF{k$>f4j8QZD0CJ*sdxJiqZeVe#FHdGUCQi{`AZ_lp8= zuSJk$0Nsy^wFZB?HB%c8!|8tq2pJX~zE^)xsba5eAVQ^EFhuQ$bAUvDl6Q<41u5e9gD7*3H^~(Ow>EDgJHp?JTG-x6QFA@p+BC z9HIKt&HB-8LCQpcXeWPf{3!;F3WDeIv4HoFk0%7~2@@42K{oN_Jlw5uAIENs-YpFo zT`u;OUPCtXZO>k}kxZ@tmf^+XV&kv`{uh=rQ=_AZB>>MA*oHq6??t*#RYiN(mj;`M zG_v~gY5KZWW*a3YhrLesR}M+(q{&hfEFvPQ*c*t+{-#Z&+i>id^@%;zeoGu(j>bWN zw81zlzKe#l_?vPr^_#C0M!g|?XR8n(VPHQ3Np#>xesGeMn7Z)bM2z7V5f*Afx=!R! zT;vN?`wU_u|}E|Be-lmKCb$J1n(88YV|Z-=gVJmY&KGC zY-}`wXJWl^+OE4TkP^Ze0(3>Jr_?fkH%Ha0`6LhcA(o)>F zPOo~S?>&5?DJv`R?*Pb_z4VL}>{-O2du#vcH8w}Gs@1%Z4*}kz4O5!p(9qESN?RjN zC3gsSk`faCjX(~z=*4L*=PRL}C7c8?1_5#ZJ6ntzO&uZ%5AQ=li%5PGs67DKHZA-5 z-w>EuBfavLw-AA(UqXz*iW`rTg77@nT;M(miYX*=2^5HE?~I7K}*s$D*-`jo}(=74r*hw*)PnPg<@Vwk9k_civD7s=U0* zH05D(RDh~baVj-2&ubkq>ixiIR@-GcaU>~Cr7A>qiPa`(Wz61aX41*x!mu!;+}UHV z>$H#f!stm2+qH53{E*71laR%6W!<$z<8J%#zWDebPM>T#d)ivYLa#a7%yu852gxRy zfZc#FEfy2>Xk$%HUJJ?pj@v4sXpv!B>Zo@iudklFwX~jt)9M??!d2e3*U{W3eN$Er z7i^=iGB$RUo{*QHN^v>Cr4}cx0lT6F=~`m-QQzcahEs2TiinAH;bE z+v^KC#GjRwHtpB{##33460|6w#~u_xPkwbB#~1fqOo`=Pi~y z>dCHV>1&}cjzC+9r0EKL8*Udz{X8)MO3%oMKalQx_V(V%3}tDMMI=Z75R;H3chX74 zvZP!mL!B(`_z5Z;cpjy61E;-1-8jYzNk{r!teN=zq8Oq;J#@f8ZiT=T=LMg7&z+>RuiP<@>x z5g+DN^`1)&O-&x>17pqyTToZ&={@y8LdHTfzPdO~{foenJL$R;xm6TbA(w-+r1>F; z^U{V83n}^&GmTkCTwjKwqBA|u>4ty*W3#>*h1{c2y%qOrX4lQfwJV0w-{amgHTd>= z_Nlh_^=;4A*C*Tt4!vt_ZSCR?g%k_vzg+8py?XS2K8hZ1&Ihl3j}OqXNU@Sp|EI#yQhgqxpIy`|ncI2=JT$3*?B*7>j#x<~bq z=z8iVg5>W2k!;ZA&i(yMPd7F;rf*{vtjChm_%zVaUEE`p2jP?e!!1 zY)L2#h;DnZ?Gh7pr<(&e7?+g@+oW4hU?<=*NGoFuILJu%Y7P#LLm-r6e#}B`1#0BI zifs$1J$)aNJmGO-(e-pk=-$rWgElux2?*`4S)|UYR~3Ly&esRvGH3in9K{R*{yF_K z?`hZ_45CYGR4GY#{tsX^lv$G}x-j}oovdW0h%%dLSX+>BCx(*gHj0GJ!C2<*?k)lW zNjL+1;~cWwi$C6EsK15U;3U=H z+XJ4`jKAq7DP+0JOHS=JRGTOaz$q$juttPFcK#O~9e<<5__r^Zn`A%N9V69bqyB`r zTKvjUD<#2o`;7A#u(rE#*gF6!p{!~F{xG?W@g^n zO~7o@=gzIK)6qeC&riKlIKT{Kp99AWo;v_>7NNoTM*l?9nJFm644cpYm9`{u1f4l% ziyU**qN^WpVwL^KI*7jIXss!7gInywQvUax!cmMzk9MGv?iAk!Fyi!GOmeaZ6zx!A zmsw1o1-l|t;RyT*Xe~!a$3`Sx9?*)X4oN5G9GE6Ty7~nLV1N2+9f*=GUbf5KDYp@T z0fW9yO?g0e^&MAk6UFQgtiaG4jp7yM?crJTEemAlsG6LA{Sh0f4E&R#J+-ro%L}vL zQoT+fA5f3K{zXYa0eV)0_E0hi=}QsFfB_F~p^kWgazr8h% zjGLaRcfCEs6DAyTWhrBVR$2r4bfS*FMMMlCbS3YMFcd}``ArFemIu=%QE?cYc3W__ zUvj1ZHAF5F2pUOIpWK^|s8(Mk1+gG_F*+hn8zIG_va+78F8XUHkZfU_#UvBtvy7nYfY;#eta!YaEYdt&&fQ9j{9Y>NoBi8OvJQfA|6~Q$Idi z$O^n5RGM&v^-%&i7uGZsE1R(Ml`qNYX=tF(X@=Tw2I4079x@!s?Z7HS{8G}?l#106 zfK3!cz)@0C@+CO9Ag>qRRyZaY=;bZ#4g)5LuOc#VFy2rNEUg)f+9C&JM@MaAP=-_c zZSF8-W@mPnK`O)M3rKc5uXlFT{A<)E3dntYbvDl zs0j}bFNeAXq>#G>UDQ}YrRN}>zJ@eOr9hKRiX|K{U;izHXg|8!Yl<28?4Rcqpjn?+ zc|TDqm#t_=B?eqWi&#g{^53l&Ts>445cmZdwl>fYrv+WJ<9|RfjdRlyA+zx; zu031NO|1rnu7VmE8ScsbfqJLjKG$=}G%Yn%-?zG|YA_D5m!lVzt(W28jvI;pY5tO% zkC9oKtnK3%W-;$ZmoeH5O{+9jC7%%j4uYOhZ4XB3ivJ{IjN66sHa2^JRf`1T4HW4%aWgp} z)-^7Js?}P!E))4vsjP57=Kbo<<^!1Nazo?e=~XUXg6_b;KAJ)8DVAQ;wBjwq5E(K) zU+8SLb$$PSs$|ejNH+C{jf1nh^>3;|>@#zPlFK}HTBcP z4T~U=g}E1Am#GO)^*#tcxPOO|v;H2lK|P+qhX*U})UQ<@N6qAz>c6mA6ywa_1YrVb z%o)iw?HwGFK+QoO!ei6cf)2|U0-p@1gmWk*D5Y5N?LBb^&eT%us9tX+gz|vt+MHwc~bDT%5^o4`X=+1%R$OoDWvvFq{sPIhmZjdhT)M z%FoY_kkEZ20xuXYahc(p^wIa(ia7{52NxWmzh@!=Qp!34oyb5gV#aWoDPNrN3t^2g z!3h;8H^_MBn(#ZeQ2}7i=<+jy!Df-1q!jvhY~3XfyXyFD0{=K4cz|&TzIoBR96&n= zDTZe;l+T{ob%awQ1RiB&_NV%u`T3hbfJ5r(w9xvX5$VpCfB>xO?$%a7I%-R}fC?*N zZwtiV|D+7E&rpu0rlvIwM1Z!X*tUvEAWVOEIb0JPk&+u-iB9!2YyCZv2kHqcAG=ta zdoOeE+a*?co&kMPPsf9|4iUGy|8#|Y&5)~C#VZoN&9zV%b zm;n723u)g}s?RxBM)<6-WO&FjOGsRIUYVF=S6)<#%gD@vf|h%`X6U}{i?qe!^ikdP#+=SS1)AmlxgWeBYU7bq@N3lX~}pc7ER?=FZlxcfyh=@+x+J8t{A15wOh2fB5M5e#k?*D4w0rdK!E$8m?QtI>ab%o^Sw`NA~PGCIUu`PmX` z(qiQY_a|McDuvC1dVZ$1Axkc4y*bi^OrV_e!;)nAgTI~lc@Ra|uD|IU8g5NiY}X#o znElQhCqDHCB~EV=@8sm9MwjE1H?koaFYk0~Yinnx#Ci688|wi$j@M`QCN1k3li2i{ z?{ncJrnDOi)dL$g%=NLyT|d3OFKnKei89^=rNgm$pymp>%UFf&!NATxKzjakt~I+F zw}HJ!*NXGc&kh;kReBlDP-;w?BgIcPqsj+riqQuwr@wy>evaIpBiM*`Jo=R<*njH|(j8_-K`iDj`TQzudjD{J6z=i0oGgE1;votiSFi}~ zF2`HK0v>=F$Drvc-ZELq`JcpP`}n2x3+NaFA?aH4qhIysj|Bz9Y^rn`UB-URo%gY5L0JZ#BvzN&^U>90zZwT99o&CvA* zitXA2jFn7UR-L}(FB^3G>2)o2-2Of+n@>qAP_A0ues^H9tK!SvOpS&8y1KuA?#VHP zWt7v&6#49CA!i^Qpwt;Arlu|hg%0>;^%TIgfsDn_$KJ<7eumKhsIQyP)@co!3}?w% zTNiIJwzaf$^zyr3y!_|$ias_slB^shpR+Kr{}N;VsuAnJ*t6~0T?iEa`~A4@-KfPe zRcKQihr5-I1}C}GTa5Z-a}dfI;!+S=VzkubytXPC+n>4CoOxAn z)Zg|Lw)ut475lWML~h$3YCT&(C}6i$ILSmFD=E(H(Q4H4^Aer>I6ia(QdFt7FJKGv zM-yNr@<&zVMh?X#XXLrEg>{Fb4t=91$$_BJgX%4igum{?{<7mD7MJ4|kJ)bq)@tQ! zMQ`Njbx00VZ6V>xxcq+WgIp8jKnGpwQb{~<1$O~!QWepA#Z+mvzP>&$uP`OWQ%FNj zMo&RAa0+04n`g|%ZZg=MhFqg6r{1?`*ph4=#k;?>N)cT=`bbFqj{Q~%s{Da!O>NFF zLDYTuQeU~wmzW3L)MIg{amdcK$9zk_6G{`BJ!vF0&s;< zHvacvv_R*@6;AKDe6qR_Zx<1u=^oab^f)kV`rC2B&E zN`cvx6?2*~l$M0Yi6_=ClEbFyXL_~)>&^V5>i3z>W2J!)=KbIk%-^p}S} zPhzh*NxLu21a{)dK6~=t^ER;&I617p-I%JNq7tCn%ej z1qHl68H`k?`hq|>098{DnkAm6Pd(vN5zv_IpT(*l+ozhdDzf2kw%4+RlqN%8G|qch7Q^JP@|B^ffjihV;knfdi{6k;*8<`WzMzioZVIUz2xOPViAb(oLUGac;&M=VRD?6iA#M;hX*X(?{_^4YD($CA8c5$;|=JI`JQJMd^uIoB|rg%FF z-v|2nUO(nk3)_;f|0<&rS3SF&*?mJ!j0`C*n8Cn*v+MPc&q zYXGypIVBD@4#WUNS_`SExhxZ7a8oaN55*aF1SJTM{RcO|h;3T_T!@aKcRx|JFq9R z00j<$xEL8K@JwccMhg}m;Bj=^VE91n$zd@fIpyac+*WalIjI&(B(w>4$vH8mI#`zC$Vuz=F7Dh-TfeBF+0z@ zo8)#_sC_-NTr6VaFj2PRg(`FOd^n7>m6D&o9`Y6b<0)HM_x%twKzp?@Q_IcEo5=6; z???!cSvPygd~qOj-s!yPLasM@JXLyv-p$;LB5uP=P~#NUM)S@tLmm@V|Av&cuS9?Z zLu6FclctCFn9!u`I_XO;ZamLq;edQ<_LjTmpR4lVOZKimhdA%IV-$vPXPj48hTL-a zNf%dENO+y0{^EZ>l+gpZ2juSNAiV(LI3*<|0s#F7lK&D!er1&2#=gyU_X@Nhv5{C7 zv=tHk-DqJ}tL=rEm^ySBVVhQh5{@EKGD2{v@)=uK5R`;|*f3dOc=4sEJZpVo2WFsWPP=sh zKZM$gy6r;;xtb3|J&7GXWJN3*db1rUap)K%weFc8of&VpcszNazDQ7f>e~Hjjv^*D zP?ENCVEj(U9m>iPap5lw-@j2%=gt}0Y?b|8!}SVJX!*E6P`oAh83uXfWex zvp6~va~*5>NXmCIzu?jKk@WNZ-Jd7cod@9e%l3_+p-mJJc3 zJ~rPHM#e}_v?bv#LHSZ6gLK<-2_!84_m3^N>T{@azx^sBy4E!Rbn{d5E&n_J@;L%} zpHfd?UuX`?$Vpf~Er$PY{*P26Ef!$)YW5_q&7X_F!Fwde5UTng9Y%peBO%RT{ET1= zDP{5h_usxRiVTfNylo8o|2}K6Sy1Q;u2%k3vnQ7S9ZXaT9H1e=*Y?rIXU5LxbK%%L zei;;apR^OmYx5TkaKMaa670RtdT-ULg$f6}An$A>|6NDeSy>0afzp!Ni&C>#7K&SD z`5A8IIb#@fhm4)yA6>;zPq)~%z9tpze}g0t7mPH{+M%L@kRj@~{Q^O#-OCt3G4T5g zt$=Qi=qHbF2jjozt!&=+1izNv%h(nAd?f11*bO`|tE4qO)9O) zpt;D62#rRhu5SazV0eg#h*(%yu(h#SeT4X=7Axf0b{=gxgV`1}D!#i%Cw<>1qtxI; zmU3mez$yA@p`xhw2P9!q7RXrLjo`dHJKo-2*q@clEiF9&^#T413r$GXK^`$!^T8I1 z_h6@PRR@3*KkN5T z>jmy7g(`(JU_Jn?)T$KXo8lzuAJ=z!$)+xMVa8kytX*aB-%wl+2TRGCM|um{?%(a{;- zzas%Os8v-8Xn%#y$xZAtlP*N;h;|=i*NQ@8aQ#+*Jw^bE3LT&OJP~w-Abs|jZGVLhYyS0S_Sr3b?lAjj% z;ah1{6y^kN8cGjWv2T}%N+qa{Ia#=ZL<%uH>};mPG#_eT+hd}7f_x`0IWkhH&=V-o zZgqfpQzaFm6YI30q@vf~j>-8#d|_CmV7mF3)v=4d(`qh*j7o}Oqy8Cp{mXb*z@GTx zZ{Fmq6edAZ2h9#Ruu#Lo!b<9X{9q?Wh2bo^k@An|S?|hnxTM>@AIMJApc3V5I=RO- zO$`Lq)P8?M{jizSgYE(uB7eaOaL5KJ_;HkfJ_NVc7sLa`iz_N{6Q2V}iBgx7rKgwj z{d+S&0mC4|NiMlG3a6qiu(!G0GK=cbKr8I_=kL#M0mNcA>}+cj zJplK^X6LM^{C0CAkXrhe&CNn*IEcynpm*g*RYyUTWjGEtvpI@Uf*M~PrOOY$zYufx zW&!%=H%EcH{}0ez(wX}T6ZJg!y&hs&{HBslpm7Ep&WFcW`di6f|pV8vjI^F1nv9(=T@m+ zF44f>g!nS3?D_0-0XfUuN}{&F`Pt3%ry0qz>aR~^kRv5kRXJDweRQXjNz(q$4e?_wF6EAOnSXZ_L}Xg=a}1s3i6v^(bN`&p)`F}9)?o{gF#W$gq%NWA|G9N8hQMLKO>t9QiWa1VrIk$K- z((q4T0De5kM#@r2DV0TwpbJ+E01BOAFa8z)4V9XkoBMMSQPEb4!LMY<@ry{>Ky?@> zH=yX=tbQ5rQP5!-{J=mHj*K${f)te>w82N%!N5fcPm)N(=&Y<+F}X})@08!q>;EtKt?e~vHa6V|0fkKbNo z9^L$en7W|*$jBIOBYA2+AiBCx;XHAg`}F%U0(DW-O9A3l+vCz+Q1F%;Vq>zj0a##x zp^-DIuG^6hzS_3e8>2}dGL?NEf?taJE@aU>i7JtDmlkLec64Urk)n+@6=aaiC^;*; z8unY>HF`kp^L-k)m-Mf6G2I0t>3Ecy}jtj$Z;Ugy8V!c6w&{T ziiXB-3$Y(#-|4-IMCJ0(%@2s2T7`2O{cW6OHmlG5rem!H`=f~h?>wEdRvk&Ux02OG z&FLON6gWbsOBX9G>8DC=7O%v&0L62;sQ*K_UTtyAvNt@!u^acwIROJqVM-1tvTxAt zTWZy=>1G*n5ZHqtUgaPO)&(f%4YS8vpaOQ%o^@oWmXDdvU$A=yAmUzOBP`)poWM%} z$U%G7IXheb)!&B=d|QPzdtZ2Jp`(bGGy?lUytp^!C;!Z#T{>{xG#{Ke`CTe+%XFXF z_(RD`f?QIK=doxg9eFbX*+2-XYU+IvlePD5us3Ap*~95)9N`+V%Y-ggknEYAy1fP$ z2^UTc1>K0scKk;mR=UjA&6tY&9kc&_G&8|PyTT+0;sRw!`pLW0faf3Hu*~X5zHEQt z;&$P(Q$mm2HngXg`;~jpfQum7bvgKE1R@IBAgnTW!ytUltyNdDe1T6)JPq9n{^>bo z$A;K?JP3fbFH5_5YSyVxHHQgD?t@NhgxLudMiVe``D4Fy!Rc~sw9WG?A!m?pE0E=* z=ePOwRGNSXa>eu;>vUVUlr`5#2fvdmeR-<%it9D=ZL#>2V=jPwT_ltB2P4Gm$G7LS zR=V`5;-n12#ZRx3_D^)S-fVGhb8_xq%_i(8DP#wDqBuQQLpVxaSLlgrjqZ? zL*Hb>WHKrwpC0}yqp-Hj%I$p~T=lEc zN6LyAG!r6dWzf)~`~kxw941Y%t?iLq+(0yzchKbH{4S1IWn^u%)46o+ZHx_$y*)j5 zP5F`2YM<)fjKY@*EIn~XY8D|UdpZ}3ze*`9_9`e9ksj})z8ut}k58KepDmHEhlM}i z)M~F7#Q1B=kn)@MXGwVD*VX3E9~Gz$Na>8c*!%`A%~nPt*biG|9!O|AWa=2nk5s09 z#Y@VQN>kzLA?A>+BK#1q@r>-BZ)8Ke+XQ@2U!M#OcKHXLPkfEx-xTR|O+Vs8hm)xn zMFO1-C*o+<(&xY5$e5(l@oA{5Hp;Z?O-tl(ktjIulp4?W)a1vMj3$a}JFpqt46|Ets^VOan#eC+?20&aXL_Jntu10DNh*#c_Nb{PZ=_l7A?*aAl0zVk58CpB zR3x=<9&6&ZII>7847#}QWYJuy#G;KeW4kf5; z>HYrttu)<3>b`9U?sXrsZfb`&OQIAu{0~&m!5FPhb;%iFDy%(ZeZ3C-X}UjcKPc~D zK9;fU=gQ=ao{|eLr6IL;mZU~{SrLjXZ3xg(vjlXjGdYsxzJ-d6jC`Phf{OaTvEwQd zVBSP$no#gq=Q6gXNh2#>H(yB3)Do4mKtHD_zo0M}=3Y&ae_XE^8Yzom$FZXFR7;C_ zn8llB=vOI;C+hK|O>ytcg*x=I&UGx>+3IDMEbSx=1^20~3&L+|j*2oSPbJVb`UG#U z;IK7NUOCAP0|5Uz51VMp^(_^=kB^VfeTEy|csFj$z53*HO&k%^^`MMAleb9Phh{(=N7Icxq9&zQvhZh?d7+AalR|g^5T~$JzR_Nc+cb}o3lJrkCraMpf2vt$~9igTP z;}5v_$nlMzXPCywR8S`dEa(#Rl;wKDu@#3iaQ0G5R4XEKnOh^1{b)FeU=s{T^?lZV*AOws)9uyj#OI?q8o6g%siYqK6AgUt;H-}bvGNy0;cwUCe}1_l_&Wz@d@5a}@%K(&qa=E^oxGjF zU$Vxj&Y63-4;d~#u&wP$=vKXc7`CgUr>AG^Xt&&3p~3&}rX&zd>F?|7gC;^HC4(5B z?mP)O2Fxnr9I7)Ev)lhnY1k*9vmOj_rImya#-7z_na{>IxJ9vyCUoE95+P5L`-(;q zMdz6I|8UTGMj&~||C0&-&vO=*aUxcZ#1F(Bs^+3;Dj94{*G1b{NI0ZeQ(>Sf!8mul z*ljCEKZ!=d`7Xa^R#+=y0pc_lH8uRae?RixJ)EnH-Lw74JCpeHMBlZG1cPdLQWrx# zhDRMH#|=aqGm6L{7U=QTY-SyPiCho3H(z)7T~{YOQ_snGfUh7AkN7=4xn#IUh1KYh zAG3dp;g&1mr#tOPj*HN%i|hy)G`>y0Jf$pHy#CBx3r!TiJ5)P6^lp@22(};k3M$84(M>*MxB2Q?)&)2Q_HB`;35vg9 zXEjJ<`5fr>)$Nna2q)Hc63C!Cf#*|J&cc z7<g5+Pe5=dHuC_ZbAJev*Ttv-u^!hqAfvd9y3|iUekW6Dfq&o@e9G4H3jI=f~UE zgObc)Du`64*0+jd@SJ7v`Jnfy2Hcmlz0!{R!az`3_|9RbK%jVhN$NN7;?b%|6*nHR6 z@@qOT7e%j+tbHC}FPU?6@&3eK-Q*_op2*Cy)<7DOu2h9WWPB4JXMFL~^s{i;j)TVS z>ZHNQ(F|{#%^#LsUevHZ0$`nt+Pqk{c^#!5*7op5WyW(~ELX~vn8dK_Vhebe#+_|# zitF;?W_6v}IyWl|GAvC;z2U!0HkAUTIxKhq(L&QNF#Uny6dHMfd1Yy7sl3qR2I7@} znaTalUc+EGPeobq#Meyk-Lw<(3c@U2C|tG>d-ry!LxMKVh)C=`HXS}}6$li>(y{Bt zx2wuhr%29wrW>`=J&|OzTXpi36w_G?61q-I%ubB(?*4u*(fkHZ4$267g>PAVfl+AR zAHx}QcmM3L=8a)Iu{jz;DI?-dVTxIA+(?qZ*3i|~`J)`wB=gp)G?;~P@@d_DDGf~Z zn&d2-f;?3jld05mnzXmyvbtA3i$*^o>n!Go!>d)5L+4Cj*>q?KCXz173SPRYZJPM1 z>5&n&;@!kP&G8M(A-uadd%W)U&)*3Pb);xo7zfo%2~@V>(DA0jr`f(FgB9R47-JDa zWTCxJjIA2WZiF7+)0(W+jsy}bv>P8+be|E#P+Dz{A^q{zK1j$WCm^A>qH*tB+Bg&{ zGAg!I2`KrVLy#6`I$I}i?9CwlmWi4o&^v7mXCGd6#}=|2QR@5gk`>xbU^*TQP?H#6D&PMKT?j%SUVuhP zSxwEqz9tlU!y!UnUz%8+L)n{0n+4McWu`~9U9?L5Q zNl;t%xT8G9dge?ihjL3;sf;G~BW|3$8*Rx!dr(b}5xX)Hg=+=Jg{=y^-wPUv)L}To zYGcc|@=gT)J>#;1M}q6MbM4rIOa0J7ygR>XoK8w-G9HBx`0MsF8Nl+;H%}1Pb|<$~ z5bF4|xE^piWwp2Tn1;^4v`)Ry_zg{XjY__P*a(NKuy7O&c?e;WrW;x6Vm~Y#qrsT4 z6${AaQt}D8HaX{m&rYU}7sek>=u`4q||-JwYnyS_Qc#PYAKvt(Xaicdjr8YgD6M>Y0zkXHA7% zjx6-*Wnx-+vk1Tj7R<)JGdn`i=Y>4J#%K8VlM)g)7S${(RWXY=h%wS;&MH z1re_^1h5#_J>c~4aBx1?H~#NODnwG)iI}m2>LxumP%0A3kZ*C_`hrP*v*||DH#f=4 z@kj5JYo^M{|3F6}2(uK)Q^3ao6#1u*RUf(=BBB>Dt86RxNQt;^H_no|`5M*zJ?!Et zCC;t6vDb2#rjJKhI6mK`LRyJ=&~UvzxS4d}vE)@>dMfr4ng~MQ$PQmR!7RWv&j`+(Fv+h;IsyOGw=D|L0e|9cV2d{1r|Z z?0x8Pa@box;YuMSA_6VjqFn0=3%7x4*%`%WyLkgq63pQgJ+69Sf-^+h9ip4LujGRh zoK$L!Q(4rTT;o27Cm+VJ&oL$K5*;&@UDf~GaS z-4|4-PYi`hzL7stHRg@Wi~ryI|FuBdnsa0xn6FOeUHcXmnwWSQQ-9%eA05@ST*J1QOwAT1jKT<9-492A6%cmJC^p{81FcHkKaw`aBL3k5fx;3 zg}Hx$quXYJ0Pnxhd!xvQ;ghL5+T`Gj?_o9vgYXcB*Eh^!#(2`4CG~5D1{&04k8DlT zTIa##=Mq`=o%w%~8;Tu!8Y7~(jPN8Cc~mD2y3#l7rQ`dX?TPTFP7Q_!uS(6hK-d?( zHIB`tn1WHBVVj$qix4QDy#i}Gv9R0;Dta9BPz=0z!`M+3Wx6n%LwS%^|GwgH?AoL= zCQo}yEFqhrcv6&>^`Ph>j(*DO-xK7iV_)9+q#6?1FglVGx!^L~L>@}yOp>ASco<)8 zNV_P^X-knHp@kIBU75#oEkly*Qn2Q`aHd+dqS-2wzpcp>gHUP@S?qx5(VmO=D)N8x zI)PgG+bZvu9+wZS?s%o^Wa$Q!Fou@eH;?R>3$#899NJ>`MK91k<_bW?^rpLPmTf(_ zulz&uZROzntf23&3dgeWE@b=VtX`d&k3CvNu*D^0OcFUUt%Ss?BkOCOnd|4Zlid3U zAB?2tJzvvKkv%z@v>Zy>fleza%DKBQi2t{C_0b}CrOUb{xO~Wi25D74iCsxWC_T8F z#u}WaNz_(M`CXuM%D6o0p?1t({RsA*)aA?8{`pI*4XqrcbY|PVvEKxWP|^O{D^P~w z^QA{>b!07iw4=?b`ES%x$EuBCegf^Wb-!v%b;T^giz%0nRPGw~S2D8r06`SH%D3aCxgARD zt-cr6VUY?=agh&@fB8tX_O=)PnW=CYCwiWj`n;_6^BJ6;tG8nCk&~^y3vWJO6W?P9dXpSH-eNpj zx`E3v#~^LH{AE%9;*)!w#+q@)Ny%g?#tKtjl(b_6SDaJlM~ok)iX4b`oFxcjk8V3R z-02Jj%;PBWQsM+TUeGs{IYIfZ%n9MHw40lovi4OG-%mx_ZYxb(LbT6-gATk!H0f^F zE8Y<$`R@h#Tc6+>)MzoDKOcsvxD^&P<zoBYMg1agJ6ywl4cy)yjniD~;8jAd1B>Gt1=*;D#tzb6xrJO%qGL?1A2@i=D^&RIM z?>mbVtPO9bcsK)1&P!0NiC1JV6knc7GhNA%$IbKL;$PdVtLaQ&+DW)<`alI>lh_eT zO+Ssrj&nKhUBbFJt3`4|j&552DP%Jwkw#XLGe1TpPxaM#tH=YMN``8K7o1x5AxIB$Grh zx2w4xp!4kj(56(g1WgAKcP++FhcY#}6pSQkt-u*;!4KPAnSNdq*^;7Rd959_7a-gm z*xCiD{@3W}yKy(-=8&EdJP2r^V`O}c;)5;RK>E-5#~y0-Kr}By;pa5Y8n*c#EBF0k zAMTOmIKsY2F?DP|<7ih4Z;#>oKhMw^H5&QfJualr|9CTT{+>%d4T-U^)6#X^8Vu=i`;*Jt$?NhiUr|iSsI1p?x?RmU@Th9aJdf zqhn)@*X}+EfipA(8qbt~f8aPypmhe(%c7Ti8~Qdaz$QYwd%G10k>gwVCb7-rl*CHw z9!#&Nz5fa;IEroar;f;vPd@Vgtp{2Lm_0PPYb4a4+3XlYz6_gCs@OjNHR&kJ8C4~g zWp`+9g8BFb&3js|t`tNP*`b6RD57%RAf1!L8DDqna_O55t67;qqdPn$mr;wrEV~$=Hvk7lnLl0of1|6g+13a2=+;w=57w@G=kY+5L)Xy z569WSl?r26i0b(#F8-~Nd28gVrBv#?%k6IU<(pKU1bCA#L4wI;u=E1?W|$Rwd6_ji`?JLoY{4h;bFyneiDKnP-hD;A^Mz?l75sTWx++9xI1WD z=?d+*K_#TWsC`*2Rl3lmubXgH8J_2Mg!pR`8%V8;zCklk<>F(L$tvF^^+zT{%TFl_ z%E2_$>{h?jSbap}iDT(1rl#5+kV@mF;o-Pm8V>LkEPG+!@(d#BSY<^l@ZXmWh+)VD zCbAtx{lMpsMh+(oxfv!V#Ehz4<7Cb4U;&!rfsY4zhaf?IMRh}CUar{!*eufpNw|mR z+k)auW&K^ms&Jxk?sWPgVVE)WrUH+M{t>PH=8l;SL49u?8=L@7%aZCk71nXuF0cI` z>Pm55R!PY8e9;@nQzfPJR~e-$i^}@>htk(U$o-CezpHa=k&NZ?d#Fcw*IX1cqo@>< z#tGTcFDD0Wm=-ekCKMl@H%NcWs!}MjH9J{Zoi*$N)J5>i5lWO4{SgtH{(Wl&#HBe$ zo4=un1hmWZK8kzsegY}(MjktOYV=-!o$<#gG-!n=-;oYC7c@!y8#=^Ra+ifQBr|`> ztxu%uLFv!%n|0m7zE14Db4aVesS)yOt`ZyL<@}Xq%BI{M`EnzaiJ}QEuZ&P!dsef< zwyJB1BJ1EsJND_n@Z1?9mn9ELXH&SHlT?L8Sy(wPV>lBY|sn@Rt`>Dz^zwc zfxNu9Fg5FWp#^sVB#!MPZj>-BW5GtO8Y@RUo6Rb0vo-7zGo?+X7eSxYvAbAv2RGL< zMpk?1#$TkMoeg&#hy)OQWdwl3)NQh`P4+J~U>UDWB*|#D-Y3X>jnK4zRZw7GxP3qS zgT$fm`3!x*LbD3e6Y1E`GnO@Pm*js>01;5z@FYZC#^hnE%+utBx9`NWF)~>T0GJQC zxudZf3akrT?2GkM`N;>WN=m{mIfxF3)yIZvYU3cAdo#KMMz`aW>D^G|qhhyn2fb3c zG!c$3{0bqwM|hQ`&=E-$`JZUR!In?<3$hIE%`oYPNMetI*ICTmKl1YFzr`rEmwh(D z$cqL7#ead0coe5hT22+qjoxE&YnAmaB?&AidrlX(=jR2JEm%^wQ2{~;%`sEoCj*Yf zFZ`pIb;Ki?m+^I;f%nP^l(Z|Ld_A~Vwf0Tkw52kkAAsN5O)-q?tgunr7fZI}>!l7% zs?Kn$g#>l7iT| zZv9j7FilBj%9?N_3hJJJS72FE_ZiO93((^vc)ef7rAF~&WH0L$Z9Mx)uZen44MXwX zhEI30qlj}x2AoU6vJ-=`B@0s41=X4Q-*2OOBq*?mni+eIaHn!P5G<@$1}2w(_E=UD zb)RCK(xMYH$07{x8^?Y=oPQhj5HAQ@L;jJ9`#P+)O^ha9jw&ang|?F~0tKZ%5BdLI zL$VDmNWXq1P+5H?A9VKPmS_+``q0|d-xL9=u~*<5FCK$xWaK1p7*y<`l86fwT!@oV z5MYo1z4*eY8l3&bP?E)lGs}lmO@7nEdX29cPoI$)`2BI*t&wTd(_O)V47;mF5e+(m znBs;Jx>|+nIxG^6EPeeS+f$vh4li}TYFNgM0em)+pzW`}+$q6B8r%P}Xh*c<>~W;r ztl=)NS9$7lj+~9LwNlC_aZ{Gs8{RNKLvf>dnG0sL4B~a!l{_Zj@q-irM^5~3>AYnL zh_U_O_vpU!)U)lA2_03mvWn%KPd-66vI#Y~3amcBETV$9025$+(tUkT zVyXD&L8|6p?+#*S)chH$=?s_l=u-YOfexxNnMGMzL$t9^75`>_X79;~X|){5e&8B4 zb1+ngY;a;US|GM4ZF2I>xIC4ZP&$%IZfSg@TDRq%=V0`iXpr4fGO0`k!_WVRsILsD za_ySlBBGQa9fAVVA>Ab)(nvQ_B8?y&(%s#iN=r)#(j}oFB_W8kq<{k7+UK0-d-=a_ z?zm!IGqYy&nwSIJ184^(e?cuOJP0D&%!{W&pE*I|K{vVyhFb1Qc_(jDpih_#vL zZ|z6)Ng!-v%J>bslw(VqqLmD_K1A5G`gE5C=gZO4Trcn?P67&y*@v8z{N{H$+lbSy zw3-6R{eXUEXqH*NkwJP|OHto3 z^G|sN?lLBo(80#%ehkgm^jhH9bwlzRD!8ND`9a-QovCCzys)`wa6ikR@g^Aselu0n zqDLp*U}DDWAFx4qx|ThHB3C=!AJ|;FEpvDiKpsklpM2||^Wv~pgc*&`cnIZ%C$|G# z-0Pd)>Gxu?n)uBBUVgRj?-*cyg7@NYHupLdqp+Xo_v<^0xT?Ke-tC_#kYX(4sv@3} z;_7LprX5P2Tx?~p{?$i)wa;_FS%Nl?m9G2P`OSTf`g6{WpAkA`vcGpCvZ`sf)t*=n z06tD-8f-DJ)!Y$hoD3}ZM@d-b&2M`hl&2niSJcG$o{-B^3g|%cgQ`+GUKpiball=m z5$~RvN@!C68u%?RmOI32|0)x=og)jkU&_H=LMBbwBW+&!+v8qs^%;%>fz`=yYZIM# zF9(Y0@hw~%dO^YT6D^3z+ft>-o7{j_V#(cvaLz;XistykA;5t74jY1rb|L_CezWOCx21XQ;X8%c;bbB?4X-EL ze=Ha!Xb;T>QRE$PB9$Ko{)A@#7JMJ)`6h=-O}_B_pSH(=-~bxkX(UePMh{Fww01IghoeaY0?>&VJjt6ns|(seikB$eAbjWi6S z(QDDYOd@%Z%j0+B{awY(frO+kspts{X-Si{G}h%Q&F3HFz72p9KC}aw?yz<2p;eYK zZlPaobZtikd0(JdL`Xg?!bvG}ek;U3}|0W}(e(a+i{^XO(Y)@@A7&AFTlWHM0BJ zH?K7Taxf;@tYu_Et*3s013)q03u+W%2@>1u9yik(^Hr?@6mXx-8hSDGo8v)gf3&|(b-WEmE9&Y5 zuzavX*Mvzaczo+MzW_su)oge#ls`~_NJr=gGNrjFzN>e_XqHGB-O15cd!e~*d55K@ zm{p4)QI5Mfr-Q{W!ua~#JKDKJ#npK!wbLBk0_<5)7Dt2Xy=zdlZ1dNlOdaa}to?fn z>{crEKTVb>!i=Dr3z$fUAsp!#|WHpJn(KW=_ zt_)AGr2EtaV<7u#;ugGzpo%f?$(MV*KU=;$Uf~t}%#vKd?cE*Cat#gVrM^*K`(4@0 zI8|@iygOaNiYDA-q$}I7t~WmMvTTeqmaYo=xI7kd4Xh8=Ry5ii*D%@~3gADkf656_ z_+`iX{H zEN%imSAQ16romC6>iugCNeKyHPp_%!L1Z8tc_}zM)$$lNA#jFD6XW*nhSx8=vtxTA zh(FpPr=oaEtg0GcA7D6&8(!I%C2H)?tv}M-mz~MliT!-;`N~Q$2d~q={rV^-%jXo< z<($kB?YO}v7IhEPWE_#A3f$r#!aXq(u{RY8;UD}{)N&}|2Al%+)S;JF5=TVJY76{F zy?=|C`TxNqBa^$wC8siS)g%dI_R;v0dREE}dTc53Bbyx$ zq6;|q&60ZF7NSWH5x?O`_C36gS-d7b-@TMCQsn=j{#)%wed)aTxY1S@(N}Ar5TG!Y zja0U$6BEmE(RbaQ-yk9!?eEW3rg{J_e%ClV4@v${eQ*Z;Ds{4-vV?9<{ zcNjm~X&n}VRSXUDH&pG|&!;{MSmpH`-Adl54YU%GCa_ri;4e#x7t4R4l;^`nFQwnN zk9f7gh^>~Hq3!-dVv6aiw2Hz;1eK^Y3;YA`^dTptQwjz#0m0Gcmxz0Qf82{tT{?rudbmDVmI(G*00DJdpI8V zWVq`^(~ivEJqhDt4c^~aE7z{^=dYv?bc&UEu&cTb`842yImY%IR>(-Z1A27MJRRUI z9&aB)yVcxfa#It(;A9v0K-h$_s%vUe3b|kvqah(2R@&xbJEg=PCyb2AvsS?@&-&2* zzt7S2YlDHK7u*d~Xr4OKf$>bD_2Kz-O!McjzE=AN?y)_u!hR@_*x71GrGP)}z+l?K z+N`og0MtCyL#CR8;T?Yp&{=GUQ+|m)d1Q3$0WIx#`xeN6=Syd8cry&GAAt!+ey7pP z3qeUb?jh!?R*7wV0)o%^uF&-n4QWBJ;DX%(QN+lhJb9Efr%`_q_>SkabMf&JPyKV- zysC_ol?|}*$lj$_{+S$xa5T{!hQP1Mem>|Oz8le|%97YDxN@7_qE zY~kPVb@O*k2V+X0{x~!(lOomEsd+k}ZXpF{;K#PyWQ_GnR2tK_2$F#Hb+?tH{bpr*{H=jY!x)Y2*b_{ulZ=r&BgW!%^!eGnmbJ2d(t-``W z-0xsMEm>h#0~k+lR7ti+IIlglu73lR2Z=sR6v{%{pF3pK3OR2&E#{n_okfvwP%|@Q zqTNJ(ml5i5lH04g3ed`xNjjL=vSD(G8$4ewaF{uOuq>6U8KV8>j0O?^i=Ke}IXgSYqVI~{2H?kfac<4=44UUQ5;3E|!1^PY=fyzJ z5s%@gf9P~^Q>Vmzs~{(dP#8lc;oVFkYsD4J2^KQ=3VPh4ft|5J^Ekrs4>`Sm?l6%` zll=;Y<%J+p`Q@Jz+ga~6hws628isGwFSnjuRqGKb76{}5wvb_%F#X@vv_qoJW8 zd*s16A4Cp!Z}%mCWWI(3336qB=*X8;(*`{IZnr_85NCzOMXh5V75<$m>(@_gORve0 z;zJD*8(11qiG2HVRo}dK+K`x^M#d~tY*me0Kxm7S=aPNfaq>|1ZN$F}k#B?=jH-9A zmWQ}0wR;w;mG&8NSISm$SWmy9re|IMBU=2ZDY#BW>$LIUs`zDp#pqW+Ru9gBP)_>O?f)9EExE(udCh3m`xMcxD22ShQO|xu2 zN9mT(1j~ji3F{9!3Ar~CG2-ukyQ7U@8wGCE0*6;r^qXRaxC%j=Y|a?$jc?yFA?Y;a z_Jd?jnAbEX83|e(1aGL}HOtjbjf9%X@+g6;ckV=nsE7FjOjDe?Q0`uhP**KBX&Q;3 zr49)GYj1Dw4lro2Nz>yq1Lu)C=1XXYB_bpQd<=nHxwNUmwD{&=GpQg&X;dmP!ceQ^ z!OV_q)-#|U=dVC%mZ;yDoPKb~{#QH#4sd~Pec?Bj&ao(kQ@=e~+QrssabewwLi&WV zIlsOCAFo>}8KmCAG$1E(Osha;+DQpNlXTL#^@KT9Ws-8|qr+wXM@j$4l+7^)P5 z+@2U+YZ8SHg8VFPtv`8-w_UcsK58riI2t_Wkjw$H06PwCqsfvGkuGp}CAe6gb~Ceb zOPM%Ak@@8t{qVe7x`9$X0dB4{j*QlK%*+K@U%?~WUHlbf)%_fyK*9iSZv4pG zLzc2o%punko0rFy%CaR!gF&2OI`S#Kud;IZ9rTJn)ZN}!5?I1>BcYG)2nlo@QSFcD zIz>Hh1=HK^Fbe*cxi7fa0WyUJ`93SOQBHy(B9ecGa%pMg!9ikezBX(FW*Dc=9-pk8 zlf|Dm@S3WUsO4IPB(p`Wr)w7IzwJy#Wu*;pzmE6q#!z&B)e#zNFMTPT*?79cj*pKC zZ=q(7=R=dU)8^+e?zb=j!I9h#x_9qaBYcTfslNQR5y(Qh(|Ut4QK^#vm|Vt4%h|6g zu5W7rLsLz&mn8ozRmo#y=AWF;Q+m>)rt{#*pcTU|;!;BNoBGCR6HfQSvxF;1CquDo|#>W9m}9GZUJApx+B zL6NzZM&4HiV_RC;^7>@5_~#Z0M)6T;BR>donoVkR+s_Z-sVFV|3CLujUZdFn-v6#j zxu!ZIH?%>J-17=kgd5Jzv?e2|E!bqDRDqqUKWA&?h=&S`kI`9(Ow0YJa;F$$9*p`{ z+AqcrHu>k>V#)hr1?#P}{tWDf9)??Q-@ZLR*^&RViNvBAN_uzx6D_>FYf|67zv46L zgmH%@MzO>wT*@?+(zsohCT@E4g;2V)BEOp79ZX%>EKL_dXO7R_Er}o=;!5RXzvbsl zr;sYHKa(xuDX8NrbdxOwG8BDg4eo)hlHp!!QtW@s8$XKP1Y@a)AWR{&!KSz4{@7aD zlt!9mUU_8SiRyZMrxynoZ>R08zT-L&)5urA5*uS7X4B7Yn}G4ANk~|E6gn1HD{)`^>$`_(h@pb=}Aa1oq!lcrlj?3eRMH8f3*Z+Gv z7LtR7=}BS0;5>MV+4JqVwn@idO4VHYotds=ibg;9rO|MZty#Q>y6V1nk#jAN=oIH{GBHYPLPX z=&{EWSYO|+&F*9K1>jy7(tT-?SkT7EnmHpN`ItLG_3TVxY9uS}84b!(C|>GG04^q> z_>h3xulqY=;A!ptlB-^K2Wkby#v=&OCcbso^We^Z%VLF;X)ar`Y@eP?2|HWr9rYDf z*cwzeOPj}SxU8F${JXhdXIb{}7VmS!`vpARN*sUQVUKx*lIPpN1p4xn>0%R$Yk!P5 z)9NH!VB~+Yd(&xzG&o!M3bkv^%x$c^|H$q!vX!R6G!_bfIN2Se+Z2IrV=Ih)iB*vq;2$3D!xMmJMHZqNGn;! z_KAkoPEzO>1V+9oE%p6x9gq zSe8quz~Y>Se-Pm?y+dtbDUh*m3Vc4*jWIf#`KCDjIuIwpK2JW|;&tXY8v<h=*nnY#PYK;lgv;+fAM}c4U?RVtyE@x+SurN$s?6u3B%8C z5`%6LbL%xc@gh~N(<(JH`RdCX7-~IRvQYaXCWZS9zDZHWJUBxg9Xa)*?n#wQ?z^>4 zOiawq>W1eV;UeLMouFQE1BL``*^-yVQY%65b(X(uz_B7d{rTstP2s#D)^Dd!7hK*< zmIztK&Ww>62xk^^W_Znnatm24(}RAT_#od24Z9cpB`UnGJZ00PY&vMXoa^fgsXNaL zp@c4)L#(uFqfUPaIisbco9c+q%v3N$hB&nR)_VN7r&{c3nckYzmoHzk_#I5Z0!dkT zzuTMc^cTtoBaPR#~ruus~& z43M+J)_#UPi=SQ2lE3;3>cmibQ0^sY~`Ev z8VYgwiwZe=$SsZIuesfXAwkPre||6%d9N|hQzgC6y9OVCu#{i;bsmJhh?ln=931NO z$Vo_kYG8M~Tt>QUZsZOEL^ZEVS8mvrF6;o_gvAf3tHlcdyNkJ}?9&AdJXC`d;2~I4 zQZoHzVeSuOoD3L6Tt`7MAI+Qr_vZ5kY;s;B^MAH)2Kr|ei&;@l7`zU(s^;c7K{a=$ z=Skw})ib7EIdkGH+|kT(6Z>MhfIc1v>pKV=;~-=(vAsU7Q7cecM~fsCKOC_+-WCRQjg{Ue6aV{xgF7S z{`c_HMg~&9HGura;&T!uY9z!q99dm6s_4x(gYy{)-aJ{fHOjY_AiY}@LC1E>bWpME zy_i-gB`RBmFN27lZmPcbzT?R`ATRUW@zty=5rEvnBq#cn575wi!a5;%7qoV^0Jk{{ zR>{}m6B2%bxM&aKzu>4puR;f6y%58X_9ucUn1Q~&i1Rk+`l?xb-DW7}up%Y6?{C?K zfh-$yc}SvqwWA$_*PJkfYEdNs$~OwvuE-%8<*rqgi`vSr#RcxSVzbDRQSeUF9o>L% zR@PyY1#0RGr_Ggx@d5+(NPu}KVaa*;yefiKOlLR-4wuzUm`!9b!b?Q-s8!mk&p>-F z<=rUr`hB7TiGbq+_sotCF?emaQ2j$FJwjXe!ELwU!hkHQ7>QLHJZw9{E4D^XvzftgdiWJM;-gZYgC_ExLnYEawLGl|kok*^KNJws*C{UZdJYjphMK*%G-PY#)!zP)?s)o6gf-)WQlgH!92k1m> zokRhBHbe*W?|g$qyum0ll<`rJJRU)xbVbEuqfUrvbcuC^IPsvNwT7QjU;=)G|FNr! zT?{!rs>$=@I)-4OBaJJ!C{(iu|9B?BrFRjpE7{zTm)y%2N|7XchXo}YJ?-6Q9j9OZ zTe(CXeSCR!AFJcFrtrx-v?2Dy+pWoe=75=DdtT8Ndv*vD2+pusN(OfU1qUV#*jj^O zU2g%;1>meev3&cVCM4kPY|sv+Mfvj8W4q%CWYmS*9%0g4Z~ReK)lQwg13cPVJrM+r z8z%;-^pWfHx(iuxAcXN&q-UUZgLa8+J40O3Ys z!ENOCaNmQb_e_CXBV@Z?vu#L7af!nM>+_i^RBNMDS^hWjzhZE`8i_U;sz)QPI%&+q z=Cx~j9R#`6{Qhv2$x&NJ$9L0Yew{|b-~wGof%IPGez!G*s(Z!jE|DMQccWn34P2rj zfkPB5CV?QKwufx>EdV_E!BsXe5Or%$qwEWG-yr>-p=&`9>|ww=GgGuXLAVZv4Sss~o9bIRxOjLHcAlWrshFq13k%J!vNyVLPYI;YcB2N@ z$^qAU&l|`yt9HIrxlWvsK$Zk`qm)14!PM&6HbX<&IDGQyD)%&XQ{DCF`}#+44s~ON zlgKp(J7^`@TXVUux|QYnEzjWM6U25_XttCi!;Rcu26M6ZpXH z-A^A`=VD@FvU-Pl3lM=(?W1o8Iah!C>@a_COon)M0lq^~xV7kYsOb0+7SK`i+7UL( z`dKJQ;>v)3o!UCC)cj|l7Q;jpfCY)iypJdTK#f{ix%mmh57p++bw3JOe?CPZe(6Fz z112pHA6^{{iQUkyWf{==XSe0Jqo*ZWf!}fUmir)l3DyN8k?NO7r`ZY#Qx(fW2#e^q zg(b1hv+n}zJ0FSq30LmZ0*Wk}rXLj=T77_ia)am+aF&_;_KIIfrcGJU4DzHpL8Y+^l|8GM4ooJjvee#F=6wng=*E<(W-nLb)6nD zDz0UiQdWFcU2n~@EiVoOt(=%)hHyEZUqBi0>;YaU+0}(<#|QAMwrIQppJOA=j8Bf+ zchh2Xa+E2d>cB~od;Y|Tik3F0(#+7%u=dkz?F$@lQGahwQ%l16Rr7GyMeh`SNW{rH zR<3?nJqBjz2gx};mos)W$WRK!Z2lP!gJ%%sM4wzoSur>;kY~>&biHnY8M*HpDpkns z<9OQVArcxjZhN<33dw@W)vCB1FzCa#x11?mF4d` zo$S}=8}Ht6YQ>@hZ%P*f%{tLZp5>@S(8%9k)B%aN4Z>yb->2K%NwHRrf5Ge*1lq*} zd8mzCrHuTP?vJ_A1ni8VW06m{9>2eYV7{Z5?=H~^9*@sFl$Vz`c#8tq&Qd*$SsggB z_lB^K0%C+Y6K6@8zyHaxb~r>3dOZs-P5sLl%{E|pts)*r?${0vfIi>0PL-%y%l))e zd&%5^Z|1LAwlh;p?=GdQo9^uw1W3|M`+AUYwCFu50fFY3jOUzbG7mTno)|M$i=&Ze z8BuX^Lc?MZm1J;iBj_eS+6f^|trYE6EZQm;TU;MDb36SUGv5;Txd)rR{=YYxMJd6p z$05=M0`Uczk}=0h4VjVQCjRK}E^(yE5-(;0&p2$3fPnLzseLhNQdrc-;vrl|LJvme zG|&aM!+_6b*eX-vUHv*E=|dNc&w(QN^mC_Dg3*t%=L*YYQMcj1a{(^Y@y4$Jay~zg ze*)QPhew{|H;JsQt!?L{p<~M!Sm0GwXy&b%8-3XNOCmvtN4IJVjO69+@jK=VTteO{ zujs+Me?R><={Rj$+N>_+uvSi36$H*Cj!RpxSp+x|JJmMlE2%dAqNUOE74DBWLvyQt zK!D}L-_ST(9ZU+VdWlcV9xT^jw&6K|dVco<>_u}F+9aZ0{LK)GEKHvB6>tm34)~kd zA|xbqvU=j5lk*JP;TApjM5wi{_G5fn;guo-5`n}7Ps;g13^o>)!bOVoA&@Nh-%j*cv$|-M=_?8=05;Qr({ba-4qc)vT_ZZ@;v2K8j{{M-1v2q>Izk*bB(d1R53zr@bn zxi*v%R!L8{2{P613)iBP(ZQjx)*+|^#tnps+svC>EsF69jqSk7%kJ$ByXa~UP#ACX z^ScJ;SZ@%8e)|oNngZ>*NRC&T+k%tR2op|T1>f-mvnoffV>%5+!*`iz=NQ}lXXpDA ztnf$0rpFbq>d;uFJi@#Egz0Hcl4OfN8_;oN*F=mKZoshSMM9``1#{&qhq^FWz>-Pg z-iYOiP`l)Fl*cN_rTy%ss^1hU(d~Pp3T#N{`3B)SQBwbzFcSE0obC<~Gtp4GM5i2d zI`r&P0yL{m1Hf~v4*`q> zWhpwKtmTZ;qI)bDqzR3xsXER|+M?Cp=DZtlM4F0zj+u9nc%^C1#MOnEdpkoX;C<|sXq=Aht_y2`Ks}> z-9VN#fHbfdfTQUtSmpi<5U$g;jrfMt8aD324i;5~)JE#3b6dtQycgxnaII#>A%E zA-|A4h!F4%O79~TU4p-$0#u4cKVnM)kY&!XHtB=E0^Jt45jIy6_rqAVJ&TR~q)gM15_WfYHgQE+WR1@01r~4$W)A72ks0WL8)5ApM zy$QWiX0<6v-9L5ThmW~{?hbrB=Uk3)!gv}{@uZSxIGlUY2=DuknTOUI9+N+YL(TFE zV5oksh0p`?=LIPVuy^J3JvoI!ih>D0P$Cbs05LJj0QE9o$3#`tb36uj_=ZcWySGaK zyIUO{8+%$Sj}%|JU%)T|Q4Qn-{G`1%rDx`9_{sHdux+}#DTe*21k-~ccFN1Vps9J^`?afGJi zdySOZrElKk;rIhhTU_)C|J3W!bFb~i##k@(mc`2>MQusmr2pBd6n#K(u-JG-3oD7) zhNLBRvE$pz)P>NmZ-_BB*A_-=kQj>#(X&^!3}M2XWB22i@Y%M zYl>b3^1+M-!68l7YXM>+Zx&k+5k)-4G}Db0l%X5L5Mf{_RRll)e7dG0EO;|)>1rk? zCXi@$r3=WwnV8=0 @YZS-J46PQ)&bKjvgWF%I*j}h(U65t9Juc9bLbPFt(I^(ZE z&*%eANI&IfiM8)TI)T`JH~tYlqqK^jyN zw2g7^d3M)eNbA&^gHUN4yRmQ!eJGi=;NBC_ihbj()g-7X|L)lPcZ`nA*vTOGLnUw{S7)_n6Jd`WTXalrB_?ip8u>yMz5_$~-Ru0-Lm z3QkL{u+ANkrmQAKMn(cWK1-pUYl^wodKhiFCw%_1;Og(X7$k;V?Y8Uz<6nPy3@N=V z`b}dKc+nn#62{)pqxyR>2kA39Q#&B`bC@s?*@|Rf{&-rApdOkg-=}q>NOi=3s{^3! z(7|sBJM2a^!WdE`AWG=pGWw;Ox(8FKlnw0kJ)DdajCHImLD3{rpGOP!G>Icb%y>d| z0x?KMVfLoRB_v3;v_Kk(FOSV579yOx{MCl}!9hXw%OUt{GNP|Q>eSHDll%dN^*y-R z`%GHnC~QDN1UTcZIyN6SK9Q0dTUSbNa|ME&WvUHJ31*M=iA^^swBV%Dw=J0sCmAFN zuSfmNDvcIO`zOTD*zwzw>}<9E@B}o=(6TuyLriNp@&@+x#=-+wM}EMeo&&x^EPjUK8WMY znX-PSP~k0y?b3uB+c^dDGdKxSWMSHY?RqtunSY6@iV6p0PMp`$_lp6y|Lrsp71=QE z>X$#s7|4}@#TZ#Q$!Jk+w%~V5{d~Y!fvfffhC1ccuv_IsUDO=cjh`OyLKF{Rftf%@ zWQ-)>fE7sk@bEBFk>#|AeU~2>?OG~_Z-(l9)hj?Rx|j;pUJ45iHkGRoyR8aDrjYF( z-(n#u_zhEldjP5{%Mw0rZh|Ra5VAof^UQOn{#=a^*@ilg?5x#<(}?+#8hmI7VBG#h zm`>`WCYU3-+Fz?j)|>e6MZ)yiN@ALGK=)UEEo}+gt#I!qJkO_KP!ROO_#9eOr3e}r zAqDxH`AH7(NabX+YnQaV52+UYp@LpIy*f0@BM{ZdJR_ydZ=enOK=3c`_n`>(v6sbA zGX8P2riD??B1cPB`)f217fr3w`L_#2KjCPE`Bl6we(b~dDva_KZ%q5^$*%Lz#6O?? zs!P}~0(^NjGAGue0}@~<;0gbs{n#Wf+ng}&Mr;toyA!IimalRbk_DuV(e-dC9S-A` zvS8PqbDrBd*GSPkCID%y?+yODx3k!NjyJ!+=liGg+rkyuEL086WJ&}=aiEq`>H+nI zWz$37#DvF`@xxD*Nns>pM6rH;IH8T#NA>+sJ&r$+>Y?O2KphZ1oS+I_`F)#&9Z~$> zE$xKLDQ7i+KobKNtebt9kHP4hF73sy+(MBkF4bA>ct(meG#HvYKZp&0QyZc!y+vB{mo*1A*w**SqeC5%F|{yTrL`ME1|z=V0g<;6JO+X z_?;cDR3`vdaMTXoV&^|;Qc?cx&*aB8%(9b5TZB&C2b9U_jpWXPR)*Yj)0GQvM9LA0a=S9?B2QhIM?BqcHS_CdUg2-n1{UFhSQ zyIAUQsVNB4f|CT*?O#xo8)GAN0OnXmK`mb#^J_Kf_RANgNu%-<&In*m-~IvcwRMe9 z%T#$rgZZu2k~8}7?&N$7*e1V!RfB6autRRg3IYznLl#o}WdD9-ZtM;vF{;2vWfia* zP1rV%6snaz61<`H9a^-@GvM5z;qg#TU7N1HiG0N*jerJjz?U^s&^i3X4<8TD=)XG} zBPkefqH8WTv>gdIv&8-}ehO9I#F=LNCjB%>{D)LnrXuZ{44zBe3Md+L&!G4rRYQ(5 zyN0fR{k1cEJtOk}bU`-_+r0$-_wmRi?ft8d%_8%KOg zQ{t-r>c!XHWP~>N2L*r$gziUVdxqB%%zqR&qaTMw@OkTl40;!>*G!xC)42&aTPa?psIYhUNg8_@xF6;df6 z1(|h$9r{R(eX1?SsZ%jQufX>ByCUvLvitb9It)%*6Mnf`!wkZKc=B6UMGD0BF-9F3T<6} zM#_8Kx&SFe4w*4cBvbfx=Zn^}dM8Q9himQxJ3Z>S)z#H8m&&(q{WsvsW9^?mZ`drN zI=s=xh-kjCeyKq7^qJ;^mo?4+0U#_SVa7fYUWT*#=Py)i1ZEO*?Ehm7uKvKC z`2~qic0gq0zt@Ydv%v4Ziq7udYh_awgnrHM#erNwR%VPaPY|>5)z$2?r0q@{c*q;{ zK>fewL+gyzP9t`p((${i!A~szS>7veZJlN0>|R3M=0`WNS7P1%G59O-eOMlLa&F{% zy{w1oA35Qo;&b``%6KxD?K>`rV31mffbf@Inw-Zfv4|W*R0G%mIK>Z`9Wet1RQCqj z&5gyO1*;B|$NBv}s{pz`8_k}&mfJ-Q3R^_4IBWFxROgu}IZ_s|8U&IH6omLMnqVbG zyPoVML06ZYV(1Y@vIE6ovK`PKWX;RK#&LS@!}*u)VWoaE9vZY``yO(TMV0BjL+`39 zms{!dtqk)?{!m`)Vzz>XoS7d~GR1tdPeN#DXkZNsoQh3Nm0>|9nW+$bS;Ypr6ogr; z*}!Q`PWzm#5g(YqF~C?7?wLPF=nE%!Njb@`G8s#1eK${X&tZ#PD?KLo_ik^6Nlid{ z;R+5U+2SH!1@@6uAQ(^UykCZ9g4JN~w__&2R>FZ8(Bg`$%uFmUlwpz&X+?b zy|r6v){)Y8IhG^{LO&Pm^8S1imWLIsdJ~DWAZ|YgFYfMpuK|QM#^^_FQDGne%;b1~ zA7m%hPM<)*8Y=q)a`#Rrpx2_m2*|n*@z4wVlI%F+K3yO?Xm>_pj2|j5Rm#7aiSJy@ zQ1h%4cPf^0td}6`itTeu-0;%KlQE#3JlHyNtmyPT#4x~%(-mXQv?M~2tLPlpf?M$Y z_>YnY@5PH3A7NKKarj?jm0p-I$OwYwUjf(Hs|9gwGHfC+n);8gv70v02oCnEZ5{+} zS~RTK)&JEEI+93Rb*eel_WNm+{Zh+iX9}wm%t)kqs^nIH*R2%@*vm0!5z~_C{`1CU z|7^nZ{KBQ4CB6uD_ZmXMs7}(dDaHboHw6ske7^{e68B4_4xLC}$UUfh%EvhzXyqNI zo$x`!uq=^+nP6w+%|aaGDfp_f6`4P$-V1RX{Y!FEE;E(*@<$+UZ}rQ=69z~-v+*4%3@FOTGflurIgba(?Sx&bu`=Ovs>xV8gJ4%!x zWDw-PP40FG-NxuAHegKYeVO>!q#k2#)$e)eJTLj0K^v6oV2ubb8JyJB;gi2l4l zhdtK(@w;J{7j^Dwc%u)V^^TDKEx3`tmGmxCH%T?*A(WGd7m#SOhnsRb!nYX6h^t*4 z@b|BmNg1VxN2xlK{BJK9J*(MpSl4^+hb89t5e}P1%D$u@oyXLA!^Iv<_~|d>>}QVO zCVJA4`HNpHxJ{_#L^G*eyWC;43vG!=s)o?K;5o0m?0TfeOSD04+R4j%o~|gA8qR#L zThZ|zNhMB*c~{i6SDiE1EqE<>&rRsIH1<74OzUPyQ>4TBA-B%k{}uwoKR9O{78wmlqby^fdBv;ublDxinP^Ya8-f%qDL$ zEV4~M?=N>O3#m-4+Z@ThlXftfJY`cYRV^PX-1c-$Tg}p~GIy=co+6{7o*{*{(JD*y z91C$Dudst36OQQg$eZZ?zJIhet=#>#uy9qs4~px@cV7aM)94@%$#ek(htPzJs{wcM zN8;pWj8&F3Gd4VwiP&R7t&Hjt_s7kbv4hJ?bid3@j!$#?h1wQ>FxteO8jkSpop>B> zOMI!u()nx1m@iWQths&gX{GDAjwvbvS5wQ-y(pFNYS*=ch5&9tiivFA>u|*OE*Dt7 zI}AhT)Yp@gc*=n^ki4thppkYuz1~TqyZ+AfeVoog(`dYs^+&Gfn76GD^tOeW<9?xZ zF14B*TT&P)7E!e>KHfUAt>RM_%2g;FlCp}~w)3c(cxvi2nSwwVBdb*oW+i#~T5T4o zfG&wYvU5pndJWr|X%UNyKqfm3BQcvxRRF>8LJZ&>C`A}a5;tXYSowxMa==oqRchli zFQ)I?%6?-Yr$&;}Q#dOQ10{1OftrtWqvhDMlonvAVU_DZeCD<{n&1meNR zZpem=LjFUcfwcp034n;dHbb9E)05o=<{QwZVT{wbf#Y);;AH0P%u@qT`~cOz6EDit zE>4gXuSjh!@^5k*26@W?Z7OyB_d0wLJJ*D9!r&XIQm@-8Mum-)U%fM6q~ToKqnzfe zF+oS%SFiBVqy%neJhysG2?P=%_BrkVXths20%gF12BYp~2fok_E+1UlTupI4L(x-8fAV}N;Y^P7oqWSls?yr8i4g>X3K{2@ zN`gdrwiHhq@h~uV?%YCseUJdgrq9jI8yvnuS-~0&IMwjn&aZW`E1kPufG6S`vYP~| z1bsXH^f%c|DN$)IpenaVsUymo`Xa|eI*FyNM(vX$T0kdDv-9>YU1i*q#K?2^q2=Y}iPSS=V`JG>X$c85x0MtXzic>3 zkwlVY(Z~A}u6d{ya?O>6@OThs?DmvBeo|kOHCEIocw9Md@_SUUaOFLJtw{Que!0uo z8UjI)>NC3o9hA7QzJngZpWCmi%N_fksD7XYXwz)(9?n`Qi6y3 z@i1-g7_*x=sWyM(5p7Hx{;6-P-r$uJuSdM{>BaGnNu{;A>w}})h<7amywqx=gyYhB zhQ+2~cnI-7z?%9bT%swaiyP`Vtd{^oqLRYn z#4$GB9=T5?=<}y^mIN!FqbKUS$Mo19U+pRJs&tlsk5KvleRH3*gu1P=kOP8nLG9)%$? z67og*Aorw^FWTzjGtr9Qw+$g%>_Dm(vnO?hFf$%)cd(BI&Kn`Q)y zTPo$JOo0S=YZBykt^2))c1q{q(ivvT)_!UdQ*xbLYhcZh@f1REXdTdKq;I)oIu!8l zwC*f?_waCxJVf(|r7C$Je-@-WdWL_5Hp4ID zRbn6#+V8qLuew_4d;u2^og_}J4*5bRX#>-a?B*Y-?$QOtlId%7I~swl`1xPH-%I$| z5TDohk?bT4fyjK~>|D<+X;^1*#}Xyz2Eu{c#7p0ldTm2NPk%7S#B^M?*3<2W>*(hI z9mnP`_y@%E+8?POWLd!)zmW`g_#VK;9;~x$&s{2ba>RW@=}2+*{vC`g8*0ybcbyEj z25sTXzR{mwZwl5ZGib?a>$c5EuA?IAY=I0k37)Ue;|u5%PZ#$=;1bjH%GL@I;m02J zc5XboF#V-}fe$IZoId z_I5khl&SpvJ?GQkZngM&wajPx{2b-%BtEXPz@F{y2CcybvrQG>r1Ld|Z9?)7{BNMp zm_R+h?%PDdal7GbK{n}tl59jf8?jf$o|m2^{~qz#`I7xX@zg`NL9MOuyWwmUiqa|e z>J6={cRy|CG&^jTOEUT7stxVk9TzN^D=5_Ub5iNotx6g78tGiDn;Ke8e3tM@dsDp zZPuq*hRozSb2g5yCpt;_p}Uy}$89$QI@-2!nP>T&gR6!-CTu1284}$!uI9aSRD^kkt6B12 zA!Z}9eyhmcQJ+4v2zncX{R~d6?|h`*Si*T@(iM^(@&3#=5V(p!9Zv|KfxA78BEt9U zJ&ro1XY2SUor~WHaSIq#$v>%i9VYO_v-PC7_blp27@K70kFPkKqo|!QDhWMIbF|iO zFX+3s?Ig0Cp*pw3|6ZY8FA;MnOYF!H@xi@WA6O5MF!uv6S&~RcARIVM0yAkB?%AKI zkh|)5{h^2o z9G9J>c=qGKLzbd0_++;lqsaN_Y#kMiuZf!w6QTLENPDj|9@$S1(;I|~ob7xaG;>l? zBmbi!y(@=4bb`I_QsZQUuPw-q$Q8jtlWeqG=vH?>KiVinN5tX=1@%t#eN3p3&CqI~ zS1y!)GIEnXa$ST%h@3^GT~)VHuuY7yD)B^2LFy(NqHfaO=Z_wTMmsbF$>JmKw_{>% zyjvZKSDK5Aabh$dW>e4l?SwTHTQtNn-mF@vL4dYrwJJyYcVq!G3ON`?^*|8BtyfuE zdLKTN8!O9Gjh`~^eryyJokqX%Y}|#EXC_Pq8^v(Pnb)?W@E8~~oy8O@2@v8X+`PPU zwQERIGq`J9*Pt7hC$m#(RO{L}Iq>Td=sHxIsk<&$|B*b5`S~wKA(@HMb_WGPSTn>+ z{sMPIT)PI197!&mWI3VzCy6pTAMZR(jW)FIpLN*TVq zAL635tmMh!FN%e*=m73c7XJrQb%gxKdDUb61uGQu>?=Yz@smXb-_QJ$Z)X+L|7LXI zoAL59YN|}^)MrG#Ck8ThBi~f^t7RM?R2un?pMdh`NjjGgFs#UT+!AynIdAI-!?{p?JZ{p=_K!Xb zfBqYuW*WDmv1Os(3aDBbh}?o!Lwk^=bZ5G+_QxTAH5qi{UH0hCBiZ)^6i`*=923BB zXwxXRZg6i_HSunoazg&A5Rt#XKAMbr%{)&z^J+Ic^=4#2gtpxIxVeA$)lHksKoFz& zo_aRX2~_64nXm<4beE92*!wcmvfxOKML|hZ@4TgHIhzS2985?Kev`YC0wIEeF0lKy5bQKfIhH9>7pjV3V&Us zS(cHW&O(HF?HY|Veb({bVnNBTog2q21qByWHjfF%ap<&g5WK2=k~+6@`Hd7Zl&EfREwuc$ya&RYKcy-K z;Jnv~N+bP8gG~Q{P?mj5hK3-gvpP; zNJDe@2hc7;l;^n;Mz3^XWNmB={gc3xLYako`l+o-G)K8${4PZNLZeT|axi%z%}s2w z)!%=@=t($c2fpJ(4<;wR@(t#!F;2&W|Q1~eGX^f{l*iKkoANAUyK zhwD;5K}z{aSNkhy@rC5^yMWO|XN1e_>Fb@2&(0V!sgLI*bP`7qd3e5<&7|gOyM@4g z2MCBv!i#2orF7p~z+X~A=M?!CoM!uk)U|J+1zer>iFJ1_LjLGKS5p3dB$C>rh?&>O zR1k5QiPh@2l<2z#%3`-}VNAe*GB5=Dq41Z$PePg1i5S_ma^lUPpgq(`2?<13q z1>1{?f=d1G6kLefFDchhH6-;NJN;NyF@A|%G#Bz2qGu<-lRV4m_={G=z@h!?L-GmK zOxdz8E_>fZ;BZ

HMzxsh%D)MNU?hqETn)_-k10MpRjwm7mNFOESfXr5cG$E@Sss zWEbeJcM|={HrERtK6ejTn@lLi{?c2o5$qbXl|nJeD*V>_BGi0<4(i1+9L${g?d# zvy(rF-t2r6_i`3!o75_Gu_@;mW!8)twcMOVM8g)#^0cCYdt2jcy6rck)-R+dj5auC zj#QZ)6%5mVP}B(#h-Elaf!LS+$0uk%r&9dz1_J{_s=KmjC0y-y99DC6Y<4yqMHNgw zRuT@0L!CBvZ8UThy`46>05M8O+7`a2`3^RfJR=Xsh2OO>CQAx(%8Rx~*=M(Z2+xeQJi%WuB3Niu<4L-H4PpGdhxnO{+E0*$#dBZ7%NTJ11fry{ z5MHWld-X36K1I;Lvh#a;M$!g`o$RIfOV2ieZRPnsd-7~Xy=TvH!_0S9r{@cXMVv9^PM^k;y!04aQfLIJp>-mp;d?; z3q2skJ3A7(n3%|Jnan5!Iw4SeLZX*l^BR$*-_vAUSbs=nS}$?RqG&i1-nX(lrEF(8 zwvCb&{9eykxMZatQQnpR<#}*#X5q-NC~t)52z6E0B~{oT-X-g@akH@Tt`hWKX1dm ziI}l~t1@P+!NY}dk(OU&3*7lWD@y4LL(v<-^fUe>uOdFvl08Y;HsGh0uXL=_c!oEb zp<40kbr>DGtIR;W-omALM(5dYu|vK0tFVYYNQIVa@$|>wlmY8-e!fKaCajpwf?4aV zEc*YCwzmxEYJ0*)e+DWb3Q`7A64DLQh)9Rh-JQ}MiXe!Hw19xL(%m5lDlH%&-QC@A zXXE+5_kO(}Uca1kJo~rz+H0>hYv!3}o^cv5I_aDR>ZvvShg~xz6dYMuTT=mUFptgB zd-mn<*!_b0rF#DM(mOgCSJ1~2hBNlT6*c`?d$XdQbAhsp&xk{GPs&fibFJpl;{%4> zhiCD^gWLuDV+uxlwK_&F6kQg!Hw{||hJm^|2 zc$+WXo{_1C@~=AO6X#}aM&&iLQtrW;sYRTeBFDTtSG-X|b0>=`b+?MH8?rE`{TrBUlu+4ZD#`j9v?-4HA z7*Do$(|`pZvHM=Ms%1a%@E+nNsnV+zio}X1^0|hwlf_s8es_5h_=(m>`R}c{)O2*b zn<=9Mra&4(>H}<@E(bG_sAJtA{?~b=w}MY3kE(v|^xUMKBHy}SD7E{?YX%^;PgGf> z_$`_5$W?f(x&i|d9>nx+ZF_N|tmx(#%=Z|$;_Ku^G;B3V21W7jW-yfvlx7N_0h}`t zQ$YN;*&Sa>vP6en6&7X^ze}N;7M#sD{?roLT~g(P@==MU4yA<_?CUZj6fmj)CbL@H4N;QgD{szgX{9L|xT<==eOB zlHjc!?iL=z{my^lT6c1H1s7iAj;;FQ{$Q&^^A$vq(py(9P_cy80zNH)8y2O>C?_K$ zBPG?dw4=vFFRr1A;Zzm>%opduYX^zqISj>#!T!#bo1d5~yN#k8mHhq0N;+rC_+p3n zC^o(8ymob4Uww}S&${y-{ft{ijXjM#oo}mmF69k!2Pctv|7Yx`&hA;#gcVp1I*y%g z8gUtM4K(cR)Vv&*C_2&&Iw^A7a*y;!6Jt-lCuYhO1<2>uqsXndy>u-*i3#j0W~KN0 zs~Yx2BytA$cZl}>GQ%7<=mSKhPtLF@nWHQf`{C1UM?76JyT|_eRtCj=CVEJEN-=%S z%q0H<=njL{Ei0GwW8Bbb5o0Mg9nvqh`?OfOo)|-=JX?zx-(ey6i4|{2u@_+~A#YWq z!)=zQ?_s_5eLx0wuD{7IQKQzMGn{h6S=a+DU{#FqGeu;#4{_SV3@Rgk)-}It*~ulS z>c@>qRP*SSu5eV`^lv@#kPcBT1lfq&y4oouEuBNr*Wy5 z_Q!Gc#|rT2X)5TsWN6li)!A0^Q4{rSv!M#ottXW{ud`Wtd;z{JR>Kj?)LQY zY!PBbeWF^@3s$h+d7ahaq--3b^XXO6TTJRt+ z07^uSDKIdVl$7uYz6Ib!IY_XFwq~?u*9tMmbVlsY+${b}9dASUF5i31JS8se?h}gz zR;h?Qye_a@{!0Rhk7m}l^NktsUOz?othfDsRo}`~r`eWvwi#Xbfr=t&dD#*t z`NE|umoLtp1%VKeA8^gyPPuQ-Pk^7v2~Kk&W8kh!g1$*g(*!2P5clSlR~7B0W>*JG z%=KOBN|>-ewU`GdafS|=f2y6=p?Sh)i22?$WJ#;pDeVhUt)tQp&egf3d=x6+HBj7x zDjeWcyMIHB{6rg%V9)?+Q^`fZ1-9<2C&MF{?M!5Pf7Zn0_4o_yv|NX;9BLX$dZv=^ z>zWro)naj^-{2o59wiwM62;tk*-_jiR-)yY=27x;;et{6u=y613quFV@g7djR`2&% zvilV!{>a2Kc8_KIA+pC*6!2%l#F7FI(!o}Q^n%ewS|#T$;R8p}B9 zS*cl<>qe~|mHfzx{V4AoUqDSvZnyxok0HGWXn|BR;GRF$7eX8?~hTKr~%g6g9@sJ?&RXafF|9zDHF%_z{bk?I1$!>*82UuO_W zK|X_@!`G5Pk*RpQ6M-}>!^Vxx47K1MmCtt7DrO(6<#3|dIo}Bm=>@Y5&b)GnQKTyu ztl3yne>`>4U6*#L_Rw-lg+$thueG&~mWka<>BWBKEc1R`?31Z}o!zA~{v8X^vZldE z@YZ>nWkuc8x+n2vWd}SjWQ! zc??_@4@7IUfYo%7)J3Tl|=2Z?d})N4TIxRhM)7y7h^(o5QNp ze)>3RFL`Z_3WbW^L^1CjV`BN79M)xpLHvT}5o{GFly~Rg1O)&WGCJ>1!Y`u4?Wn@P zf{eJnD9ou`I2A=8&~F3ArDNH@o-mm)&==?E)OnSSI1NEy#ZP<_L7Ss9q%umoxMjds zB2m7_Hq-Y&J)>>0M9k|cZ?xcF%Pry_Zu?kElKzj{{MmP+hls?yiI{r0zbAIBnMbVN z3CMXv*j?vUI+^9oC0WK#p`@}aUbA`hOFl|sQ z2ErZ$lTZP;w759veRk(6tFo~`^+@Cd(ik|n~nBGfv6Z8#Wv z#-Vw9({D=baCYD|qwAHVA}+;LF8>*0f0sJ>)ZG`dJ%C1a)u<42c_>;PPp2LfU1?>E zada+P5eljiUwM>nuY703(OAy6Nx{k?dP$4&u8CKWK}bUsO&O#;2jopgpBboYn5nub z9%RnVBgqg-8I+J7+2M_^+MQWjA!4+&sLSS2I=M!)Q0$>ZqaE9x&W zh_0%BxUE@x=c-1)iy_Dm9)EmKO^QNwr6^}XH~GfPM-T^}zjhKP$53@=7)yMWOq+t@ z9Zej5G;8m1FKd-+KdEp+1!UG#?)YQR*@Kr$8~MHU*-cbHAz*eI()l*!6Iafsy)sJh z_OP~z7iswPJg-dZ2T0tL$G>wpwve_?zV9E!7}U@tZ{Q#|kU=__Q3}4LbOqEOvv$Bt ztOaw8ewh7>g@pw`=tFvk3IdP;iCgj<0x&}^cBCti57X?&bG^swXs^gajC;|#{7t-+ zJlLvE-mRm2a^B)tAd(d!{kNyOq@zs>meb#j227>^ZYM|< zKqHLv)m-pn70aVbS7JZyKJ}WChr9)AdF8TA-l)*}#4;7_rFhF=0y&q?){md4f--6r z?BfS&O1qh@of{HvtK~1QJ~&k3(@6 zVb3yPzmQBSKboMS7YQF)v9`9(&B^Iq+IcDD;{%De@P+QbBQlci^Kl*S$MqOBOK@aN z%C7*Nal4L9uU;=$eB-@&{0*D=ls8SdE_R89JFWgF)yrSF6iBY+Fg!{Q9FAl)jJB-n z9{sw<=V4NSoN? zAD2t@YJRj3rB&X4N?36JrreRXP4p7~sqWu8DdyYkl^!KjW%J)4u?AL1B>L3EOG=fq zkSLPD+pW&dE6{kx_N0%;KA}N5$@wdhwhig+zwb;~L(EiG{m;Qsgs3j3+}`A@($L66d_ z>gwtNfYA%fMyKL0W{;Y_`*y1PEcGSzyWA9YlcbgcIlWf1LazbS0d12}MT^7n*S0c# z6DxKjo2(<57$=9gKj0G#Ew<^fs~1l9)nI76Z1ECV%4Qc~X5$GKr)ZMn*tYamc>mkk z;pmbLQLj9UsT|M{SnZVjinM9McWG&_ce&g35MEQ~_p0g1D0aWckZ?DpN^^}b+ZM0* z?-m}X?Q_Fl*$~EEV%XI$nflnFX5`Qt$KTGG_hSJJWfDUO_FdoBv&D&`F`MrYIt082 z1tanB6{O<<__0Ey%(~xDIF*x=lRebE)_eSW!w&?YfeRw9b=CKA0~!WqsBE-|yP z7yGqn$pAQZ^q6m_#tk7QOx>?3$Z~RQdt|uc556L9v|J!#XiSX6XYf9%G#Y=PM!=JIlZMpx|w?VgCBRpN8mri2R{*57&y-_`uWXgNXf@( zeGvfcbZ$hrS9Gu8!Qk#N1GXQ@k9*gwvsntUX1-cUeC0HE#w^Zg+<0EXk{n{tNL42I zvDc8_idzBk%^3FpiIdn`0kz-m*RL~x{_DJ!5EYf)vmkr)2LSG;eBRqBBmmxBgsIeu zWGoSoJvB<{U66WP*ZdYwKu*tnS6AwYZTQQH<&|ig8Cy@4mHXJdR$hVW^^<}1bVjEB zmJ`pd{OTjGmpqF6UUvH0782FhwP~Q72=cpID=Ipkk?dp>4C=+=m#sjdHN%q+``YpN zRYB~;SYBsFS_e>zyxyD^{EJ@_6R5xYhNL6|RGb`P~ zs`l-}9XkwI+qVIEbGHncgsAGFieU2h>0R#uQ_oz|2nH#{KknMzN<7mXyyyduMo!Sn z3=bYm6^9TK5J+zD**_HnKbSCGD#zKgP9c{`o>%j21hl!hwHESv8;Ig9?$Q)lN5eSJ+4shpdNk7wx(Qp@<`vT&{fbPzgjVs zxI6Hk@%aeYomVwk#$+WVpb8@u9Xm*EBzCyThmU&ItF%iDrUFKs%SPA z(H|5|OTAyXVL`OAtVR1~Ep()vMM>4v3IH@XJWk||b2uybH=c!JUO5}F+?u6XhV-pq zDK*~?9b$dhBcq}u6;?rUER2dGA1rNP05JKN+T>*7d6YaBO$3kbAYTVk&7TJSOsOr<%-=Ya?SRKhI&dXu1ej&c2TTVhA&?tc;nEFd-=#18zSvB4Mtq${r zRXrC&=N=8C%4346p|O{Ph0QC8E-h=sMJjKL@xB*dVgPFjv1hn(2p8vDyU91E1xGiE z|Mtf-chD+|O{(2P#u6}njx{7+WE{PR`ubRlEx~W0b zD_S$*z&5Gruo9|0u9zx{*;O%(jF({!NFj+O%`eq_w?0(s%^wb!p6m zg)1Rud|J+F`S(R%rzEC}EVO_T0=irS0CwKd`@iT9{*JVV%+Wx+265AYNP@nmbh)#VHD9vy+eray|bvMO)>Oxvg(<)YL z-qbdlGBMZng}33tDF{aBshw8_waof`Cm@0VPDaApw-`8CQcN{}7Wj^|YWuaE;~(h) zXRHweLh_t^p-P9`8Ubfcb@F22^n6Z91EGaVY`PDaHOwz`>xla@et z{E%7Bkd~;MoqQ`-yat{OY}!`ECHzv59?AP;n{U)d)(aF}Fy+95NbHRRTs14$+Ou(W z5e(6ws^SMazXmAUX@FdRV|Eot?g=C^?Ru+YVBTl78f3@G0{zv5B_H0^@+U;huA7GX z$O)KYfB4H`GA*B!BeY?TW?8G;MvwHi!@)&aoo{1lOIXUcH9HBvBs9`?&etlMWkYQiYtTik*{8albnssT~^DQGc4I0CL0KqZP2~H`fMu8hm zX(miw0did;pFzv2?+44)TqogAP5X9^*O+(&-kCmy(?it1nsISFa=QoCRxfkVY8!9F9PT8b-M~apg%w|Kd-;c_AdZqPnnr2 zrf-p7fj77Bo~G(q@EFP~b_Q*e60FV$z}dq`45fVBZ!eDu)wXOX$9Vm@6DRH@aijGv zrk)wlFp$N~n50OHM6bi{9iO;U(F66MXq;RU6pC$*^4r>^X%ZqMBQImYxGAu&%l_?= zuI-88FTI5|B;tBDMDny0z|o0W9%ZGA8|6>U4QJx&wGuc=W` zkrdsB1R-j04e%C1`?W-1t-t|83dia+wII&(mbNy_qhEdaoQS;KFc=j(pk_LS!N(4Kg%By1<9hRyBdHiiwU|`^)yVF!c=q#mVmTHvaHktg zUqv}2fqZ~WtXJ!dUH5C3=>M^eg~pG=HOy%IL7U`=IONT5*toABRlgLouP`HP1a5Dx zor^(r{tNZEa+QKFid8kEVR?3cJ`P_g6Xc0gQRjOJye09KeR*TgDow!^A3IagPy8L7 zGMHp#ZwhDpr&W#cP}oJFGn~5MJY;baO4aB$@%>E@c6$PU75`P6!D0!F1t_^fzmKe} zq`}9NxavpQqeOvhw`RulRsEMY_?iGthvN~J+G?0u5XXPepaudB&hHM@yewL^UbuNB zzSgICrT>v95YFNU6pS25x_AKij5C*MZQZti`$uk7!9Ran|2iZ!mq(uu*j0L7V$d`fe0X5B0-a8R-qPuw4LawaDTnZuyHhFffz2)9v3IF#NPH1Kfu1sbL}kQ$KH@{X0T}+RU@^6UfFP!fr*9kI-nD8^h7@o&^J);Rz2&uyywM}1Dk-sx9 zeJ_Em$nCgG&=#1<3ONg*BQg^Bi$*^ksN1bY9Z^EnsvxAp4sA!~vCr|P=Tv2G%EbND z$G9`V4Tp5NlGb5ae>@h!+e`6g5KTHwEC9_N()g}Ti%IC7m%al(24Yd;vcwG{6t<27 zj}~z9KfXbguJS1m1;KenLwH+z%YOJJtQYz zN)NO)nd7MXwYSb=p~O)(wFl|!_=2vRKt=?>m_^uZ&$u}U5WPh1&+j&Nmi(8)WCk)+ zouv1SQqmxXYJciZ;T}8ST=*Tql%pV_I%cTcxk?qc&kZhj%{5ZSYTD^=h;XxW%BONkm4M?&dz~?ZDM3D!N!2px zfeb~%%iFnl@y&$S`r<~^?3RFYw(YB>8R1SYxAEio%C4Jz#poRiu62;zasuvM_{XN2U&esj-Pyicv#1 zs*w7G_Gf@J_fy}kQgF_6HI?u?rm-$7IjwGrdn zL$#*GmN{(Rng;6L4dt1_b=A^k`(Jh;FbLQ zZ19~%S{^e=HUUFgMH9u(OPH;4laN>hP-~0I2uK?N0j^*ANBK>uq%?T4rY1)!3h}8$r zYcvm#l;3nvyK4%1N)*OZ&}!G=k-n)YlFh-n^!%CGi*~+L$=R@m#+tXDlI}iLhmg3S zHZ(Glz}Dw0RMh{sk%BxxJq)VKT5RZnIz$kVlSE7CFsR zLI^S_`MrF2RwR@9U0WCvcRDXb^Ox?IKLySa+w{^s?s;((>a!J462oSvAy;{x`MH3u ztcuO;=$rQ4`}57qzSJ_Sa^^Cf_{G-)huj(f<=o42#Wtn(LJXYkxgAg~(r#;pd@k+3S#a?~>mqJdVD~rEyHN7o zJ-_J`JToc3%;y7}xj7WqJ0rl^H-i0IHb7P6)=+pm1@POAsyL(Hp-{2xS_7<^?&9j| z>Q4v`jz*!a_qh0K(e%?r=CvH6>sK_EN9A+apgyvquee&8t4#c4TA(SQjhT=LU;yGy z3xb1FV2*@j2H#<|2>byS*W)F#%@S#u8Q%}mdg{8`u~ zPsK`XA{UgE=KGcI2;{O@!B&Eu5vH+)S1KHt0+?u}$ID-9m^8KDJ49Q3)l6stk(U&5 zAGT{{&{sc&RSQf-}=l94D#PsZ6G8_-5u0xM>hzIB2>IF z7AyIE-HHC{W8|x?ILt(BzDdm|ErIwPKJlm!^BsM1uV(ppp#2$(S4F>1 zMb?c|EQs%Zi~&bYjh^7&SomNp!OdpsN_t;dD{)I|A=>xSG>9!%>FVOAIbXB!NNJ)> z2Q)Gk`S+Nd{Po1%`+E-s9_eMXmmi})*Z-i#59d}Vktw=s(fRi=rB3YZhbl@IZbX3o z-@9~!4>|N?Kn7q2@GxoLFx7h5h7aigzd<3FByniCne-X7utD8t1xaKU?loYW_b>h1 zB@)`Ik4nJFaH~+QN&4X+zVe&fAh$zz=Zd6bM~>qxnpiRJF8iAeLL?YN@fbqRviq2i zjxOl@yF62Uq;jvS$`$58A?K_33XGN0hBUS=dWb}EuqKUX?!8+p00-2Yc%er!NWRtQ zM#0c|)!)_@1e@na3>;)DHh?zCt*F#$W~1t{)g;v*6rvaKH4fXRe!#8FH2-`eNKTK zGQZhdZBTeU|27Zlf1iJ|C;#gCJ|=g@a{QZyMy9Nvax#!BN1^6#IgdC-e@q^8nF>CC zJgJw`ZOuQI!2^OAt^mZI%%Vnr{}wvG-%&e?5wL?T&?%)4Y3>|#@ z-#uvovBoRfM$#xyp&_Jq!8WsNGKuQCm`(U8n(pOEwyL|z z?r;e^pfwis3P)pkaF@?z9pU;<_z}J}xPo=J@ykV&5HGYikg_QR z(eZ!DZ?BWxvO4^Yisiyq2mu-eK^HXK66ddC>m{7~*5K!#3NaV(ih|@d8lfgB7AZt1 z&L9&`ifuaOQa7fftgNi5*>-;4|BAoL2G4U#2)`jikyZ)(%z#diLO$E8JJULz3X@_C zJk2rH0#;pu`s1Uawu_NNB;w1Aw#$E67B=2sGM4KCd|R*kVd5gnNBlOA!xxwDD`R!u zLuel76QbV-y5(v65m8ar!{yHp_e!mX!kF>EL?~V#>>-FY=RdvNS-9eKgs-AWrgZ~w z3H{GtDeVM6L1}@14^*cAIbA73)>Sf>=VC2#an4EyERjmwGhV}AAEw`lh7PjuWCV8C zR}6v=3`r)|?bqNBN>2RX7cJs`SlwaKne{)&rvMHpC#LG|?dHJBJ{9cvyeai5K1v< z$OJl%{-gXBpt+Pn$@;A>V2q@VM?^us7 zb8O1RvP;}$2kp9_$ZX&t*$A6j;A-796Mi*NTcx}BsyCy=Z_S2H&GPb9c!Cg|Ja{qO z_OqW3H$Wp#f^Nd1L_HZ?R{J!m*V+ z1!#GU%UR7z>5w??Es2edLAERmd5r?@_~hq z>S*<1GE|w{G6)LlV6=iXQOEHz$PzqMdS}FXr+|kp=x1U^J_AkfkDc8J@UrsovU=_@ z(7uVbryT(Pih~)9=bp$Vl#eM=0q25wIz@6O9{b_;d&RyIEmhSLO5cY79BH~kjaphh zq6ZMF!JwM&qguj%eZBAFFzgx35L1$%H<83Yr`g*9m(OM)?&ab4aysT70S$2`$5XJ9 zxLEKJ%vvI7CK>htCnz3BF32V^6HaZ`D7GRbD4m(crF^CupyO3{c6L&CMy(Xge8R66P- zqHCrQA;~}@?fq;OGd$(M8*@sSDW>v7tFez$Z8OKCkjy39S+Cpx0nI zBi_ewSP84zdKAvt-h*_pqYn0golGCgGE!eVB9Qx7WwMhJy_ z>|h`Yp_hqxHRj&R5En+AKsc4iW9Hk#`!l1k9kx_FliAb*ZbE{d3B&?2a->h*TXbXWLV)}d+WacGp8bMbU0M3-^QrqS?iTqWjIg0Xs zV^pR$Ha!^OW@SJtrXht!mTp60pt9hfLH5BCD%B|+k3j1j2U!?x<|XNAX@OjmGz#gE zLi#8~I=-8ad*>lNMF0?mr7oCRS^{DU@;n7xm&4jYj6Kb|4m7Ni!ES7y0Bsk~rtut! zzUcF}KEO5>4$q(DcOfOe`RNMxDTsUGU(HBQH)p?_{KgO(J*HY1LVgt4Df0@nkx&Xa zS>tjJBmeuUw%36X&0Ro{{*K)^11{9D zokPEwyf%W?{H6g*8LK3uo6w@QVGGy)}8Q!P6Pz=Wa=8b83UrJO#wpZd1o#r$tFH@vXd4sSwU0JP{dPeoEc z@d8^29Onnv*FXSGgQTT1ENY^1?_24y8gQPh^Elc(jG#qOHlDG z;a#gPawNpJ%~#H%gg15g0UB$JH3h4KCFhQ=A7h#Gnl{Zf#QdA;=mE&rt@0%=FGGRM z6}lY3OhX{{sq)AiT84O^^n33t4{T+-7mWhd1e&Xa{2o(Ap~Q)0GZfs)JeLc8o<}b8 zb|7K?4(~OPw*7bU!F_n(>BlwZ5iaXzsPOC=Gx(W6#PVJv0>=#4rikd**ZlksF92v` zJg?h4S_Z){;y}tqpdliH5yd#`M$uZcdq+=>FfjatF{XfwWWj?Ura+1f9eM`9S$d04 z2iQK~q=f)s1cd-p$|((S$}N!HrbCjXfm)3X_>G@(Nd|b$_ zS%n#rbLO#EmAE_@qw?p=GhE!p`=Zz3Vp7?Yv9puJkIkU&`+=QIHnNw!dcW_D0%5j5 ziB)U!p3@6no6i<3`fw@A)*KXhyA4l)G7sp!*l4on;SNWJ5`Jo_r}Kr4$3%LK$W@bn zz=EVTA2;}6O5jccC&73}W?!OEBfv@wBY6~I{LNj5HuDk7X znY7|o{inB*U%S+wze-~sp^nFM=~<}nD;a!o`Vvxe-I8*D%HWhN;s|<32E|>-BR8_j z%T;4Lp=k`1UA_O)5&)_3nNu!AG<+}*`J-Pqa(h*jDh-Wp})Ua8K1yAKjoodYl*h6NW!~FqbI!VjT4tp4Yqps=85B zyC6~x%MHS{4<67aXl&`ThINGz$USK1Mb87KklKxKvEOLB&fuThT%R5cOo#=KhsI7h zq5UgVjR)Xf+NRce9!WB}f*$xaQrM)lP^5rc^f@8|&rcY@KvHvn<63zX zRZUmAfhpasVQy?p#R3Rpb4qr$bSy7|Ng-3|k8`Pj{&mBve7hOR1tp|w2y&6c3?NB( zMW2xh>;H3Aal@-7{ZMT{$(?bYpimzaq3MYk3#O!$6fPS%ylYoIOf-SI-s63wwnf`< zAuamW`E}6PD0QI+b}(#sk*g1`=&Swm^Fsg zjq=k}E#Dh;Z_1KaP-t?eL#~oAf?m}6KO-K|?DoxfkVuJ;U zgOVEwwlClYAWP!gzcJODmq_Lvj;wi!U5A6Q@IOSuK1m{+3Qn(pi=x*X3x8u^+RwIy zY9rbM`qxmXu-`hTn}jTc;HaX^gf%~unQlpCIg#)8 z(hP!fmtLaIC(sngGu;%3jCv}1L7Q%AZP7yOS4`!yQngk=+AK2)R>(m+Na!VHXIsKc z!Kp_06Z#MI+Esi)K_oPt3cP1{?^IJTuc_!Pt@`%W`P;q+O>rhLR$Ki)h)YeEO5i0`8(rgZmq7HQ>?3D`2C24|t zWKbeN`p(a9*BugIU@xzM6t%b=I8Bx8yym8+q%r}I3F4+SmC(M<5JcgCvy5s`Row>? z72gU`x^xUHQ~*Pz|H&J{<0OZ52(b}u%edgXOmG+N-rnBsKl7kqM*(+7uA8wDS!TM8MtN?t!MvQUaaJBo*e<`BG+r?J?+Efz*#%lcM+ zkx9a2iB#nOHU~|U?kumpsI!WJcyy@!2!d_JF=T9XWWw=;?kd7`X(kVr2_U>g7$6lpLKQQww=gWHu z>f<7e6?8l{(9DD_H?Lldbd-rX5ObO)rv3VbV~D8nvh5q#uGwsUes#w)Ox+g2q7Oom z-2=i=fImpJ1t1MEVAbO^ASN%Sp#j1g>V*={xL|R3`BV4sMWmaC&RRW1{VGkN}&G4;Ch zF!LV?+ds1S!mg$GZ`Z0;yA*6Uz)OW9GfuG=H+C%A7L;4Fs}3Uz{ck_&!Jd&mu3TwThG7^ zWWDh40n%ZIW&dDn4s?qfTF=1M`QP7$&Mz#WGo8i8#_sGm#-JjCApGm}b@{Ulmp?lokJIlN8zZ?z8ZSF%#|?x3 zh1}XwyUr_K@!PLo0n#YR^SR&z-gstC4i^54$wu`5UG^G6E?nBrm6dK*Rx7oj&**ngLaWYyI07Yo_j7fGYu8Y43cJ-VQMv0@I$UCaTw)AqG%Ri;pou}@{X|}}jZvqT$PP3N^lfdQ!}HEZUj#A& zsn=G=>X>kS$W_4)4j7o7HHHQJ@Kr~LEc~<{`ROz)YQndlGc(OyUDc>ieucT! zP+%koc+%qf2&*AK9flUe(5^1v6$J1w-OrGb&9AQBcn)$XRRslPxIV9tM~nkkM}n~0 zZAX#{g;oH!5`vC+89hmuhlhuT7oTAx-|e#4dm?nLeD^(T7(*kB0aZUft{Zj*)Q4bN zPLbKe_Bhz2l<-FT~XlK`M9`YHPJr(l*^@7xoT*_a%>8c10&BO2H-(N+L;fXvGz0mqF<=3Kk zEdL2fIpvR3iJs8kREc{_)h;D-37;sq_QH;9Yby5g>i+M{{Qr-;uD*o+b~~5okT#$7 zvuc$NUyZ^*5Gh~ZvmVI8jjjA3)~XDv2zn=$fzDN9rzAjD7pNqpr0A3~BaFh3{%H71 zm~K0Zl-jG%2(6G7x%OLC(~lpLlt{CRraC$PornR;jx`GCH5N|wRXHbizf3ZcHd|Fk z$PKv|C7=&INs>|3K(>Sixule+z`($RSSTMC-1(z0=?^V>+#%oEc+;pQLa*uYVmE9? z=$eUx=-#2}LRVQfR#uT-CND40Oxoi34%zv%zk2Z3(5<$+)2$}KacM^!={te?CUy)- z7JM{aaG0AqPY!o*1*7RyWU0a{1inL)E)v@;i2gGbGeH1GeBrUE=#{0YZRB^TaTzpS zz>yj|q0`(4^D}TCDqZX=G>P%>D6BW31Um@SG-4^*U%*D|V^{zy0MFtDq3fr3KK1qW zlP2NK+t3I#NC(t>wW<#w2SM9}28d12z9Ycv-Feh<35FwK#K_N3)la~`e|llRjjdC> z4jSXC(Vm5n2(GrYL(jv;aUfPo)fNU!|fL~RNMF&nv>y2a}n;NhZB`{h50-KFt-0OKl5%Ya6%@)BQq7CwzR zK`_(3bBAG0{1~*-Qn!U7j9o_^?d0$S%0%4qkPL z`Y}g#T0oX+LGr`5uqfQ2j!uFf%2~?6Z3(|40Qy(0?I$NEB(Bq;`T6;PU2&TJ{tO#2 zDr}(nF`nP2$9(EbJloKZ$Q$J38jDPO(u4OOM;t^w1uX>onG$R|nM%`i(mr1o3jL8< zNR=d?Lhr(BH*OF(>v@J;y-Dn_bk~W6A3N?h0uB`!%Pf_DhE*ZQxAA%>B?oYBXemwI z={8rp0OhsPESSh|pUYF0)DeLnEm|U8V(jx0#Uf zBI9-*+sW56s7+XjwP&YC-QCazV;kCqRCJKGD4DfEnt{Uj1GN6T^iFgN{6v^r4@iM> zwXY+>c#vyi6GOWsaXTkrM7Ycyn)oZRzdj4?z_v)j%|?I zPIVRWG_O-k=@R!&LFgE;6BK-?wfFC6QY`jBTTdw^4>z~U6usnPZLjZef{hsNZzI1s|A4L{VJ5pZ^*Mx9NW}FNL|V<-DXzw%kK5} zcX!w@b{jDTeU|_^p&sc6J5CgQ04pbr1&m3DeFGoNN8W@-+SOc5`^OPm@TO6dq0pkO zoR@!ndI@|Q6iSUNOq`s`w0u_PWne8NaS>3gVl+jX z3R%_du6ONs078}&+Y}QM8={ZpcQ18ddViXqkkQaNUI`K{vj;PNvhh5D7r>eq34Rl( zTX-CrJwYya^EJ>zByS!;+T7XMZM?ak5E()BCM-ssPYBdfWu#CDY-PfhFYS3*C4Di3al41t+N0u;ys{K{4)NI5d6UL&P|d2?Tu08hcBJ9ol2@w|8PEz>$V zjcKc`Q!g=hs~J62?mjpq(#m^$xakg&hNi1aRMCZFIH&qf%uCcOXx~MGo)g>3ANep25aRCK?(EHx&5JWopG>urcsy#ihWA@`8PW z=xND;r$5ns!sOQuIz&eN{QO#J@@gG~p(bw3MlE=Agd`69{0|#AmfvrKu=s`U5eOtZ zau#03Y5$vX8IZ`1XvqSe$^cWzr!i6ripK5(Bwfb^UD7%nJK(fv>k8e93TJ;0EA$N} z{Y|ERIvn7COeeh|-Yk+Y&~J(c?Kc3G@SCdW7LosDD)e|h3G?8_DhP&o z;y4hRBALS(h=YUEtwu=ieL})&C=Q!|E{nX5O|$$(5}YU9`LJ%00zD8Ngku`mEA>{N~iqVGOnJUp@=*8oRF`|cJb@`8AtA2YkG>r&g-LgK62C8f)>XdYHa zfd}2#h@6aKU)04jAoWDD?PL&jAw>fJs_nwOP-ef=)uUk<{SXeOJwUt9m@vyWq* zuAQCA@!CdSXcZMnO^~DrA|Oc`L~=#}iIRgLNhE^=K@bo`Ns@yE zK@cQJP@)nggOZaHY%I#s3c3ok4qX`oaet6fk)ksR?ytU-#{W_r@<#+0?xq8o1*E^ z@-T8A0Grh5Tnk-Yezw+eNhe^=c%L{7*9-bH%;)=N8FQynYAc>1rT&5oNCcP9qs7p``STUnZ~WW)>=0 z|MR5_4!oKmF#|^pDZm}yptSz^!(cxgy7uXo@IhP}#ie%#-=C+` zyFV3iPB-5LGK=U-nyq};laX>wi=i)lpdD>^V2(yE1m>Gy0UY6&*pPor#ynl`0+s{E z)bQnt5Y0IThAUyXq0+anDHWc2uRe$s-0kL|Q0&`pQobkU8>O zKoQIS{=UhxP1n9Rx6+5I04XYIhO!-quOL7^4<-fP&zJM7%OIG^x39Knr+M%e%IE~f_F zfBRI&#ayEt4yc+Bjxpr z@5NbCNh~9v*>M#VaBY-50xQ#?3(|yF{0q=p$B7kt5&*yc_ohuSIxFG0bl8<#N2{B& zvxmV4bLQ;|z^8uw`UR0F>ew+6-AGvEWQgkTymdBLDb+;`3FMf=O3Vu0w{Qizr3%}( zKZBb|%0Nl4Zy(y;kmpdgqu%!b7`9W=soIoN*+d4V8I)>iI)J;vq}ZgSmNtW;e7VeMBHiCFMy_V%q_3w?Q(A*Sr;^r!e(O z+jsfSnU=qC^9^|o-@hmW!DbXh0)I1l0s?l}QS6y0AXM`}lLCK?pu3G-PcXF(Q~<)C zS_7!?ctU!1cXxNF%)WEz??3hfrPRC?eG~){G=g@QAK6ZVWY`$moWq}q`xuX3Oyc4- zG}o*@fx82L_)5fi4n&$8qi~9#91EHhF2T7pGDg8N{hV|i{kK6YUBY!E_3tDE9d%eClouyZd+e*>Pfa&eC;l; z7}k|Ze%0(9kCpmBFR~!1z7>9xmHRVJi%+CV-0xCyv+ry23L_E*M1wR&EUzn z(rxf-)792Ft$2S(O0v;XX3*ocBU-kmSHbbRi3ZF0qUx&|SGqE*_MekYny9`O9kUVm zor2s4fskbnU9Z7{aZWbApPf2-XEWjQr|7{5yD0mfO!D_OYn7b-ZGrQ7hd1r;^&s{0Em;eL?_jjXFVqn0u9^#osW z6wiJu=aEk=-@Wj&_v1$y(fbG*=zE2a)_my1UAqInYh3ZAguAB8JhYDQ>K)!^4`vu2 z#x5&5fq3a2LoMaGacx>(^4H9_h9t=142uMi%Wkq24`wJYdhgWwi^~=Z`0C?&3_j8` z>d&?3B0ru;P>QD%Hk`l1QV2hzo*tj2OL?tqY>WJCMI!K7rfX~ZhJ~rXed1<#!D-m> zrH)vtnBu>eW*m)vGi<0=E|RE8C^W3zV^H(ZqPs2+^z0BO$|Ve~Zy0LyQWak!U?;7%C zuY!M`IW0Typ;Sruc!Wk@Dg<&!-kH$5Zb1AMUQQ9%9#XQt9#Y zRqM=QGw<01gk6^|*Nle^dC%5RAom6aH35^@()U~NLHx&rfwsRGYU)%L%8u(7!E;-i z3>5M)uNe6~P8{JUg1@+Dc`+k1)>n`bBmwXTz99xbVJ3h1zyHEH_%!iZBIew^P0kE&`&kpWIMsC2Tju*h=#gwIw@!3a4Rsi}2A1JWBF z|9nXX>PZO{b%4#~()$qg1$6*`-v|5gvF;wE-%_><1S>tnfAMTaEXT7qZ{9#NW}2|Z zmzxvNXlexG}U=(V) z09EupL{LyjPD@GIn;{sBl90DuAG$m;h$z%86K z_XI>c(-I)LbW4aSzJIRXnxXaaIe;ep0~sh)K)2E2yGDGo1Mf|ViHJa3lz%;lg1%Ld z>YtPJw8x(>0~Id1*2V|%Qz7omlbSalVLY+THpGLR+kkjp-lKud=v}%m;%^4u0I*=$ z{%ODR*p(HB9na;y9Ce79l|6VoH*9uZw6wICbi-}xtD->~>~>xt=tjgp`xf>~^}W&| z2-}q7cpI|BFI~!o2i*ACZt)NZnkI-d_B{~d7R%0G>C_IEcoA-N0*I~&p>qm()tcURY1kxfZ3vOLYL1~$$S`NN7bK*l@!~$!nU=7>(pP|?FW<9)5a(>DN{U#LrAKQ2 zd_#I1pXCfE78&O;SY(dW!TslC9gT>J0t_J&WN16mrPkkIk|Qpm;#6p*6V8NM_`wgU ztHJ9Pq(0j?_FFS4BF?Qn#a>&E-F%Sk?+hYci_anWi89Ww5Ke&gw_K1LzWn!`_u?ne zJ;a_cRpL1m#=v%5pwfJ!mIfs#1tLa|^y?sAmbwKe@|>colamw7tA`~i$7c2Td&$emon+mD6w+oJdePvJ^Cu(^`}DGZsje*m zou;Oy`6y&g-w%V1#FibB2*5j`oryj?DqyHFzhZ;`hKnO{7&=C5v-)VIvmqz|6Oc)1 zA8ZDD8mOhD4d3YtSLpBeClwq&a^%QBqTD2yi}aP}&!7MC^5}w71--DN@cmE!UP%tU zln*+Lz<_`OCxBbOjgOCSj~rFe|LDTS!{Y!W(tpf0gL>g9NdlolW|2ul$UC~Wh;h6T zj7$VVz?nbcX4=V4W;V9KfA@jD9Qw8#nhpo^L&0R&h(KFi$VHhiT##PH6%KnB18Qg74L1of4p&_V)Y_p4X0UbF5i6wdd94pTC4EI}8bbqF~P zc7pmN9_Gt|ybL%ndND6;?pStqcF?tc09^yr5P>HCvbIN|RWDn&FdzbLZEb)Sw}JNP z-v|8N02b>hexl%dzd8amjNc_BB!sg>nfb`Y<-;Q@!zvHws}?ZED11rob@ww2R)mA2 znb#PMkmvdwG`lgwvO!wfVs`rZI4}4Za`eldC3Q zo){Jm>?D!W2BCDp<6&KYvnqj34h!pH(PQJd(Ar`~ecjji8GRufp&a^mH0EBX&a%{EDU~O#=h%Ef7$1b$ zJxQ~rNQmOaEXbqZKc1MQyf_oB13pO(9U_7)ypIIqUUmW(-40WiwXypr2Qi(3?8 zFKGt=5jzl6?;+ka@n9_NBa!@WT`d1q5s5_IC?hvYHcT>9r;ouhc9>Z=p!wtVAbbkZ z@^Hw;y_1St_xa6$NH-?vpM@}#Qi*IEIXSRn@69jaeP!0PIqh(DN&Y7*;>Lp%;fPd)|4?=_bGUpS8zoZ>eM%T-u z2|6=TCWgI$0>1+L-^U^f!rph*s4dU3YgDruB0mbROCkDpM=im3_i9U1*% zap!gq@#+EkpHDm%D7{|c*VIE?cFxJSN#H}e0z94PipN%X)ODEBXFck^zw6%Zj+noV zd%eBAnW(&P*6xdNal34Ie=lewFuwSSu?+b8s>c>dWXqIm6BKC}jT)9(Yj#tgJ9lRU zvjV{&BX$DFm6hKRbpz^m(|@GLKQsfyh+UEM=!)qN#npsy29m zc|`(YFcL@ldPZYJcNU77n3!(ex-}My`2;}Nj~V57{z>Gc$azGr#)!RP+~aTZ7FZ+j z*vNVIrOHIf+=S>72?I6<4tC3gb*Ea%WL|f#?_7F`uWdlZ6ubz(O+nR5{6IFQu3`>Q zKTe!P@)|rLnAe~9iPOkokVAj^3~+~t>yoJRoW|-J#A6~*W&$h<#0vodN4~xSMe(2I z@fxrZK)Bn17ltM~;7%;~I>FI^H@*#A20%8U>6^eP0>HfwEPy8vsXN!BTg<@t-G{Oy z@a_%G5Fxq(E!K}uh0YK^CWGq)(3`KHi}9W(LmVZLeTD$E!TiwzaKt~BfS%eLy%^mJ zYK;=>ZwjKN;5>lc<7Z(p1KJ3btqqxq+@>2^&R{K6uKV3 zsM3Z@xj>oaT*m;1{`Bcn2o$wD1wdv3z5>xS{FpIucCJD!HxFhEp{br*t1s^wkq zLtz}m8XMeUdB8+jc{r3&fGGus=og!bdKnQB0TC-)^*+$vAO}9H2(lh6t6P~0(NJ$u z4o(&V4S6hlp!i*H(=+Nd7&H`o6#ytJbbqLj zJV2TYDk+TslM6*VYoM(d%0@wX8LYzD&PQ{7IVN|5eSmxio`*9GfT9e38$hkWahpVk zbO1-iF~fb;y?O=?4zj=DspZh3i5Q^P+P$qHlx#@9ovDC0<{n|MO$Qv?xX(dVWZXAv zXL!hlLb!Q7kt>GPm+4!&vXp7(doPrGH^_V?{y;Z-<&@@!zAKpI%r)?ZJ)@;|qtJVK z%H4G%{q^h27s%|XCD*;Y;20w_aM<7XAhvAn*P*08+ykOL#MpoLfEN>XpwjtmJ9C(f zdalOB_Z&bCSBhV~daJSpw)ni4@k3gXum zf#5iy;A1g6y~OIuXY_DlW!F0X23dqmID>dK56dRt^T-f`J%EC1Z0*uDE=5L0Qc_Za^PL^{-r-_qz8Nn8<#xo#z!9jmD$c&wOPmJ# zApHOquoT~xAgD@%w4-{QiRuw1g)@DxcbNstlh6*k2MguSKvbgXMb03EfEURA!Hbp) zfmyk+Du;FoIQG}(thD2eKy%=&0{P+~?yExv?O>+ zCJZPJM6YRqdG8j2unIY>{m~O19uWoq`*S|U%;Nkqv#KGtgupRF(xy7X=^Ip>X<$+Tf^W9}=SxDQ zJ&bxz(?9{vuPc!lA(;R1zPwy$zBe!W9tfilo@KRi1+l0Gk(j3;db<1&FmYC#YAEG* zM7Zc?$m%q*dvz?_04=rc@rU$4Zcf;3B^7+u*HMVj&N5;P+o(^S0x~-SYQxs~?Z-y8 zYeHdm*EA=#d07+=FLHI(7X}5(5FJCf4+(;G&Siy4P_;sB7!SmW*WZ<>co2YDPEw&d zd6Mz&eAgBWM04jpTlc*&?zKs%n-Csz_CH* zduL7uls4F?13`Dy>r6m>U8Z;E{&{K= z;PlRg?(Xf`Qk>nK?-1U*EUrk!#-X021hnuPegYDS+t1sb=|-ZUA^Na=&jiSD&RoX+ zf(^!hTWMMpTd1#6pH=Cqu`a$bV4A)b%${6W(X?L*9z*e=1{eb%WP$IriNufq2z`4NieY?UiGXYjJu^H7=nNN<}kiCujZ!5gxqrscHy}J_eH7qKUuIR85SC5H$W#8 zi8x~w4@N0j+3O zs|DP%%gzS&7zG|ezw&cN09N__c!7Cy972fS z379^Zj{bbHb|c<~{GJ=f^+dx}5p=A;u~mTCT1N^$)AW@tOnY zD3G+ex(b9#F8Dy;M84QkaqHjXyOG@+yblH-m?7q}P7fe-;1fd8Q{d DQ21@@HtQ z{J~Esq>`@TI4Pp{Ud;1TSr&>-r}E;@N_m&BES)@@(-x&Yd72629>qBmhT=@v6$QWq zAV)-sfE%h8IO0D#;a+llYj|hN8}~bL1r%K%ZK!sf4za`>z>z&oNeM=|!FzGr9i4X| z4sNQ@ZQHtZXiF~h_QgPYAqUW=&kE;jn<9PY01;gGcw~&YyYhaVUq}}mS-BqKTgp#| z^fk6&C_2oyuvJo0ieys;dd2z?btyP|8Cls`k^J&Wu)O)6CZ?tkIDkWi$q%0#r(2PM zSI3UYz%-i_M~T^n4c2%n=oSLx8cd@!3ln zPfJabc--!Lq+Z9rwlR7bWUHn)ct5S^*Y~0{+v(54U&rS-7}oQ=l(e=6>;`{&n5jN% z^Dc1RKs_R(QhA9r2IA+)>{|718rr%9s##)8i^4_W;gAw__rs*!jhAa z$SEtAE&=KF#@_p(L@9Hlj&FpPy3+gn8acFD-qQopJRcaT0r7WKk3YQNV(-J~3F3(5 zILl;QSDL?aHpgy5HXZ`mgneWv*gmjTiZzW54Z()_H64s`*In%(1IVwDVq9Vs6&=|h zyY4)9hK>#l&6F-Z2*N_85vPI0cFnKInxx11>AdYo-Zb&+CEjZx&OmW=TpI0$ zjRwtV-vL|#p?-9_Kdesh0jah)+>Z1M499WueHHZ5K3Kk|cix_5=!R~#CcO-2&uYT^ z(ih+(LBZgS8?`T46>S2grKFse$3}nOhZZuM14!S1o#UCkIXczR3oKZ* z91W9hII{~L9*j?WUV{UI)PbZ?(Gr?6rV*epN(^Q`RB&@*?+j~_=f9mpT3Z65Q1o9f z$bO{!0UY)sag;8^?*N!3%_~FM)Aq*V{jF<4Lgf%82n0(3-VI7wE-WzpXHs@Ssca`c z3@AZEI=HjhILxngkN5W=rrT10vExIU5K4mmayRsF@YGz^SNFx&t~A~1AwFYC`UJKj zHyhgwoGWNQ)d|gz5zh;}?sbtr7vLq4MqxORj7T4*F#zbtI?q|I(+8;tu=MsbGMDFi zGDFR$;1D#he4o99$ke3be4uDTu@e#x=}w6D7cJ+ZP*@u@z}|wHsCuw3Z$TLo7l(U< z^Gi)UxKo`@Je?@`u;D`Ah%ds|W;DNizVz}rF?!!U=jh@0)XMe} zn+kwS^qU8gT1vheM7#<6AP}#FB4uuso>U3quSWTI^guo_FrYn82om6=@>nPsm$QAh_? z^?Y_M|6LUskBJZnIQXsmnzPioV*$`(<>kHIx3Ho4X__eX+?7u&Hvwbs>FotSU1c{a zvf~*X7Di4+Ch9QZyT^=6eF9+;7|_#6IyyQPV%JvPzVv`o_I%{w@Nu`sS#!hUqnBaT zgWoloWPbm=x|K;hVsTK=c|vq@-5XR1fCje5ojpF$hGT(F5n>byAK*P1UwFOz- zSJxi@;jP-}GlTbyziV0Xn~#=In+okDn9TXmS(3CD%^rP=F8vESBFUm0nguit9Cs^A zU`BTJC=)z!XXh1e?vCspVu>B~k#Q6zFKPbXtI0klOX~X6nGk8F_zMssgT3(;%tFN> zS8O@Gjw+|F=Ze+yL zTafrCaEg(4?|u*bcN_WPCunefl>ZYmT_E|tV5S#q8!c`(PnlL)3vfTc9gr96e7yg{ zEAMkCZ*w5O7B{+^&R+7cY~~QwvGFi6=BIII9(#cS73aq3AgGl`*2@wUdB|b1ieY)oPSWfqVMeMp=x~HyJlJNc$n8IgRcx&K6(ULk^^?-n zX7>D4#QfQxM=RDo^cwt1uR5Reao7H?Zppq(9@84uEW@ zbwPhs^Nr1`pJc|`C#HvZuD12=tT#}t3z~!t>a{m22_OJo>Pei0NSXvU&JRvl2I?JP zk$D@by_2oeX}{CL`2xyx-w2fSWCVvy{UL>qWGJ;&3TVfT*HaD+ zmT5)(@OZD|IR(pl63J@>Jw2=gn11?LyiQz1fW~H34~NjMiDelNf*${y*(hP^`e1>O z2BxBZIt!BD@3}Pe^sG8YUnlL=7X!LzVR}vv(Xk&=w>|y6BT zRA*YtzV-a@HCyKVbV8d2-!SqpaQ;&96TnKyp-EGa-;f>R5TkfO?&wB$bHkIXFF1es zTX=z@=V*qqaWO)>CuX4pk+2**{dfEg>$%VQ#o4H@$3(;^j2#QNXFa2tL`|a}eT{)o z7(lr47)-!NV6h`}&Wd;4v$Hkf;|#<}s02pzv3Xi62C-|<4TyTqVZWZl_mA@>ocVMp zL$1g~36hYD#$medVeU8ZVI$(L4exISb?_c(Xkwc4w_U0$I}h0Je8K8>_>#ZhGQ#m& z)0RJBD?z9|vi(;_B;h$;Y_l*0(|hNQkFX{ZaI}Knpglz;%g(XPx7=hl7QQ1qqL5bJ1U~>&Ti!j#J3B6JN6g2hO-Md#s2whN?@~r>no;a*)la- zyG-HvWJ<=2Dm$0iZDczinzR2d*RY3!z#H&$60BAc6D*ReZtjOt$FF~=rdB_h6Ev(w zKD%SvT4}Yr_IPsl_~#auy-E7Ys)Ul`hW7;?<%TD`-sinjYEm?}_D3^CMT5&*w!$dN zai3(mN%Ab8p}G=7#^ya=lSZTI1;MC?L7d{$nfmw-);PCsMSnEDvgc^2+$u=U(W#O9 z=$t9AsyA9^yKU-8?n8IYK>q4zrS+ZGb~Z{*27(DwoCHmp-=l{R^0BFZ9sgprWM{&+ z_8rg2F1d*n^)x87DFkbA^Tc{bddJDzALGJ!$F32m`lww^P>vIXY+}x5`1NqH0cDNS z-Z$CSC#GlK-x~Z``88ed`mRdk$KlkN-!GDi*{kvruPc>$y_^|7Iu!7&Qc|2|U&!Q> zcXwXz$`IkPGqAa z2KkT^NakNp6BqpOjMDbkzc;uNj=a`^Z@T-{u%K$i3rFkX4NoL9osa%F z*hml*>~~4e$ELty?`FfZx<9|80>^ra#uqbnuj-V!My?J?f8d70B=KGe-6IdS5fA|4 zoSY9OFI630GVyC&KHj~HG4PFB?{6#)BRr5JYqK!5E@JbJV-^TmzEHk;ox)!Bjh0F` z!=U*c&cQhXf!U^Jd1)n=b(5TIUj|jczwP zp~`Z^duiry#+AWJWh{4c-o*WR3-FbRdb|xl-R|jyF8M{)7XPgxZFtt5% z;-;B@f^gf(#!|Jtfw>fRm1ooRN55^#qBynHC&!pY27-uApj*@YG=T;L(jB^3;MdlMi_8_}y;cCg zw*?lSo+}A79d6cuNdorN4v+DXENIxQlQf3h(TlD=BW4bp?;N|cM)CM%676~X5dC-Mz_G5C)R@>_2Z$d3<9T$0dokr!!?>P*44Tz+JzG@WHAp)@PL^@r&?fNE?`%t zkw*|3VDr>fwCtBk7#ebzJYvtxk%q5fX>M|P{Wz;7FyM`rvf$gZ3ug{rj6ZWfko+Je zj_N`oa3F}sp_DNtWgH~-h_40Tl%n^Q{rcCF8lmG?4df^DkFeq%8M-J|IPh!owa-+N zfyWrUPC(!e5g?M<|3xZ0x6T(2>X*327aUI_1GzOX><@`*zJSsE>E-L~C&hfm-4>bc zIXL#6#GAeb?S-1e&HSq2jl9`tM|8*%an%2xk+cn~){7+lf+DF`o5n3KS)t=`&fSk2 zkN%AIL%IrTC~7jFf<=J%C}74L9=2I!@!1EG$bO0C4WQ3YG$--b&DB!5Hd@&_Od5h4 ze$?nZJz{*OxkQxW=J`RLgL&13D+InH;wtw#PeaUeM&1J$K;c^np$b00-+{6tEI8Qe zZ`}Y;aF;TPiReJG6XCr?1s#PmJHxgOah5tizET?%4p znZ@=eefYjcIY$aoo#71Z4VwO(lE$A^G0=D4B0*Wkg$|Ca-+f5Za1!3V{*Vd@TK92q zakckECMD6+&_KF`70CeEb<>eYE z1WCY809vyTb$2#1aDHuW+_(YBqVp_Jl3O!GAauO4;Px)xTAg!i;2edxXU!N)_9Iev zVY9!=LE+>n&>R?`cB=Qct9(<%XW?jOf?95@9Md7#VVUfn1vP>KAT6VEc7=K}<%6** zPw4{$?Kk}Su3IGN`#oDJVY3qb)jnbJLYRL{+!9-X+nrnuq+hJS9vn*N`S0!1#cv0ZVgVpAYn|-tJs@R z0Jd7G*Sah(qUKxoB>PI4h_!2Du<<8ctz@@fONQ<)MZ=YR5~6^~&hrOK<|#zrErnFj zA^RcH1`!w*R#r`*S6NwO-wq?C$^TC=Kb^-lK(I{!u9I!hn|qBmwsp*Ac13dY7k6b9 zZTm5-J=Q^RJr@^MRl%Q;}4BXm$rLex>|P}$0VuWLdR zmG_)e2p}eoIp`c|f(s50f2W=uzHWn*pqK(=I)ASo>N`JP#PY-`L5r_tfGek!Ko*sO z0=`_B`tc>bD!XLxu@vG#HEW-^t=w&+Wn&t zwHzWLw*ibsDx$N1F#NxZATndiXK@l8x~-qmbWi=9GI1Y_H29t}(0Tn-NQh3pgCm71 zO@4%OY)Nz#8G#pMyvp|?^gI~hz+`|e1nG-F5+K-M@yf(p{V0JoKS4&O4^5Fbg?V{- znVIV?d~fIEK>-pcif>8~dtSgs8vq*m_AE7yFU3!TgPGu9aJuvBiNcH)vg2*#|X$#!Vq7);{4YB zKL8&QUj~vqKuxB_Ap;cNu~=n){ftU=bu55Lt;5@YC}zgbqB5>==@{$}eyEhoK>3VT zBqPP@drPDwB+!#CnPaiIxOh=<2QbOdt3bjyImyz5S%7pMp*xRr-pLyPtCo?$GJh2a z0;mIZ+l@gucsHnvMZO-u<&+qd76hUFSM0@!YVv3pw zeV==ebtNLvK1VM826JQ!=vRF3zi#bZxqQZq@>(2J)&dV_+mWYjWKp~YqG_Nr6o56- zLA1mZafflw+FO&kw-n;(XzMgxu@<+VyvyniPiBIqHA?x>NqZAXvlRwp1e?|#5lVN! zQ(gfW8)VO`t)r6w_Cg2+vLwXzuH5io>`oadc#2zI@!y^>TGwKT<9)YkcXYOhy;@FF zaZUTDQ~?r^UBEgtJUw+mA(q<~ny1^l$|)%PURw(d4ZVO|MH{X{zkPLQFP6^xjt`dx zya85|QG{S8Qcu$pW zUtesO8eoFvj-k8rZO5>~p9->3!Q(N7{-l>>chKbtr#FY{4tgG(Ar6^NJA~g!zzWwUmZOWa{NBUiDjxt1pqm(DQ_8+u9Arww) zO%lqa1|%HF-!B=ctn>qwtAAB^q{&Pi({ts*z`}-$wOBs)k#PMbTio(xW7+?ij{!&W z+0{{Xsxjn|2c)!?B#tJZ54 zEvQzmStMm`(wRm)tRs3G%Gedc98}>sa|Gc2+A1Y&3w{Zy$U?Xn~FRo>ivu`fkd;HJA*o`ZpS+-w)ED~RsUx_@3I0^AbRvYc7 z<1aB&G&CdmB35HP$40Pl;$TJ9SZi-+A{#!pdyIt$nEGW0fjSe-U8-G7gxlFToO0`YH4ifDr6?E-&%s%@}aW*b|i z-E!8#gHifBgx&cN6}{3NwD~5Mgdv+~J68Qrx8ZvKR}QDZ!zI!oqGLkP6@VhHAxHSx zu|?Gfzpvh<|4wB8p?9}%ByX}tEJX=@Oig`f!j(E+-tZQ}${=R~-d+ak1NCEr?9q)( zFEe)8XgWA<&{PX^*zZn<{p}Z33_gs)1K^71~NhL6~^}AP@ngIU@ zAo(8>><%o7TbjQw+2ZO5D1=j|D>bLCsGGnRO1Vtv&5D4(a%e9uY_mK4SslR6%nQmK zH5koQf8_x7vCuK^T5F2)b83Bb$vi&N|8~(aq$=w$e|;0`rTMTyhDOVV1N^r;{)%wnyW?Qn}b5*2_VSh>neaMahFPw_;51N)JHxCXt92 zXl}Y6zn?v~hBou}r$2ils!n7XYkLyqHMNRpdX=SV1<`>=kn@|0vLcFeiyYgbut=DE=f*3xeTD^uq8K0^ASUoc z{?}C~S)RnS)6-;RF}YJ|GOO?|on)^gcY$yQKJLKVB(i+t2CV<#Z~XloBaJhkeyW9q zR7A7s@wFF9{>mOH3oN)Kxm--@>5Dsiawg>|A)F%uBQ%}dnTRNMACmPI!*8$gLA$68 zf?5oJ2PVScb+=vkS!qi8LaVDaEb>~)ysu2!=u$;bHujInGT6v?t{9bgy&(?JS*Sbk zN2(%%%22U%xW_(EhiKo>RR8gPj3p>UH4gX(kuDjhc%+^pbK%}OvAuMr$>#23#xASX zo*SzE!Hs!>8nQJr=B2-a+fC3XN%%;Q*n^8kWJ@?_K&Tj zP9i{HU<*G!ARF`Am1Rl}){W?+3ljN57{(T9Z8Oqi$i*DwJR>Yp{269y1_MftA}{P!8)jGj!ay3< z=zg`dqUY4f^C2PQoLhKXe~QW8vYZS_B1u!jxNn$=cf?2jF0Q`}+Acyo11EX09s>*4 zJLE4b`EqD@fh_UzDXv9T3W-d$20N-wYU?hh%NWU&=cj}ciQ6sU4V^e9LJ#H9jEPBu z{<}=jJB4_G&=<>7g?00yyt$v?!Y4tjx)&I8CrDCFPE+>7Q}M0rv}cl)UPMWlD=VTK z(N;MPq9;hchIzv~fQ?DYd9bz0VX2DO^4y7QJazamEtmnFGpgxdXL$bfKm46?lrIz20qAUgi@?E&CgsE*iZ?_Lht((%RoK6Jv1190;6i zgMalrf8cIEfFbk4-a8p+EGjB0;`FN!rcS(QRw>FzOiZNavna2ue5YGVid;T8NNQ71 zm1=@2n<7Xpk#k(MAUR5^dRqM*2kLt6`!1{Oh)(9qJ5xkU$SLyFLJ>*EcGN$?F4R$; zK6PqxY6?`$_Mx;8cynO zkpWu^Kvt~o0-5>4BZNN_&%2NaYndBvpKnw4GSphoZO|NR2VwTyI|m}Fsh zphkNd-Sq4N!z@A--0%ZnP;xqKLTC3Lw#=BO_25*;wSWun0IO zQgRnn2Bn=*=cdAto1Qt5WzFa9|Lmo=sA_g{)H}C+qxv7>uUIaBQhN4Cvw!U8m*R0L zh0sE6Fpj@m;XM>|u^!Z2^SZF;hwJ73ScZa5<1B*ganPEn7HT~F{rv%DABN6Fc)XKx z5>yg)kB`@Me0wQ)?AS5u*noflq3hQ@Jv?B#+tP5EJ@jM(E&0moYB$Car1dYnd;q5d z3g-(;QKexY$(PBk`X7{6((%o0H<&Y8Sf@8!mRG@fELr?PrX3Ee3jW zl!Ia)X2Qgu&e0HYn7AP*DA@k`NOg5JGU{^Q6IAu`JsVIFt1N%>Ceg*gPg|-!WmwRs z7ZHd-~dvWvKa^FUzba zWT+JkSk?m7E%!OQE-i49FD#Oom6Ci70RM>>A|NOn<|JJ;)~}+ykH#FJHc~Nx(5M_5 zMkU>Rxf%KCIut@=BjZUjcA+84Tr9L>oyy$!NoFlzKeh|)P49_I-4HW}PF33}Vj$rM z718{FKU6mF#e+7>>=x+2RyTZ&&lgdPl-;Ps)M7+&pHKYh{~7z{B8A`WT$RO`Hd;Hd z?B}JELeFU*+nIY&m_J*AK2q;gXL0wYD;OHjCpl#G->$<6Nae_d!U32j)P3>K>M9O! zVEX=p$6cTwUPyyROTsf% zjiWgoKz|y0GzrL0>4hCx76=#oE@sQ{Znyb6KucLk|=Mut?j7iiPR#>Skh&7qkJDEq#D zS{SdXiH(g7f8pTlJX&NPn#FSQVlc}-G?3{s90u81OF9_JV~d2JdCYEsl=k(41av=z z#y+3}HU`6{bneN|RYiQpqB7eZ%-H1(1{LH~i3oNLmr-Li$yD_jtH$6N&kUG%czwuPgi$m+B{RRo@RA${E|G##Pn3%Xlp`_`O zsOu7){2E|9*H}UK%*F~($oBoUjSaR?sCI8RLzBP&6CaowBE@_Pc~3d@biL(5>5?b4 zIBWE-?7O8(1g&h<#62^U9h267@NS@t;iY~+++(L3iBDnUSEq%DKEP3U8lASbwl;8^ zeB$d%uqO?RvH0z6f6%e6>H7Cm zzO!3!zQZxtp{(V$0BoU%XRZ9IsaeqnE%kQ;lq?(b_8fIDj;fm5WeG3r-;%MHc^S$h zD|are(TN@G$Dc0N>T;(bEYDdGC9|*s(e+Hl9nKJx4x3CVfW9s>N9mRZNDgn_yjj8t z1jaKj1v@)C|F^sqSg3Ee1|&vDCr+7xZGf0Fp=8x|_jDZHyWj6jWK#$JIQvr=k2(p= z<`OSa``u}&#!KXe@;vYoeOi^|#|L@l09I+sQNK!X4C7FU3WSpU9T++6L<+432B1;k zh5j47+n^NHWqz@{zjBq3ii#@9?+n19az7ZTs4U$*yP#h`Y?9&atBLeJ44EGa z2lRNke@XYajoE4(q}Y7*fiYH2%j3eN^eek^fA6+ZvqT4Y^QVm7ewdh}->40Meq$CO z0Tp=yD6F%K%XiGjR7~sFuX-i--b3^B-#)6Us;;iCNu-VypFsK9IMUXo?a2eS5KIiU zE;K&6FwQI_gain+Q4hIC^n4AO*F*x+ZML$6*M$S}PgHy?qqk03xi!8BrnS?RP! z>#amfxPx2VOlM*rtlO1f&l7oeF=8+n&}M(X7$wI7^9o<;bh_ql>Nqg$V$(CRuU zh3Dqxpg0W(dJ%J{>GRjQgmdv0=sN`l%B0W|eSX*4%(J;?ecskvgJXdB zE$IgVni{DXwdtC>v9>F3*@ema(Fd>S^wYQ&` zIuQQr16B8a!%c}ni)w(n0K4Pl;>yp<6ZP5ma&T|}C+-w@jWJQ^LH*#MJ2Y5?`_8b^ z^|2*=k`7fj0LM|BIPnn*7NnS4T3bcDHdmm_K=PvvoeKRie$SJ%w2GuP+jE>RyPORDe1NBbwfsj$d70qx*sqzqJt6Hh4nx|}mhCXv6#)+En z7Q2m>3Yv}{IzM*uGEdKyozGim3?8j(W;d0WmqRe*pRsW}M?<`Hd8C3br`X7^E}>HM z(l;n<)W3W88mCSFyQJ7yEnszBl4J&nlV@8t?%(+}GjqqlfC5q~qsFFl3YL?CZ>YW3 zd2eUD9T$8>*-n?bYe#=2*Xd~uBS*$>2OpODr!t*|Jb$I0)h!&5}Tu%D+1g< zjnQqk=)R{7KVb6Wvu9*`sB`fzGk<$QoB(Ag9nB%i%M_0zl9G}V6ZN&V-xQM|n`o+z zvjD!(-Fx$hUv(nmUfbK`t5eb43GVV?d&{KZnoEX3PUnri2JPRc$}kAc`7OdB;crA|oDgIE8^DGZuDKo3lJ=oJk5cb64sBXqnl z>DY`}%Vxv-+d(%jYCiY~xg8`i=fbjXhm&45ZMyD9rP*5M72-vmX0@gyL#l#w5SlA@ zFloh|X3!t&w?$)SA?lQ=e?V7FvOuUp?-RnTY7iYZPILd@qK2Gi(}gd^LFe1DW5Tk1 z!YSQ4!d>hA2f6x{B$pPt(!4E#ZvRMx3)W6(QWluJy})w<61GAuxUGwUywMUwdSU18 zajx8c@w-cAHBXQ%T$JHgi@rR6)0M~TwY!cgl|8iXC7+)h7=7z5pf~=c23N{@0Ne1P z4_1?hw10PdqgIAe1hQGU+|_ALw>iNQY=RR>>7VGGnlqhrMqNyK-YFH{W61?5x8(gT zR7=l;<{Wxv3ktO*!=});U4Ou(^Y6#|{WxRQw#QRx!+lpY?vfjz07xbXNPtigq2?~; z?p&wFyJ<6dqAQYbi`|3NG_y8cq6f-S(h?OvY)A^szr-4%3)|qSYhFFu9SToTSCg@4 zR;J&BOw|`F5!;Rc1RFOf6Y*y#I135%-Cr5om{EIaZ(iy;|NT30y5vlA|Tkrz$mB zU#Z%iTkDnilZ_h-!_PU(t>VQD)YC-HF~x}!<&=>JEzf7L{x9O*I;^VoYa3lkEJ9fr zgouDhgMgCKpmc+Th;%6^jdY4As7NR!E!`y`T_PzRf`UkgNH=_A;@E;e9KfAU*ABS)1 zM||H5F)gWm%Fci(#%%xQhXs`qoPCHSziAN)~JYC{ilbzVofm zUU9Whhh^uPRTalsdrP90iEFL1in<|lj4w^u8D9^;P6ZQ(z_2x<;rK2CgkRJD%=ew% z*Z3I4LKtVpX}PwLH*~IDYpm!j;4!Et_u`Kp#XV=(iME?x7nz&i3dy`1!X3${P;PxX z#=c*nD5&sG1f=kNJ1XxGc?Rt^M;+7GRbTyd&qyBniAD0ra20X6)v~hxXvB*?DxK3Q zTwmLokHK5sJKe!^Chl-E{gu_V1;*7k^taX4{Q@ZV(O0~zm`s9xxO|EY~ztcSGyV^Urr|LNGCoo=jC2ar5H1^?%2z5gc%hj&jvi*ypI9C=% z&%Qg2JOTbYy3WJJ6O4Gsyjx^GMWbuM3cL4L zvc;c%Aecb_In9%K?G{vSHs;TE0K?hrm%VC%39QU-@Zk}_ew!GYL{B)DF>~L6#WNV+ zT+G5(*qM{CflS?5Rg&u+{jE3aQ`B$J5~kwR&klBPp5%7gzbV;g$FhQg87pi2yUdm_ zGxGCpDKP{P@Jsfxny@#b7JUR}3zZ8Rd^QdZx>@GiMOyaH;P%$yT#+SW;&iYF*kvT& z=KSLsDf?gaaxG^1sYYL{EFl_l~45Mz@i%UdS4VORDXC~iWi1HKrIO?Y)NdMcNYY1xt*L}Ka zYApYgdSH5JN3}$}sFG6x?Sm+`S*(`P(naa>*^x|R+|oPW1Xz9uGF*vb?ciHr!&l@r zX!Wpsailj3={_nu!0ZA)Kv}4TB6I_`o)7$3#$yD&YG|asNcjl%)R=NtP0qQp)R!`5 z9(x%?4~r*N0+o%7sYa7*cBJ72u8PfKW#xb0s(V7_s7N!k7${cg{DRFn)4k0dz$*;} zI*&vC7>}qt7Onuc>o=^XapoopUhcNeLe7iIo{jclKHj3d*<<~yPFocQKVx~bHxT;1 zC*kI{5kZ(s;)O+@-CiwcPA$*kTK8AeA7v84-A(cLMjWRj=vbzhNNThkg(P;qxi;l~ znYv5XDq-I0EAl!wa@*j8_)*;krN1lHI`J3Ro7cCl!gk*CWh^WwhJ8#(xxfSGMrvzA zjE1dXMiF|hSbZ_aA$O8bPm^_MF>aD|nI-8hpLk?KFr=6HEVH3=kmP)???l(iEuLU~ z_31k<@2EP9yo|`*FEl=d#MHcdfm2I$$)HW`T@^A<(=0LW;**H;MnXl#MfNmD=496L ziiRoX@JjBqsgXD{nS}eaP{wu#bq zMn^Vl43*bz+Q5wjTFwl<^c17VGSiU;<9Le-aIpbg59>rgP<7Av*F3+{e(8PGaDedm zdwl-WCk6dFzJ&XCTn`UXH9s7I)zp2DxK2H)Prc^u*bCJ7^C^v2t0Vmzr>60bcjtj0 zBwEo8RHwM&g*h!H(B4fHG4zY#ArUkQH7G$xykRYq%OV1Ek&t{cDEk|Hy?{+eD3zgf z3*ShH&jd}TMrlv4TK(v0OIlQIE))t3{JZG63@|7gF*om(vp(M%O!QuRUYiQOi0S01 z>WH~dDgj5x;*#YDK_0)GmRPDdj;kY|IvengZzc7Ii7WtU{O^RB9LSvMiRkm)u*D5> z9 zm4!+22grRexzz>+kMo~I@2328IrN+h%dX3InV_clQRJm+E4b5I!Ed5D_=Jhxe(r^P zTiQev7F#cQ{ugAyGgh(?!WwKYK(BwpT$?z7(zL)FlIz-baL=#n<`fLcJ-{iaLM3R60i;wY;os{J#b z2U8dK)3=3nZP{h5ZEq=fU8c)0>``WQ$_;nlby}l-1oab`9jEikPPw7HJw~uTIv3Wm z+F+l^7j{b!{_*o&;%bL|dReU1rm0W;?=;0toReD7%0~{mJ4-Bcfsb?MiII~UmG3z; zFB3H6_X8pRPa$o_#cIH<`yDIc*5mo$igRYcuXfK~f_(Wmnv{@T+PzHMfVRl=8M8yr3y zovHB%9`~txuzzSQI!QJf%@JVim5csNivrL4H?Da+CV<`7$qX|zpJzLc%6D^1xLAGg z{=_+wk+&+M?hU$1;{=cJ?Fzn8(Y>8+BOGoIS|9dyNp z?2IaDPOMwlEjP{OBFgxF?n8C}SEalV*O^A}4T>95$P2>?8phiFPAe`}10iFAVHVB7 zqCVX@twrrb0dl3LxZ!rysqa`j%&ql}bLo1_F&VF< zg;cQhJHkA!Nf!RTRG=>^%}+V&0Ka7=M>CojcnD{!x8ev^>V&;`g6(GrQ8^XcDd_HS zs(qZe2M@8@6-;uMjHMd?wr7Q0IiGgC9P0c(g1d+Kiz~eJ$1*+rqa;l~&ap zt+M8SoAuXmPCZ_LU7qE!SK?5K+L;oCYTFNNlcEsB_B0vO7mXDs+TtINBI+Ub2j6PJ z{`Z^XibGgeTH4!Ryn4lO^tE|V7GT5Y&zafTpMw0qut6p-ek2((B7aIo1}A)KWoHXr z0R4VqNRGh-6I~KOSgCYphox_}wzNcDGk%(twE#>R0}PuH=DU$)O~Zv#|MllLd}w}W zTky$BDVSjhzCRUC4**d-$R)S8o$rorBMO=qdE2{s0K{DF2WbRcHv>I4m30n1mrFD3 zkY5lVzbx;m%Pn{sllX21C*0lLzZTzbuzh#+(LOL~L0`h64ObZwE=Q$@B=t%Iser=(GVGq6%Yw$JL@ zN1hC59S)0+&%2|es(KCdO5VT!c(hvyeXk(5J%1kB>Rnu196^a5V7$pHXayD^v501v zii(Pw6a_jgbmqE(m9NeRtGVfEr~~l6JutS8=eIQiL6`TH&^(`9Ok@2-m@`?J$vGhB zP{)APR_$c1J7;OnKljHdXOljnU`njPd3Lk*#TmNtrc4W5hY^Kq)=A=0vRv>#)|PGRZ!!AV$au`?BE0*y3BE> z@I5rXPyU?ljI96`IsM7O-#6oZpXX#=S>wQBg1l%V zGOsNtz|xZXn$Z&_0KsLy$3{k`t9$;a$5HcKos`{ZgQs3q+q1>x0k0a*=_vDJYFe83 z>)`l$piF(k&IEJ|9T&x&oy$Bsva{I=nHU&8IlI+zo*F(Cdl%*~#Rs$`m4XWmt4O=L zi*N1E+gRTIhPiaKQIfEC!Dsq9KCEu7!+Kez{G@MI8@$t=t=s@+|HdMYm`=g@BGrzy_vf}bmw+iHR))B`h}Z|%N#krmewG&*IUSe zyrRs6*UwL<4@y9}GMWwcm-}5*oTqM*`agZj+lsgem4d<~FE8n4t&E%4p`SV$)j`OS zA$~n5-z0s)DRB%=avNv2d@QJ}Ku0?ipyDGU(rVHHi_HU1+IJsmf~10}axxH~Qq3e` z%^y(Dkcp0ujm=>mu5#mJURhh~)sHDH<+~Fe9sPj8O^n}jf7||xFyqrVh79ZjJSqnq6D#|`a9+~&?cpWF%v#s~Cs_SE~=AtB;tkDxE2ZNv|*397T-27;kbfg*YasLI*9 zPjGxIw4MRK_4t|%pc{;PiabFb(KCQuVyS>|KC!a05;h2=_HncYCB(;%2Qz3Q+D1w& z&D%Rpy(~e4e$Y~1Mp&Jtb<_*@An+JHW~5jxGqTOFzz>w)EVx4Kfh6H@ex8N%2ucz2HS^?4 zbVG#!j;#QH|9e%g8nhuMrlugK>I@Amh`o$1yLGhivzIUF{lRVgjSQUtxS~NX%Acus z(K$+cb#6q)?u;41uKOw13vf$-Ax&1DOGc`4Z6kJh5ryX2{psaL5 z;zQk+RP!r{&ZNRa%!dXlq36OXS4e}v`yBiu*QvEFk*N#F`A?*+RR=xF&j z{ER{gin&a(zW<2V`_!T7mQWFL&w%fd!hr$ehr^~=>Dk#1c6J1Wgnm%2N7Xu=!4@xA zu&1lU3p@Qd-{8Dg&3R#@$J|@kf(?|Ztot)b+6p%ZQLwn-cR+mZ&-;4lQ(LgAXpa;` zh>ON11MR!hkY5_|V?Ych&CO?!Lf9OMRrJ@dU!iFPn8+EKnFvh+g^D4=3s?J4hiT!M z!0$g1V^30r4}Y27Ky_QX4wi+2LOaMEL5&?a(ThL3q@|<~zaDUagaa?%FAod$J7*eV zZK;Ai!uPe$7)0{(t%S6qgFTV{W2ir1zxw%G(g0&KkE)GM?A+H{+!L@fy!F#eTbs`> zBCyK?zUBRNvCiTz(v|nOT3u0&IAQDy*Xlp3Eb3j&KP7-97{|;Zn3y))WvjHq5h9<1 z&YW(1m$>eKZk7YN#FFLbFCcRU#+PPx24CkYD@f zI_V9U)6ZuPU78rSruA?dl_sf(61opQZdS7}oc(!u$b5SxsryS~s&Vir|h)b=1_@=Bp2*XtJ}?H#y5Qmn~*QOVc#*?$+7 z9I1K{Aeix;~h5kWWMBV;rn}74!z6r z9sY`nb&y%J{pWOnpGDP5EuFw;F{~*U4PvcG4tyT&Iqm=C8;en=O4BRU^se&Bw#cnw zj?D0OmvhJQsgTgeLX@^EG zoHP%KO1Pj?{awoq3t>>e%tZnVc-W<{egTj3F*C~~+N-@dZw-ssnUncsq5hf2_p=D! zeoXDw)ntyt+Mq19ua($(X9~2gWPu{X_Kix2TzqBCi_p$??a`Zr zaIVsx+XW*4nLjhf_z-Zixa&uF9#Y8;0s7=_DqF;N)oy!!R0(0tp@^(8AELid+4$P% zx6l%k6820Y>qy(}_c+lLG3DyY&vGAB#$A5>dbm5oZ@SLlvdsuU?_~n2re;4AacU-~ zyj0KqxQ=QzCewkQOlKv40B^>;{Bw7;{|N~pXKSIb%9n{|vwI&pA5PQr01^f-&1jHi zjx4O{(C^aZz)67404#!GpKZe*t@53ArAj<8lKi^0;b38h_OEwN_0`XIx(HoPO8~Po zQc#BLxR5cfy1kJW7Ch8gI^LwQXgxUB7nXT$ghWuQRl?nBj}+&quIJ0q)g#wLsOhjU zN0+Z~X{nn0av@Q)r(p?9@j1#o8cFOO#vc5|+Hq;91sT`1VXPts2b6Yp0fZd}T$|&D zOj|i5O#W9rd3@CDQg+&NxPfxz=t|wvNFsRBG?6PM-bHr0x_=RKN#^aVu$+Tn_XUY5 zubc2HGQyLghUGSsnvy2Tc?qiGEtQ6+Js&#=9G~9a4I_Seh8Pqu8UVsJ$55T}gKaX# zv8A)RthTE%JrvUDHLj?bjP^IT;AngEJ9fU&X)4jCDUJrC!((~J8i z`n#V86v8~O-Pzr7LlWKo26Rtb^NoUJ$ho@`_McdJdS;m)9@)71HsF8N5#bL)UnJm=F~pm=ce~5n zYabY})5}Z;b5>tA46Zv~9R?V9Pb=`!k7sE`F>oLR;wR!G@3%g|mZ5URB=E$NU22 zhXyr&e;+2^pAmp6?q$r@48bGA!5i0oB{Bu5F7HgnD_V#SxgBqqDCttP#@|;C$VbmZ z?Vx^)wU)?%-&!mSfTT|^UIEI2{DYn8uRDN!MqOZ2-LoP(uR7q{EU^t+toyCP-?5DE ztWOHU>40nuZoj*z6yVwZY|DYd&fGiWVpG33$|iId72G6MWHO!{^qBrsyndH{lkxao zkDu;%I~aeqDaJ)r@2WpNKRo6_y@iKPA}m*AjhVRp zcGi1Z(3kPeH=A^C;OQ9{d}Szbhw)*MSpn-B*`SVZr4~T4B3$O`ofjRI&+eYFBCM`_ z7WnP*yLhkNZQej(!^%ni=q;Ap@`|BB1m9-e!@ueC?wE;e05ute5K8qraX-C9u2wCi%l6 zJw}6G=GWicIl$#zOA=EbI_fy76)GJ&%!oPA7P9xAx`^5o{b!_VWk8B5IXStdW^=_6 zTByKGHZUmYI7Wd8W^o7)f|T?|EO(bPL{G}kgxdjfHuc>AVEhi3F63Lt80MoNxT#E@ za8tpYYH)UT2FZUzU=`7Zd=??{0Jl!ymjKm3D@O+@V?(h<{G7Fn_OC?X5Fo%FbF=uh zsA~xtR-GF2lk7XVHD=`<$FFb`^$_A3w=QmIkp&9W#Lu4(ZGQsEQNU`5^QL@5Lqm%e zGJXpdlb@p3NHMRi{8T}swWlidAu0dfYyI2IrP`8Sg_Bm4rOCyaAy z#28{I8o@~bl96JXe2Z)YJa|-qxt+`c(LvF#8%*?6Fn(Ev-iF0VT~Opw~Id4kWrZHX_Dd-0bNTfHH2{juqrw z`Df8sZ43k?GraN>P9LKJI4Ai~z&C>t+q#cnA%pV#d2BZWS~r`e5Mh;XjTu;Qf-`!y z8@>>-dTTdE!QzRrrw?ImB?Yd-85}@AH_cc)e&w^chS3^@EUs;3n{S`|GXY_Oi=A5% zU)q{~B15bY?=&2SWDe%vBJzw@fG~nfL6$6V7o3yWzx z&`6xVR7#R2bCRHFxSeo)hn(e68a%bjHrrg`icyWHlZ!*w+@Fa`M8v4*98O$7l8=VG z?f+Mr=u6dyXzDCpiZmkFd)P!@`#SgsSbcM7`aln4-}G0Win04|MaVkd#~EE$gCI*+ zl~>vbPqap3#S8otd}{BcLIi8%h@dF(TCUO!iJi}*av*crP-FC|0FPI4|oh&U7A z)#{aV!4zsUc3D1I>Bk=DW z2FgJQo{TJ_zFuOf9wyGu7M$K8Kk49|vY(YFpi2AjiM}WlibIF3<)%-Ev5_<16O#Jq z@Xlcv8#a_s|75zg0Qu=k<`rry^q2F;AUDiNZye%CyAr6oLgsYp8jz^YT#=eRAKDl{ z`3*QN0#l#@OEU!$w6|*ze3+wz+FYf`Wo;Uy0i^aWpIl`I4c+deb+9i*B+r|vo&*Fe z47Rtn(gqnB8NDb)pK5sB(N&Ifax+0`w&%|Kw5}8`+wVvq2fO%Rvjz{4DfA81Vag#t zfyDpGGL?`b2I2flFrL|H%gkhf=TA>(ihc0w*Dq+!?5bpAW21Y11a30xT4jKC7RqGY z=C#0_C&0N=44x%FmrfXhJ~i~sJ`uL4Eh^{_7K*XJC{aK_3ozQB)3^47p=Gs8Bxx1B z4MUt*tH@#<0AaBA>8PnCslyvkv9?|o7`qO>MJ-{BmU{47YusU8|0 ze(UTJ2<*+~ErD={i7L>S{Elh2Ot3wvZqnSiQ4Sj+_|WE~H)>^#sD*#tJa`(03mZG$ z5Z3tkW`fJw<12&tvl6mlR~B{rulX>by_4&*3aLdaZGf*50AxL2SkFvN&2fOiz=CC7 z6>l2*=OpN>wtN62>(DnA!5YpWeGxhhn2M#G)ADykIdCOeADmuVQY1}DO^t4$A_h71 zLqx;>7VQw$&8`$r%tzT3H-2L{0=)k^bdhkC0eWKSx%wLF@ZT{r#(V&DLtT9hd}ct3 zc$Nb}Wx*Ihgxh=50|$(q=rX_zXE1zwZx1X76x|*IZI;#%Y<8?H{DEQ369!O%CnVN@ z>620jij_7DVJSKNDli-g=E6k{pg6K5uLu2o-g}#sol@6z38&PMuBV!P{eKJ8Dq{&7 z@k>ZZfH8`A^JaM6@VdV7IqbJAo=4(n=GF7g1Zjk*>=STq9fS9OPLXJw-S71QCk?Ul zKH|G2{6E0DLUt%F-o2FKWciMc2-rN?*l>c=6~TM`6Qfv=It#?JIPd+S)*d4nk{iN$ z2rxXCK`kn_9$F^^1O+XF@2Ow_XaO*02mr^OnzXdEii!#)93U~~-V-!_WCmMEBq->D z3goFAMTW2t>ob&;r`%K+->plR2qeh6MdwuH4z`_J|q1@u);=;mJ5WNMRap-91zA)fzGtr0-ung=hj*CMfsrL^` zn9m0SqWA#>NZHsHq5J*pb2^ag2T^iZ_RgEb=3t1_yC`mD#m2lppOtruwi|XtKC2-h zRc0%wXJLhjfLuB!e*4S#kmI_`aUhd$;S6?MhHf$`-t9!eyfSPzm0;oRgCFh$h~<2* zF#z9C%Mn*|1B>7IufV~L>bsL~;FE{}hG!7zRXaz={_K1p=>owLSc+dADNHfWaf~B* zcM^g3=i+i>QM{_kyZ?ow*tZLHEupq%sM@bUg{6r?Q2FDr5i4p4+*E)3G_!~?)g4Ef zte%Rz5dEOh$QJ0?APs+rMX2aKx%Ur%CL%7tW@2s*zDi`I^HeZlyaUSEaV5OMwDSh&J* z!7Kv(v9PeGrG=S|t-Wr^r&iNGRQqG!vTckPzXI(i0aR^Zi1oXd;iQHI<5{hcyu&qp zu7++AFT8tq00P%0BfkKqUJ$&)Mw+!%B#~${^p?Sf7J5n?%_AA&lL)>Wwu%~CR{v;h zzOmz?w_|yrdikRv^`^n${!WfgiP4>b!&i2|hYbr;cW@~77{cnAq_BD${F8!F-oz49 z*nfAf=Z+WmD!P=?*)B9J((nX|H9$!ext*xj4}aeb#zG?g4|reJ<^v)?%J&}YXeBVE zLmq;U`RknJ83=hYw$^~NeW|hqUt!KKgqYy_j=HOaP~L_k{1r#Zek#w@h@s9W$%BdhDJZa5PX~KiOROX zK+9Ylk@v&I(Hf%XSL+wAnWrbHB~+Y1=1y}l5>d_bP>OnvPR49?3ND`_=d6WTk-=#i z45DQ=Z$edWgAC}~Cbc)Ab)b8G{xyE(i!?%)H_1Zf38$NFw`nykF%&D4zVoTIqn3** z0O%}yWD4zn0d) z{Br_=MQGH^(H@C2okcR^tY@oN)(Gd$a1yE~i0q`W_9CIZo z(RWfX1tM*2V8C;@Sz2>LP!Qy3LicAn_{I=>6(^oIav$A3tQU(*yS*eg}AGCcn(!O&f9du-%Y+b zTIm83H&Iaag#Cqn%s1N!jC^QA(mR#GiQ-aQp`g~4)er$}AxB$YM>8g`|Nav;S^97x{8os&;Vog3gAkBgBu0h8fyuOc!PMsPpzON2G25?olxt!b^;Be;{3o8+gMnj)j zW*Y>nl>6k043_P#(6@jGBKXj-blLK7wT*)(+LvxHI(pK+-yeF;bjoagfP8cfSWc=9 zfBCWu0YvnJ$%?Ej#4!QVVkV|pm|AKPAWS_wx%>bsS7Msvq@<+8#44Jaq!bi#)jY#g zg|pUPdogCB7EgQgdZ}PuHVhBnxhf1w)Kaf-hCjeONw=&9r)X})!JRo6S==4n)q9_> zX^i!W^E=EEpA|rL4FRO>v$NqllqYJZ_e55Ls{FR|egqGkWd8Aw=%gsT8S$a@1-aGv znm)vqyw7ugY+Lo*8@VZDRy+u8*Qt2-eiGb7ySrQ|EHNN7pQEyWZR{lt-aHzO2Mv(( z^-$sti^$E*om{xM`wU9Tjs##}zHIqk8rBKKZ8AR=@@jJ9Yr-gNIyInIslTfX?f4bz ziZj>tfZmCk0&MNaf_yoL7mj~B{gYRz39$;2dQu6WA42Tx0 zT*NI52`2^7_xY^{akiW}!b~hIY7uNuQLH%>JzT+T%%V?!#gc%eTV_Xaxzg{i+q-XQ z?-kNc?ZZai9bQ00^so^N+RBelwa-WezI+C{<6ab?fvK>*xv4qUlNY?QS-ZUv*+BJq z%5I~9kOJ;S7$*JF-80|a*A&xS!M!yJ$X!4W>7$1}*TKBJxcvgqDvVxXG{kV@-^+bQ zdn8WNPVqhpU|kf|JvNQENXHVH6!!x=DS%gZiIA{i(dXqV$jD{de;FMe#a)6HS8-_1 z>z|_I0vg3QI8H#}F_oFShl=}w@N_=9MwC)&9BE2yCgy(q614O*w9@TA;5_86ZYtg! zhs8M`iZmkbLS;-#l&_BS)S}O;WBxUy8Q#(n=j5V*#Y)$NXDk6VOhBjBDgMlv5y`MS z`X3!K#N9YOb{!mlK&iAi;J^#U{h}#`qJ)ZT80+-#@Gz8akM?SgvT|=~CLIw zZcB@EWdWDp_w3w$3ef{pQ1H%_KZ%6}HU*5)`>=QDeW9T20^r{pxRz`e-eD;`AwY4) zp?@-UUI<(yS5;=J#-9WU0JcAGgZCvvvnaHO`fZN)Yw6R|BZ2B2S>q+~3Oz*IY|2LBswbzh2~7ILqO+6V&QQZM$w@w8Y}|1DEu=(o0yoS+d~)F7Z`K^3i@wiLpu+5Op~lz^_>cFO`oK6?@y;9NRa1kRbYmoadAg%QE@`Nr`dL4;C=>TD>{tl&> z@=a@`m+B8bY+w$4I2hPbx~NIInwlh=sV1kU%0GU5z+w^V;rKQ)Y$FW2 z9Af#etosZ4M=W726ctoJ*#eeON~aHTi>XzKHa? zxXH-K2rrXh{{Uyb%r$#(JKKTqz#@+lfz(m?0{?WTP={KePtDC7nO#@wBK&j0ExrNc z-xVN-biGgiBD^*F_Uc`Un>RPitw4m*2tqg>o_Lq(DCjJno|?*-(d%p34F;~lHLNFm z-<@yTvwFhp(*Y)s0tm^iGX_xt!N~KZCxkGvVGLlxrsJmf6#RQ6rKK%Dxmk5lL75+# z3p*&lYvBk^V(#tlv4W40f_3x~Skg@)o4zO>bOIqG8Ea$u7#g%f6a`=NX^Z#(6<#P# z^CA*YKtyrP)YKH%AL@Wc9`s|2Gl51dV9#H^%v=-@W;q`S+Yb1(YS`y__6``^P1@~% z1q_F7*>7;K17EGY-I%n~ZG|^(+%QomBp~2czsgOzYWkPUnj+V5(lr^nQ1D?b^#2-m zT$rxaG$4G`TFTz&f#>w=^hpM_7HRCa;(ka6KOhFjpM;j*8m7Q9r?(%INP1{~c}ysx ze5J@c%H0qX&Exgh1VPSU`u;=tE|bGXapU`_^)NTTk3YI7TiVgQc)4|%&xSh_v7={KM4 zlX5?9XdFO!XAs{e5LgO#U(t`@c-)phCT|5NuZH~pO=7x zM0F;~bRqSMZ0U#BY!o>{#UWSA{51*w`T_XfaloX*EwLdy-@J^}I7O+RwO8F(=EmEP z0I>m}DD-k1L(GdNsMlR5ygvc7q=PNjqAQv=TB8J-8+I;)(7LtrRyo5yU?=l>vDopw z!`=TCeSE;t(z^u|0qAHDa>KK8KL=e(&`3cg6l24B&)@dW-WC2Ol!X=0KAw_kT-Q4L2Hjv{DSCbn z^OQ8*n$mBIDsYK4{ryc{)mOFl7D9g&CQx(^sDy$OD9YF9Qu=T*K%zCSg{a z{nOR~MXg`@(kYatUV9qQ;4g8U^deLEPa@$h;O?vTY5HDT=Os<5Q(@LSw?FpW6%rK|3q6wMwX#- zS~rCwHqmom&(S2vXmAE-+aO~hBYIQ{ss)f&hStPbXWnY@RRtNEJHNRh1%e{w)Yiq@ zjE}p?lsKcv%91FnH$`w+0W7S2wN&@&;+FBMB`Znf4h}L(kcFWZC9bERM@ps@4bx)= zMI2mw>QM_bQTm-QZgcC-yE91l;_cNl^NZrp=i|s8eK^MtnXJIld5;@DJM_<6WPn%! zWaiQr6>^Xr90O=p=;>l46~LFO0osioexNz-iB zQcz^VtI-V6V^6z5wTJ%uw%`lireyM`zHM>}7uYnv+$+s|Q%PeWeZu>RQardpBLv7i z5BFFIt|KEOCnk)LQG<{DIj5ZZ(Um86pNSJ4xj$@CP_)>V_XyS9_jsfj@%S%UxKmBR7N{488 z%*1Z(fH4LTIiNEQZuhQ}Hu^LfUr=OmIfwUwt55@?Rke5A4u(mqK>J8LODuUWTJ$yYXJ-k+!b zcu$=+xtk0j;r}1Xk$6^{-ItVXg6I#<@_F}VIDbjmb<@?!$;beY?uTOv*o~oEQnzl+ z-cGn59c~T|@JFBm0oJ-eLe_gQT$mCQ1F-Rk_GWOE$3)|EXoi2i3T@^udweUA-YDb) zWy4y!KDE&diIu=UHRDaJ3@MhubenzxZ(|S_+IJ6)2~!ZIp&5rv002WE6$2ko9U!FC zPDp=dmzVWt9<*EpJ%ti^Xj^|58w+V87`SaMjqriwH0f2H?d@#~OgR{m%T(8xw1ff~ zGbA_$G-kl;9SCExrmNo}xwL%1Ku;e+LPWux4tf1$9gB;&{~En86^7(w>%sP;j2|l` zY44cv%I%ijGj6x6&R#mTv1!9)R>Cs`@J1VeW+bF>m?sn~1*U-V4X(z6c^r5W5Kl&M=;wOg(a@Nh zm^cd&sd~q^ZyrE$jE?a4hx>&%+xw89^x=pUDYy~iF8N3#(gg878nvhtrg;Hv|8R%M zAp#%dsGb(9TWz858E=z%na6*WRsoCC)DE`_7jA$OaDZiiZ)9bkEA4>(byZc>f_qzn zxF3`yvRweD@#5g%xOC;m{QO7LHc~}d&^49a2b;FUnhzjIA0`PQ3Z!gL4L>J!b#+M% z`%O+x0zbF$)2Dwe<;Y+PlD&r5udutl7&eYic@xKcHi(&M?=<{AHsHblIDBbq>(%dr zNO(}s?V-}W*OE{J8R$E8>J%MoH&xc}(Qjh98QQnkmT@T>HtITchJyppaU>9SDo(V@5`X%MX7tu&!Ds zD}m_?1Yl|kvAt^F;$rU5J^%?V<-q+9z~9uW4^WB<5dduz-r{;7-9l;Tw~#3CACYR3 zlKF*TakcoKgI-&oM?u3nrMG$KGoRQobaqIlki(MpIvBZxGcPg2N89j*5&%hQF9 zQHwB)PfR3ny_S%Y!j;_zNpfhn1FdL5NFDz=PhjErO-p{VF%P28>TP>5oI=PEPI;tw zJF)h7=mN>P%-X$AU^x%~y0Nghm@VuiE*u^a0hpK(6fGu?;^N|<)F?nQY3Bkp!58IG zi3OVmz?KKC@axXLp)H?F@ra0|{%hFs&t`^ODR?fO{rO2t_l@3LZMKjea{V(jiI$$A z(1k!V>P#UrU==(&Hz#Z7bik&^RhRVi@tDVBSi6w*07fogH~H)^4V06LExKv1URCr8 z*-iCN9Y)~JuzDb@3e->l?&NT=yYa8-F(Bm=`88hsK-R9|C##Yo6HNzAm9CQgpPyDj zcKH8~#Ws##?{V%xTuEc1$y{pseM^?j%0^3BneD|#+at4*vp-o+(mB9s=jp{CUk_%q z^XqV~M1VKl>pne}H}h$maE3BClCo(V%=+E_zbdM6{JU^m8rXMH#=9zi_z_`e{7Gw{ zu6%oMG@2G4c1xMr?0Ty7GeJ$TY(Sl17sJMVB6Jzfvy}JmXJ2FwnP7`U>J0e}I~!Zy z9c+YC1)t+f%2==`$`oGjNM9j|&}DmGrGD36-qia^=f1P6s}l&(vj?l1nx;T)LtC31 zU{2(NAS)fJi3YC@8FZWvbFWEXp6#i!f1F~6d;Iv*Mt>JZ1ltvQ|7J0F|5x~76*2Z8 zL-@Wc2R-n9LiDe1@za`K7Z6b9K1zGqto6<=)nMt3o9G)k61Lz2+`pn}IDC%AzQ@)| zYAhd7liiX$&BJzei|26%&PcJHd^;)Y#z%9DR7YzaC)*YqWo*^a&kKsIidVOAUN-4+ z{4#n=>l^4>$Wh2ut5CC?ynNId-^STs$ZO~RfNvd@D_`&Aoy*_>>1^u2hSj9)j{+fto z|C=YaJ}G1RwmsbyV|wx*qoUi&e0+&0k#h?XY<;8RlDGOjEKvfI1J=`uaf+O%T{l|oy0DR1ilT*(>^7)Qf3~OhX`$G!%b73 zvoYUEDdQ@L%9Qw(hX-HzGnTJ(l6Qoeo0qp5pNp%e{0iCfOS;uI7G_^oLyLR|QOUh6 zktyLQV>P`hZ)<#hRNr>K!0qlMI_VU9FGL{Y*ytqv#r50%K3hFAu^tH~)Ez>Xik6*p z&OYWm4tI_Rnps2;u!Q*+>Vdx?)X-h*o?UNu=Ni@TW{X8Ga_qd5JvY!$xm+Kbw(qLd zazLV``%087;!Pg!>L)zra-1XnGaV7ftr^~k`ZKMXCAZeuqQU*QoZ@u{B%*lLdwHUv zxoMLYt3qc6&9iwxS5h~ zzjZV^_+9n#P1AY$Q~ne$J4CnRw8~vcR{77uUn7Gs?j_vLDnz+qqx9K>-VI6z70t3B?7P&+Fd4M4GjANP3A(uy9#e zSB5{_n;yUq>k)LQ|4a;~89Rc$X5;Im z@xdB)UQN&Io>mv+U+31lx4GA?`mgV$GpX4T5`A+?_lWbg&2!W`*S5PaO&|a&#NN*# z(;=c3sRZ^ABxL}rod(4~s4Ifrl;>&-Bcx@Nmo7oI8IisNcW9th0GO;;n`+}s3G6%<|!{6~5oe|M42K02o?qqX15KbiUoG=qUJ~34Qx*Xq=3l@Y)B=PTI)0!& z&;jjOz;%R#s-(1Z@Dm;cg_>FYBo^{lRV&9JHUtnKG6H1pdI6qU$Ug4{I7)^l0Bc<0XW42T(&8$h~*BbK=Jj)s|jAD8&bfj5+^2jl9vn z?>cplM<6-7`IFDc{o^ZmufC;{fxIrP2ndhKC%@s=oo(6!Og}sgJqFM}Voa|&c@nt- z@=v?h*S?ZfP^>Q`4@PeUU$hd5AMKi(_eqd94q(`sk0 zOgbyz;#`7VDS0Tbqt2t5%&H0W4kw0crYi<+xKVR zDS{t_Z0kRMaPC*?%moo#k^PNd?_TeTB(A_lMg82>`Lf7@lB-^7=4)Rf5Q10f?DYc! zY^8Q+;=L+)M5d@PT3{Ry5do&&MU_=mO@&bPf5Rc4+zlK&sD)ZX+CA0!2B|HJdR2p| z)Sq;h+RzoMksQxN0#02B&A9?Vg|_h9AM}UwSc`}70Zx@+u7T{oYkdd zJc1%Iq~LU#dM}0m3wB-@xSBl3D=h2*kTsNuNiFG}e8|HfC&2-;d*|P^&(>>`un6xe zVpu>qKl0JS%ku@v&Df(md(UTaW{d{!Deg1VZVb-8$dbK*xvr5Zp-z2PQ5+3tlJo7k zI+*0l(?F5S`e6omG+C1)IGAMZ=K*;r_#z)W@EB|?@}YJQ9eU40YXYfYp+U>_XthU0 ze~%y3Fe+hSAlbAUEi3rtbRVMaylhsQi|#8;WUKK&Xal>NdgEYW&FmR4frhe0(Dm}! zO0cnj-q(H+1@*R;mKLBqCy34Mw1ppNq>@WlL_u*ii3R%o%72ju>IdrFH7tdF!sv9< z=G!ts&>mZdszG|csq4Ui6=+|wcMtkAx-C@ zAo!?eYGOhO8hw@iMNw7M=g$|zogqbf*twQc&=Aqu7z1$ph9tJ7hI$-y-ad7BUI*U`{*y9H=x29_C?D{yA zIAL=EJxWP?ELCY(;A@?oonOZgE}5rQY?*aejVE2!H5g#dY>)Odf;jh>w`0Q!n;+aYX^GI^Yr!K zDcqEhurgbQ`uokwYv~$U1Hck#f|G>zEfx4^%XPwhX7hLmkV5)_Lw{S_0?frO&()a7 zNPg?@LN`i)Z!i&_m=^{Q0|~3zc$cy{SCxQ&gO7$&0Oau2e&*ZqV@J3&D{S6 zgL40CafWXCkL9=7Iwf*0Y@d*Yk-}lt-Y?Z^AQ|L zD=YNpy^X*Vt$gi^*I`lLdhG)&#G0K~zw0@d4Dd}F??QlF{oYU&65F%pULNXzmNen7 zz@Y^}tAYGn-CgS{ddn=}ore#Jg@0LXxC8uB)t%FhQmyI%?3kJvOcIb+ zQu^TFs?yi1ESj^Q`<$Q`>jI8K46ot~!{;5hSu|i1F_quJ@dG?0N4rJ0*cN{BRt&7g zIZNZIgOqOyZf<+<4!%{osOz@@_xUr2!Lt&Uw^uEm4^RCRZN#P6CNc*~V!MglJ+GBd z3Z4rIfFVvOzWVWyd7~OVKnd#vmA~W5&cc$}7cAde1h4zqmeOwO(|~}WoRKoyE9i@; zAqCfbgSwM9+5kw{D2d{+e9MLbKFx!KtSBz?Rx?miNVW3c?45zVOmlrb0nSKR+)pc3 zx3(fiM&MZTLt`i&xsTVL7YYMFE!8hDRiv)U>g=2rKXYZewj8?NSU z$0@JxI>p-dc6QfIS%Gvhc^27ET3=J)txou>#k*W?iBSgnO{vgg-t~uOnwoEQOO-`- zZatodCBl#fBxe>~S2!`IOo`}47JyGu*yk+n;%aapiVZJ5{nVUKHTtZu`?kuv3s9(r z&7vE)vn$g%vs^(SrUAx&Hn1qq&fEh)1>&{vEryKSEZu4R9nYzOCoT1YygU02mu=`$ zS7HNf?SUez_n{C=?f%9TZYOd8tM*{|H(n6GJqz?I+}-f46`&@?gpz0% znY~=R0nX^NIiK6xv#sWQfCXh?MN`Z10NDOXw!Nu}Z4HMk3$I=)M>DOA-~f~0Oc;}Q~`D=7an{=CA(DOKqm7kM6V!}>>5O+W>Ff{nTJ1B9aeAHdh} zc{NA*^9wWWQR=DR5BC=a8jU>6I^PXfd$@Iv!$CL z2Oj8sC!YtXc{LvD<6+xrzi{wawQ_+6_+gXwq<_`@UMy>C>u(xWKxT@B!&(ncM}Fy` zex_X#JV@x7;$iMqFtm=)3GnkjEAkE{nkssN{rK^g0Z#SNq1!MVl^eUer%s=a1?M@~ zt!@>tS+l2kOhUDNL#N~c!0ASEuK8X^ho5klM$3COd$HsfITqfdSN+Z!Lhxp)b7DA= z{MCtv$ev`yb|YTTpOY&~)BfjfYbIL=_0p z=ogqlT@CQutCrSJd$DX8`h}%0Dq)@grEF_?EFLy177JIJ2i!YnGPjS)Ly45%@vAm5 z87VrrRY21Q2t9{CHqv#?H*O|s_!ijO3AecRi^DP%4e}^&EeqjSujh!thB7tCwviL# zY@C|UiZ$o6fhL6&XH}6<;dN9Ex~VaddEgbQ)ZSU^3g}npLG_bk!6Af~waf5n-n&4=aTdCD3 zSdnU)0;OIzI}7q7NbAHlAynaWc_?}N_H7WY2Nee$9UXV~%Iz6A;-|D0UnpUK>4~{` zd;O#6MR`nnSxxpks%OcnxjPV3Xs43A-qJaL)X$emL_|bj-C#ijYoOM59p(Ly%mL|@ zdHm&nvG>+dRjys%C<;q& z^M2yC5Z-RZlJd9a~2WE zcuHs-mwwMec92TF^lek_7NqK!WC3TBvH)R<_(7TP`OB$Hf%u&};68Vva^7Fi_(kR& z2y}!M0YGZ`?k-kAdM{HsM|B~>Q(U?tt@?eSp8=$my;6X%f;|G{|L6d);0|4TS5{Vx zL$1nGl+A%DRFwq^bz+Is8Tj6S?rh#3Bs~?K)IM$GpzT2a9{rJ@0@T`31_qYRe;#=d zz@!|a1Li=N_x+TX-Rb72umM$WZUt!0;NfwV<)cHX4@49w?|hh?MyD-iARfT!E*jbL z`+0Csakv!)T%5#Yu9PZ6;Bq7w1jSX9e;|61L6ln4%JfQ8?;l~(bG|+e|*vE~qM@gS&4uF)kHZxN@S$YEB+114b zM2K?A#U&8Hzyx4xtgixd!S~6J17>9}U40+)5rFQa+(HDOyVDnhm+76L3kAM8JFqEc z%_Jm3M`@=@)fh*GE1w<#(C!0KiV1XV-xe2j%H2nII8Zz<*ONuBLAzoQ58IYQK|@O> z#BCz3wcqkfgr3$<2)={Qod9b$W+BcZf+VntfMtFU#)8Y%WQ&s(AD?&4umaE#Svfh- zKeO?5OClTJ;o%{A8^{u;wcqKy+Prok0d|_o*OQ-scmeVlha0WJjH9#RM=kXDN)e4CD zuc4`fZQfpqQYKDzCbDR2I%UB4OiWG1fBAA31aY8RG?fT(MTBgg5tws&1hty<{g#T? zdd+U5UvacrJZiN^fO40zd5MMfM`A1hC^W^B;={Rb4r%{P@z3G5eQf&a=7|d8P*+Ze zmm)@sv|_bDACbl6179eq(H8VwU;?|=T^|AkkW_x3fUlHWZGt#>;ytEW7PJ~K;S9tw z8uMG;Q=fqILo_)xK~tfi8&WV7AG`BGJ8o zySfKN0}cn@oBlfw;C}iL+@>8zCiS-fVg68KJ;Jc2IYbH+^bZS++19}i;bX6 z&ug#~1QrRXmZb0`{%dL6JxIdN0Wa4pU}82tr+e|exdjP7iW1cS%v2fy&$-c#SrQHA zThbU*iqmETKN&vN-Oa5>d^PNTD1}KlK{1yhJ|1w#h#svU90<4{uK_nRoPDM3=%{Pi z^!n<`GH>hKw{M2ZAW&EbDr7CKmh2dFMKvaK@}#TXj-&GO4|6> z?rI0guWPCO(wO01eOVtq9D%&HI*V~Jc5`4T7F)G~V!>H;Cmx@>8_;!Jw#g9~LV#h2 zv>T$^PDU(Z2z0=2paTFP1$>l!&}#b1_QndflZU#}8I%%1oh_CvzoJhfNR%6M&F%(Jeb!vPCjD{2@dfhM!87em8`-f2(0N>&UCXI9Yn zKz6>mkH&9lAZ%YBnT9oX!sE`b0qVUz1Q992dw>=m1E~x3S_mG{2&n%E0%f6pf>3x9 zvqgY$g`;&2+&=p+k9-F|A+&?FUeg9hL6@cR;`qYF(JC}IP*fZoH2_oW?d+rhBCYHY zz`wIc)hrq&3K+(I0f46JV-Z7H77-%A<_0@_iWsq7z^v z*iNE=WK1GG?3p&7CImEpLBV4dN&f-*fB($*E3vBwP>Lcj9Ow%?_2Ko< zxDfj8;Smpif+ETfZ=ky4;mvjBbM`^Y`4VkK^BC~LK&}j{^~@Kb9)X!^1?reU3Wbi24j(Ts$ioN0Z9FJ# z8VEWj5_@0dNNK24=z4*^RiM(rc=O?dBJ!FoXw`vufj?PXiPM zfGQ1;P6l$YcYglN9z@^W+XI2_RN%6$*R}v06eN}d;tCr(@tMPdhn;0j$#((#PL%qA zp&_8#&ZQY!J`JLS1Q7uInGZra1sm1(3GA8;JbBuKIUX`n6x_#P%Rn_TRi>NJ&}56J zD^UBjwHeGN;bloP?9CR?ar%Z2g`Nn01?^kFSl1)<-X8@`PPu?XTc>F_|IedoxL3zu zU-9xb15@8n47lI`Ce<*r0Zxx?DMcNM9X|9%*|{v-eCX`fGb9@Tc>e4>KnFC}G2>$>IHYmmlJb~kr6M&_zf=TP*E}FD-v`Ll9$82M~ z>KtfqZwFXcnMSqP?gs#ug7$LTz|{&?V`E~n2h#x*njs$)NPHBVnp*N+T9#OSu?)Sc zstP0^bp`)dOkupO)hzLmK{(%A3m-s1G z`s=FZ2aWDv{~}>g@}A8Yb$OI()t5Ilc~**hiinAY3(GCJ=TU>aQ(!7EQQlZUmqPGT z5_m?nbmt)5Ue7Q9cn!Gp`y#7AQ@35LMC4lH3UG3BTT+D~S9THsa#{`1M~j&AC!(|* zpy7>;or{YL0BoE9+@R;NMZrV|6xPVp7D!Ke(_hfr=m`)kfbIj3U9|rUL{ugK>U{&K zNmN!!%E7?E=;t&*u{~M0NaR~kpjadS>;2(RKffrp>2G~y7XZv>#g78M{OAH;8~6=v zwg)WRGY&)E^>bvgu`ZTqU0@Y11hM2o!ol`4E6msfPGS@f}cPwCoUx9Kl{9F z1mrkT@nDn!?=3MiQ};8(z~C1^{D5|KI^BL-ZJqkL31Eg}AW!M21H_u{L9JvKP_|7U z0oX?2eb)-&v3t|cI!o?=tY`D|iQ_{Ri~fzK*dA^DMA zc52a~7Lf4dzrz5Y$Z^CYU;*XsGgL+aqheQsu6+!)bP*W=Ap|J+uu0m=k5c)7gSEx_ zb({Vnh|q#0$Stb{sLxseK%J{O=)A)KN7-E93%}P@&1mT>DgD`Au#d_<5>e~b@Eew{ z?}Eg%g3rMiTYh{V zd2>4a3F(Le$B2Xg;7O?hpj#CT3^mo3A83=d3rTPjtLO%XO@cN=KkP*CSMKT2*;!Kc z&Nb_@!`62Yht6rf2Qq955R?wELBVKxBO{}|y*<(X82+oCT-eh0wBQ8X_=l91S`|lu zO3`(`monNu&}aDKjRyb-yay21xT6% z=?Uer7#v8!!X=zxwh3x>%K@TI3cvrF8`z@B|25{bOg~VX!|cHZXqJfIKHMtQKEArC zp&@Ytc-7Wl0P~O(KJe4UY*mz$;@08oWVAr?Ck=jtqSBb#ztGuBSg85o>kvl-U^~^b zFF;gFr4WJK&mJmrWFYL=8W%utV+4T2O#L7bkia^%#QhV*gVxFU+yJ01x(b!glB>W_ z)8Xgx|M%#=gVI4`G(SH-sk!Ia*nkfSnxzgHI#}@m`c*(T=L;f-0Bq@%dr3$LP*9M) z2_hvxgmeajjr4Y)3V`kudw@QHmIEA5z9a(mzvfU8@0K4>guB1MQM{T93=WpdM zpSf4@u50m~!%)>bS&|6aX$6hJI?~U50XeaN7~zW)21xV7<8px7_9Bqmp78)cq|%^j zX!eBm@b-MU{|p?R`h{=MmdN1LuS=qJgLCfr70}UynK%O&uY}sP`{Y}51jAbwmorfN zHV+iO;Ub9jdjemj?e=V*Oa={5oC54`^&%gqI$2cRGTH!fhI>Jq2`s^I3r{!amdsa}l(a0oy!6qH6gJ2}x^g+couf>TWd zym+8~8}Pw~vVMyT(!&31ddRmRrltiqQyyT<;j)JR*C*5{MSsAZ+!<2?)omMDc7i!ygoljjt9k4 z(iA~;fA572GMmS77q3)%bGM-Nmo?{nZ%l*-c)t9#XB8yJ0-!~LLwEg*ZP;n7uKU_t zTR>c?%AwI!nq!HIi1chQ*H}#tf#+*$r!RB^5jmYu_U7PUi(n#w1~Syw9brH$ztl@E zYYK!zyU4*FT9izBxW{&|`)gd!y!Rm47FSbK z<2nF7k~6`{_N26o<#<6<+|ocAk-Fh%!8IthUGoRf9}qZ^FS(Q$_{hmSH)a1l3g5v( z1khGAp2cN%h6zCD^?ra;`=+R+v$#9)5SrFt11h8SzjA=Nbz-)Dee?^vk7E$Tj{zoL zpx3Ow~HX!^3U+Fu<||y`hZxwXc|ZW zeX&1?8Nm)^(g8ZXQ%M&nL}hkxQ^7LTeqW)9J7{#lmiFHhtslPy%2mIE(5*;NX%s3d z5pPk2y@*I4$a*)Q0>KTC2JJO?e0-e5sHazDE=pT1UP&E4U374 zYz9sr&}kJqO}Lntpdlv)pbi55aQIwVpAV?2ep|!3&`?oPVZ8v*nEw@s!08jme8KPw z=u&bX(DTj^0`a>s0je`5W*Ep3x<+#^===k=0f3l?<@ERTP_whKuqc0a1hQeh58BqW zvmdvw#zL$R0kpsk0ld2@YBy@1T?7RMK`lT!xvI>r6G-d;)c_!2M7ajz9;l}J1^&N_ z1N|}7(|{fr)X###O(mXbb5>~1`Ve3w+W~|129O#QXJE_w51iq3ZJ=7Q%JIH`Rox!Y zO~5=g=GxudETRvjuQd0naV1NDM&iKqgHX0BpdX-`{9fLW6Pd=Vgefa!xJ z5V{P|VgPTZeMPL*B*epW2Rz$gp+OKA&?~HisMJbw_i0jEnkV3q7Z4CoYycGrP=EmJ zuqllie*7pgx^+M?2Z$ZifpP|@tmvawAmIViHnou4GHueFE28qjESw8 zlQ{`HEBNcf3z<>O!rIBifrL@a+Q7*~#Kg$X*aVrMANilVxXtD3XjQ!Avh_Z%;EXv7 zuySgHL4d!i@$yHB(|7&F3nxhw`LVbATV&L(?VV2v`3F*plE%p0cE`q2mALR+>f_b? z$#K2RiE)8u#SXoedo{kTLUZs@=xbNgK0{4Siv&ODP7`m(k4$A=GMRd$o)8-up{l=M3#Wi8A z0G2@W^l)v~t&7)G>8jLcdaq0Aim6IXf!4uawWszAZ9W&n!}pn%@4uO%XSB77Hc&#DhxFj)`$VCi=U%?5)j(PIh+^n+E03gzfEbn$x9cs~wUZf1-xx z>cw7?q{p1_)Lgx?SNt85MwmoBT2BV&kMWpp{Z*}K)yEpH>0Z+u~y9h`hEkKjDg0CLYg>9MZv-uEfi1H)cS{jUl&(Nd+5iS{poSN4=% z#$O%$n)V>!!ebgN5tuCrUYulNA|HHdY?lIS7pzK6P!1(7JKdv-F!net%2HoiFB92| zD7w7VfM=e1y6387|HUWl=8bE6wNRT&t(u!wJ-48CblM!6(M%87GGTy7>)TZAnal56 ztI=G#6u)WL==pM-Ms1oE!-~DWlTM5h(+jx=W6&!3DZ^pEM;-f9O!%(6`0ZOJOv$8n z+yny2y42KD)Aw7D&sui*gInUQC49fg=P};S`}se6{P-f950=W)hKFvvGU5AH7i8MQ z7xs5FO90o}B&9Qx^_;U%@zfFRM}He_hj-$kSX}cBY~`vp?qn~KO59UAJsOy8T4IOl zbhonXl_iEd8f094k`tKe}Drmw_uH-f}k{%~LlE9x)-?)|`%{fnF zg1Q4FgBJso<-mL^@(FzYEBRg02!NrG>h;$0RED_NHxpN!|c zgqsd$P%khm7MYkC4ztnJdUIPnV2Xdfbm~W=`flmOx60gdN>^waS%>6?!A{MH9OGeeW_pf02-I`&Lx~N93w1@cLqw>CHk8OYC?kLkTyF;2ev}%e#xv z$VaKKJ7R3YQu?els7$kRFm1&TWA=+>-oB$P-AR5Hjucj8+WkokllQGrY41piY(m$9 zsSv!c+UKym5`}AuYY~0{YxcB1iwpT_y%ZKKlcgnDNI`FGnMEF3`8cqv@aK6=z`j!& z{z3(_i|)}{i5oZAjfNPBP^r$q}2Y%D+)MGES(2Ms!PB0sR}! zhFIH4;>xO`U2(qzjh(PaH3M^kli$@lZ)z;woZR*gAJq68`vw~!;CWz3A)+Raeg-K~c=W`oHN3lZhN;oV>F{99_wD<3 zpU!Ey+1w~!BIho7PK)a?Z_L(geG(5r%7~T=4%c*OeeBUmjyW2&6Ce5FWU~PUvuH*B z*5`ccgwMrHl*%Q73aAvl@6%G0XY07~0Y^YE7n0biN!J=1Qf&GU4d5!;mO zER0)AJnj?I5A;8a$<{l<2{*+EUSs0q6j96)HmyGKIP`mInpyh6CX$`T1EamKq?+57 z@X`~GM*VrxHMF}C*piFIvzZvK9FLX@35k+;c$H9*zI51|-lAYX zS~sp!Hn>ll=6QkJ?jQPYGIq5 zf*f;x(7yN{Z*^l#?bd`&y1rY&$;`%Pc`cUBoh%+we028c``oq65jHN4+HQUr^6b5V z|2*5kzC_$b2g_XwqG2-JYZ}$_$ip*Q(dyP2lI6J~*e~3Th`%=jC#@;`#^;GTtCx5? zUq-s7^=A^%{bVQNYoA)HZMx`S63=LU;{RzWj zZsvu1`!akcg%4?s7Oj#IUS5{!`F7IwD+TP!utsm&M`FOfU-kb@X1zm42 z`dnXE_$&%+dF<)6-XHHQm;Yqq{yn%h(eXHJRQE9)1+jBWz=r=Ty7vbz_$)jyzK9>y zOZU=9dpN^L$LyW!YL1GXwdUud5A`d^15fok&UQ0~3Rx&frsyZm;1fvj8@`oX-_p-s zaEkm26*UeHFVNp8VKX^#Sx5dX`f{Rbr{Ro(;a&7fJBhpI(^&YuGI%;n@#p9DXBO2= zZ*mv6yQk^@Tums@~;-LV=M`R%% z#Ou`22fOI4hTo^x8+eHEpL>dEDK174t~e`b2mR6~f3H$tSujHki_hNoUVLSvz4mgn zfy>}SDg2t+QO^3uSNtlZ*->eK0)0c|Nf=_w95^$B@jJguu1RP=!F*o6{Qdq-W1<4R zSJy0DE|CPUx|T+JaivjoO9kr_&d#6(u5@roi>}!XDM~G zxfLpZ;uSEOuMA^N6+}JW=o}fx*mos)?&34n*kKlBlhUv}$pu$Gz&zLfX*|^GPEPjY z>a$Sn3Hyk}w|vvJw|;644_o0RzK zT@ad=@f#Ji!>4E+v!dHU&G&nwN^vyB@d>h;75ppLQ`<9)Y!)ktfw2$|a_R5p@{yMX z!6}#I<=KOKLUc*L%Q4Py(V5LvO`WAG#D#2c+#YehvSw&`@+hA zS#)tb{#~~OyW8HQ34YVBQ?!QZGa1>V0TZxcAs5g6l=-4$OUd-KJLZkcl7TYI+)8Ra2+4V?jWoByGCfd@DEIi?v!+Eps=C6|8)Kj9-`R>hA>L>Yl*LtBZ(qpz!C9oA&8 zJRc>uDo8Ym`#)9uDZ#PBp2(AMXzwzxBhpb}h5i$#KQBloQ-VgymAkKq;FRv1>nrA; z^5D(`+|O- zz=1Pn%JrgC#NY|4VgAp zHExc$ICrtNVZSOBwM-<-N=x_?x$9N$R#hGcbrg;%Ifij(C$8|HwT~8zZk1wE4&Mq0 zP%d+YrhY*3(!Mcml#(mQc;2g)PSZ$AR61)`S21pX^pzyHopDLL;iqEq+*4)F@9Ouu9kKI~k0w3i<(a_C`9@yC#-Q`(8&J*h0 z^668X>+5pWJSxH}jw9BEuA{>VZuvpsOi?M+Oo&JG6FOMT-Lc6hd++Q5-&VqD`SYm9 zT4fWx@ICY(6=ErP>@Vz0zD?MzYp(rg26Y~B!o~)IQp94ofo?-~@j5nc;fQm+gTOP1 zTIWDXV)QHImEVz`;d&}fvY63hS<>A~sn61ifzSL8ezKG}ua8GiHPI6-AN8I}Y6vk~ zEKR$+>cmi>Vk&hRNLJLy;X^IS#!pJCa!c#R8; za1;n+6DzL+YwsQzo3YMhOyL76M%h}w-Ec772upl^8D>O5hAeqF;xjE( z^XgH=>Zam=fQS%${a!c`A6Z78fta{pG~R3v)8$UmMogMq#n|)q)ggX|w-lHfCZ}IT z7w|YI_M>ygoZ(60_H|$;iMA{@ktCE1XLOD6iH5df#R@Ge@z~1W?$e^_KUySG)ho7q ztHosXA>>@7yW1jPK^RUChh-q4z`1WMfw=b={kcYb z{|C{4GEZ&)*ingN5?ZN&Oa&u!)(va*Cvo+!JMGH9GIb7`na)ULK8-jcH}O$0N~1$N9>1 zsMK*C$=7jO(k0lOiDHR3>&7~ZpGfH|=`)29>*1Dn-)s;i)|HdxVUu6F?s@UXpyY*% zd&Y3>$bQOxs-=HPCqYC1M#m}6F>0zjEb6tIZbH7o^ zAb~tj-KU2}Tt+%WwbH$FU-mYTOK}t21@Vn9oT%=L9S*7w-Ik~WOqt_ zi`wq~Dz9Xfeu7@cuPQ!3aE=nH=VjYK`C~a|=gr46a<;aP*WsqQi|K!!c@$7Ws`~Zf z1Vv&^P{LZn@8+NRDflIR%3w)d=)i1<>mJRDKZEGVR&#o3lD?XdwOq#}@;#?ap{L>~ zn0p+nfO7nLxA3kA@u>1u`ST+NHhsi9v9Sf1HzN=^;Q%j})eWlL!0@_Gw&_PG3kCWF zRS`i69B@0Zi?C>>?s^BNcI z>VOzfK%>BWp=ke@943t0bEhtN`*tfs)$aauJZo9w{%o11+4XvrnBkG9SZP^D^RfIl z8FjgWSmVV7bv>9qa|WD&sbyM5F+aH@f>KTybti^wX{&8R&q3~ zP4n6FSY)lW)TAE0Tdp(axO=L_@>bQNX2rv;_GUfhEl#@$4ytbUy+P%~st&3q;l)Aw z#mH;BQ7SH_*|@9?zc>?=z3(?K@RadY5z%wjX?jwWVw_otwe-ZZQU_Ny|B&6kDvP$# z2{W=O6L4l~;4}=OAsJYcqvIKh8|BrHb@|m!OS;AXLbcYJ2-EhppneiXwWWDS{w{s$ zy6eOO8CS&9wr>vF9}IZYS|!5h?NmUobqnUA{njk&V@R z)KMSYc%*<%Mj@Q$T*-HtQQ#jVF`h%(gNSkZ6Xk66sT)FEz#g(xZk6IIRPMrcW(JHa zG_*C_Jx^8!k7ci#!-31>c7(M;B$9DlAO4?kA}^!^7eYzh2^>giPP#zuih9BC#;^O< zWqI)~LQYD*6Uoj(g)k1%m3 zT%yyW!Kq>xJSOeK3F|4NM;T&rg zINpHwLKaSr3MLN1b~f+qY)x#PNH~!hh3%~E9F*S~7=as#n7CLNnJ9@1{W~bc%EiIK z{Qoj2w7a1tJamR{n;L>6+?Q%9!}4a&Bwen5JY zY`A(F-&A;p=>PW?Qg9IThj)TuB#6-dt{^1{0}k@=28P0)$c_HgSNb50-?;Vf11Plx|mp=(~5SwiQ!Ei9paYOI$&xg?Q{m=R^)c*;D zmY~a=w_QAD055ql1C>6A5~Lm7x{kzx2HCIwPS}VK0tylgbW$It>F-DIS!BNg3G(SGu{lJ));xP0{SRq57V1Hadi;W6tZ;VQE^sfe|@K1U$SSL z)xCPq_DaQt%P`ht}l@Q=Lty982Y_xxCJUJVDkIsIruOJyy(rx$WMR_@AX z&W^(^!F8oLvQ&1eHau>>%51JpSF%yG&NCa7X({5RiLywFM4)@26a#_|-JkhWK3Pcq zb5}pc1e7E~2?JJ_37fdF5WIYDG^sJA@9F%rF)hH(_$>B!z7jO=bv)5ojr7|uew~Xe z56`(A`jh7qJfE0gb82WlQk8Yg9Lj=jZR1FiQ1D-0rJXi2bhu*PyPwI|Lj{>`4bApX zhW~F@;KCK}+Af3nQy$J>ombnwK>~+Jup7}IdYGtd?>xI9u_fhYjHe5E(|!M&kJ%xy z2{PU_*I%y|WarNr8!VkNp~vb=Nm)2`K|(;b0N=!V(;?tN&c4@`n1zFs5KImh7pm2t z&|SbtN+BaLy^_K&Vc;&Q=iS1~oU3lF+WMXs9ej2-l!MeXPt`AV2JWBEja3_pFBc3> z1`KrWptA=Daa7M(I;rlcfnyU(+u58vuCLomn@#bSn{EE{bGE4#Jc~i^=XjQUNsNkK zzL$RbfB$wrMSma9@=K+xyRN0`k5Aa@EwxO|_=ykr?*>2ggOw;DI;;?QIsfHUgWKr{ zA`E8X2l36xhH@$O6X?lAhrJc1FtZ$S(onLPhdkspoz9J3$AGPMz2dH8lS1Qb>B z7#?*tXG^-c!}5`F$1i4RpsyW;)ZHj0N%?WZ|9~#;zi)${V2|!}W~;M}_|%lm!XIBT zf!$`g;%>*Y33Yz4bwHQT3w`Yv-n9F1SuQqobp-)0GbC0Fza!mRDpidNj7UG~bh#SM z-*hU1h($daXQC)Zp6>|EW#q8H;1MCwJ(WO)derL?Wgtiddr1&A!Ks8wBuDCNl1&36 z@omJGyhf8JY~X|yQ$=sn<7M3&{;20}Ar@GkYjrgDhe{=T(*O3gBCsu-1Cy38_8?K) zwq9D>VYjQnk;6XUSLIXph~K*!TaL}vhlDtrLU+Ar1c|xi-*Z2lvm228@ktf|4+0xJ z+v{ywusM%S-*#^Z++V=1b`SEP6|j?NS%+CXe9!;Ce{VY}a{^-yp&Kka3uf-gcUN!} z_#6w_vBOLbw;iLG=QtYeJsxfQ+_-eFqKyGzMC||vk^U=8JyA`F;RXi|@n3K-;7trL zeP(IBG5wxJyu}$ z%fV?<7XQ-Zg$DFZ>p}d)JL`GIkYfWv-(Y@q}pI-sY+||mpQL2`pJHMh==&;Vn=+| zjg!#rGB_Xvp!^NY2~@lMYf1vLzs5??_y1WuDh2y43No_WQL<5OH)WnhET=6^nT(ox zf-6oVa`m%`3^gVTx(z;%{l_-wCn<7O$?~|$4~fBjFK7xGiIlA7V_T7+=P1cC=emo} z{57#EFt406D`yOy0_5QTm`A#RY_E-jg3KijQ}x3~8t@RBkRMrd8oR`88KZ7DR*5mn zW~bmZA}Qv+dIY0TgXo_mf*u%T_Z+KNRa}HVEGQT>(B5p@AVWNc7b2+JSI)Wy>HtTy zR~h}&oaerrWo7o|ydS;W`%`07?dB{Gy=#UAt#?PH(Dj6asAEDDM)40jm2ftrq$l8k zKAL7&C6Q1};4`z$&(xY+XqoA7z=YHBxS&22`jdRCv3r4}IOl$$w5>s9t3sco44p`I z4LFEsG)a*;4hdo$j>!lC!Ev{mvuIhA>m%Pgu-#t5E&5DkdaM8mJVac?LPiBqpt09{ zXH>=d11&I5>ntH(?)c``<6)zRk`(n|z~PiVhJ$<;QR2HR#krbXglJ(MJchYFf7wTZ zC`Gv*frhU%?6Hw1%%03K7yyntI08%M9m)jpgiVbLt315kr&*a5Y>P5bJ7WeN4u{J> zJNKL`_dDbs{^i@&m`0z6SS_y>WLdU;~zW6&>byin4ptW#J6R(WOy_* zo!a2N)HAcUUUGjkhymSypGo|)NyX7sYLL<4G&;iA&~rZj^AiW*Agf1C|{c&MU1o9)|^Id^`~*8>3AQG?ynXjl$z}L{n;i<=Jig^E~vFTSqTXKy@YWp zy4t>guY6xn>G(7sOrAits~4CgpJRWX?qF@eHVD{wN!<3xG#h6@e zGBlKl`*XrzI^CvuIrWY-U4p^g@T{l^4cQwKm)u}uZ%ZrdSIp|6hnGo;0r`iC9Ll!svCj5>?QTjb_TYIC)$qSg(}^MmZNj2QDc0}$0p`DG#AGl zo62jqC0&2~{5rIP7@WsS4z+i!2p|g3Qw|{<+Vs<%PNM9mU1yw_xjro*<{@l5i^Y_@%(ayox`Pn*T~ zRNU8FJ-Qx}%cn_COAo*|B6^!vcCbmdtEk z?$ubDRTK-FmR|9F@%XRM@SDv?e=TocU>Jc@7CXUzeTi`(gVUm2eqA%#oHu!PPT9}n z-0jS$`qLkxP*2NUn5kDbQaZe-Zj^F+a@$gV$<;AwzZ|Ewcbvkg##ksy`r2pSHp}Zq zwzj+8Dr&oQ09Ci@{d4bEAHyRjuG+YM`B|2}W-OY8CZ)$FWggiH*o678S%P*<16BGummX>W&f z_PgFc`^UEaE4urLYqRcT)9BxFAzW))9_YsD^*rj_`Y1I`b+aV~Cwcr_{xf)}si*Of z3YmstaC0T!;xIgR#!IJRcU!@fRN7BTDoMu<4GgDk)&1#y471G(aoD1l`TZX}u9k>4 zj0ID4UMW{sk{Tw{Ud`YR9qW7&!l`|^kO{iJ{x7bD&qlBze{lu4yR zg}?`!ss7%hB$X|_h!y_+s)v_ttUyX&Eo$Qoe^6lKX`b#WU&;D~Z_^-#eOD&RuHyZd zwrMtHPfM%I_uOaa?!hqMh>fBj9R@6->SC`^DTZ-MV1Nojxo!w2(fq_Cw>k zDFx~6goAA-50#JB$ktY+AY<|lKHbyETt&t~&+*Z|W&A<+lA4c>x*U8iJq;4REG(v5 zr!)bYILTryM?q7n$W(E5GH7OVleQbSVH{6 z*wPTa<~Z3RjrC$!Zab>b{3p0&2y^;z2APF=d;95sx1Z28M6q=b=yC**6m@^HuvOvw z@ZQE_|7{}E*BPT-3?{Z6%Z39@qx`}2Kk1JY9p&H36cmKnxLey75bimuJy#$;lWLF~ zvV~9q|4&j5!?wA{@<`=h}tmQ~_bi)xU+wmdhtXFhUGw}f}tIVQkNHUFbcnWI+q zF!6bEUoXkSNK7|QL>lZ^2=n3~=o4)!>%jg>O?9ekm`d^99dR9Z{NX}Z)2eWmSa6$O zU#C79a=WlDasNg~P=as--@xU?KkMR)<9K-a^5dn6Rm7ej|Ig^MAd$_^pA*}x#1m;r z+A3U7o%fQH`^zCFD;JvoSlfc3_jbA_$z}(+t{YPQ>s+}w0-{WP-nAE^I+WX@UkauGWn*VoMmswp1sHZQgq;|qK4 zlZ)EkR zrFWH4R)=clq#(Y7 zc`X~Bd20a|?z1yadDO;==xe3t)) zY)|%R!A4QClH$BpW8r8hqqoJt`Q_?! z=5Kpt>_*gzJtPt~qQw;1%b(fJ>AV<;*s1>e!&Sh;0R=by<9SW%otl;p^_I_mAS_s( zb5*ybAFu7r6&D+*Rj1*z|Bh*6UQE8Pn#E5Xlg;ew#fLO_G(0PJ3u5AAF|qbaTq44G ziVK&Y^u@byC}p;Hqht)2@mB}gz;N(w22RnFbHy;@M5YTMktefss+tvTI2Sgy9YniG zB?6UTb>vVP2opac`A4IDRmKy&3jAde?VqjPqHT&TT3jEfw6&b{BPnvZYup4LQWh;? zDWYIHNU8Q8zhn&uGD6d+j=@yyV(PQ`rA;$u`gm$eZTtL`bN*#&63GzOsde=-|6TPK z>z00&Rd2u=Ibz>rzsc(M9~{reLd?~!M@}Chv9G8<5m^syM)Uoz_qJ@dw3oE43ijD| z_{46fS#{GO)K)lu*@+u186+1NeI}2BeVG=#bY%#MB^S`LZst!HNdrUK|wUXE@~oRhrSWBPq4m$MwI`&|*m@_1cGj>2Hir8&W8#iW z?Yxyt&LvBX1MP{<#x%oQ+uSMR(G3#=+?Z-lMWYT1ex`Wzp z@ge_Gk5DW!;gNJ=NFkcB% zGsTOyXy)9P%wB=f_8avNoS6ZOr#$7WI{Jx|$I8q=-wcnI4X|~1xbH_F)q508TuBhK zr*Nl_PHg6}_73zyI9=;zl~|^M$$EU`@Kh-)jr+Jja}}7j^)@c2G84$N zOO~EZQ=BYHw1e&a_{*a3kXWhb*! zgz&<9TW6680R?Kj=e0-=kPujinkuJ9zgCLQE20yi~f+87l{`$R^l4pAHWxwF&NEKSVUcCW(jcm4J; z(F5mD@CXiqEhUHi7dL~0bbWg%r%ib7HCM!jarNc9ll?@R_u>osxGFNV`+Fw{#itpl~XekZnjN&tJz%^+rdd?NU14DY!Md40LQbaYP zkF+?B0IV`1wFzvgBA={3t{-IIPj7_rvBFK@n>&hF&nH{Tq9 z!l;LhD-d1e<~VkLX%x}D?NLAErsqw>hm1!(Q1L{bpuxx3m zoGtU^$ukcl<}mb!Yf1s=Z=;Xe7W?@Pylc&zF3q%xA3+I<`1NATeoE(z-NQxU1kj6^ zazv`Sv4Jh;qoGB5;*t1EKdZC@FQcWD^Wh?DBIwg7L@64+V_sE`GnoxnboBvkx9pKl ze-?V?9Pz zQN>`dqpDZKP0o~2$v^+vNqjU)e>*?`odO={L;1KJxI!T)npipXvPlG3-d3ED$%o$; ziw2#A$8X?*`i)v+h(EIFVxN8hHcK%2KNH@dazR49{2djdz*H^Pxr22_FN+(cwh`bl z4(=!-n7s`hGE5H6Tr;4ds=N%xTpK`(=$LxYU?PYR){xIKEMVe-s-+|;u{Tlq<= z#)GAIyD@4sNnuz|Y3hFXbABC4)XY@vs5X&mY4i23EFPg40AgZog8+1s6 zqJ)Grh=52tgmfcFOLv1v%K%D9cMc-m-NPtIcQ*(~r^L|A-T1xtH^VtQRzB-l>(mI) zg=8!2{%?yI|2+@SIA+cgA2qjn<`QHeAK#)`usFKwZH(Lb|4$@;O<4WD2H+`VPg&}w z=z*?IYFWz#8k7v7rZzvC;!FQxsQEYa5CaGnqLthMbc= zllEoYU!d6ktx@w9$yPzeRF14fQ17}s6c5K?m3ZaP4qF#J$MA6qnYsU|x$&>#@}E^m zj}Ge=UyjlJJO6B@F)drqC#Db1L-2-){E9V%eeJ>|rXF`R_y_M-9v%mI^!jl%8e=wc zZD0bxZRtQtN;OW^@T#wT{n|6OL;m30q_;#C!68nE2L4QXTUUBb80V9GwGz-RCYp9N z8XybJ6dyqk4Ew$iNe!;OK>2b1WQWh1Bed{7SiwIlLW|W?dG*(WlUE-LiqlCdcWX^X zZmkIjWph-^IGI)4ZuR;pz@@=UW;*Zn4|tP*{cMRHWEvUF&vXqFtF>%~0<${sz^n7h zNlG3G%uB$&sOaiGXp%N`0HR7y@gZ@Owy1V%Ulet~3-vcu4ywAcC3rPz9 zwr~q7gO`MrJg6QsiFi;y?ri(R#pcD(N<>A8hkpT?fjySJLyeU}Ab=e5CyOSj{>jsS zr8{u1Lp$Y|v3}lBoLDLPoRQ}`yF}6&43^B<@9DHnaCWWh5PGGXJuRwcucp?wT;E86 z`}?oFM~juBdpxQ=^d|-~q#z))>|=`kSR#Z$R^qOd6fhk#^P3-Q#LHHDZgSKRb)D13 zkyx^8r+5EVFu;l-L-)Q7!KgP-^&m=BS9;)CsQAqZ@p?=}#!1%K7I%O10}w+Nq6?Rx zn{`gqBHI>8 zFeQ$33|Y+p!AZkQjmOn`bBR4}i?!253P0~8VuSSu(OSlVk>|HHmZ^mtH2N+@=3)GCm!EQF_1IjA_WkKB{k~% znMqF~N(V`V08;eMWXw5!Qe@1|rUE#O<{cbN0M`Ir_cq*asRs^ezSI)V=xQ_K=h<}x zVgawB-O*>gm?yd6d>G+JzbM$sG&evR(5!B81t6WO{@S|Ef`!fbw1dec34mfHiS2z2 zC7xBj1>xlYuKii9IxsyQ&#)=oV*ZY+(s z1c5D);oRfIGew#^00fkTW2COCkPH0+ z0i^;Z*EPrm{@)DKVr3Dctx8BLJn%FDW>LbYlq=)i;@Hcorl}s_X%PI{$kRr9WiH&` zMy}j<-al;;qB`e9gRnWr+A@LOOOr<0CKM0;lIQYYt=GMe6T< zIDvt*`&#$=`StoXZb|P^h@D+UI?dl&0Xkn~-v2COjc<&xvvr|YiGZu;yKOb`Eg$h% zK3i<2BqxLCAgu5jDO(`1wk6x2NCx2%2t}W8;{NDSy`!m0Xj&b31@VuW&6?Pn!Gg9L zSB^e{Ki`_4G4`nuhh6tiv8}2vHg8QCI_ zI6sfB{!AjVQ19t@!U%32Bzb`O}lTCmnBB95DFDDFHf)DO|z6FGePH zUO2t=KFIn`5d+pC=A*~SUe+b03$!xV)K88&0VDuUs^8qT|AO^ah(zWGaOoE)U1Y95 zyLE<|@tL3Sj$H;mb!c87ZhWyeySRPPIDPS4Si*2Gd&ZqC;Fp zK0mNHZd_~6i6p4+#s6t!DBQC=g5LC)A|}2AH0{jF&-h-~eVcKtUmV{JHGf$y4foI_ zsBD<9jEq7D@&uA%K1h^d)E37#01`Nu^6qYr<6MK#ev86#OO&ZzFj3*FeC36RfeB7J z<)N)ZjhXQMz1TNL$wtbFY^)3oi?^-@q;lkF{aB1K*BMjutL&UUx)d8uo%h%|^_iC> zD+iujw6xtgH%y#71;FzRRrD|Cy=CA}EPQ#oM9prmm!TI&y|I%0@Mw{?IE%K6P)?6h zw}Z}$XEofq&3zXMj;FJ>K>Og3G96=|BHQ3}41b|)egJRcmv4nETpKdE&dg;Ffxt^2UQEod$ zq7GHG3w^HvuN6onk(#p-5UNSV@<`L@*7{l<&0m3zOk7il`9vp@>grh zp0ka-n%TXQ&oQ{%KcUucQ*~OBl%XJ86E-sm|M|=Q#*Q{_dV){i{I3)6T34DYHamJZp3 zBO!09vdo8uEdFQLofyKD6p6l>1P?q{h@#sua9!yqS3;>b-EH(})Zo*z)3;HSI)dpM z#3N7wud10$3jxi4n+U>(j&aohj7+P^C%v{o-jGihXimp9v`_DZU6Qe3o=8AoBSS=2 z_}U>6_$|lmK^b*)N~e75^&Vjp?IO|6&c4 zAc~!??M`YPW17(aCvDrrvSxE!8p;kDg-**;IoyU}B3r?7R}1-fNTnW~v=8EW@!lA4 zUB5IXnVbKoY&{l)reVRiQB}7exIO-N;zWlqsH0?iQ{xYx%QWV65$N}-MR zY8re&^g1e$9qV6kZ2+a!l}r;MehcS+Z|uzd_;35TP4%Ca92(xFLIp@PL zx^xL@S(NynU@)IT2*WT-4c0)EpXue3BYMqy1mY1I#Llhpi+d>LVU5j{V!L#dLXjvc zvQxtM#T5eia~dGvrcBdiySt&qb_}<#r2Qw>0uP|&GwqBGJr5Pc7S7_G=c9e=V#l+A z;9@a$lab6mi7r~v0gvI{VBtptKJa0f{5muNE;eEcnJ;B_!@9j>z$-Z_aJTVlOt*TO z?$@B_?){^GgBD1ncsCAX3`jZk^?xVTO3L_5rOWyO_=oVFo}k25i55m|tEgRN{;6AH zr*~wrkwg@qFvyK>`7uVS$l#CFi`Q}D`o z%jQ8*h#{yZFhQc^4{!2dMd2_o9RVD(9< z2^gDgU|xaWZZbzPhRjDLpXUp3@xYy~ziC#iEoWr!5$I=|us*9WnZTd{U;t2d01DbG zUFcE~sWbn4S@ur|v7k{Bsa@5`Ty&L5}K{Hh#qK$8&$lz zEp_C@N@3kD^kWA}qF`F|_U)aDZg{-M!t*du+zK)`X>>`yuZp{=gv&gNTe%@Y0{K0j(>s5j$cGqSiFaS^m?r(}e~lwl@! z-(LSyP4xdirGbbjmO>g{_Lj5x!;t)HW98Gxi-)W@VDq7{VrEcmGktyVi!r3#?MOkU zXXWm<9#Ua}d^eVFAo16f{zXJqqn9c@aj;X;)6ot67a;&iX&@nt0kB&l#k97qL=6Pq zZ_I;c@r__i2hp@uLfc(y zGUgy!nFO+v!8qH{t1HeRS}ZmY>W=TV8N2TQiRkYm1^Sjr_W03nl@ zCJh zs@yTKA3=TPb#}=w0lSude}uwnzEN)AdVIro_7_FAa3l9;lEBzZXqlhj8Ku2U+ivA# zY1fV_ZgDg%mH-g%`0?!s5S};xR=n(8Z7f<5MgDJbc))Ar*=xCequn@t@h5DS&iJb& zGs@dZ7XE~v$s4%C8L*^CY%FX9`2PIl+R_&e;cKci zi<3a~H1r@*Q5qndI0iYdDvcF(7*TXIwg&}Kti;0EeTjqmwVXf@GYsP8rJ79)EaV== zz_Q(53_FTw*798S+xYjR;K9ZquL$?U_PK!#--mVVQzd?$CG&SOe65NJc_SapqA$Q@ zT|$iZ?O`BsHvlo+aOdGgejd(4B#HHY7`GRdLYW&XSIL<6@VEmA5jFD_Ye`%ZXeOk&MVlUoHIY$~XuP9M_P=7vs zH7(a)YFm;t6H;LVJZZ;hkb`8vrRh?Fr>!jW}DTQc?^7RD_q0 zNSmEPAmZn_%%QjGBb^p67pF+xn65aZb3B8--_Ek~w?R~uWGgJFR|rKda3Gtt`qZTp zI1m8T;IvT_{i7fxFOHa2lz)2&tS^}v5OFOX%{gu>VOm$K$Mn24cj1PPoztIM2=9qk zc2dDbHpOcD*z-F`b=umXF)#J6I{b$cP<7cyuyeCfgcN}vCsJ6AAsR*nBt@Js(7TBH z3%Q=;#W;>n5Vy?rZNfi{0YSKznzb`AFx?lg>yYevM0L_hrRp?5zTWkPTFbU!1v(}a z^=AfWoAhZyeiQl31=9=IG2__+ic$zAF=F%^6Z2n6`n2o{dMxq9=V#N!zR=Fyr?O-K zK2hPu1W&IcJCWzG!eaEC*QtB_7}>r{#NF`$+vDg5Vb}YJq>JJ|DGwhqG?i4yBhVJF z4Y52D-qInOnUH)e><8$#>&Ch7sP4$G5iuFDD*;X;zrK6|J{~XxAB7B0iMlX6nqt zc^>yK?ke|D8r{EmBF{j!+tfG6;(p@)RwRVWiaK)V^3_3ySnQT*OYyMo)ZOI zR7FnN>C;x=lm+ZmN=EuZ(Bi6IZG@qecZyd*+sZ9d?@%%(9;7Qk7nT4p)bU7w0C@c( z4y+WSSW_>_C&+f72qJs=PvZ`7WpndPi#tg5G!I?Or@XW%K#Gak%q=hZwY2uGjvV_F zp;MptZHSnKKh1&eEv$;jBcA`AhsTSJaJaffnqMsHFFj9F2sr5>R(z;AgK0Yv8AyDT zN*)hMGu?+OM+FBaNNL7Zeseu)9SHxpzkNT_F8ZlMA30IbpRH1Zic{5t6;$l6XOTe3 zL-a4CWPhD6-Mb||e)0z0h>JB`<;rY+i{+8^9L?iYX{l$HtXiN6s4 z$l(l>p|qt*z@jGJ04lfVeNR~$g=5RbPYaLhq}z8AhpVK1b!YzI-`=9buc_p_l8c`@ zpASvTCMvDxM=YfJ44>E7+&b8AU}X2K#t3;K)W@_`|qjSux%-Niyl!%4hWc*6E| z`d-hSe!J2@2w~y^DB93=5@#!0i_Nd0xO>l#ZdH77zb`3n*PwLk?2eBME~qT@1~85T z7U~((g3VK~@D8=eZ5Ng+j{pFK$1*XLzi}vS0ik8MUhiUglZG+6ZA06E7>9@mq-9 ztdH_DhJ;`jJ;R#us?0l#?p}%@myimH5q`b}ceIJSJqI%%!Uk`D4n$nP@6}0RXO4O| zC<(#Xj$Q0;3|S^cDuS&1ZyhXMVD`>eS$a{RZm0DojhVkGV!}#%pTFSJqddW7Z`VpA zmBRKr#rP|)T0)gxtuXXr(wdMo9QU?)0L20ruIgfH1t>ZK=zsyu=3`{A8n*#UB$vVE zRP(7R!L2~Z`x0vFFGTtYbE8;JDQD#KGMMhQc2KUQU839w+-lYc!wJDr$b~Yql_v30 z6AJ3<@Mmfv-iUDRW#v$t{(Noqi$XaDoeU<~hF_Uz%(mok)4Hr<^8yP;%6OJQUvEcS zx-7nzVXd7KEgdfU!U~Vn58fVRYEpDkm-RIOM4Sv*FJ|;;eEnQ2EZ)`~$0@ySG8qBv zt0S+w1jl!0o&paQPb;e_>}2(#{44&3w2zaPf}E6Wn7!;;n7pe?Xh=*Pl;&n1jZ5m? zuTq_vbxK?)1IzjNjmswN&bq!tk7;|ylce-}KD zFUi@iP3MJ@yVNuJ*AC&(y>7ti@Q0ODme6&>7IjeXKz8bAGSVh^A5?XQY(uls>Q$3( zLI5e*hUOt#I*xKr58%~Zwftxx;wmA!J>Rs$V=G6``xgdv6Zjhxi%Fs#^_j+-&z@Wf zMavJDl4eUnd58Hxa|ZRx5A5g1vo#!g!DIwbR+{9%Upnu8&YI%G>Um{ALc}YPuo{rK zm*=VRJQj^PFlY z+d0*{gRelrO3>y)XJvmw%V9wi@_?CK~o#MH65-FhGvr>B1bX?>I6@TQdhSyNH=7AiyZIl|DO{H(S?$=Ax<*-Q@CVP>6d&XE?^`ymG3YqCjd4@FaFce$b z#Ldn}{>E3&WO&3O#%%Q<{>x_M(L}9}ep@UWBq;g8+DWH;w*9PEI->X&FE=!kv7y-? zP`Dd#pw0e6F=Y=!+Uuxa!ERppT`wig>+5jKfFuTC z%6jadH_3z9cRr{efzhzcEycyRgQC5m;Vpim@JVjI46NA#_4&`bqV3fKQqrX`|!sU4R7 znVHc~*p2=g@^W9;vbsTT_Z}HLmghSQ=6)Do9sh>|1scduPbhB4VeSCp`XgN#jijzG zOOf5KV(^G@-V%PG==HhAMSyp09y=w4+}0MrhX(g)LGWFz3zCD@ukY!S`!f#V_7k`| zC6J*$wZWb98{6*fCsg6cWHE`@T&~mfew#rXr5KmG7R5O7g)lvt;HC(D*`s5_!3A=P z{hvf=X6n`{eK}VFzj3jo2MGw#O0cvwDFwmjlW#14iU!{xS3)LQhT8`^m;U-SUw%x< zTg-woOf?2mfAA-{1B=?H4?cvfHmTGbdo<^Y;M zU#-r02-as3^jW#Dw3Z#&(=96pIb`Rw7fZkRs!W!yhd7i5F&8N5}HAR}~fwiOo zdGI-%m0%k=PhkPQbJ*BUHw{5_6?hoDXZeE?>x}VHMEi(R$V-FC>)7UrC1;2A~4rkoG!^p`l zno2RT8VVreSjbOGBDfO^YtM^IcW%6| z9qZ%2*mW7O#NpLrPW|D=z=wq(zR!lxwM=J_pxu_*(0dUCd70iv7_Eym2u%hWNzjb* zK+5RR?P-JeLJz7>BKyVOCUQmTyhM7AuMi`w&vBug z3)G4)p1V;8FvjhRtjY^xQS#_eq<*YXTV0`1PTIj`mlOkS6!ks`9nf%gJ}g5Na|miM zE|r;}V>9{E;Le@$X92XipH@FQ@RjHej*7apZXNnw9Cc@Hd5fUlUM_5{Wt=nFx_Gd` zTwRs`$CV24>Y<5vC*8$o5j2R$h=}kxGUC3}8RU53w$TfhJ$ZqTcbMP&_ZQ=r2-@3h zA{IR!-y{DKcoB$LElji~c!=0(V%rlIEP#fd2<-*0@&?Qx)Bm@+!+(tLSbvoyoB=Hx@mS=?97C>wLc&r-%4kjW&{){bI124)xj z-8QL67AAyLJI93#fdKB!OCrF{C=$j>NvZkeS}xuJ*~W%C53b^RS6_bodx}uYDJ_YV zPuQp3=z)tzdOX*ba4&MjR5~^L&}vai!0$fu9UazX?k;sy{Z0iN0|J`p!e^0Xp~~Mt zC!A8!j`yLZs>y@?N2JF_0W3YO!+=y&g9-V=gP!rMHiMQq5d_!khztV`78PrFCTN@F z)VqOZ`s8^^EZw{A5oKWuFj?T4{Nlo!Uea_JJ-p4e@wK*->LoODXOkl$dACke>il=p z_YZZE1T-8`@f`$c$1N>i} zP>UsUfx5tAdERrovZ_V5Gh07VwYbLwOgObaqS@K<${vuQnb8x6QLMi>+3W=^P|qYa z)eTb^*bId$2D+j4%7>4@rO`oEFd*)vW5%D5XOaHQ|y(D5M))(@_FA_E``Vg zw6`iC6GXHD3lI6g^=Qfy{;|;CSndXiXytROT8k>WPGCLe(3=i2{lk)uXk{$$)-G)^ z--viABmeNBr@*IFjdB|bxB#m%d1hjPeLb{v2UArS^^gXlbKGFEZSaKj7?e-bXa@oD z2XuO0+~2-O_7(&&aml7?H`b_zua}~iTys^gM1aB9pXQtj?C!|vGa__7xA$$T&Pr<5 z7tl_^vP8y6uUs@Mi#yeSM}s^%z-WEMg5of53Hov`n<9#Lj3LqQ(v+Wa09CC$GTanFy;>jcRX~p!_X+aphXKxP%MCgHjoLNv& zauIFUUg+9B!wS@OX~NQ9JyYZS+)@iAvp^x*s9VAW+7Ut5qG_%FWr6Pa!fR%?C@=h7 zwo3?TNTq^_>oA8N0t`lpe9RJGod-1XCI!%b2<7U*U(M^V92QMvd>GMky1RsTA=IR7 z0CUMV)buRbyg>SLS}H*^udESU1*pH!*uoLa5}nVAkVwn5sHR6xBNrk)iiY2jq~SW|@lT3B$R zJ-Shvp1T?NDSxU}i>o<0B2O4+w&VEQ)$oNw?9Nsy2V2IyoSubP5&x(t1nw~Z4tSg8 z`_`m>uod|A*ylcm~uRDt@d5^LIUG0_ZK$X;~YjloW?Vs7?%cx~)G<@{p^)n%mlwEKy zOr(_wgZ0>(AE~22&u7~(#Y-B8ElQnf+6@padLt}J6aNrwDS|TvB_m|8Og|7GxhB8<>44IS)e!PSDWf^A%J%{wig zx%;x$GweB*n!JleO9wTB{2oD>7js6m2-E-0Jd1&&@-} zQ`&+?_+fL9zE>=zC_hlYj3c*mGo&&`FlSLWjK1YSPP~?++@}nbl$D1`70M=S3cg=E2*@*y! zb;VyX@Oj{O5UIxjbGZU7z=l5wvSzvLEj}~u?z{CrfKG^vZ3f&M2gJV6Kt~jVhrCk) zK!=Au65SVfwh41XimYeYTWnS@@wi*N%&rzV?1MHrONEbqlhN^8esIulZCWY0PHIYW z3lGFZs)u&`GFI{*QyYHYH@fr@2CzrpYn#G&Zh_cFQlpSXf!x1z9gOFoULkOQ*g^S1J{#XJd7Hyc z^n5d$8D74zcP+g;{lsw;dEdH@b0tqXMc#EZ*%q3K>Gs(>ZRB@ST~|OR{rbg`tAMdn z2(2YI=%Zer?sZt->+>Kxaj4Y$ESS13r*_Da@O|WA+7W#2J+|tt4$DXq+@z1?Z1k+5 zI4=_`o&|`|wcIRo(EvKzNyFrgVB0qtb{fPDr^8m-HLJMz74xXsdjh@Y{X-0ZiS>+k z6QWtM*oj1Vr>V`8xI|+@tV23-g!Q8XEH1F1gd5JD*-)$}fd-ewr? zx$QVhO~TK5JNb+72Wnaz0eYAgc{2${U4-L?Omk_Bjx@~$?7Z@}8hbA*GEWoQ4DPE> zY3=Mp4%A(=SuVD&|1J|i6(+pH1?!#N(`YRr?%T6CJWsc~#t-`{H^xRUJMbRC6!djB zIrU)N`md9Cm^U&+U;p!0Psa?Nda+~>FZ>Oqv#4b619i5+&pC2~44S!`@Y;96OvKGB|12Q$o0CIEsf^oqa7hr* zj=i=SVEKMINDB6Z8U?JTMU>Rw+3Img-@qSdj}>+HMF75}84gKK%K?HkY52S82gFIn z%s&y=iL`rV_L!vk%JYt5R| zyD8vwNQ*`P-c4)bVf!$8)(oJ=y)Nq|R#GtmHy1@J@M!u{ z%Q=Zm$$#MzCY#Dx^jbMa%$L`FX#+3?Yi8_$_`ne1&TCf~dcXja8zy>r~*i;4CSw*WUF>u zQrJB^rkj#?s?Sbtj+CRLLF3uc|-!)avP}=;J^Aju)p&@?^4X)HgU;d^(#r<-j=U4i1X`eA623TXZ z7<3l7t1So97eQR#O8jEr0<-IuH?6~fSAQXF<498B5NR-Ah?3{X+aDW{b^g$t^v4^y zy$M3-T_$1ka&t#Y5EawPcnNsFG}O&Etmo6Yf(!*dH7pu_pnQe}U5hniEQ5(#d(4UO zGIe{Lf=?N-hShSKDE$rSh*-^N!(60VqGmDC>h$5V-9BG7t=4L<&NdtXV-=53PXT1mJw@~Qh32&hnllhRHX{)WGqsDGKHqdjp4%&2kAhH8LwGMH zrdKzt!_vj)?0R@LUhF(0qR_}WA1^~bZ=8FiVS9%5C`Y}5n znvZ`jNoF4vw5laRv*N;3TDO61M~GTP=eA^5t{7ijOT~L1Sv8BSM!_ zCgRIgB4){E!ytql{Eha~cD}bn)tpGB22A8Pk-s37;Q%>1hx#PC7C4`B3PEjJWW-b$fcB&lgGBlGs#VRpILwf_MAOo zS5n6Iy)k`)cI{j&q@4HXumenQYP@lz^V7;MT9hK>g~ch5(_&qGJpC&m|2k~sP&Bx* zJ1dW0P577MaOFloYsKAD!!?fSx_7FGvI6i0P!)zhnf#!D{<{PvXCK=}yij96xy$-v z6)-mvavKoASEchU=9<}kJ6f~79EkL$*y&A{p3$|8=)Mo(Fn!@0TF6jkD8nk-uCU&m zP?tMF{h_;5iqI|^1b3$1$M3Fa@Sd3U;LdU7U{+pCK8>=YboQ=^VFAwgH|R2XpDGE}O`VIr5JyhFy-8`lJ`zVxjOz9FN^*r_EiRm9)nLaFKzlej1i+`1&4c zCMt)#&S6@$FX?9^kM#Y71R?xzT4u`LNzT4Vof4_kd%FS?RQ>D7QG5<^po%^R^TYSN zap0)==v5^&b+I+rF`flXs%RCA>R2+6`3~rafbZuVkO}xpg&?UT?HEJLp+^m!{myQC zD1;CyKlh7TH&vk%vnOua=Ih`ZaZJ64)6+NeQTb@MuyL#|VPjM~42JduVMBpO9P!lz zqkDemU^aMef*iOse5^DI5;afOKdOx`5t4-u-J9Gv@|4=s1bH@j^~sN*R&PQ#ya*ey zB5sp`O3Rw7k6RdZ3Nt75-)ESDa(bxJVV)oZM8}1W(c$yj_=OUOVU1Hrz)(!n5xJF{ zi^h2fS)GI`$1ibMK1dn?JgA1x-qe+$iUEz`NotIi@E)rIFcJ%|KXxC5w3BGDrM$Eg zi)x#^216xAu3USbVNpKpU)i77pWF!J!E*L6@qp8rX?+2n+x7SIbt^bE;7PKZkE=W9 z%!dn{*OWNw6ne98aZ=atI~CQpIG(cNg98Hv6rT$e^Uu|jR8pzv&pLT)VHGLIP*6jC z;D<8Lx@jcVN0#XCT%8sK{6XsQ&(d$N&3kB4YMn0g%+oi5aa>}F^9nA{DvQTzPJnT! zNJEs)bTR-3Rp3zjXa5)U+pi3h`Syf+BE>v^PMq^Ot`dvEI8c0O>043GDNNOqu-PyD z*B4z9b~q53tzc454{ z;Xi~Gq|6~>RD9nc1$dh1S_9O1zVw{0mcGDPpYtg7%J(wHEnC3YIXJ_eK*?2kX`$+u zihR}&kF!8D3L1avK(-b=>EbnF#a2kV@ndsAMx$?A7kRck4O<5bOlka#gJh|;y7d1g z{+MEoD@-YVL6xdb;Sne%gV_;)R=eW6gtFKdotv#65ne>e&(8A*RS7w{f7n0XnQI{V zP&Zq<^C60zNxOEHw%1y>TP}gDKn9&nR{+yI)R@?XIQHS4C^T%kFXGG^2OPvNbj|NT zh`-^S`DUE{HIb*qN@FF}7K0s8e+D0{$`-&%u{1uqZW5DS&VtE>2WFK-O03tOBhJ>- zdi@rzxms@`>CLoAzqqk{ktlB5_1V@cz=iToUW$zRo_jaEFCTdJ@bt;e$MwI}^H952 z-cB>_brDU-G8k*CUfUqVTkGPFxn06t){T!<0fFrW7Z@ubW~5O zDWXQ$1rv2An1Yzuno{kKFd!=|1J0nK$s8a0%)H9`5_3E;kh=9SNkGljV5)AWZfn2t z<(2kd3;+IJ@x)UZeYe86z{N#6_eez2y&Pt&k+eN$rOWH1c<4{hHgky8_5e{45@zxU zK`7{6x3e4W!E*h9?d21!&o0M)BwUPGBR%oUDEQuq^aGgPqy&zJrbyc7HdPIyr&ysT z{iU&1O*mGGRA1K53m14zha2V!*tJRDrk}^U>jW_d`Gd6K5jOPM1-cJX*JDGQ7!f3F zO-!?hmoc+pcu)#O>^eK1?142_G&=4RA-Fi^=OO!gEJKy|G#5Z$w*m%)#Hq6h zCB@!!sE%XLITl5GQuC&#?DK# zN1}z#-2$P#%B(%>%Kkp+*Qw$c{b4VRt6WEraOYac&`3b0&FSUH0bM`+%{7xQPy9W7 zzbUR9GjjhnGNm5efn{3ebUWyX-_tCJN~K05MgmzMF&bRU+{h91{+I`3u5$!@VAtyS zqIP|c^t66#lv%Weti|{F$K&1~Yi2FuC=UA6=1atdyU~8(P*8ew7VhvwXQkWg@X!LE zBQ;*{?O^wISpP|(+GW2*FSOcFC3Ux>7y6P09=E1jeq57RE8v3Ms{Tc5xgU~XsUuEY zr16 z38j%K$p5LKnIQ#`{VS-z2pl&qT z1SIro`=ye`@Qj@m;l!>%9u(o9w8P1Ok| zq26xn)E?9{%M7Del&Wmm#n_qi#rY#&safd%jzlrtoaJ5dF>eiYR3vMwgbi0_wr;Hz zH=}y~wmgx?gH~@*9b~WPU<7WAU;0>ueL|+&+HE^$AHCD7pOz=&E#p09lh;j$t9Nk8 zck}(BSmdf}K8tm?xqjFNB-xc1O(Zfv^oE9nt=&|1eP8M=>DyB-HF^S3vJWK+rhi7G zrCGKj9%&D;p$F8MyxF4KMWSokcJ;PgmSILd#)c-SY3(6%_AkB{_ERBwtMM8gt2$TY zZBPeJ7R=gQqD=vrxurM7cu>!N1x;B|t#=pA&L6ir_xvZEIzSWH1}m25g5*UQN~^f> zDE1dJS3dX?`~1&lzK}={PlLA_s&)JdgHQv{SMLOO_=l!)8`Nzmx8NLKT)Yu$IhYWu zDIwd7{&c3aM!VJ149fyLO7)SNC$@6KD^H=BlFXr%ANDudn+R8(5f>V`ayspvXbmpm!Db2oe zd`Gf3p&HLJ={m*uWwD_>HZPth{}kS7LS~AE5Dk6f*&WCK{!<0R`6GAXj@Y;yVO(Zq zV=Z|1xV!KrHVZw=mVdXRaM`KXL*OVgXK6P)O3tc&SGcJ;j4BuQJnWGFG*{a-PjbVD zs;_ra+TP75?$e{s9nZ0_|Na%Z$HQN>wzdt-m{O!zrs|AP#BdF9@XQn*U zdvHvM`gXhMrQKPNs7HQc^i})58&u2NRdK3f?sB3m;YG8R?{rMo9_-36c63W4Pl8W; zng6+P0^h0O`rb%7p8zCEQ#r-%kvGL~zVNF*UV0WHepMFkzF9|pd!z)lt~2QwcxCu; z467UG4?z6IQfSqkf~%@%!%vZQj3tj|590Lap!h)it?Wj``7+6Nt7la}yl|35bDkVo z5>Z9poDo19o>kZh6Eru6Tu@ml;*;cxzSyL8=m=54?^EXqWy2&?CXMCoQ@`tBK~1&K1^2zl z^@3|@5It`}Yw{;r=89+HD3OoeKF8e>e82lch%`($wO!6U4c@oO%cr5ff1zbwGmf+| zCqGs!8RzQp6ocLW32Zjy^-WQ8OS-%|aR6;ePvMtGgF!e!In6Yn8;ks#&HJtmt#4$H zAG~|B>kS$S0h57T)8SgM{O{^e5Y;dfdZ-P1Du!ZsVD{cc*ZfLr27&iJ%5AITdBFZ( z>Q|AF48I$dj!bGPJP5VSwn0;*{~|ml@#~?Tx0DakuP*}zF0#nH-4GxUJ*?{`ID^t`ZkA5qUt!-4YV63zA~V&c1#d7@_NO&;*WfHebKb{(7b+XH?W z9Hsbrt@Iu>3@dnsx_6DU-2^2odxJ{RK!?2$WOkdBh6#TqO~W@_cr0A`vV4f6-?_t}!C=5o8kOltIYU7&*M)tJ3b63d&=}64tS+lRO~Szg@u$kPRCM|k{~YujiqQ1 zt|-zInI<0~vu~mcem>hk;vp{c4c7S$bt4RLKEErAYoV1X%7b%$c25|r&j!vZwULyW zKOi&2Ki=%xyQO5Esg*DB^C#QBBsAi=eSiJspJYBVdi zV?E3xXyM~Gqp|mh(AuZcG8CnK`@ADBVkh+s$Yp>^{Gn?l7tyLdsTdf9T|V3IMX##H zyJ(!SCfIc4uP9s_i2U%!Vry-CtlwBiVw(wQ`B3tQRQjZ<06hbqm|C!2%TR{y$uJvD zd%>jWq9HHc2a83PDx^X8L9`x~dQaDY(P7xGoBsMjET~cMn|m)ih;2dqRAVAIuc&|( z#5LztZ7gAs;*REvgvw|=a}<=H%77Hvw$+VPH%yHG#NP;@VW zo`F~ZK*ghaTk9gC>NcvpVK)=ZBBoba%~ei2r&YVp5+((ygSV<_4$J$$&6DI9NmPif zr59Ok=F&&m0~?G?x7^)uS{s=9+&WSNA|fS!EU5YTO<(oySv;@g>n7j&VG~NDA7K5N zQ_{hD$J$HV9vy;-)G6>@bKPgPm186CNB3QEQ%7BpO4^}F(pldtYx&bVL{T>ZneWJi zD`lLKWr>d-6HqrD@Lsck2?$RH`zEf}_p_H-!B!W*4iqMwFaTfRj(VU-k&M2p7Es#d za>_lu7ngJ}Vd|H2W0|mLmoi#>?z^HcJS^Mo6T3n#Mb#hrmevl9i-D3ygF4_7yE2MY+pLl@JWU5F<9{t{|?aO@)#v-iX zqQSI_-}a~T%UKUBu=FA{quiHDKRoSe`nZ2qBfhI_pD-~|?RrWOjt+f#o8;cXRwI*o zb^HkXE2pQV-`OchByp8k5zPAzR4_hVPk5K>;-9x0Y4@oxAr%islyz)Fjdw`KSFsHn zkJwj;{taioli9@1%!=`Wo#{)rBm5|FhUn7e}n@EEJx4Wu$$ge3s+izscFW&H;m)pP;p~qFP5&a=8(iWf6P1 zMyTDapCX<*SYQ-dcEknGXJAVh!ff}Den&!dDZISsp``jMWQ^`gB}<>`3+!D>Rl#0d3w5jC%7%mNsNTltIi}j#_H3;0-P(hLj zt?q|#wwtf@&)$;wx9Ap3Njo3y6joyZzr7Y3@Zn_y{<)NB02<^o7C-RT@VBpq7IsOY zRw?w^^L^TO;%4b5$K&=Acwmb8bDpDUQ${)#i0T7{HQ|i+DRP$vGwx$84h*a|W&Tvc zckldAsO%>ezF_>O7$bDABH)QB z++qrAoVlLas2`d$LKs>Ly5`P1(n}Bq>Y?o9RHbCG6+1*1csoQx;@n?Qju@oA5^xF* z0Lq+!x@WRUge;}|q3wAtdR=mMmQYx(JmIqPbE73e;FNUK+P$j(|A@NEfGUG+i%1F5 z2+~L+Al-S8Zlt@rq?@CJl!Qn(NJvXdDwp?Vdyrixc$jx^DIv&ctutnqGw%7q|U+yhyJ7N}ZLDvKC=)u%`P|SSnYY|a= z{982)acW0p!=53*dS%w4W{drJDZ0eZPV|r9x}DxYeg6C*IQe9-#*`&;M_#egKx7`? z$8^>FDN;#twFAU%zIr&NHouiq+lL_5c z{J+AV8i;vaHYRw<^o~7w6-iF)Z>_3?x({~v;yy(5o4F~gXWIIaK;NBXu3b= z%KcXpdPS1zjia|cF-mdIgj*lFkEtNn3MUHdy<v!;mF0J@-NG}YFIsXwHs|nWH{Y|ENt50Dok=NeJYHC~dP!q8E4@&Q^)Q$RzmvyCb z-$&)K=%MOQ6Z*{R-J9#GGnT$rTKNSj$9sBUPw40pq{Si$Gu$2(ez}dqrlAqNGnpNGR6`-G=_aN|f~ZU3>jI-Y?~1ecIDe>jL&eE5 zlD+*0hqOr4gow0sg>dLFMs0vZ*u5pwOnM^rZ}{@DB?we&B4G*K8aTFfpnqwwB!6yw zL}X+Z$7G+izfm#;K$r1SNQ*ku_TrD_r6YW5LhvhJf>M?@^Q$rU z_tQVEe{(JnFDQ&>&xF3jPq|ilRDPRC@ES9H@KXEK<9#}#7(ojz@Hdb$i%fXkx+t9w z*Qy^a-=tZ~hi2@#-eh;1U)W)1siDe(UzyfYW_m=(Zy-fCa3;^QTgC+jiZWvU) z9`=kU_uzwC|7CJIHBxkW=4C9e$RUIkI(4<@n#%iO0!w{aM&BL-~iG zNaT8${_iJmV-4VUJAtzBz~aOXYq~b<{97zyZuK(wrChj=9=DsME_uW=9w6%gK#Le% z^6abx)y}8=zUzJ!jKi1wCi1-0kdEkY({W;p(p>>`p%syv1=qcqisXjV)5i2d#~}B@ zq`F0V$3DRxpNph_{6&n^6@02e#%a8Z|Da3JSDQu-psH^GYU+iXjoBNb=*B-s@kO!Ty~0dY^=zvv0>#DUZB>1#v9T_+SkjT4slC=Odn44Y(iQZ{pwk!oKv(? z+*BcWqf}bC^V`eaMz?QF5)S0YEw`I4r0|172$$lQ2U#8p^S7P(JExRm85X*op@!El zKRCRaBl_jeGsmz0B7h>C0!3UD&k1Zj9}a?oJi7NL3)Q(&p-Iqt**nLK1@!lhcaAql zmT~w+U8IB19W4VBMQUvC?d9K7HIs5f^Y&57S}Td8hSLjn5s{;t@$Gr9{n>C^t8rnu za^j*hI{qsZ;JpAM1ffHslb-uDErxtnKT19`)dFyy!#)a{)^uZQj9d8pF-pBXz4Vq< zn&?9e+_f#)^wFCoK>BiHb2T>Ne$A5Nn3z=avm)f5SgKs@LTG;H$|gWh*;6uw1SgsZ z^g6HAeye!#;cf15rw0H1z}P6R;Dz;y>0l`HCwj&j)3yVPx#Hk3TCs{`G23D(MGwAR z`v4Km#OU;W_d$MkAZeA-t#x3R91DRuJ*NnwLi>J1-C8P$S@;p$uqD=sZ^d-iJrw{= zc3jw9(Jq@+!?^L#=9YA}tT>!y831} zcg+x~rMz${$t$wI5^n^HGhtct(v<4z?JC^CQ;ybBQZyLw|K{O+JrsDkyKc!WazrKI z55HC+b&&ZNdNtP>$?-TTocR4C$F*LL^VyKkKj}2)vD{ml-w7Q&2^by3{bVEgm@b1KjN3qu9y%%Hmfc~a9UxA8D> zPXMBMwB$`#dG2*t&*Xvh-M^0+ZR5qeiy|@Zdse&w>?$DudOf>Xy$4=*BJ)DMpp^6} zU*}Ha&s*}&8SD%S>CBrPJxg4$BU2iYP1QllRz-KpW?%_DZQG|SFuG3tl{!OP$DR>A z%Mn(Yv^4DxYavwW#E$LvujqSrS20x+%HMz{MNZ81a z;Ieipw!m85WAPs;RF{c9^W|d;2&X)+`QS)`YUe!zV7FL-q_j*z?2j)MR(~D~m7Q`; zCZfe+p$-t?HXAxz0`#=~^N7Ae?Q8q9G5x_}zXN1iM=+v6S}T^U0g&|*?7gu3$gyLE zp!AWG;B&WN8YS_`yX~iPQ8Br*!ODU_iQm>-T^Sit6dnEUPg z^%64la|7GjO+T-S1~4xwjqkT#ZRR(OJfqoATN9g6V7DAxBBC|YUK zaehB53n*2RL5U8yb_57V!xO}4e?kU9I=9I}|GJ+aWN^&o>a^VGe2*G`Y-j@0kVz=; z*Ja`BgKy(Dn2Th8)@q`l23&t-A1x1M+JLQmD@rX7*M*Us^Hm<9w{0|vK|c^^dlM{9 z7sO;B3;;OO(X0DL>B~y(c0qj%5ssOhM|C!@^SSt+Egj(Q{CRcnCMg+qJO)RYor7`N z$)EamX-#NseSQJFSSR$?-Fi@g(Ddk>0#R<=4%3X@q_dOSKZA(wowP71PngV&ErlX%Kk`jA;6i z-=+Ki3OXM9F0NgHI^r(%)pZ)Wo_zhb(a&XJ%QC+GFj4m|rs;#AOo=9%0;3!=xd@qo zq3ME-NNs_a;7GoxM7}^E83Up{2?{PS#Kocc#WpdeKb=iqv++wmhKo^$D5rVvP61uM zd4B^%Y`IMTcFSyh81mC}7lugbPh+=K3;4Q0FRp%emwx%wNJn|D2r#`6aC{8~uv(g6 z8!%jNpN(sXd_0M=RQlA;yR-Gc(2x8+&|@9S-9fy0)2~sPb0zL2AA06PRT^b9V3jq! z(#Z9@et5MPy3=h-x-`e$QruJ9Y-|TWsSk9x`sq^|(Y`u-H%^_yJ5sO7)t~y^czOb; zYF|mXoa8RYM*MotZGvjHy``+ip@?{4k~QD&vL)dN+>lizQp?74&5Zh*O{mpbJm$4+ zR}j9>w=UA!3pyg@YYDH(=bJTRV@(n02#@!LWOxqw`n=uao}!SV-MQ~=iB-@s3?xzhtSA_avd9O z2S#xk{_ePdHB3RjE4yR6j3<$AXaC%X4UvDC|KO?IL%*wL5~rhhq2c?ql6^p{0Y-Rn z$1F0eTR-L@9t>~{M|UxBY2TS53k^G5%ZPlneim;{2%bL4QDcyu&e;r)GuU8~$Y1l| zU1X?=!@TC02zn2@Y)Q0YwpKP#;o|1Ppj+xt7$#+3_!|sC@HnWQa3LZ1p%rKpeg=5_ z)N;UPU1M%w_WB;JiWl@G=n08H4Ez&po!6?ziS8MokQN+k`JVx*HRda(r4*>zv&p7F zR`4Oy5Iv>$(-a?No2sjVL9v^67jZ|%lmvbn7v5x>bl`B}^14$XQqE`LHfs+8Nl(izI95{B6cA?X#R$na$mY@t{>RHShLi1Al#{5!f58t4l8dHL_!cR^Zu)AqWk30$YVI8JKpY5s=dp1 zaCA=;vZrpoXgR%Bzi3tY_w~CvG>Ow#Ipt*f{rT0$C~%^bxh<2wvxl~+n!B|{ckf>_ zwdPAoC{n}9d7vn(O%PCilKx8aNQJSEf*r5tr6YY2D;qJ~yAG&}%S_wUUtjV&p&3Q7 z)5-FnTXH#lw=}+o&(Lw3&~is$Zmw_}yF+t_2yKYa+M9U#0F4On7z!ILS%QQk*9O~ zy9d82UYD*GWb!)*x+Es~p?`UAAl1DQ5p&tE?#~^E3}=+x+9p9|GT>)C^pX~((;5M< z#ZApgi%<8R#P_3b_v7n|FH>a&^3$e6d#>!;^a3vW35&>%mJVDp_ulSVJ-++vm6xP9>tZt=ZNxh=G6KcaWpJeaWC7pGhk6=_QC>AnL=<}}?W9Tk(A5mxWKsR=>+?wU(6&L|`}qir>Ymu@^k<$Gv$-ZU^@ zbn?dZ&oqCYSN7~Le=v2Q%_@EPi;;|Z0Yil-l@eWFU{5I+eguzp9lK7fI18qm3KiKRoYA`{-gb!*@qTLx`R@kvxLaLTIl+B082L!|>)7 zAxy7z(;S?5WsLgj-b4)0Il z|K`L)m#??TZw&9C?-j}s+*_E}{BQ&>6+pmAdc+H?)sA&sSHBJYk8U8cC?8QVp;mde zZ8nlH$_?ZSJt-p+m|5QPFdvGo5KTff0nu||_E;MAr^VUtaQ?hBC|IdvC}Zn11wx2x zOOxt_dd@WnX%OO%+~$1KNc~pOQ~Qy|2(294B8PHvKa*2SDSW)JUSA`CB7^s*IJH$y zeOH&PF&&D@X@+EEk+Z?!YwQ7GC0Gyu-Z=dg>aNQNcb=$b@eiNJuWB8t+sQaxvDPr< z`sp4D78_xU>%T&ELFhQvkJ8jaNvP~Jk(%A(&U~*cBlVC{0V0!#qnnDx_T0w6ZUCZf z3c#pD?c5k9$lw@*AO6d)t?6z4d+sg$;#r)NBkqkH6hv%jIFx%7fP@R;WI5E@h7oOh zcircarqmqU<>)mU#VmFm03ogteR#U_-DP3X4%+rslz%s|fi9OA?dL+B+wQ;pgzS1) zx#{h$kq`YC87zM-$(-{_W%(;Xqwtj6Vg#wP`^S{!8ZU3@Zpb($ljuz#wP1cQZZVMaAhW zhLrvz+UidU9hJQHOE$g$0RRF8@&OsO9}Bo*Fm@N%s}`)#I^gYSK$HdiMv#68`ukZ}R777?@^x0h;bb$G_Fi0NLc%VM6lK?nU)i9K_fv$h zt!n8JKjLM(JSu>P;ozQs%=wy=8zjz$mEfCw>F8h6g@P>ugK>`!(sPTqMRl6>ZP*;t z{x*g@#=ZB=x^?;O@%f(p@-Pthp6`;S0{MA0;Cfi`{5~`OPlsEXMT$WpKc&TbhMMY? zmE-96HwW=iw{7@i7RX1e+dyXO5;ARD8nO<$4Hx#Q{9Vr>S<^+*cdn!0&qG*f5wwhF zb`>HY^CU&+q_2t`q=w%@l;6n!Ljn9qH$_ymWmI-84%aefSrlW2=UlTKr0B;~y)!J? zKVSAAdv1CV`Uq9%S^edb7k^!mOaFOQRVZhvxqUuh3nSdY_^$}`@~~pYX^23jx3`0` zclxDlygfj3z>hHE9i@ub_AwO4CJiUt=DZU?$`h^>rSb=F8#JAT3@gmPb6If|yv+MT zB=xuf#W(1y0KH_cA(&mRSUspQ$`8DAV;>v#egSHfDu z={PAlPU%;2ZMd;vJEYM+4hB6VDSpf>IG4ZUAT209-*lzNiKk0c!$E(zWU;T~@Eqxn z(QHmWBj!2$9XggUz^ruFEML048CvSpd7k%MHgJ4T!ypzYv( zGHH)Y^p=FB9@wD;m>iptVcZ-*2JKIUlMxlpY7}>b%?k&qg}i&IS6wqPlm_uufHDF< z=@@sa!d8PumAV*o$nct?LGod7$(Qg_(lJ{s&QJI9?$;wSc=cHZy0ggIFa8yp?^jTn z5EUOiC3TQ&L}F$;>OSZ+?9n2x%N3{Z!5%idSmkbG%JQ01kji1#2T&^bjY&Iy5P+v? zVf_3)AZwEmkO8Z+#I021e#LT~_v4OhCjO-^BZ}DX>Z$>RUJ#~^f7U)fYEdTRm{Rb4 z&6R|c&lOEJD3%qGJ}LNesDESBm=*=ruc!kqO{W%5)xp!!85!QV?hgY?r&R__6Ob5P zo`vo)?H8Xg#9;l!L{MJYqBqB=3tpiH_@Bqz&^@aPM8XXV#INsH#&ThX_pS)Q4rBfW zYfItr+~Jk7g^$!w6%BuLmg=GaJs(8bj^&#kkXTmNS)tTDs8g?DExR2gchGX9Wgsq= zc!4MIvP!F2y!LgthinoEoVHJ}!#Yjio{FoRb@Iuo;F))fmKo{&dX$@Kc7sQMi*8_H zkK7Suw4F(Gay^sqn`T@`mf}xxL7Pf)dMDiT0C&1|zhe;}g%`!8kHx53hX4AEN4)y^RvrcDt@ki}Ws)0q*xK;tz`Pk~s#uG75S( z+W=a8N;XpIXs(e;>%ZS|ALZ3n+lEqCfs$hm`Ec)S3E&>1SlUN#CF}MS>?9|n<*r&l z0`0IeYe!$vy2c>=7{G&jyAKblw6c@ZPZYN&VdQR37_V_yFY0BOoBVg+u8gsd7vB5V z2PP$@P=pS&oy@U&&y&B%lsR@VVFwkfep{fzeRqE}Os=l(mp$Jgq0qU*n#v-!m;>J8 zL3m<0y1-7~J^1&3z0tj7toKYz*<<=ThfQra;-VgTA#K6_MrBs&@qc^zY$V?MGj%NY z@P)%s5Q^@Jo`UHk_GI~Z`cH`e`&w`q_XwYtWy*=dS`A&UR7^(S$`kBvvu^-#*=7FE z?B9KikVUQg+EWCaZQ#w7#OGOf*zgCbhG|Muqrv}Y_A$ow^{<#DirYanUHsKmN!5a) z^K+p%_n#^=8f5?37txKiS$|`O1%ZCL-g3N{hAn6Pq|9y6{{Qdfc#2ScV4e6)UCf#y zWHtscAU-v0mHyS~s(Jkys6Es%7L@L5CQS3Nn# z$&tYf-~>>uTo5b&S3sN?hZs+>k|v0Qx~Kn64n?GMy{hXd4~7fcN`$Gd)zPPRv)WfD z-wgyQGbkW22ff4}dr?zE|GUQ0$Xs;-5_C}OB&>AqHucFL#lf`j$tP2Cn8=41-UXx> zm>dDF+AYdz#vu29#-C%IK+L{5Km0?KZeESZ{L2~eAoK>6h=2Y03E=cC-Jf*>>WW{} z@x>JK(Bajx^CDTsvf04QrWV9`Up7$IYcc zy|9)3c0Jy#E?;=Nm4mTlWUgwtSBm~L+G!vDT}KZpMEMOJ;;pmwG@hmw;N(aOf6mY` zG$nn~?@z%yajN&?6Z-lifW1_^4{a}4I&2s+@<7U6>uQV$$4%rvR0-i z0!Zn~N~ix6T%V8C$>P0c&Q_4XSUmxr+pk|FAPY+(&3RV2<8EC>7gUd$dIcAQ5|E+E zt{F0Ls)>m# z@d~^Y1WeWfqYl zk&TLe+eKnNAjzmJJAdTIFP)Vvnu*z>j$zZ~~HkA~_M-(XDgEk(WEjF7y zw)G?kq%E$Q6jlt7n6peS=7oxTLAR_Ti9`XWDwkK@%6VWsvjFjeg>AV zmOrcLyn){V@uoxKu9RQ+tY}tJx8hX}if&)x#2*G-VK_m}H_3Yi7SovzW!B>W12PPF zmFntsB+YnpF@tubbPpr8tgzLJ^*~Pr?B|BN{^qRm%Bwa_Z~bC&bnL$!-5eYpZU8YiHO8Sn$l>KL-TCq#7SZTf@_Xjn`lX`&Y~00Tu9t0gfuSY}|gJv)mtb zw;6&XLW_@1b3BvbXsFJ5(b14p#%0gF?AhpN&X5{$o0CA0idh>qrTaJ{bT4Bzm?uI3 z7Lzv;U%^3tMU$HK z$}>9q=0KX7(y8#Gnrz#Ho#^AIvw^zaND;O=)h}Ka+wPK)7!32$?RqkEqKuN%-%y-| z%(5v+VC6=Zy59-fY<*WXAn;MGDHU3Mb-1!9M$g3Ufr&R&J5OIY5f#C$9Iud0>Sqzn z@(uVFd5;yb5Pr~Clb_E!($=~4tGOR}e=1y{Wjc`23(&GWCAT4kw$8`)T@Hog^tD*% zu74XM)LLWP@d#P;dZE5*X9-(#igrW@`F?k16K*Tkz44^e)M8bgrd@Z0Ut*z7{r2x8 z+u-HU=l29{IVlNE5$7gLwv##)%RkNw9EGiSWpuH|O^IewpzG0Io5Blj&KuV$Me1`= zuTBs<$llgxv7cosMjv*Z=A5?*vEL@wku~dR(a5xRN7yMU-PHt4JM6I}oUI2>3^=!m z)F(JMCEy^;EG&9(laAm0>1!@1Xs)3_f?(7s`K)trn8hgYQ~8dTq6Z!1Dcd! z{n2l|F2B>Wn;IV_Ix>Vj;~#*b^E{NLqv4y?OTZUge8t7WkCpSkr3OMj+>zX?!MW2i%%37IjctMfs>h_^M#Zi3}jJLL1EQ zjCMcSca@jSd06ZSd#^4>RLJ1DjF1U3PhL15TJOu$ZWxZ-t)~x!FZ%K_KDmh?fcopT@}mS zB~U*6LdQzXRUc+0oC56Z@=}gDD-kfUsG{W-MQ#TDb#!^C$La=*)|W(2Qav*5@~dZuZAHnL%X^N zk!P{l6{^6XQoFji$r~(~F!3Is8d^=v;ag1v3bC*r9g~q*%5JdG3p`ir`X!h9tS<9_ zpU=i);FKCL75|n$vpVma74igE=X&m~c!}LkTFN`8@*H7wh%)C(uUoMa;TbPIpFuEY z#oHdWOZcCih>#_+#+!x0-3oILKpkyB`T+k^q!foea+;f@;ki%iY`4{tIUvLWx!KHi zxR+jOFcz9a&w@lij41;BQ#v|(amdKfWy=6jrf2%~%VlEn%dCfAX6T{rNkktKBEXZpvGKz%+m= zW-W5z@TA@_*&hB`efV-Oajq>VYz;B=z7t;;{luOt!TfU-JiQk>F&BNBB5C0#Z|vr> zCxc!l%{{V&k|3y ztm>QtuPMLK0f($vd=My}%sy@Z|wRo7KPX}b{s>t^|ABwZzE3^1QB;~x2m>l|wOGQTR zwaDo-Igs+Xz!W8TdCI1 zeiExLQ~&i7d@{w3kKkZGb73lInjdO)rOkL z=lc8-gm&zG$C@sf9=OBxz7eyV{nf-Fx{|6>gm3AVqyjQ!46@IBoJ(r{0qPPl{y}%en~7+l!Q$pJf!hPh&vnkzQ@li5)yT+t%WP2Us>84D z9j&#_o{K-}RM@N#+W>||;S-SM^XxQ1<+}U(*v1L(yzu^{r&a%=2O*7m)gEFA zn0T0QT2Bom_+alcX!c!m1j|S3W(I?0(FE8*pYoa_ME`gl@5o`2Z;D~%YTpY!sgLOn zG!?1?@-IJR^%?Mtw}XR2fYyyb3#cBWID4ve&Ge+DNMB?=w3N!ltH2|2QW*PK@x87x zR@@yX3V5aP@skbLRx7pvh;j)CQZYSjVq%+@fidvY-)c6M@_V`z??H|ni;e+|fCo6*IUYX>*6JFqn~t?P8=yx`<* z(+2v(wZh9kMIIceIr6)q*a8zY5g4Ic94=qBUeYT@E5Ki=amA#jL6N;j4uXl~VOsiI zFrtoDR4X zA!E~WUBg^E5%XIv<^G)2q1hHEPxV!j!SSwq0g4~ad|jv$=Fn|K4rxUCka4D)ai$#( zb!hXwGw=#fqVMYsNB5ddoPXsL+r0|B0F$MhiYk_$o99RY?~Ujw@2nq70mB^`QYdTy z_A^M?6R7v;e4W`=7J3xa;LOaYiD)lzq3W?rLp8=T3sR>Qt3aqxTjLx zBNN6AC;|C8GSDTf;^%+aAXZW1udX_Xc!$|r;?PH)OY*~P<57u)lU zMdy>zHvw522k(_U(m$%AD`lG7V#S58)|`Mug1>8*v0o!FU*h8-04Y?#!t~WfXBcfv z(%|}~eM$c9Wqsp%1>MKcCkl_F$Px0O4@yN*O~aW(cUTFFQyq9dz^BgCdR*w~C0U}8 z`nF`DrLv>xq{5sv(g5dk0Gyd4R_=mE7cJ`^p9|wBx;?f?rmx$>YS!gu8&%1rt8+?v z3O6$N1PBu~=f9vj`xb(_fmv&0<=eFWPb`;IxSmZco}ihMrmGdu4vBIQAM_P;w_TY&>^enV1pr3#4nD zyFMCRt*{((01tCckvJIgQc8AMvEm&e15v@gn$Oszr zpa&Rybj9d@z6>P2dx9`#fDQpZLZ^mX z)601o4~RG>0Bei((>O>9aROPrQJUcaS%vXoOC(KM^s>P2qoiw>R~^V1T1a1Hx;oh( zoi-fM%dQ1a@7z9k5m{~Z91Qg4`yOvd&E<l=k4P)>mqfo9heWQAW@o0ywu?;S-OB#ChhogIC~oyM8Z6#BJ(+SSqr;E}Pr9d|GN zz4DCY(*G^nQyY^v$_}om&cyDL!%fG*!e!4Fb$Gl>{B9&O8=bqC_E9X%xRO<|uG6wp zu7bMhV`d(n|jBMmc>6GU~xwpVTPfHGX#>#YeUct*(IR!_Xen_@5JPFi7f2 zKYo88Gau2Rz#HlDjWE#1k6Ex2yLQZk?X8Aqm)n*u@ytfm&6S^gKR9ia8(7>ihK?l- zn`2@$w0~&{6{7f+th$DiGyAwYBza-kLF(HK?fA!wKBmR%#|J>&ekMH=Qm93*ndFKN zNlZ){6Py6~js?B{^g$;i#sVZ>RVd5>X}Th~2QrZ=HutKS?t~mN=CS=3S~$W4lVC`I z@|!5lod602i2B&^x|)P~7y<;C(xx&s{fND_6js@%f7^d*q9Gfyus5ri0Y_p*fy7A9?;|u@iAK=Tu_58m&@|vm+93*_3^Gt{Z|PwCDU(z zaGL1OqcN8GEoL<2`4}xCH#WWhhya<3G91Zp0iG2)*#C9UJm8>nx`Fe!^C#~bSW+of z9=#@qR*N!_k*|2fEXbIw|g z!O_=2%OHu{@|W!FXWJz)pASPVB7{#RHku+5*O22QE6CWlF#}9&eKi~epFfrQNc=Eh zew5L@G`;3L^>Qz>MGC9=^+$>* zDMTAG>QyX@{%_=&Ih8n}srNS~w{brPXd%kA@eAb3cmpZnmHbadHIN}9&8(W6jjB_t zf_ngQ+cVa=y~q-4Jf1kn&y0STV^!A=OtAvT_$GnmGpqX(q=2#3ks+T+TaQ~{myO*` z?h@GOG>9zSzGlJnZ|+#H`S5uo5}@GSYMIc8DNGjuV6pw0|GS)Ud+ z`6ZkCJg5nf^7rbj(8Kl0eY?7zXE{`BZWdSfIB(+W@#aeF7l1md4!IZa^$aKZ?Sz6Y zJ+q{>e`|t9XL`Nc2^1M0%s_CJNVj{mro;q))tO82B6R~kEMwfU)ZttThQ2REVO`2h zsNK)t zUN)`(F|Uew4NkA$=7E*R$x?d59e`X5MMQkJTilVG%`59&QvxYuVoh&zOb<=kvm;aw zLHXv8{DWO|Yy{?8&pdu$$!-p+r~;EVoW`vph*g8!PADh6AvOBuYje4l6>jl9UD@Cs zw$SIZvVzF=53dPKbSIQZPug}Y8v9Z~mC{<6{K?8hp{HbMkvK$QG3$e8dqEn}R=wl? z5M7v{^8bsg5dzE~c3G@kYp6xI-^9*-SAHky7(&4E>J*x$;}G=d7qXILSsP^0QaVXh0mq&q;u_0=jh_6GjE*xpQX(k%u+0+H|EM;CQ=R+^_mQ^ zfKty!~fUw%?G}P$Y?PI$AovhyVFZaqV zDbx3{MW0P2sp*<|!%PsBNG9SMoeP}|fFaLb zxSr{*SZ2>~!VVEWL}NVo6++*R7i8P#_7wyc-ASG|Zu@Na^8JJ{EC2*TT%r!D>Rj}_ z9|qriG|1G9gk?)EzGQ1e7+l-aA;k}Oo-T z^AYf=r4=j7&4RffB5SZ72a}Uf@IJjED}DaeFeavAt@l}i0j1lB`Ki~mfG`@PSSkhF zy!;~&Iiu+64&On`vrmb9aTYhUwnnobnzrK_3j#In+=_N|5WD|0g#(ED2mG+!?M%DX zNgvd>WeR#~hQkDH5O`6A(zEl4;Xsr%5~BZq1P;pT>e9E)-}oj&!}#sU_c3-Tv%^W# zEqr1SO|A-KUl{Iv$4WIbXQ+Pi5Oo5xtGrvuqICwd z%T2jkzOqt$7U3Zf1I9ArXD)53n&nl1q|lFn!!Cd=J+4l$OAtLiAc>Cgx=Sk^deG_hc<>i*LFdBcyZv`qFKqRhur6u26zQjKO+w36+Vt z#~GSv$wYB&-qy~1N%TWA{L=%Vhf@06oJSxS3r7CVt^seWoMjmPJSDULZBUPuv^4!? z*giZrA7Z!ip+{|=F-s-t2=nO|YVN|XARu_VD1CE8$J#TADrN5f1`ZZ`ge^5h{j^d_ z3x^J^ll^n%=4A&rl;9qZusI2Z@L+)^MsGsr*0%UVm_w{Oxe(Dyu4^3m9k5w0tE}K9 zF<3kfhQ-I?b(1GJpWi96uGlJa+R)cs0K^A-l_^h{zHXlU&vzUWm-j!YZ5gpFkiGtU zGKo>J843;$EBR5vCvc0Qy0`SY{Vdho<67j}zOO7Uls~VV|F7|n3(DBoRUQwApP}Vm zY^f=yjbd~HBz+MB)PhI5n8EoVY@r|Z+hmaD4V8P$!*zlMc{M9@oGmRo2qCRd-tVsL_CU~X4Ru3xxOr1JY}fFLvxDSZAWqvIeNFSSX4NYZdIDSB& zyASkYbe76cSgw9e5S~N-CC+^UkdK`KKc>F)x)a}25-lL%+0#kw$8u0^DN$dOQ@|x$ zQ`6nXB-f6roh$x8AacLixl@hih?fd@!!6=r6evITKq8h(9Cic+N;iYaU>#C#03k#N zm|Q#031Tn$X+6FVuL+Iwx8$49U#G~yT=Lc#caI-qyH9DHhLPs8J!F|xmx3|X@HGL> z@Z)I_B}WaYpC6duj=0K>vDWLzRJ3ERGL6H%JPO{nn3&7ka8Hd1KV;vDR*SqCbmwm^ zCHAZ9k73T&jML(th|1V`tM>FVqZ=hJi#JE_dp9WPds*aQ9Ep!~9Xa2U zQ{NJDgN*f>wx7-sEE7xPJ7^#x!;@K$S^KfPoDf7YCk^*Z?<&2`46F8S%8P&PkjpCk zJvNyD)rb=1$&rPXwe^ev-Uxn@?TW_Q-@B9pq=ULTUU}=)mlS6bL|CN7yK*RowEC!ny1{LKU8hb zCbVvuxwEIRWudbiK6Ernp0*2HdQ$I|RPi-q9ukt@Ws7fPr$2k<$Z-3A1VP~I7MWKJINoq@fuX(f$SHLlpkk28sX^iTn>yQR z;$=~~H>P*p1H$Kk{~z((Mlw3j7}hw($qqk3!9D=L`TVLlIIuC*4g?VtU+>2z4C4L{ zAdb^VoI}y;b19I)n#Yb>y~x^MgTPk%r#QXq-@@CG7{_T`?khIZQ;DJvx0a1006n}{wvs*F zI(pH&5)MFDFwSA2S#isyoo{MVgkv5gj8%4}#6k<(^xQs;q>%JO@k0#BOuj(i_RlD8 z0#8+a{dFspHD+*m za51U`1UI62;b&7Lc`~JZ-VLzm2riRJeVd>aBs@yg^*u)xobp~M$lyQYn&TF#@W1l( zlnA>#$3%AJ#q$lQ{7#+xZ$-;4wFzp!mq<#_swxPlZuu{M(D``%yT{6kogqalo7;mk zw~NB+C)c+Uqd!1`fDkSahj+?Wqj$$N`bYu@ka0DJqTc9du;pp!^fxA45ZJq{&ZKU_ zF~0K625Ib-wHFs=PDiGQpuxKg3G*{FIS=3JVF?2?D`OC25`XZ-T5mvJ<3kttIM!Iz zxdR3G)!V+awZtxH2#|(XjUsJbbc}ML(l=VLt<19IMA0!>jQx$n>fvpev_Sp>Sbc5+ zV!pnWm0~&l{~3%#U>5a>SB6Jybj*8#$$b=%NmpMabpvZ8eij9z#Acb|pa;Z`CR-jz z4kvs6zh(Qos(tYYHkufVWqQy83!$E47Q<@N`3a?QL`L5l#_^2tp(m_?bQMK<&gTVo zZ)X8k70hxQg-$H52;toU$|^am%im^(UNZuqVkg|Y7Q`*Q^znp|gO9I}Q#C}+c*B5M z8KTp5Ic{cdt*Pc;0_$v;!!XpRF4ryc45eaQ)Z0sj*5iWa4S~MKancVdT1G|^^FCHs zyQ|Iw;~ss|4U8R!ilOCNr#A%66LoH1d=0kn9!RX)bc_fPnU@eL#x{cz6{wkcVKSim zoa%hC{vnaNRl*#v2w3YNitw&n$=@8-eBp z{OEG{H*2@XAq8M4@=q-Prg|Mn0sDJ*913Lk%{6?OL!ceJf9+?|4tBlk6nkH@%Elbd z|D5t=aLRSxxJUAc>+ic&NHVFqQ%1?}qI;B+==ZVMzrhR#g4(J% zJQ0{gp2^X2SkOJESEe?~FL(2L){WeuK|^v;aXH^!Jur}g6Rp9`>9g&Q&{|j#JDr;Y z!VV5(xEHzq`MWC+z!3{9!aj?N1EHDmCNd=;g;KU(n8z>MrG&Bs?J=0U5dvq#;LNt@ zSkd_2^>>rd=avko0ERnw7Xos`T=kz{2g6P&YXDdwlZMCU`;EmFS9O!om7OEYFEgvWVVvn|WdM*{5EVfD$xf&ns0;cKQL*Fu!w3 z!%Tm@<*;Gsem&3r+N!-rhKNn{wXX8tJVs;KBv~EID*0APH` z%Xj0mdOBUlI(^45ufOr|c~l91m$1RE_*zF34#YnXQ`^$7d!~zZ{$0rF#Q3_mcql*9 zS8)5s_j!}6^xN}-G|%c2=f3O@XULzvAL>^qaDoAZ7hICI3=Kp~-2dd$0LeQFD|$A&XSNIybrrqv1eu z&9h6ji`aXSxDnMcQ83buzQ4PxTg)PSyV1W2CAGcz{kA>yK={_s=iv5Ixt*?y$TH1e z@KEfSg(rjb`;vcQeX%u)?yYN2K%BoUW&DvLxQj;fPkyl@xdvoaq@bVX3aQ*Px!=i5 z9sP=Wd_Lzf@6}S#;&xh-ke8^7Z{W>;Z*n|;JRp+p^LKTIZfTMWaXq9Ve`YfSK_ty1 zoA0|(PhLWbEi>_sZrzes%f0_~#jmI;h4d+1ljy$2imC}?*-RWK8$qxl!g=Wwhuy3) z(nrsSb1Bv%ad=1pd;WPQ35kXnhS3*0W(t%yi}c!K?}2o4>hm8li7~;=5qvBY{XUju zBzIdBc2e-`6k@2_`@`57q}gEq{qlS$nZSj7es71HD-DS8Z)6$v z*+O6!2lu^wz#g`lTau_egU*}el6&xY5ge{4yS-adPFx)7EPtbkoF?VI2zU_TM~9JH z5+Q|#M7-~Zl>D#bLOE6V4_!!Uw(YQ9Q`?5%R6$BroM;@PVLN0ZqfKfNNa2GJ)`{mM z4sLS%i{~5SA1_KC%&b&>(GC~6)`%OUA|hEge|+yjZ>E)2zy)4>B&ZF%|R66P=Q z?=2*)5tg-R5~4h3bAU89g0C{Kei-k1RxAr4-NV;)_)Yt)irf4B3Hz&FP^NyTNv`Dr z8_9a)_Dcb^lKGn#OQ1E0_OLk9i1;iNP_Ey}#;`MVkFqjT(7PBkmBN-Ze8l)oj9+Xi z+UNKgBCvE>?9&X=Skx$$+RB{?K@UxOaov2iqS;zFs^VhLVq=pc?{SpOPwv4S5PLVA zKLlYYVG;=O@z}Fohuan(H9=h8paOGLtxWu-sgV_rQO~a|j@QRszbv<9bM3K!;!vwJ zk{>7I5Cj!qf_|3MjAN{py8g-o8wuwg>F>g0RF#c4_LT%5WJ9oKcQRuY|ITj4lP(Ox zqh~|})~hFTVxQZubPA2aYl*o7$&cLGa?~lkW)J`G*uCM5ZTbsUUc%tOhpG3u1+mSK zI$p$Z88B10t3)s-8pS_YCf%5HC*iCNV5NTiFp)_`R3Ao&=4MRoC3?XxwO9I*^r+eo z_7A9i>(2UN)ib1uk(wW-GxCv|Y3MrX((u5*lx^_UmWie6+-Mk=NiGl4D-@7Ni*Ub* zKa3)izy8cT@&GCTR$`?wd#VMJbP(P2uKl~?v)rIPgX!d4Jo?pLS*zyvny zP&0^rm}~~Ynq=Jaz7ej1#AiY@W@cn?P@hX_D+|UKPEQEY=FzwF4GYeZY9m0!M^G`; z`jU!h#?FGl5q*xnEnf>Cjk7D8i@Pd>z(HWeB+L2fpwXS-LM5aS^4ei^st_cG!+oLB zHP%I57bXDcadNQA^tU3r{Q%1qnF#*?3%{rHyqO5ZU zQsTT>4Wu8P(kT9EYUJDaN4Btb*HKr=k0!~6odOXzhKi_svr704P5xKZv(v3T-lYa!L1JNBRG&(cOu$WXX1G6EM z`F_ocGagOxJqeUOfC(-&(=;y3x;=-UNYJ*Ij;Id*h={t`ZIJ3VzN49iNHd}`I6{tS zcPJ`tirPX#q_}a9SQV!kPI;L|s|F2D^3QDFpOFxQy40w{=zzDFAu*^Nr2XT1!7dS9 zHb0D!hkVtS=UYKyYAvK~5oGXvmB+fhdL~e1X3e7zO?C4qSut?wrWK=jB)=xx_7}{GJ+3`@Un1d~f`R<*DaH2!dj%}XGGhLA@ z;@~f;S>owm!|%67CK^enap!EmwRDceK4{mKE9CP$!t$$LF3+I3>x~e_En|m#kH6^b z+nSs@5CDzvixn^47qXzQw$zlzHq2SFE)=5Ol>F1L{y{m=l~9W&gNG(Tmr(Cbtt#Kz zTK7&{GZ+&nYu0a6dn}4+;xEDrNb-I@_9_=R0@E26c$|N`my3+=nP`QU6CW8+? zbpskJcI>C4k3Dl=2ElGYK?Yc;aX|J55&Ks>U>o}%@h*oi|A>Pw|8L@8ULSO-mkEpy zg!Ulue=h&;;-J?4`F2oi|A>QH`w#KdZEVqfC$5-(ud-ovb?KkbYi)l%T4v+1IxJtm z8lOR!EnW2E_TJyK$LxIXe?jWdyPPCam&xihTPPjRK+0@#D*K8l{}dUP+s@K*DzP3( z2kocvxZ3Hbb+>Wr36LpyvN-rNc`!GoQPbhIK4BXsum86b>-cC!P38N4sD%AnY*y#} z=6jm5m#6#6+I|L*qrUF(kMtbwu}*E?ef#l^l+E^eeGwZjTE~AV4-f5;_^7rt{lt~W zx`AM=39z&vT;xW?*ZSTU0uo#VaJ8H@`YIn*KF-dS`e^+UyQ)(!*AAxxY>b*7JF9I6ey{ zpvS`gd(UDbe2`t9uE%@Szx^s{DQh&N)ys)vGoxRgn#Q<^LCUXwT+ZqIi|AaNq>adH zVMXQRYtHJouO`fzjC~Zy#$z3{;-}TFy2`xa{m;#-W>WM*PITXQ>lJ6g3h88XeM@&) zY0LHN+ZmQOul%BYyj2oqTH?H)3?4}QP3%8)7K#kf?$In(@(R+!CD!6O|Bf`^w67n} z@cHkurgjB4mi-zjjFHj5W@T(-A*7I7QgHsta1hq%eQ1~fXiw31dnII_Xcr&fFLi!n z0HrQuqCc#!%fUwG3l|I=KDRH`%Q{FO&l}#4X2+qqrW+aj3<<-@WfRM6+K~TMngT`& zwE3}hvL&6>wn8`08@5sL$>-vPdEs&WTK50cYcvQ*_HzN0x2LxRtVq4JX$&6M62Hax zmOJmb3w|$lkHHEJ9~gV?J+GN#8IrUU*Aw5%pw<|9tJ4abC2!hdmSQ?w`7fP@Me3CZ z>HB$UQ6mqgzV%0oxf7aY4G`|rWf1O)w;PokcAP#3*C3D zy!q|EpCi9E&0w}YM?`Vl$N$)Ui5Coc=>jcK8YNEOyJ^T;v+?8YpSx|z)`dEmFtPb z(wFHQ!w-Az$Gh)C8=l#;tU-umxU>H|x+h?OV=UKUdoBW*bg8YEAq0s4L)R z1tgce zI?tZ?9^Ahrtoz>Dyj?c=a5-GZ-Ao5!Zg{`=%%`9GGK{9U7mhvp_1jZ0Y*4lfWjBan zzOqZijIr5i4Ylp-2wp2R49VU_hAGHZvMN05j8nRDizHi1F_P zQuX__sw<=1yUYEEd(p&rj?jBsd+m`oLT5JPNU>gHEy%z|Lr%S7pNhtwA3?q31OL(K zSNm`^yYQ96#t*!})7m`|wf4*2SW*T!Ibi{W5VXARSz4ORo7-^RNj*!7&BLO6nb(_- z*WHTwWR%PVthW93)4Fc;0~WmBG0wpYAZ}D3^NRVr*{SvSVI{<-+$0hXFLlnC7K)Fx zKHU@B!(cukk-=+r9yxzSf}Q|)H;~`>>It&*LaY96>(5=QdHD+dnrLNP*(LkIz8CY| z2j(XI{S%nmHAartgZq@lujqZ?#Ba0gnqP4>TUg;xl6coFn3Dw#JkM}<`+8o$wZGW`{vJYRozz}3xTqA>`2#O)ho6mFKS8wE0y(Lgw> zNux#DZ(R32_R6JmcfUI_Uhk@Y!D|DDd#)OCEK`|P&2hogVbOa5PQk#=>zCe1%iv)U zo7s-rX8-W1VKJM24c%4UzMW-%{`_O|;GUX8>eHuq-?tfqGz7{9I`O4__=5mAbxR?I z@djZ-1f+B=uXgm~BRF$kA5LS#*Y*KFf8WV!=k3Z#-CEz=MoM))!$g!s5G?WfZ9fU3 zxhj>o%%9SL(1&@iq5YZ9F-2P(JL@)OW3&%{?MR6olp>)x)jIhftf>f+?cL z5k8lyxyN1&dJf8<8OX4R_E5)my?7BoQ{2G9z2(%lHTsm#!whG4IJ8_Aa{7gy5+5Sz zG-Bq%Un+n)>UYsM{v7AkUgrmxb~skpj4m#6ZH)^~;q$z2+k_ucmB%)HnOuaAB0SC> zH+x^_Q#AwGIOI&kpIH5-4d?0^BpUx#4S$ZTi7(|1LXbkEeyfY?xT=d6yp%qsm!8+% z>eG)ZLP1{yrccNPoT)4-K-Ta7D(5lYt&A0=rojqBy@*!u;f8W&0&{i6PkM0a7dFoy zZ!CDPQYuS(`8?qkFWd5HGFxk`PP;oQlgqNP_>x1Dob*=_a}WV#T`uWLb@lG_53cH! z;OYPyHLb5;(1PW5sB=(Ggk^;IH8)=SnyrXoX`i$D32r!HfVR*G|uh&g;#gw3rvvo4AiB_0@pBB|3L)*d~crb#$m5 z_R~$d=yyOoEc-KGm&N?!o6-dv0T489zB)gtdLiGAR&4e*f_(Y!K6&hKbO5<5o~if? zQY;8pvEA(=ggCO()8X2~0(`j?bxs)S-T$qDyhA;N7Vq;lR67zr(~Wc`>Sh0(?;4Vu z!mFdIlTqE`INKFNITL!C6$^4?%0=|^laHRO`5f{BIWOnGKu#A!w@MAvdQP(dw7MU& zwdnNC>OwagV5S*>o$`n&e@YewQeZpXfQQ9bkkhEpJ8DbLi}>tgq}5Qo#8a5|c0?_JK3U z&2Ur+!yPOI)DPRqK@`#@d-p+wRlh)CfY)1_&RsARo~g}>KkwgjOKp?jl};MWO`pp> zP#v0wYv^+v4cwWmt`RbN6h)ZBo+ft!tbZ9T4=n=O!*|lG=`BLEVHLjyrD}8L?R2|V?>8AttN>$7HPx#NOG6y5z&a|=Z=zxma3=s5AGsBjmt_s8AHo&&J!SFz!@n^@*F&r(OZyH%nN{b$|AgB5O5^b#Z3zR{yM+mC^g z38uy6#ueGc5Oj<~Q@_h)=W7S|(t`I2=Hn?xRIfZU$ZkOs83e3Wmez3;zamViaivAL z^Sa-C=hnPHp1wzDDl?}MvhvN+N;Se;AM3wpqL`QxdoC=>*Nj$k^X5?9jw5(}A-kD3 zGfut2D-_doZxRQeF$V*R0!9xp0 zEp(>4F5twZTwnaxu~Wm0RDM5U3+`NfAnQaMb>P=)K7mDGhtX<$mUy_R_Hc6Ph($7| z{jzbf3rN0ywu%r^nAR!Ke;of~-H>Eh3#x-hT3xAo_n3}~^4kIhq+h7xX^0N+;8{mGg|}#Y4v|L z)uA1Ie1J?Q$7tzuk)W!rawEWP&aG0%DgP4^F$)}ZE~gdP))?KT)JWAuIy7q|U8 zZ%1GQwMYrJCD%Zxuv3Gm#n$^0ykC0QzZH~Ar1pZT^BzZ(95>;v$eKsV z2LspW`?g7xVqV*fy7@X+@YKTtk5GEo0kCTU4V~6hJqPr(AV1?ARChk=|1!3(LJEvs zIrUb~53SE~x3#b`cP&}lWsSI@u6t_DC!S10l%%-QZ9KMi9@)=FK3G_%fKd0AyQvb8 z$?(Bwbsk!1%=bTelXpCi72xT~`KPl3J;fIpg!E+z2fhzhBM2<&7134pp?#=$1&HE65$J@ZV z9|O!}3Ilm+q>GE}iCncGH67P5fT7)J^hZsdo(BU!CT$i_J1tF>f$OVR&&|vEx1B}n z$$iltS!Liq;GruRDG+p=!b(>Cw*Bkl=*bWEAH!zOh7KR(2Z)>Vt|G<{+>d1%VaYs`Vs#d6mQe?hXXvM|Vo3??C+VKsoov zK}9hAW($$t;~Un^q+Y8oD3f8~a|m!w;3F5qDRKGM347nNE-v;&1G8~4bu7JY zC!J6@PKPWTY@NQ5#m#{$k6zx;%!ayv^J1^?E|>yhs~uXm)*iJ&uaMOED#*8c%Qup< zhmec!ug?Y38h^Ul0lZ2;5WGi12?>!CO=zGOazNTq#LJxXd%0b6M1hu4O@03S!;qq9 z{{dZ)qCkYr(#jHE>}tF`zNkEY;J~~Qmr%dOlL1>ty>gYS**h-1N+u2HOl9tA&+kC; zPGjx900n*-TcPow#GZk$v2XOcj*$WETqMASc*nyr*p}Bkfz6BV^id^PI~F|d%R5*A zB-@WvdUG1aIwV) zmQ_Ht&fjUpGqkG;`b6S}GY}W{35T-NRO+tN02K zP1wj{=V+=i=bb471=8DvuY=M89_{+w6qvfc%$z=E1z7&EX6r_2E;GK%BTMTh zTc=8U^Vv7FtZ9wT&g-^3YY^jaFC4k`0dCGF^?`-Sh+9IBq$ zysqaoXul=!SEGH3t`8v{Zrc(3IYvnoA{Ij?|+2&k!%;C#vsQA;xqIRt-#{P z`=G%)Aa3QRFZK)RgQRu7F(;bwQz)vs9ord~n;mFgc3ZrToQM8a<*)Pd62ocMxmdor zA8ouK>U8kkjJVN8%v$lxzc^|x;{XVZnAMuvY`AdW6gS@uv=4wnA*)H9jjQK0P%Z{U zu?g!OrVfs~0U}va1*ZTG5V59e0y?$=1x4nL805-rCvk*%VOziEl5& zKIqr@K{l_qA1L_SDi0Dvym+1HEi^o^={|=*GFMQYf;p_%ggpUh8XgNNEa%94a8|(r zCVWu!tgRgU?gVlm@jo;+)EF(AxcK{Ab~EkA8hn?--yYaN>b@sF@m}I$Ta|^5f7m;9 z+V+yRG_uN95Oqhr5HJ6@;~-HZkZ5;;`D8EYwi*5UznQp?PX}6gkTOWJraxnWLU}(B z9_wJaUGE_L+ad<&>1i9XAewiH;PfH zYhe@$j&cANKe&wnr`j;+iH=(0>a<@x;ZlH|shw`-w#=%I{sI3c`MkKj&+lV5C(n2s z^^~KPv!);JT9JSR0-c~_SFM_DwqE-^kpQhX$OwqWbc6&y1&D={PS$2(qLI0hM0J98Hc zVm4+bcH)0ta13IWHZG=4#0+9KhAyTerpERrrf~fHaQ}Xb$4b49bR}tP%m%RXvc`Mb z_0A6r8exh5oOl^R(u*T?o|pxj*qynO_8O1}Eg4U(x_n526*JpVMke)#@?=+7DdoMX zc+{Ss>Y?rR;dYiZReHqq<-*KNI5rDG!aNS2ixjnj>wK3-eye_hS)B>)m>*j2RTLpVz^1J#UfKM~t zQN5zRIvtjaTdYSqa@y>t76$RRemTc#(eiy>BSe4&VZjXD4XIyK5-zIcW z{}cnF9CnP^G;$IJI|ecrpHy;#{p`<_5{Tb!B^=2^x+Z?!6p}j4LUdmnaO9nn=hunK z;`;-Pa7K#gY*S)OHQwM${1+4#4&T~uIkEMJlZBq{%heD~W3!HXts%yU_r^TbM$>_& zVMdzV^f!cQ%)(C_5smWekfrxEsO%gIW!FBNG_y49g{>1i3kd-49SS~pQEufOK5O9+ zL$pqG)?|lb@iyxAh^lYj(@cs6?BDo;7e^MDDcXUKwcoBcIOoDHSFI+P~O(9-Z&andYAFBi{h&@`5)j#e9#vD?UnXo@v$aanZXDDVT?DOLtw zWjv;gVm>`8e6M!hZ9L<(*ByI&#%@H2S~wx8gp9mUy#Jxpn8|X^k%C{o<-yrJVn3q6 z)Z94Hzfca`C|_y+c3OpNSsIHS6qSd?ZMAOR_(-f(nPi1jg=O2;SUvMlw?#5NB2SW{ zb_N`_)k*`ePaabX*Y~N@c6dtD(K-h@Y3?J>!_!~%Z&ercKxC?dlkH(=8ys;@S4c-6AYC1wj!RGut} z-2f3|{|?&_h3JQa!Qi;!!-7c2j6H09rNNk$gSM)oODm-^&F;l> z$QJ(6q=QU55TE!gY-5CUXS9|2#L`(Nr^un(ZDW?N2e_wFaQk$^q87C8JB+FbIGsB# zNOUIzp3kKf{iS^0X5$#RecxV*D)rmmUorstdK!!se;3D+|7;<+F~cCQ_^}DDk%1+P zWZXb>HC>r+xw;e8vVyTIhJPvMvH7$t%8Cl+WS>E{^OvL;=`TZhRsiYGt&IHMyq6a1 z_dK-)7VGhYFm9_Pe6+3s+@NBfI90ng^3Zvpt!bm_jVyHjoJ~mR@|^*xCFP&*QMt-e zf5{DO2^zc0)_P<6@Cbd%-(cX~3K&|ZW;i$kH4vQ(X|CZP~mGIKSLQnrb z*A>H*-`QV;xEFBp(^DidxZ-5L!h+E?4|b&gcbEmW+RUodsuASToTE#EJON&XMoiR7 z*0R)?e$C&bAA$Sfrm^RUK0mqJ9a(U4JUWoklBCm7MwbOWp#kyLOQB>lKM}Rv{E4|_{#Rbd*i#KvpokaZ+Gp7ro`T)^MuXqu zR`AP?nD=z)wyW+X7(Cg33fu-EwR3O{pxVsGjsRNbLo`AAP%BJDXoC9$M*O{ssfVo4 zex&OKp1#dOU}%*z0MzqMmeXG=(e*TO*|zX8Wc1AJjs|cyi_)Wpv$MmG5TNB2{F~3H z2Z8;w)mVykHUN`1j>-VoCK3@kEiU!K^__HG=wF!Em|+p$(+j+N!L%E3nx>T%#hxn? zUF&}hBQ2`9lBvGJP1NYw_J0x2Ecj%x{{EEG`TC--PrW`V&>;eR1iZf0Z2(Jc(RWw`@t0&4jr#bnblnRb4?Vd_o=dTJgDob-yGr`Z;0agc1 zi)gr2dQ6@_fRxIvmz%8+P%_QcZ zqFNej+~-MZpRg_dQZw5^~U0n(J_C!6239~Ygsd=ikXxU6SGaxz5<-9C4 zTc|FSjc`w$R$d z>ChL#gRn7%4*zJjIv3M=&Q%6}gBLqL8X#?Xp5BO?3XHIW*F&lkoas_6GJRIm^qcP}6ko4#Qx{;H(OWr!n)GOT6kEAA9wjGT}I(FM~-}oA_6Kf$Q z^fnfc(wtZ&$mUsemKKC#QdQXW_N9!SiB|y3I#fsML7{1oErq+_{q=$xZPf@FvLu)07CT8FxCh{09H91@`2af$2 zSPN*Is0;CyLBeoQSE+gQ5LEYN^s}*E`It#a+17OqtZBTe6#DM==h$3Dh6Lh4M zMJD|_t|=N9TRBk(RK@66hMA&F8W_y<7xOCo65mM6`LvyD*(2>2OI=)6Nk9)^oIm=D z2&oBFz6LR#_s-R^J&JlS!*`D_2+Q6cvwWgsudq1{0hi5DQjxgX^afpkw?UvULO!``|F{3f^oV}N{I1|-0sAJvV z14ncguI=PLlY{`11v|jtR&Hc}m5H&mHVVKj!Q1H9(;kwoe2iSf_Wx{vOX-?zxufXk zVct`A%?EVuZ-V=(K{8rmf$7kSS zB8$Jl6rg8+_P}8$3u!Y-M$90(=ZL$S0h>>)Hj5N>|9rcBuKU6rrB45qQ6@$T8nIO_ zCFstK`@YB~>y`ql?ee~zuKjwB1p9v7@R&M3#t(OKp?y|-cy1WKqsag8mp>Za!oG{l)m!?qo-B2>wnTZKFliadT$1t)*F`<|3ET+!HVj-%Ld3 zZ2Bu~mP=+JTgnQ!xNpOB!@u@yxyDpd%m9*2_*VIoY2fXjfB_l zU7@St)z@3zQ0cY-wWHJEi9QQ%x-VY@$fvTeeI^Gap1iEt^pU@@=b81dj{`ol>{Fq9 zyV7JHOA0jLOXIIo#q*X*I3bwIlMCH|91@`4_uux98C_TSX9!pW7s z>ZFi-t%!y|Z9YqE*_GNkCf%$VQti919BPxRn*JTyo$CqOLE&)Yh?;iPgUE0v{* z`k@5=V`k9*ZU6oKrStIjv{7!oGv35tjWyyS#~CXr$1il~Lsqft@^Ez~5n+~7Pj=HA zM?Q_WWGv5YNi*fUfMK{7q1GRw?S+4(Zju*fZCmg9qKr-6#l#E|%%2Q9$3?`)P zATIRSmBQ8C$Mvadev=$5G;0Lbj>9ILJ}eElyV9YUGtfKhtDt#o5(j*dDQKO<1D__V z#Q>W9#_`%ap9_#bwO3vA<`;Gpx)uIbI5uyaXTpnC1g-6+rQi;}OLr|7fSFdK+5XF9 zx`Zg4CHkgg4EfuIWv6D6Xym1?8DFF6e4Zr*Y5hXh@E+@mVhiMaxp%`QZ3=&J`%ivK z;GbKeW2%(!dW;2LWjkrRd{uA4haJCW*M=E><%lrlrn`U*ZMbDJlp!ve)HR&Irjq3| z_6(>BA4dGLJ3RR)CV7_B>O`3)rb?(pmqkWW!d7|I9WIEh>)+i)Y!DNyZhs? zWk9!ceckakS+~(|Fwld_hc!QSoH5@{Ghj`>-zxvzb8+J-s@0dQ{0&xOxa>agZgeC{ zSH4iR`ly=X6XoW7X0e_`KGChCA?iuQqOPLRNBZ|V)hw@vK27Pu&?OksreAUQ_>9VL zCAU!$7Xk=_z+Gc*P^(NeCC+bd<(n=T{q9!kYg47|*&L(+{b&Y%cnaDMzq|~~5X21L zFxWe!!W(TLl4a;Dx6j*g=LBcKU^&!UwU^70;oggIQ8XkT6o2GZtk|E^=`;3-( zqEA4m{zb-L;4;hA_`FO1S1C`=__;tt*y7yPkz2bN^x5Nt(DQkG)NN2a^G=$hDT8CD zme^+j0-P_vE?Ogp$2EOr9A+@UKq)Cb?Wg47 z>MqM?`P4b*PZJr7tqT&aO>Ig9v4_?|IL#&5bZm#SX1=Mb(T8vKHOKbc8M^Y&b9}Ze zcgDHO=M7zID`a;)zX)$2!m93M3gqKql5)s^g|GKSM`f=t)~6Xbk6xlK@zMrI`w z^}IV3V`n9^q_k4!g5JW_B6hKy-)Q=aXMTJK+-Vmx8WbZ3B^X|(eo6vNsS!>7ieo|t z@6Y@s${2HaoA?*8)&Qk2%bvVByL`$S$*%xIhjr(`cqwKP*@8<4?VLS;ckzYdXnq!* zSzjy0&aS>IY@ZfNzk!|3%rUisIq1rPUq=?Yo<}QbGS2z@$X7GUQymRSNzYWqJ!hMf zn1b9En$YJdDb-Xk}kmrYF#Dg0kBEHdu zH>;A8*$_NPL2<2QTZl@$P1taaQ5=?L(el3dAZk>(s7+Mhq7?a1!HQkjAWa1Ww_XKz z(4PKx^}vSsuH4h&QT;h?h34+hrt4R*ysIzGUD$JopesRI|QTKyiITC^e zypSh+UzurIF{ZRmB=n>)^h&PY(bG0e^`9N^JoWU?v+KnFDf2_3(?cgFnru$YT$`mF zf?t%((?!?VD?_fORyh;eN=98eN%y4tyh-;GEnCL&JG(O_`2EN z%zzC-u>$bPr6a(ArBju4g4Q?nE2a}y3E|czQ@XX`M8xbuiS2WsPvB^_;Gv)2C62^1t0v$K z;$n`(a{@by_!WhdQJjL>_%Pne;rMAa$y~?LfLnKbfB|bGhmV|pWX$!VlN6q5Y;jZJ z8f+!O%2z5e%%>Us26QfZSj8B2_}avudpq8uxMEUH$0|o)O>3FwOQwF3m_**HjbpZ@ z*fABB>Guv~h&_>an&_Ef(ci3ols zYZXLnKpL?!k)Epfr3?Av=JaMhD2~t;^Ve2a08`lUG0AD!QzR{a%koAW>qZ;PJMW9> zqwme-qJ2(AMp#he1Xrydk=;ezQ~x~aygI|1OhLvYaJ!W#nxztHqpc6H-pxaqDnO8Z zC7`eL)_>332pIendJ@tmtl zET>5%j_iXt5j+thHizR4W_CzS+)gAr&%H@%&pYKvwR-y{$W{%88F17IpD)^!uN!rp z#_eLu_&!V4c0?(E-7||Kg45NZ^$~lw#O7B#{vy0b9v#qGK%ipA&_w5Xt>U!U%w=fT$Mevrt!`mtWrjB+h1iA4q;Byd6#rxv4SRF$ z=e|y0`Ux{zGj2N)Ekxvfly61@bcxs>K@lVV#9(z8Ih@vRiF zAIl-o=FT>OgRe04Q?Y!5z7}52OvA%Xda0wdB$yfSYmrH1o7RqyxT?EiLzi+I)$sZlf2iC^l31f zv6>%87)Q`oudddRt57)!D@f;W1FarEpNBO}49T(E&Ak(@k}XW&%-|CtjN~sUjLXpw zqctDPqpg4X5HS};Xp~G{*pdJfs)u@{7`-O%zZvhTU)buDuD9?t=S0`baWb0iH(Kvf z=ar9Iv?8})e0iDUyL_D)poORDw#cYyFNopDaJJtS$rE2q1*EVib8q*hCT+$tnT^#= z!pX>!(o|)}{q1_Cry-8}3uean*D?nKyONZ~DMK@Um7k`#!W@#3M!@ilUF^E7@kyO1d>OO-^Pu$HgzUJpRZP#1Zwbi8Yw7$AJDwLXb0MgX5Wht9&6{ken1PiMI+?tlZbeI} zI0-{~6skUGNlmhD*}W1V1KGM;UY!eP2(@V||^)QvX2rl!)2B)&5M7B^wO zc(N`+!6v1z!NOW{8&bELFr(2@n=Dq;S9;h$%fwdSQ<}Dp)S}MJ*f$nmnn5xaOWZ`X z-zzKdjL0R)G-BEFudwYLGzCqufhy|i-*stbTPq%Rqf-vsD6)or_e%A`NL^2$25#lm z&~S|MWlmEc^45Pz@L`UCmlv6(FVR<@6T+u( zrpNIlFVwNxM3Gg81N17}AU-a(%yduP_SEl4B-096CNlhq*d48lFAeH;Cg3iqaOj6> zgl$wqq5;D1@ob}N;Mi<~L|#@;vASO5`qhGlLDo)h9kPiWA$iR1WNh+rxMV19Ie=c% z$bX4bUEnkx!_i39X!TDyc?3sIAOhW#RdItVs_p&yDGx*Ug@-cjQvcJPp{?HWW32#| z@pIrHCQEjdrk{NU;$)>Mkq%K~cO~UpfakhSCli0nB7zSJj*xwtH%4pasrJo!KM{bR z(4TiBLmdcA(LfpRFO#lIY<^hgZF@zMs%hp-2Qa+(MM&Zo5XB*F>g&B3ROl1>`Hu%r zAWr@dKmN{$i{K;c&O=VfX%8??dO#$yi(qnn;rv5s`69v)4ZVk5fKChRB7i^Q8rBCU zA{LI@@ASd3qHI!kwEB>0Q633Qfa^Z`CA6Ff=3)Z+k@x56py$?w$Ilb_SPy?qr&kB2 zndiyX*Xdoax#GLlgu%Of#Z!0aIGRWOs?$e}32UVL3V8jgJ9>vrz~v+SW&M@(FhNIl z;l$zI>nzgFb|2TM-{`Fh(l^!X`~I@$MSFl=yZ`gmp)#EQzT4HE&2Qq$ORGX~g=};^ zpQcvNX9rdeBD;4-e;*Idj<=cREwU7OPyRP5QJL7unB#Kt9HtN(r#~FbFTOpCA%r<6 z7pY5a;+0|Fss((yL>{7ho;`eCC2iuk`CqjH333p(Dj#|CXY&>lhN4I|Wu{z#zC*mw zx{`ce#JI`@#)jNJi|Pk?`7<*{VDv5uGV~`_e`Cn_Z%xql$mqEmTF3M(RCA+IAFJp#H zwsW*A{6daQoU*o1`YeIaE)h>p%>;`q)%6pMgk40%K}0dW5Qdp@DJqX8T6<+ld<_F;c` zOq96+^rtVEVBj`jTMo6g-!UJ=KBs;5S&FtvLwmq#Z$Tt}e5Rfii0%e^&q-|7Bw+$m zNR2+cepuGQ!5Ze|Q_m!ClpjuQV3nOoJ7bSBb z-L5Ctmc+3B;w9p%uN1hlB_$w}G6sQxiVM2;BJ9j2u; z98LUfCor?g7fp0eTbvpao+VB$PDWcg+YZV(f8CdiUqH8HywtYjkDCO}UoK3-dK`9UUWFokLlZj=k6#XQ!XAjA-&_eXQC`S8!Wj}p zMMeu{h-H%NwTIBcA=AEx3O?kdKOptKuo|a z1@MI*5C?x8kD5+P&3c{;eDhUrl6f1#B=e6o78hY|LRFkNOMlV&%t{OC{x0zu zL+~_LsTjXB^yIhMk@gX#3vLS-f^cK5q6O*`h8Hv`BWz)}Ruzg3WUYZbCroTGR`>8F zGba&!U}IO31Go?JMX-Et$`*VU{Su7{Y1_GY$>?% zlQ5sCIZ`6NXh<6!N>r3dK_(G?ga{*K=TGu)wcqS{rHarBS;hQws&kZ9aCa!OV%C4g zrzeg;hhlK!WJS@7spn=)5ggSXq1Z~faDms5tD>Z(5h&W1Im7wYkOGK3SDQXp{C)Aon&2yZJ!Qa)3fQyQxTm&%(eH_y4wiJfSka3Tn^ z6hclZSr*jUTLjfnVMIn|N!3YZOT~;hVo(wM#+wE|I=*FoGJJA<;!&kjl~?UJ`8Ll! zPg7bse^BgMd^ImMw=yp_w^70}=Q}q$*I43SvNve=2ifX5o5vWTte6JMT^cXme(O|uyTr*U-S z5oTGIbz86HFYL7((oAJcc+6qr1Y?i~t>a;IQ1-6nnQ z%3gv#p4XI=ed---3ejf`)#N8_rC3F-=!5TtP(*z!}?`Wx0Rn@#HFZ<36}u zSnk$5^I3Yt!YY(bmyR0E^BlN&xWPS)X+CLo(mB;Fu=~;|(%5dNmBqAj^vA! zEuJtKTvTFSdMGq$uD75^Qi)i(Xk&gabx&pAw>sI{=RFm+gSM}H?!2i!$+6rx?>;iQ zVIQ(dvJ76-p$+1*eSw#qFrlt;O*xB&AI(VvMcHt zMNhk+k=gO^fZ9)3^#4{7$u_Fd{|!IoO+QidVx|D3>^!EH4U6Wr?Ic$g!l$ zrD>!#?GNla9+Jk@lk|O8e$|;)js3QnR2x&WxgZqxhj*ru&)0vh# zl;yi#8}xLKZhH97t`FFh*%vjY5Pl9L3nTQ;csRf76$d_?*ib+UzfFrj~CGeI~PxfoB4q8+R|&@Do_EdM3JOEqe|n&Ru@FyU@=>+)xBi zJYo1@Eet%m7dY7*K0A@mNwezhmD#PB4(BsNFY()_ii`Fe5=~#*hg%i+&XWAp5(}8$?tgqF&C&YaGCw$zO2HinC#%t9S0;cb`er-oz9}WA1Cq!;Y zSH%av`BE?l=skG6NG>F#74-|yyz4xV?Jc`^db}_*V(_rN>D<1|R%YlIcP_opT$ln} zmTq6waVpL9QrZ;#oww*tZQol{-O*T@0o4E+fmFYvN9^0Hi=t@x96xYi@Xho85QnmT zQ;I!Rzz5zpYPH-=JO|^mw_kQpFdNU+=&2X6lmw1gv&8z1FUFPHEc$ zd;`_|o=<$O>;0!0RiPZbt#>;xQ+sbwDj7-cLmG*eSK-GH?Z+CR(15Ns z0?MFs$CrWb#&^%z6~_9jobPY@M4Mn+fyyO&VJ5$%sKB>Qf(t)&gBCYk+tM0WI_6}t zKaWV~a2vpH{vdb)sN=hS{_ZawFtsrkW7j@jV9$FSBXD=Pw#k+0{HgEWRXA2-WC+ub=gmK0k>zGQ_tT9oN5 z??PA|X)M)dr%q{7V0!`5p2le#dLEJlC)9p|IBN34A_k1P@TEcul*#Rkrq3lrD!&^e z(2??3ZN}2rVrIAHy3%&7DNakXdGl&e(FjRgso9r{{2m#b$jqLSDS=C5DkPUZUu;UU zFUfv*eB(Pw1?{jbvr%x-2=fd*GB?7~XI_qBKCmf-!Po^)DTHMuD+~EhLE>5VFTlZA z)3>8`xNgNN5-jJC$RJ$BF^{j?Swz)9!aAYQScK}+xO7u$_K zN7)DBa}mYRH!9exy(!F1pIt^>z}}|Fr5m81qUxhy`Y-M2Up63)A@3%^CCz-wBdK33 z%bj-n6o5;dDB1Z1r9l4ltDFM-Qx?6Tj3~yp0*Z*lLtyApLn|n&(Pgj3-meA+feaNv zdWg*ldfA+1B5by;x;x#wB8#?sKi`|XK5y@ro!%R*v%TMcu2*YsH}pzTRmoL2_^4}^ zRM+5aMRQ7@2;AUHIb6>DJqh+kc`zNK)R(gN?;iNu7D;lZpj=SoPXyi7NV!jRZt|}e1D5+-HwmKJ`n^!Xw1GV5s=;%_fB{g zDwB4otQv$P{2W8FUrprqz;>{;b&jJIv<4{7EM)S-S^6rwHL1?xPL&p-h@ezGJ8MFB zzL%ZmpmHmVd|yjEjGi-T*9tsN36*Iz$;B1sk%I4SSR2?4L>uQ?2nc)aa*N%Gv_qh2 zKfhGavowP~0&w!R``AUtb07jhtJX3e&DafWgFa;7Rgv52YSnuP^8(~OKy3SXF-O?f63z(BQjUXA zlw#XbS+_z~c>%4_@ndR#-dAn;zVAL;cYV4eDOpc)LP|A((wF@^z+;dh=PI4?mm2IK zMwnk$;_+pd%bN7mFZi8`zl$g+pVzT*mGRmMBS3pA?3C_|b#13)Sj{B7hWnq^O&9qRm0wYZmS&NNV60>f zLxasgrKKSKTu3UpyLL)9*z9`Iit?g2uf1mFRTcO`NEfot9$s6q^_z#Op*PIVa4)Ya zIc)8f@_trYqUL~%!wj{Xzb1>q8BvB6`6>|v(i$H|?6thJISkXgO(||c?$>raUb3Mi ztYx-Jy+eLweLJC5o{wf#(jCd1fLIO#C;O!=j5y*j<%uN~2aL<%aLK`^YZ@I&5!<-Q zqM3@!05lE8Q$H6&N6@XslV-)lc;Gh+NoNfKN_qoJ##%@pdTcP2E)z>z*+L*CHTI)UlI;u4YReJ`r8*cgt@&Vh=dps~i$ z4>Ik0Ovh-)3IG|1Ng&O#ut?J~M>I^9RWvIElfe_f*MkOmIi-K8K2OWs?ESzRYPD=N zyRwyyLWERXoD_FRNr4$s?rkrkU-75LSApt5&&3IiUwJ7J&+G5+csG~%=EHGP*XO;M z1TclWEUb4D9&eQd?o)6gs#cHWlZM6Ahk3=`C~Ee5w3kBndi;sTe<)|`p7xHkUU<{E z%;%6$#rd`B=kLk-;pgvW!TMJYp+m<-^@58fu>%|=o4mb8kYa;CwWtd#R&*!e3Ozh0bxc*Yc)PTxKojR2VNpGf_6R=^7?(P z5J|fwn6wt7ZD?ZjwuiO4*2-l%BBnHbO(ca4xw6xLRGLTTOlm<8+ww1FL zVrOnfM%o`f*93xQI@iX`)FbAEH|L2hM!Z|tMvIInmREs`5g*{-Lf+cE(|2g@91{uG zq6=u%;$|!gUr6a)gN-;suK@q4)h;l@k}AYyk?miYFZ{M&eu^QN4~u`Pp}P8x=1Vh4 z!G4%9Bd&k7@&lGfvS(cS0AU^`q0z)df2vqRL)ruIMSd?zh2VQB{l#3jY&I*#N*hzo z@Qt+JH7Q|RBMzkGFcA$uLh(Kv{AVK*l1>P5s177Sk0~DMb-!s^Pj5hQ)=nK~t2$e( z0z9_2ogj6%RX zUG@Pq0IN$+4gHg)9EymxP&x&&Dz8@KiEm9WnM^;HD*W)d>=)Dvi!WLr&c2{KbW3-9 zFhc+dQ98ry$P5!~G(P%bfoF8zM~ZnY^3Cs4iX7)T zW?RCBRz-Xe?LBk(Z-LK(81=w%ihJOZp@r=oHOYtA9TDSBwwL2m(`#5O|Hb< z7ns~wUEiON!|q=imQ4}qI~8R{7IQ8QA0eap&Dvv%SE0D2LJ+4B)g%TZ$@B)S$Mn$9 z2_4MJT{DCvbVoO#*YtJW*1woPrg`dt@}y`fgQ=;AxiW{%h_WCUbyGw< zNOZ;tKa{ZsV}eend;AV=7#<@sr@I81g6K0#&5fU#GF=6(T&vcMF!gVIl;!gv3&^;@}>&xBk#%q;seq5*DehORcP~NZfWr{ z^0v$C*V1bui^BX^4!$PSbv2R}0(rrL_U(DxGGKRpK@KIJKmMcWiTD2 z)g=R3LHMeqfKTM_45R+!j{AqbkjC^v!BA4;2Ti)VOgdUhDfPGip0%D6^_j~LK3ue$ zM=GXAz@n*7^0q_aU97tyNL-OWkN+~%#Zn{^0p5xU2o~$dLz{~C4Be${yR$_9y2RFj zWva>cAItU=;~Yd(!j^}AuQNNV;;}zZ|D`1%YJLB`k!;$fTun{<_dXO#uKW6UW&QYT zz-zR#Aj8Nq6O!z%RRP&Sjud@D+>4MP@Pwq`&mk10GB-0D1fT!YC`A0;elYQ>Ct=GS zw_#q6fK;PkxxMk8Iy!F?yXV9TLLFPT+ew=yBr^*;U%D@Gt%1H3yL7)4fqjHcaqvsp z>mGddQ&K$VZhj_@Nl@llIWo{6ukQdZ2ih|DuismOt{k0-xUp zoxX1*yk38<|0Zw!^q0QUuXr-5Rrkad9+BE%q0Nl&2DORujT+|(E}&o%vVdEFff=`; zo*b6!C~+xFB}2bxG2GDLKg9fc(8DWzI*+oYV~&Y+?1DJ|>YuS;qu_?rsAZ3dBaOX- z^6YXM!XcVHF-F~lz;GomAe+41@R^2Wk9UJ6#o$AzE|AWTo{j({n`}4Wf+%&578iCZWhC(yYC!7$09rRfgiQg1y$Jt<` zA<58vCnQVH49#VBkTrVsNDxswNA$zf#Mqks7eS;E7eS%E3RQxMq&dx$$fPl&S zKqCN1`WM?DOoKpLG0Nw%2;ilW%N zw=pCVV<&0~4#J1?&X{d?b6ZF-buLlIOpL)&aLf#djtkoxzB=B{Zi;fa42heo*MKR4 z$2$2X(j)V}HyRPg4Iyc6>3vYODNlEI_FDtaJ%48tiYWi)eEetEHFWo?J%NvMAd#R9 zTbs*TIy@+S2@5^-0JV9tgAqEJ7Gk4haKiUV-w5t?<-wAfy~5h3CLh=Gx)rr<5w)R$ z#Cs9fzk}T3KaqZMaNgH@ssv{9~tkb1-Tb>adoZf*QpfdIZ z?;{Xgky}FnN}L`l4N#^96H)ZNXY{2Lr5u)u3+tK%y@e)F7ctm*jb0*DDf`hZz#XS`XcU(fOmB8Ir3rr z4yMpjU-Zm#?;+d#)InQi8JIFfS!`pNqC^$07n)O8pC+MDjuq^PY1)G7Gn!NBBBPnO zG_tt7adv+6MwFVc&|M32U3Rf>L_6RjL*p$)jECSzO(sdrwJDmFkvEdOnRv^u!s#FZ zJ$u+2!A>0E4%!&t*5CYXp?MS2@KjoUg6!-7FAQh&6J@O}YtLGaToq(p7;}=(L{H+f zn5D8b1Vlzb11i0k|IM_8o(C+w%sp-YqLmAI&|gh?s1%b$C_kR?Bl}fDzigfq;^>Y{Vx)Vn{<7Mtb@h zhyX>#YTjCs9`lHyhS6J^d$#T&N6`NMD?EjtC=R!PaVQi!z7*iyq% zws5%9T8xOf`A?^=$Jp(ygn!q5ie7{iyaj|q$0FmTeL^9H6+uz4ZYXOa1EvO2W{`no zq!TeYCJ-vkjY5#z%Vh>U@G6#e{2e=2;YfRP&P+nmV8zyG)QO=om^da#IW11*wm~DK zQIF0tQlXjYX9;{=mp0zbKbh;+f~zx#T1|kc2K@UEKiyWGVsciAmoIy4I$aJJXza}m ztaa9TWlSgDob|XMqMgXeA@WeTdeeLMPSe(t-lg+U4#FSXjUrb?+jYaehz8ci9my%p z-OBLjJD5n!pPzg&S`B z!awnu&(}dUjUTWa%{lLnRHLOH{8U3dV=kj@Z)%ZhoJjh0^ZE=a5~xXt>Fa^gitZTW zB~>Y+IyHuTuCW2Jk)j0QpCtzLD)a(_V`#@EW3%+n4( zTc=nEuXCGnlu$%jGEHV_n|XDIkPi^sT$D3$=Z9+VF*IKKB(poNZN+4@ao3sB4dvAq zys#W&)|%7rQCR92zQF>=Ed{(px*Xb)48_f#vDn%M1*P#~E<0LwVp;a^&w8$QZGaY_ zvX)}ZdZH`mh3#-!Q0(t|mOH2ZdX*C`w_EiW!F5WK$howuxIG|GO7sB;yPiuq5om7yZi)gh7wb;5&AOD|ai84x zE<>?q*xf!G^eF1`kqg||2V^2^pL@i^eD*3M%f;%X;wn-qKlf! zYy@Id{}u;e1X;!1nqjWSVx8yE88AF01HcmzQT?31zAR%62M9YnVqt+7-`;g}44&%$ z+B~sO_TrdXeIvM-ok!}CTuG|m&H`yhUF=aX?_sqJ##5kQB{~!vda{@*`Z1J{|F^jK z>OaCpf$~DM6Q{SKi(rIW9w)>SF9#xBq^=RXtvqI$A*Oj7tS$o~Yt2^h3vQ>fDpX%f z?eO&PDGSbzDE>by95DaK!omLwk!$7b_?N-xIvVQR=sNtxF#lzi%gXdGZ~GrvE+acT z{@-Z^|0~O7{nyNce`C1}Yz*xGXO`QoCT^=J4&U>s`ozE?wfN~kf+-@1Ni0^x0RRpm z;Fl#dgwOorV{!i4vEf6?DpfExOQHI#piA@h+<4L6a1r^@;X%PG%Ub*GQ9;W_RT<47 z!OND)ChhIglgG>F>EBEGZ20_9&3G|5QE__r-p~LBv$RX~Q4mTM++#IR+e!88vQIgmF`_XqvdbfPO7?mWqZ810IXAOs;BS3<89-*-CHYSZm)a~ZT zY2w1eSaR7NakcWXq=obXzO!e10KnzAi(XG1T}emDE;Wt+C-*xcY{>O&P@7b^WNrB?aoL^o+Kg;RxjdLbL}@D(C^p76PR%4pL=acu4a z+O)+bP)F=djyza)LoQG}d*6I!^XifsWOkVdlDhJkCe#hnVrc>P8kCXZ6w-#AgOEC* zg+vHt@Oe@m5W+aLUudKB!3#nQj_8If>v1_lNp)rNwoP34fnpCu#+oi62FUm#KAt-Qp)n3lhaiW@yJO^RJef@J&Bw8_T#k}B z0e#_e_zi<J zCrN;^+L&@MoKVDsHdg!*K4)U_?PBfp#UQ2Fk%Y&3ku5JtkvzrdG-oR7P)%K|&isi! zY0p-)3aHo>C8gAngvR-jZySoNArwe1pPx4b;zvdIhyk@7B2;Twkvk|CK+K)h0-C!q zvDY?|{U;qX$TQ|CFx5pb;(EW(Hn2Cl>t+FCG!{?&6iQ9ll}+*{k{n^SOQ-w0O>&3` znJyjqL!+!?hA&Py7@MQqf6W3&*{y~%T#5VS1S0)y=?U1%pVhxd9^fF39UiP}T;l)J z#1}5h=jf%fAro`1k0rZOzxNLJ9uZpHKj^?~;>){aNEM(+bw^d__{lqgYf0wLj_s%-XrqcCkQ#Q}c;sIMUi@PIa)t!>Rk zI?g4vlF~)Jy{rHu*6O0H1Vh$meh!l1tfvHfi4cx+lKrG#KQ5%Ago0UEh6-D`@@fVS zR_q2U4j)6z^np{dDR0 z_H@g;ww6waI6oLZ8aXcXW)C>O;$nS~_L8e+t?726Us{mcdyPf`U)(OQrLw z1YH^zL?g-Amzw8wc~2CR1w24R;eru@J55BvZ}_u@uR-490UsurYC6SfrMnjZ3u17z z3QJsTx(@i`2tv6j`+zH#+IFs=C#d_V<2Ug@6sUoeu2*c z0Cb* z;=W%+3R2hox+d+W8|><<>Kl01`4IX)De|-Zt0U!q6#4(brvG5mf3WF4*z_N4`VTh! z2b=zbP5;5B|6tSqZ`hRM-_N1{*4o6v%FO;>#iqwR<1yHy$*w-aIOD*I^>qCC{uKtP zqAgMb3RsSJZ_FXy_4Ie0n-LvmqI4}$=$=_KAb2)lWzp$7c1%4+LNO+-~M?a zBSZ7V5+NUt_mV~_*>P}P`Ebb6X@ZZ}g{g4PaMOyMXHx6C?d#KMs&22RkNaz@YaN1V zYuR=GWE+Jg+|SozC*L1=?C+$VuMd1yoWXrx$=O`Ekxv@*%4+x7;@<0#qNPm43;%5C&w3Htol&+c zQ%FmkZPoor;ar(L9ca{2n|;YJ8n+@-^OyREk>kMLaFm{&L`WvaN0F~;nR1%@L}gaT zIgWjb<__6%(GUCj1DWsvs_ImibBcR?4}vL3jcvzkIwV^P?fV*Kn^0Wd+Apou>BHus zdh;xC@9FkvdGDGn6RIV|vcZcgjdf|W>6wr3-k=7&i!zp=CKpX){UEuMybdb&ngl_f zUW44~((t=gnhsx{kyK{OXVnF7J4m42lp7?q7UJCan`_t(-_3{Cb04)I%QN1u?;zwn zjatV=Vtz>TAf#l@39F0TOhZH`g91#b;!kYDppAeeBCDT#>_-a?b%oBy?@M;lua=h? zR;7S?g;>X^*vzJELz}Z(Hd^0M!?%xKn9(ldBCcNdqw{Ja=RaKct|RY37-+0$TqZp+ zuf`2c$_50Sm)<+6IwTiyhdST+T|X8Tx)0SYy5pZCRuaC3JFVvO+wQx&dta?IzM-%0 zAFb}Xx>P#P?cdqw+>b&dmL~|H)~Yc-gk(zAlMO_Ct}~ufFH=e<#5jtd--{L+hKU0U z@d9sv(wS{jvrhF#EyY9hK8xQhh=#-A0E>#D<&s2IB5$ScU0*Lw{+{s8#=&XowcLvW zBlK1J$li(1QL*o*Aw}*iu{6a&bGj7fNgD-!MiNNfc~aJn_y4}wd~hX()&2R&EqX_L z0<#iqV#8*GQv>mnKW5ec$wBS!t;P0W0jD(%OVuR{wMxqDBKzf>K(~+LAhsdx!lf|c z&G9sHd@Dzz`?D+l>&t+5WkCVKGI`+2{1Bt~elR~X^AjF`; z9yb9*9qyFaWIn{NT&=x@Kd(RYHQ~nWR?;ED#G$9=G)X}Bm`#N|v}T}BsXU80rUM=% z5Q7Jx>r(TY((j1_6r8gd4|?P_1RreOY-F7*n~fnqo_0|%ZUooMtrq23t@s(TImfTa z0wS22C>Qy+Mv`$zjr!1fol zXmRuz6|0oEiq#l5u;qS-VC=0@@x$KeuN}+>zVSm8FsIl6Go|QL>f4?x^*h}NazR%c zI#@#r+b4lfbS)+ZLTDqCH3Uwr3qFW=pbCu*LS+FoxLOk9z%6f>>I+gxzC;FbMiDYP z4CGuuV#Xruh+FYuO1es)Dy7h2S5+h9S;N3OBk2f-lS8 zo2&`e&)=4^#>^qD;QuVlvn+wgj3t$SpJAO+7?S^Gr-y@}2)sepRtxB6S)FgNrSt@2 z7i`Q@2itFxj~(7bb`NP3exA@%XUDo!#G~*r$dJ>f%YW$oU4lTl+~{0(F7h&PqH|Rm z3GNIiz%1{DLxbf83h_(NXnS}KDv7{~K>ZjW640zP{sc3O2iVA+lCbAnSgOi9_b?B;xxGEC~)AIsfBphom>J)dtbn5QdODsOGRb z!VoF+VHEHygjwY{T#HCPC0rwj7axwWQgiOPw7nc)_q!pViu!$kr{MDmX~l!5U!^yh zhO3Zq^Sm@Yvve;|@uTJyf^8qq5rBW6SMtb2+WmB+%2gN3EQMl7l(_FXgy{}Wtp7QZTRH8zHl@drU^wGU6y>O zpa9xZ;*KBU)+nMEyAJ}Snm=7BuE#xj8n9k&(s|~xl9v^Qh8$28F%8)2waW#e&HS4E zG8f=F>YbveFv`ZIy-25~n^tkUif4a<5t}?LRD9S%1m==HpDv2H=PjpbN^hHe<{LG=w6mlyjHo_q44?z7bxAru!Pi$?@y z^Dta>PDCLFC_>{UHSXD2DIR8}h+XYs$pE9s6^Y(3?g@N0yo$p|UMoCallaf?(NjDb zyaMl^hd<}PbuPbrec}$jQt^Br_fwqABUC%AMhhhKAjtB!Y)T?%k+lM7+Lez`CCxL~ zz4{j4uJpVhsZ#8<0blPinyghs-*lejzyO-3=L0CiaAPl-U6X>%AqF#^4ugf`*@yZiA94lLsKU2a|tU%%cV!y|TK769SA0NrjQQZ{vWqu9p) zD6R|Myiyk^v3a5!FH~f*U`6a*qEik5#`tYf?jF< zb{f@Ai2n2=bHAEsV$WdRf-w^tuQvWIC{v;_p>QP}z6JGW6nU`l_0+;LKC*M)@O2c` zc9>@cl$xlh5dL{gtQAJ2_LMY&0wtE|r?H8mU`MsOfGnyxBW)6pg)<{l@0fASkluW% zFKQx4NxxFo4<2JDGa=L{x;QVmLLKK!fR02H?I_$SXk(OP;$1KnowD&1(0oBK0Vq~D z0Q>51V!w;RLV69A+@F%+gs7g63No;f@OLoQR5=%}gE8e5#{^Mp?b|+$1)(a7JVQu! zX0zm_?<|x>cS&|xCzpVKH#0!jx|R_?O!goY9fCosyj&`61F?*^hYb@m1@k8s`{Yc? zB{xJ5G+-|O?%)7*$+TV=f`ocYQ4#*`{y5j_I54PvQ4Ps#JmcIsv45ZvcqTh5y72Ew zNoF-k*c@mEX^Hbd7w!CI?n-S+0asVp01Gn?Slm=oCiw+jll<$V1Ad4)0JN|<=-LD` z^ZV!U6!>O~#eCCA*(WJ2fCA|&RrBwQktkAQOOE&>sk~*xwZR0fcSpE1W3eN-CjY&F zgP8c`q1>#kG=7D6o;VeXeza=zv-UK#Vi60al6lhB1uw%wdikce<7qk%n}qVmQC z{vGl;^hQp+JHt%WM%cqTnDZxzS`aG1VaM9~Z!RaqGu<+#3HhP1l%$_Tyw!USerdwW zd1itOfiXA|U;qpIDvvp5I##2Swa4TwX0|gNnvLI{ZW4+()5*&Zl0Sg5nbR( zfdeFzXrs`G7{Q8FNr(*i#LfMZu#lHg8E4@oVYq#Eg&Ni04&7pfF@sY6*&J$NV9DRM$a_!JkuFtED-oxQ2!izY5l<Px8;vnDh0PIa-Z&_WvwG)L1h?((->(J-h!ZZs8{3& zx?}Qr)M2_!L$^-HG=f;RSq%wFjLPkn<59yU!BBhyJ~m()F2ZxPZx&vh6Tufib0A$v zTnXSi#Ht$wswxJ97JUIC#|`jj>riC~q}UoDkwmxXH7_5*{G*AS7v)=~6#S7vZ&6E# z>2NbK7k6VLS3SeBk%G3hnfwt-PX!&l>{AHBvhN~4j2J-q)sk2V{>H*wCgx0VhB!Up z)ttpF$wi`a_)Y@qOliX6HCrd>OrF!kB^b?CHF9NIt&(n2ZAKlsoHvo*fcff4qR$CC z+y$|C%0j(pi{tLo6U29T_5x}1&IYbRPe&Z!DK6QxF*g5v{TSU6N%Z0nB}%yq!e<&C z_4{2KyLOr%`*P)%nJ~S?QyzZ47fDy2(+p+?p|00}7>l%Sx(#AQSbDo6QRG>9B0LY^ zw8Y&KZs)7i#Tz>akoiVo%&1x`85%>{-FPqGvnGyMkn>OxZ)*40aSYCHQPEBeM5k_b zzY$u;wH&Tn3@A)df>psKPPY7+ZIB@Dn&maLBvj-IMN@0a^DIX6cTKCE(7<0VNI=^g z#RQw6=6j%zO(QS|g4M2Q`Tj|jnYbD@2iYY9=D!A}M1(To)HdFPDb3j=jm!{JxZPiH zgCN>B>P0bjtX6{NqqU7WsSYnf8HM7PxLEe}#D~Z1E(;)`Acc($MzPz-Oh(XkFJ~uK zhZL@2FpA}!s`XE&NCWV2)>PJzA*cCwOwa$o!|2p`LH;d|+kip9I6N}KXwWOWI$qlN z^EMpRX+t*u9$$fBHk_fNO}FX92dUo4GDv=)Mkizm7OsZV=I~D75L*(?zL2!XcHRT- zzVah+Sb$kbo|MkAmm76XK~(Da6(dpGV!S_QOo7Ws)9Ez-<_jrb$ab!lK323@rAF#$ z14i{}%*mx)wn-yBC_P@}qp@<~9kkIjnL3+_=kv4ps>}Q1)QAZy+$EQJo`U=qxQy5E zJG>Dlen&=_m^T?+^0@+(W@6}@`L3rTu$;$LU4b8@>E^PY*uFJ%9Q&racuBFGd{02Q z8S;yAlsrQfq)+2EzTrUUro#-`)U^_F%{n2>e|w_`4ZhYEr`JCCAzqBhWuFm~P~6q2 z7|Nk>C_ZNV>yH1KDgbyxwn39L!LBU&z6Xo~RD36H~foB(v)%Mn;nFC!wtCG-uRP1u_h=oUOEJi2Z zWF^vJ$wOX|Ceh*brgI4+@4=qs(E>&K*8HPA;2k7m zERw;$K{S@qref3XDaHXhhwqH0#qJUEqPTpt_#8uwrVN`GL;s`V>SOU+n2;xUY@ZTJ zED^UXmxtS%si-3)6-E=@M9MGz5+x%kFxxFDKq0v8odjbCF0lS|SQW%>xtzK4?;?X`JpyOyxxaJY*`qg&~jt#e?m);Qt^A`%# z$-=V?Wc|amZdov=P)FgQh4<<9_B`;H#cc6Ks@6WTa5Z+&T`mD-s=bpgm%&ntnY$1H z!GgD*soe)N*Sq?yQJG@Vl$}#7sra4Vy=LBQuzREucRH))llbc}Uh}V#4*j;Ecy!x& zrB!_29_Z*qrdf{zRY1a#>_&E|GZneVj;H?5sMWucc(8_A3V#0(uj>Lq#R%wALiDx3 zMEERbB7VWbL6+T{{$L0kU3CrEDr z%OB)fB|-N$Oz@6d)h;g;=l%mRiJ9n`LZoZNFa)kwU-=9T7h)KDtu6l%c$z@-C*n3$ z)<*}y{=~>Z3(>ZJ%JwfmcHFoz8FmqgPIH6P^!rVy2rtayJlcg%+%ilGKb#+IEJpYZ z!LBBcs*-AqwB=S4mq69a*|6>Ua@u z?%?l>DH0EGsRN_z`-y3DwO&h2;tFps{0)RQ{Szm?O=B>1h-fe3LP3>snTf2|9M%|- z;78%kK7a34eWyaiO8yh*z{2ov&;g3>{{>34buj+F5Q!ZBof`fhMIy()hMNAnNMvTC z|F4KdNn51i@lS5Gbvnm@QZeIHjH9qoGoF0+_E+NIWgfGMRH3nz@JDl}+ z9U3S(HN=dFoDNb2hMb{{ommP7(kcy(LM$60Q7#_T90}jTBQS&D?O&gYHUIq-oqqqp z&m0Yr&--M_yGM)L6VQz#RrRd&IGdscALr*Dk6=zSuYjphy=Hp+Vu`vo-SPg^A(t7z z6u8L1`w!&(6e|EO;Rnuv8>C;2{)2yQM-uG#UPOMeSL{wf@tJMjB|KQ>L=f-cwpC6| zSpX6+ea(216#+gt27r42HU7Ek9796%*`1L?OAM$(ei9}DvdB9iGe}G z@*;)2w1wHS9A?!i6Vqwf(Ns=A6k0I}^o0~`jYt?q*zEEcw_u8BZ-PS}S$dor*Wfb9 zym&kf;t3d6WLwlRa|#2sxHgZ7Ew~rL2vK@-zgXTVNcEJOE8)XlFk=9sDT{o00D)mu zOC0?$Xk;~izT;NGmgpir$~I}NKm74L0?r`u1_Gn@Kv-x+@_Em>YoC*8s30rpT}4S# z8K5v5K`3|KSgvbPvpJw}u}%~U6++#>ylT@L>*P(268jHKBp5hBI-x2ak&U3C>w+|I+05@x^6mx?r>u z`Ea~|FFhPqF(>E!3FsJ!TcswrxM0AttoS-Uoi>1FeMDT+eIJ`6cZycg#S=jJN zS1gbxHGXM@vcFTOqCq2gV*t06_@AQ3A6~KBqfyV&^f76QW-G@FPt7>nfRQ#d@*?)-QLN=!mg9`H+j#+*Ij=%JEw+GOPtd4L#L_p-qyxy@qU~!#ZM%IXu-ONZIHtecBtEZsqUXoZT zPajGrYm#@(9{J$|=Hv57{_62Q z6~_;2N_)zLc$447zTC-eQ^$3`kMGe(LO{`U=wiT>X=tbb(QKQiwhnfH&(`$y*eBlG@|dH=|~e`MZ2GVlMF%wu3@ z{NFB4{>G~)I{nR4k-=x6|L+$Q4vtRv?5zJ`C^;K5!+%xoU2AMQY>pv#pQ=t3e#P}~ z(P`FHClsq)Rv5WXI#6aRp96vrKq5ep+o1%q>v z6`w}-N9M3%u|lyh6~mb6p?lFg?CaPg!Vz=KVUdy`q_E(g2^p_JcEtLd$n-H~@CSnYjnkLU<`$MM zOzbO(SlmpaQmf4G8t`)L*swe#NNSbI)CYj}|gksjob*AC2Oa}Qa0#zwy;xpx=b|a#_rTzdB5kIT3+As z>7<&C(M^cEgyl?>=;}Fxet0ExQ=wfqP)!Wi9fYd-(wQoMcYO89faa+?XlEe;FV3pAQ#vY={QWKB$1g$dq<4QSc}9}fnhwFB45 z9a_s*k^vEDnLJ}U{XD#zBh{sh=!XZP7XUCAmndvNu~&Lg_^!Xbk@*9hqVN+$mk>wQ zHMH8^9U)H9m6gcnl#9y6l`^YEvZOqW=kemPQXWbRg-4uZ#-ZBjugaU)$KK;Q*k5@; zL!1hLe%ljwh``pSnwvYS-q9(pS8bEEbd?7EKI$*4li$aCFY68DTo%e&8npKgXH~_n zbgD7EnK&6)`ZhBw`+gjzAZ6fuJ@ddNGQ3mYANhQ|=i{_vIJskhrf>nHWI>A%oYi=P ze(kn=JXV)2T%yRbFN*ILuI&wIAi3eRK>M<{CIam&Z<&8?^vBOn%f`ywy`ro1~;z zEfItwg_E_N-lfxzK^g5-l(g%!a#xQ-t7Zx+~xJMjT@n~sVjs> z>ZeSO-B*g2Qq{Jlpe*Xm89}d z`E+X1KI)e&u`r?qTRSq8nDUdYkowa(Ub<#7cjzX8HUJ-|Byf_&QIJc8)Y z!)K1qHu`aUP5IWWQ)jK7FkjCm@^tMoJEa?qP^D7wM1$gG1b_e_^;)Zs;Q!GV z?RvT9)>Q(PFSJBoae2fpw&)5jVnIbDN%8BV;unRV1oH<@5odAh)OTNOjht*~x3JaA z%J#VuZJOwl-qW?LB+oJre55G+Rtw;3d+De>SuQm_=?XH!D27 zQ>=BKjot37IF0oMpQR&8@D~%K<=1Mwdg;wrmAKI&af3P1cvE=MhLD0aVfo69FmJvs z)w@=+x8(yx=wUd@yV4-!a7GyC4>`j22(9*;feQs&``+#Q^E<+SW8|HJL<_pC!Ln`J zwr$(CZQHhO+vcsRTefZ6nmaQc^LNC25#9NE9`<=yk+E|r>4xPN-55y)y&qDrSGik~ zr1vdZ;wE}9Q6BOyLA40iPPBx6+&V(%9Jka#4b1}>G-m3?EFar@4=!RP^&Ih$+%-)5 zxal7G)_f(g`^WuAzlPj-JAZqS5s1bw{I0@c7L~Kz_ z;tYcKfWjSElvBZ>DT<}xa9z=rn2V^0SMl(rNhm0zBh!s>D1LGg$ISp*2^dVgh#~-@ zg5Ee4c7Hm(ogorLGx$@VsHEV7!PZV*0hRsmPs35(6_gdn5>EM*UQ<{@JKX1l(Yhk& zwz!&Q3SYCfigx_@f&^<`bh!!gTH^*Br2)NR@fQQ?Cqc2T0fZ4Cts68`kv`anIbpGj zT2Ew*oqSQgfEZQP=eULISG$lHo9)@b@w+Aw@nyiu~p?Az@`g+qbZg z$~lL2X|O^n9Jh+Z#{%c|A_Vn~fX4kNLCq$yx{${FE@2HuAB5rPn}1}()Aqz&286Zn zHQ3OC&`)H;3!hXxNNo$CJu!vsh}zB zcaStVUHI+*CSF6@ghb7|eW>LPHyIV(()Nou%vMy(C*t$(1<0_7odq-(ABuGLD;dbj8v_;Y%`FthdMAfGi#YHY=%Z+N1#ouJ_Fv zjeLKQ#klB2Vm|*DdKzaj<9}dlng08hEMsbC?qWf}#=y+_zukhDb-C@Yx)5(}>m4#t z($jCj4KUVH-GNIa5_9MzvQ^%HZ6%`OGYLtpeB+Tz)((y9DH4Z0{g`p?+x>g*+xfqT zCTBn6_5Hk|KFL6!|3JLH9Ol~-`3)cTi!q{qWhduJqVD*8eO}-CBP4y_s}FMi4(sF3 zj_Kq3ujBcD9`gHrb#C9am#5n2fm)}Evq!Vv6Yaz>-#qm0aH_-4gr4X;!h6JX{h=fy z#U!$KjCgxfy-2Vdn6wWozdfjx`HFVCPqy=p&p#cjKfu*19STP{=YLN!|FSWa__92Z zZ(iQS#{9*)7wm-azp>&s2|L_(08Gq94Wtt|@hZR3#x`4~K zHz`9YM1Y@qP>@^_Pirkb1>;zJ*)sQRdq=qGtr=`z-gLs3RWG}Q^2l`d3fhF^^^Cfv zo_b|6)ti-jWRs6(W7(yye`_yiXYNbdV9M)2Srjmp@}u~SJJ08byUl7u&e}l8nCsiI zAB9wU-QHYP>zDkZJ!qA*5cyIg$s67Mb1Axo?-;ymLI>zh!tzbyOtnjLk$){)Sch3B z8eLT+`fLl36aV;jf7q%d)&%Lb)?U#Fxg^8x z7%tOs^98e@&I7ex7>z;W{V37&Q6HI&JfmPFS#`qf0?eZ7u%&-3 zmQ~5*OxlZeX3x3JQ;Hc?q&PN=r?!3Dh;3Anwtw?BFaIopML>RU4$E@&xRN^2jknFT zy^lPz|Mu9@)kjfkY)3#{; zMu$A8$Arw36hR#a>6LE;m_v%CtV8<j^`+c=i)q^Ek#6(yILB@F$!|Y{o60oicFmq&EHFToCU?O4Z=uk1nz%533o=Qy_gMxuk7fX zaGloRNZ}@I%q?0r-YxP&=A`wM7$X?K?v*s|kuZ(H^}5n=Yd+5Pt{-pgB#G-dGE?E( zX6!*sg=SEiC;$U6+kv;6qB3G)289?DesjAlicK*(^?*9W3AeAh{gq9LyH}geo@+;I zPFIE*Z)tNsb9S?p|5i_Knt}MRkFVQJuTyd@Yt3WI$rvOp@pe|CPlN)~JOEmrC!cRuoU&l<&}IxEd<%DrlUPq#GKlHy1p0;we; zwp9Br=`|SuZH#>y;ZJ0od=S|f?~)@St4$eei%gg)7+{MqXGsA$mSF@JMwQqwo2%Cu zknW14)a*!>2@gZuy7?;&>4jBv1(_>9*0JNXvY(CEe`k8>Lr|1vU zh{r3|f{Nas@*tt!lv0~R2x>+%BJ5Xu(jc)_TAf``JgX3MoXckxeSC5xseW1~UaV58 zmIyN6p34w7+!t3|lFtFEjA7a3n#P2vZmjcoNx%vfBG>{s(B!TMrTYqGs7cHqa9Wb5Y@tn#ZIu!t89$u}Z8?5(VNhz+->ZU-ndyJsUEnSX9-GjgBDq z!$KZ4briP;3AzW^GSUYGNoid`4}PuU<&7wjSu7F9q~lS`?aubz&=d97@yfE4Hd~HzOJ53y zX!0Z!4LPk7)6?~lSH2Lf#+j@+2&?SXfe*;!>)RHiq=J1$=t4GP*b3Hd&R4B9R7@z#WIQwDm)LD()Mkgavzn{nR-O;s=e9 zHEVVmna5+8B%xnBigTuzeB`BDFMgR@Q5@^VzRV!hM)m`*^9kGu3`oL3Q#F(!6bX^w z=N33hSqoXiT$rE+^9Iz04tpI4Qb7j5ghh-uYv+G zk>X&&$Aw}4ff4M@avs1SzXy)2QZNxm=v2qYicZJ*~0%q7ot;amS3j2BPR-y5zbhsYX z4Yysv>V*nQC_3oDRbO8ZV|+iS6K(yzXzZV$%Zd*{7DuQe5peW$lPatk{hwt4;|h z-K1CRZUGn!XhA%rg67Vf>Q0-fo>q5_a+jeX##zQgWzr&mUXg!IEMciZAlmmKq>GH{ zUH~rD=`98bShe}Kt+$9wjmt<|Bka9lwQ$3H>(e2g7=}t_0E;{b(*h8vo%g5$6k^zQ zIj>b^7XhtnRVK&4NzK};Y2KMO$T0F}OCWbBqbU?4OWO%yl}65EuRwJg8dIF$%Nrhj z#2dH^P>c(0>H=~~e#lLD%u_Hfd+X)xYA(80^w7|O)73kz|B5{d0T`;*a0S&P7-&x- zp3yhsOPtjpNIcxa%oV;u1PzbHbH>I>`?UE4SS%;bE25VHZTpyW{JZN+^$kPA^SKvI zCe+ASY2CZt=9$;I&3SB(I$sL}FM}zr4gf++bO!(@Oe!+!L!Qux+Mae|EM{eTOK9Ij z1?s}uuCRfzpwSKCMD?FliT$MMsMopGk$hkc7#J)$YmE?ZG~T+<#9N2f?IDyG29}2$ z149fD){H6U`}q7XUCo9ugtyi+-fIwN0h|425f;EP6a%V&q%xl?S%EQ);KY?%NKc7k z5}3=Ue%u`Pt~o=kbk{Nc9)%cMV~-H*8mc8A8`4AZUpBGYA~5 zK*T9pwf~L^NJc^(OjCnlS^)96_iV9lA#LKf(=mWSw&_*t@V3ff^a~f&||7p+8>~FI#byI znkIXb(C&Ab>V&J=L+VXxC*_UF^500 z@s5?*I~Neeu}XS)ImZ+8Mt$N5X}jc&VOiRN^R8I+mJmC@K*=FcD;TkOg{~KCVv=dP zdM(^B=B6tAdk}#X)#}oYi4B?rSXw^zRr9>0YSF?#NoKRNJ@R+;0|8NHS>!CsY?HF7ii_$m#Bb@E*ETni~g zE{SNfpIOd(i3MW!5%+xIaVaM9V@FcZ(t(q8(V_wbS+|}=Vme^B?Aboe%(@SX(GM`w zK8wPuLM=XPyoMDKlhQ1U!v6bIu~vh24a9Kdaw3;;25#@}0J%T~PGmf9q=MuGyd1b- z=K#29u3``vS(jd4Ub_C;76w1244#9zeVr%F$t?z*6e*a{cxU$7s%*`i(<05z4xK3( zE>V(;6=FDTo2gq34Yb!Hr%~3@a$$X=PFKIB_768%Ek4Hv9cNZt3$VC)Tvgk%qmrc8 zB?l^xiPiZ5RvTpuoeielO0W~vf3sB$47ev{=yqaHZdHYxf!tk<05g3J(D zMe-nKPjlA-$4uq^RODU*$5K)=9z2Th05b*HDqSm{TkoBV5eIk$6~XC=*7>-M;r zvBCUuS*I%^r#-wdb@v`(FZDgLyP^cF@6WGd-IPTtJXR7aPGfnbVA`$JNRUFT`#^n| z)iOrA9m4b5Pj!c{vM?7IhLei0U2WK%gwF?Eb>Vf35<(9B+FM~Kk4t3Y@FNIMxSVE= zfHi-6wp4Lc6kz)E?o@j_kmqKSnpATt=#QO0_gTLafbKA{FSy*ph%+xb$gGUrcYuTw z34{{kaJ>ucQZK32qw*M<<*9kqjPkfFhe%BR6als$Ew)9$(^wkwJHxq&OEh77p@>OR z;7k3s5t5W~vyO6GA2*C^a^T2{26(Z>Y@Wm<;KQL%eYY4n-Y^Ef!YoO=X;CWcd-QYcPBQOgQpm}3$raPn`{ zjl_bc@OpK?>a!awy7Nam^9<<@xA(Opu@Ah;Xd*aym)dQIYa@@d5rRWQ%0m|t-f0WN z-g(w;w@xlU#HeM=jRAlVKcpzmb^bCFcjSNO!FUwUQhB4Y9#v$L7NT;s7dL`)dsiJ% zVbgOcgGtN8C->*83Ttt>(qpD zuB&gp`kWiz31ZnP(%Q0S`<~Dq3s;S8|NWRB&s>Y*qL_PxxpceP^|}wm!l3LPZBF^W zub7odZTsYmb3N8{U2F;!UA1X!RqJM?!nWjbNt}dQGXj)%OcvFlhP|G9bCgd~-g~P- z!qdhP=3>u|`ql7EN%Gw>5MDZdnU9r3G~L&hQB4qds7d)dw(U^N;k00^HDB*Eno_*m zVyH$uORAS$#J5+yfVF;7he$W8Dj_WI47{21?(KlIYngpJ*WFyT&R!oFoNl1)SKIzO z@LmweCaTYdC2S1*1-uPzi(G@2#~RdX^mceE10WmOb=#DGaxtDwtoYtb%ljSroGa3W%fFDE zWqY($kJ`!(#Wfj()L+bg0jWDfpp|p`mX?x9TJXxKfXHi5mizW?dj3G2YFRGYPi57; z6ahSYmvqygY_ncXkE|6O%k;pMZ*lM1N`b5S`~u5{3zE3ecL=3Y$3qzsM=}sxhUo^G zZ#|J2t%#6|Iyj4b6TM-K4zrmmwcw&VwAgxeXk~qc$e-_Av@(UE0-1Z6c#A7rFRFsN zI^#Iw>u*(;7=++D=jpbe2su7gDE3rmRk6( z9&(P~GlaHp3FZz^HS53cI?0(F46O^x=E0TUlP1tvU}0 z@A8ujXVCxJ@Sb9|!AQIHRa+z8DszcrH&fg7&2yy3*%E)SNyHH8mBGzLy^yP!-*Ynu zmzzl#g0j+70h7%U^e;!pbVq%i5N_lEX_XIeO_+yr!%yLx-@*%5?fjs=i6_rVZ6%2d zbWl^P?jxK~GV$A^yhrmibgPCyFGcSKJ}%ECARPKVr$aBTz(|0j~`0>#A z?a~^k;DQIOzZu793fiT(?80cruh_Yu6H=^W)z4OQ?YgTbd~5d$`_bmfH!=_&W5ppLBiYD@9XRH@6WXF^X)nyfJ8Uc;(wa--y77GU5#8k9ZU)6 zg(XGA?Co6s@9+|X@c-A6#m4?G+VuaN|M?GMwZJ`;#oPLZR#E|s2ZRJ%k~^Vn&k+a; zNuR73%Gfszwz_RlCq?U;Vy_c*j(5C4W!Md(ged?+=?Fqe37;Au6J$@!#yv58{;s4Q+cL>o2s`%DCKW@)H0!5KBGYc z9h>s``9WE_SSFXtH}8{6tIMK!(J{%aP^Jd*A?+UOA&E;aol7RYI=*zI?SHPJ8L>3H zX|J9TuW~}Xeb%d5rQ|V*s-)j3gIa~buGuv|t52vpc@(JhpnijU)GNJzO{yB|;W=3< z#czN9AdAf=l|CVqvatMZz&K+8aX8FCvFENjDZ4-2W`PT)IMiMRMYDxvNvXZhgK$+V8b1Ro|FA4h1 z^JB2zhC*(!9-Nr&(dkIpIWlG|85a>9bsbIMx9RZb<@;X_j8vjXinuxk6H^ltKoQ-O z@`*)UVN$KLc6a)%Z0&qInQBmZwg*;LFQ=-ggCwUkiQ^1J)kNsqvrzj9?+D)VukH>? z3^I|jp*4&Gl{Ya@CStNLWXbtmzmNNJ;#xyKQ5!p(4#AlzJyzV!J-=16UC#Gm#FjlK zk$DeDKX~|&;d`bY6xaczr=Q}_(<}}15`@|H&WYja$FFPqI5#{lH-~E=ff&G0ta-KK zVPRyYEVgWsYaNac4WFIU4e-`IIo-u~xJb)sUK?Nsj@m>-?h0}@h1F81<~9W`S|y^7 zufy&u5f_0g>@FXkH|&k5E6e$9Z&d4`HK{E!w|h-~GNwKA| zac?sruH=6ysGyOG-*EcI+RM5M1pa_DXsuQ-e}??H#v0Wryz8JAEUL|of37hIJ3a)@ z6hSunJYsOS{W;4xYBbvWj}w*-Iu8H7!-LkxNppZt31TRAe~*!^FrUeHdt~-W7p(wN zua0`XATx-+hGSu54sD=$6zuHQ`+h7k@K94`cwofG)C)sxjh8|DFA$JZ@DZ9 z2w8Y28u@kxi6p2uN!i{*Pj=}+fmPqpEm&nWaTy2rYGS7mrI$z(3z&BfwVt(IEymT^ z0?To!UZBXBV8B6G3(gZMmUHY&JfMNxZ8d z++DA_!0|=yPXzUPt5X}ZOh11TT!kZKh}$ByO&GxT?)w=nde#NCr)|-=7XTx*{9ksd zK><4>y#buAZoIwg|Cj{Nct`kaTPgx#{-}XleD!1U^r>$q+OZzp7+xR7;;?YxuoND; zqh*Q6cS1ZxddO4ANI-@eS^~h&#Yse)^JRQfV}<_l{ru>z`i%5Y;G~|d3GIYYwBU@^ z5M-=@2PsOgI^y`XXw&H|_M;6BM}qc8uOFaR!2iO-ihG0a_uaAmm$mD`CPR5z3VRgf zaV-5)5EUC(B~l<6wb7dLZn549UY_MID!jH(^dwV8B%59*jq+i>M4rO%jEef)E-tLz z;{a<`X8h?NBMLk}eu5vZ=^Y_rL5Xyq0FAU>P|tzwsIA~zAa9{iAn$A;t?la!J&?ct zv(N3lMUT^%_W5U{Apc_I)F-aPYa$rf=R$bX$;pL_S3*(L%RUx~tW1oonk4IB{2^`C zJfU_Sk~1~&wH5LhF;=uljxp<=5G#HcaBcCzELFEkk1m48lI6<#i7!i#1|NFWkef2& zU-i^t1Ya!aT3D;WwP&+G=N02;y2LOlQUiP_RzvdQy#E~yPa>VVr<%Q4!*mFw2LrX# z5HNnMxXNeZg6}896xd+J@E)A*l0n!TrbU^?QkUge4(Q~Hp6}h9$MH&NvNju_cllj? zY<0}|DWT6B{NbuqxU;%HTVEelmsgmDMd4e~sxVIv9D(}a&rz_VDBjQ!N-diXk_xu2 zW(WD{8R@M3eS=R!|0V>Gfd$JQqJdPi)jiL|%ZcTgFXsSNpnaQn=1B_!_oRXXStWFR zf;hP4HadQGAkM+~odw3m2GC{n1POg~{qDP3(BVASeV9)=FFDDlR@@E^$k}0?pwavi zvz?w@wGrS0qD~a&*$ojRaN$%$8zwFU24}`gc3XzX@4*6mbUso@#mjx*Ux)@2bRtYA z1dm||W`NY-Q(j-h9RThE+mm+yx_PJXQE1=ujg0m4<6OO*J(=It}%ey%oXl7iSBqk7>2J*PW$D5mC$M zP7f|#zJuHf>4}t;%IYDa;|D#bTiQWy7I*UT39;`H^|4#sA&{s#0Y|(?x^mGvv}+9r zy+yAUuzmy~Y2@YMV@(>NoPI_$GSKNECV#HLCD{LVqQBKQ;4QtkR)XWDwM01*!~)GT z_q~@O$DRcwu9^vz{=0f4tl=cF0oUbptcTb;O+L z=s3TRn6|t(12xLz2@5X1gILy1`4-GJZW@F?c zppmVsQZAMU9(OC@vL>DPb0n|r-Cc&>&P()Qg$0ndoVBo_nOUr#Z8s3)$+DE7p-C|r z3Z?;CTx26>XPTF3@3xb$i7Yp&f{uO=>`d(|{th-dA^g4rjT~mVQ_~>z} z8fKOjH?Ox&cO3>=O_1${QIr<>?dgZmI9se~aAA9geFFV8AXJzbCltyJi=WTC zbRs1@2zUn&#v;HfkLCkwD13M1?+CDZ@t?b)M2cOun)x+1hdA(b8unX`WTY@)?rfIg z{uH}@wMhvfPCI=K__PHKkY$V=Dxc&g@DZMtYFoe_JFjNKu32pDA@d z0UcGU3AlC2?)Hj4F{j~I5RhB3h@*G6e=NYYECvS5DEUtW-s{0*DEnCo$L=)cfMF^-0nyqLZ4EC4`Fue&?(VbD^NDrl|94 zF0MxR>%B`F=M1-z5^R~S1d(Okt_%p%75^l)2u5CO^$@KwuMu~e<34wBG8S_7sU~*9 z+{s9=xHDbRL{R|FUZYDT(3nRyy46gPH8bJV&n1vHgkTKO8Pyv;WO5$P$|&3F3X*w2 z+87C6Zf~**sGCgxiTCz~8C*YexBx1e&L_N$;_5OK0F{KwAgBl$chd@j-Z}wfS|p0I zHA(^8Xh2YVl0CFzjWGeDVX6u8*4(1>(2nsisZ}W;1yUUIO}bS>`NIK-Ln;At?l_O# zkNu<{)`NI>2rVI3(vjW0U94Tj{`FqINGB0B8gY0V4;fBnHlZIxybO)Q>5uCRB%PQs zqIp{+r7^seeh$}|QztezshZQBd9LLg^mB?q*|n0OcX976wfnA5ptSVnA7Iimk0{&s z4r;~1mBUi_@M&_4;G!9G!074?a}u8?c}DcR4?f=ZEwjH57a-b*FQ5HqBYKE-97sLDvM@I)!0T1x$o z1dSxgZA~b3#y%!4oc(~@8YK@3{$Mz@a3Zpnu4+77a46WlQwNP5vxB5U1)m9@iW+XT z`Jvo!J-r0oU^eLWo{FLIVEWYdNr!BxO&_+-0nNE27C-42Ve@D9C2TO+cf@-Jzat%XQnd>l_t;kCV2WjHx>a)iiOI-k) zax#Ujkqr}s)&=H8Ce>0&^+FKo#aBmGFgO);mEdZMC6M*oWK+=|60y_;oPVT=6e+v> zm&t_kR(=VWe=2LizFGvVptJ;q=(jzOuh7E)E#4$~&^*OC>i5^&18vCrcz5UOJKiB7 z=iVJZilzwt%cxoC7hjXwSLu4kg1@c{B5Y%tZqATDHL+5bJ=t9LC@JK6B*MMg9>PX)kGmyr=_CG0)7m2L>*JgDLZ(L;;CEiq zckcUgb7@zCxFA_Ty;SWC8F;*i+yVQOq zpTz5j<{gXj1o?d)gJGeL?IK>oP{c;x$hu4dgs%RkU&1;r%ZLgJ#>RQoXUq9g zz1!{VI_d|c|NcgltlD;!k@40fqVm=fQD%pmz`{^KgB1O1S;Q`WWy)XG=NdZ5JR`rC zqd%Q6zkKdpJj)^#+SJ;-yP*+J#lX%1fwR-fCNyWT{~5!# zvwE6{Hio~pk?O)Fm6`5F%N>lsqOYE8|B!C$u-7=-x1@jswS(=3FHMJ*URb35!oaUk zY1W{b8<N1WtlzeIuHyO8S@xmE?9hNu!;;(c64bloO`2=l+0GV=$MK zMGmg<+2c^RIL>W#arS3;GfgA>c7aqpTOmWD(73$}+~^hh~6}hj!7gEu@Oc#HWy7E@Uj5ADGP1(x)l`80PX5zbl`?O)MdD ztgX_G4QRl$Cn`|~DwYoSUcypt(0>w;0J6-S%NU8*pskS6{SE^Leytp|3^<)-9+kft z;Z=>HB1qOwH3h?WwjBjfk@hxGF88EXUPnCZ(|EQYdtbNPAc&W0Kwg~}RDO|P87`Tk zfIIA2-lFEJl*6UV4El3Y_hUc7pu^tZ>&e}MI6_u_Qt+jAEJHTu-t94Aw99=`^Wh?* zMP21kJxeR3!zn|c2$}HnRq=E?-Cn=zaKlGt`M1W87#rZF%SRg12rXCB457;V{2v=$ zte7x9kFOr?tUt)^o);_MG-q_^E*e{F${CRGtl$I8V+q&88&-y!ec9@MDFr{q{UgBv z5>cNsxu1yCSV!wg0%(3x1&x(T*~v+xc(lA)?osa>M=$#evNL&rG;#zMrLs~|3mNR1 zKy$AvQm#FT?u#aL&d#Z5*u*w=9jzWq5%}}PYWC|rp3mp20K8ULsi0qWOUzjCejLxo zli~661_61Rf1XkFxu+FhyUr^lQI?{cx4)AYs^MubGC1jH{PApTz$z}{C+KEM$aNvR zyI&2|77+=w5OOpDCo}SlYIyPdv&Y^IfBYv*q=z_@td<=L*>tGbi%vVfYu1%;d7-ey z35omb(5Id+M+GdF!1?z+QB;Pd%6ialh@D>rcOS96C42-q+&9Ll_mpfohOwQtpn0v& zbl=1l{+PTjU6XW5eEN`YqQB&cq{cWKyP9IZ;K9*~nK^Ei!+9~#|s(A<)EL>3RC0InJ1m+amj{&Mkh_kGf|O}>jUbB}~S?AC6k%Zh!Cbls*! zFI(K4GyIv*Tfj7cRAg-k9l5}!^D@~f$H9+@#Y0r5oHKQtsbr5ese+&(kTZN#)WB;ia;2W5~IM+7}) zHimL*w%8OglX30Qb2aSoi^4jup;Y|gvgUdmT4FS6YUoJ2WZoRA$@xBtL^L7sI6ZH| zj|dtWrsCvG%qaVW_mf#KJsA^B*1Bq-!It0N=j%_VgY`BK@ zIA#FpHe#)}Z%9JC*Dr!}1c7!EeQ!a&^_nc7qrlGs72EeQJe{pRcBnY8^F^E>h-1ut z!W&g|lI6d|I7!HtubmUmNBMVde!$Uq4yC(quCBi^zw-Adi6>XCes(GK!uk4Yqv{z` zGxGx}w-6cg8q7ge*bnuWKEK_o=RfFJ%>Spu=>O(%-D_-hZ!m#e6%yWgg-<-=u>$-m29i@4wRe*ZxB zQ}pSf>?h>cSw>g&pciM8NPfY`oAOo1_4cx1DHUPpiQGmp*;gprcrVXWpn4{{mfU6H z`sRL2UpjAg9yV>M&U?^=@j1`CO>~|>WUF>sU&@|j$#%NWucZ@YM%)kelO<9= z_OEnWcYi{xZiIi*Blj(AHe5E@ElT^mcrQNf7l$L1izN#*3xK7NaTr5k5i$Xm7NC$c z#)oXZ0GbFo&(R(^Uc-&FQqH6hP%x8FBrh?AH({P3exg_e`VvUOlX%S$crcK7B(xKN z1~Ea3{|(q1PuP=qK;cN55*J4k5&!oop1{k%%?p6iD*#P&*`45wr6PLhm|8_$u$54- zRcIPT1sh2RL-JtN%FurPD`85k>sr9405ttmIHDqc~?HkkHl}P_G*d6&my!F zQ%spZC2GMpW~Hm@Q+FTN?BRH}usmJ#37kB!*!DkRo$WuU!z}*=_5TIdjg2^dL%4@X4qfB9|5c~gs5Fc#34)`bDCmqLnhA<4qyb}2QAG_@bkr3ALlVa| zh70ZhX#1PLy$hJ5_?G+j(Lew7z3=s1X^X4eZo6x;v)}oKvpq@-gRn$E8X>lceA*_D zx6)*WYupo&u-HRs)8`aFp8tXHE*t@=vwSL#tGUbMpd4bIPh8^bcnK4_e5*VeM_}9) ztFY&XcK>7o-{xn`AViEz5I2-Ip)i~97qd=$z*TB;>b*VZj5>MvnHECvD@HRp>(sy1 zg+i#98*xb5oNbp$yDT+dpn?$OIG3Gu7Up!z-vdJ@KJ+C&hnJm)`?=fPoPY(-ZgM9W zc?pzP+yaE?P#|jaKqWJ!fc!w^*(9gxbc`}PsX#qI3Sq$wP*zOB9h5Q32))&A|jQfGx7ZhJG{~)>p{12M`$nGJAY1tKu z__3m>3h%N&FT?}4l&{QpCYP|UkgqIrOGMfNla{2l1+8ttbW85b!nq4tuJGx?s0*tv zFmJw|0DX~tdHVe1IqDPdlav>#FMV&Ap7gG`eOZ3t=6U*g@RQ?*><2!K5rZ%z%P<3x z*pvhYC9&~QOpqcImDsdJ1_5(OXCUTZ7-Kt=FlOS6#3>3h7UfwMhS(_#Gn&Pb7~?<8 zuZ*wEml@Af#8bsnx0$t5a%Skw*`9gpvpoYiCVHlP27E?8Q#r;ze|-kOV&C)LC+;H- z760u8YGe2~vm9=mZrm^2dJa7Y9@CCXj$4ik_i2Z^W8QJ`IOsUrasAwV?q7Rvk<3mL zI;`v&*wd^I#vNpKjP2pxiJ!=?NUu=t5!{2lqrH>gncQQ(qt6cd&btoiozyxZ(Ojx> z!HX9yoXNRx<%{P|UAdJrHs<)|sm|5T;g6c=*GZi+bnVp3$IkD%?j4lvYqgHn+g$C_ z*AAcC-`mJrhPRjef9RAOF$B2^IOB}V0}g3_3FEkblyUGs$Z_&z_TIKH9lq}UBM#|7 zas0orIr?%27>A5#Mx|=d9BPK)u14%~f|woC0uFU_AUNm-OFLlH_xyn3tU+<$hu9Ag zJy7b#4(~vC5FL6D==-7$Wr5=04^{!j%KV#2;4|YhnWxI9PN&kR)TiR%Ksa`e)($)t?putz zi-#M>90%@;4pqkp#~8;h|p@MIbM~o z#W&s>>o!75L{_YopJ`m(Rxr8Iy=Y!7uJIequf?-O!2H2h6}?g9qJk1BQn@yGCey;q z1#BBH2hQuq`Z0Fqd^UfUb{0E7Ruo^|Z+C}O_(?;@FtS??C>I`esU;_}Y+LTj(L(dy#%VON`L{DoJOx5aAO4WMpz zmvtB3ZQYeuO}G8(pBp~iS66g*+UuFyuIlc#?yhcqcTsmw_uZSO8wGbucgQVgzo@Q` zo1U93I%W3;?m5!SnGT2hq2vCZ_1uN)&bjaObUs?H-;wvRS$qEe{V)6(d7l1vpSQ23 zM|k0`)#2#udPQ%#W-QC~M&;HMNKz<+}uRC%BOysYlj&zx?h>lwUp74%(2aM4=FG5Fr zlgK^W5nh-B{Sj$g$u*C`p)tNuY699p_ax?$(EBHQD+J&?nAnBTAIZ=(NqE@)_2KyX zOXHp+`xm5K$*0HDwb!%K--7b|AM}=7!2C$jP9cBS{4oz|*)j>_&Hju_Ott@l7Od&{OsRzYqQnD5U?3 zvU3O$1XvPi+qQe!)3$Bfwr$(CZQHhO+qSiH*uzHbKkVK!sxB2#Sy`FybszkwUM!#P zt)f94GJGS$c^bA`5=DyIb{`TIemW~XkNB;MpvQiAj6x87sYU_th~$5-#v&{iT(3qy z?)>62⋛2qXve$gfs2%0kFfh3mCKA8x15wX4>|JUGwQ@3D4yp zF?#TYBGQ?Npcx6X7>SDi;RZKLMsAP8cJG6}143Ku!!!AV|4=A};>QjhusjPmJt_EJ z1`r2Tk1nL=Ek_QO&#;7EfP<&pH}xW@T!ys_VDeGO!2|#+6ga2H1hVZHQlNh(!h{pe zM+B%}D9>%d7i|WE6`3)|#)K!$H$C?oo;JV)Wk{a^<$Yb~z_dCE8lBIcMO+$)eBLfxI1jGXmBGLoG@<$8AQU5v)?^S&vwq|8S4X23u-K z>w&->qtpcVCRj-+G{p+<<)o_7n~`OAU1?|8sR z)3-pq4UTqjQa;tM*`~qPKjtvs4TFQ9iV07ciWM&!d)}N2EOuDh4F^xw1>k<$4eA3$ zcDOZ=TC@Ny2_!twDBpDeCVve)jwTMpI!aiGF`n{6}^tXaZVvEU(wGG z+0U;uuQa(0>j9pC+oc~jjvg-GLJQulE?@jilM724koz618tiFWh${&$kE@Ip(&X%K zUOp;N562Q1gBEZDMBi66V5Kjf#-#s>iGS3p0r=glzw!sjY#1&migH0Fp8@_}Pkm{A z7O2L>=vzK|v0o&hUr;>V>EXOlS^U|>W}f1tp{RZkk{K;_95ZB99}h;qaav-q6G=DX zjncq9oCo|#7Xu4d3#SXthB)|BGziEx^r*KPum{pfA-J$UTIl|%!J?s~;Vf_rHlUy@ z5Zp@PZXoPi_*J?>`x2D}*%LJ+9WKfR9%#2a%!ea{chs?3xafZ7oj@1(FCX*Vr$M7( zY~YPM;3o-P1Ft_bI~?U5x|~8xq8&+2c&x(gyoU$wcUzhClW3SdZ|Ki6aMdg@WHpGU z3r^KE*%{C}d?3AA9jv2YvzL}1_<;yp%YHVBG<9K%{H#28Q_>H&cbdEi?a<5!jlYrt zKe!_wgZ(rXvDvg66MD?^pGsc;Xf7z~EPzcnuo#;sW|KO+CjRK_0U3L!nZ9N_7?hXU zlLKnItU+Bly1x+xFLTaM-+NEA$j6ar0!2CalmUm*7{IUztbs{TAm5~ntY}5+0VRTy zu7M$l-x|Ovo|I>U`e*&D13SCIcRV$E8f%TDpI+i?z#O7Zl?X9}Q-LWkC@xPSkerLp zbU*ibtCDx$Z-8H1IJEn!_wrxlKR~{rw9wrs@=$5b92OYNSsGH+p@o4SsWOGVhB#5V z`~Pwqns{j?Vha>*;thfn@fVSv2p;rpWJ7~M1IvF|!%nm}WQrVkizS3vzYxYC6~>^K z8vROvvt3o_A*BAZyUg5WAl<0wzMJ40EJ@`wyMAvYbW<~UOe9?_VVbw%!5>BoWRoV4 zS5Qwn1o+TGJXt`isS(!0VY{k+?0e)t?_~FjepTFbBbTwQckXOFgSmrcvx)NNg$c@W28&g>ZxSz}7{M=kZS%9t7R~J{dkCK7l_-C-zW{5$;*;;qID2 zLRn?KW(j8bX2ECj^D^_0^PL5>L*xgi#FiwN#JYV|Q337b@1@Uz@DQ6pIzrS+*v;Tn z0l+JX=6ateg}LP9Q5Pn4ZKLx7aXJW1J&;ytlE#Yg2U2N;)dl`f9(_ZCdXy1-hP;5= z6@ssUCtiLQcLWwFZtUZF<4^T?cSOA?)9BNo;%NkSz!|!oMh< zxh{OOH$b0z$Z$iY#U6@WHmQk@#ANH%Dv|3(OlQs-DupWKs1E5$glP0k!SW{mqL z+|B}=i5KU)ynHp>({b}&+9N?OWJb$h4N{$s0diCClL=f&Sr4)#?{4Bh7UEMJf`3uQ zoK!rsuE{F7W@NV5s_@=+mEEzjuDOAD)%I?Y-B9mn(7E=iTAGyDH}64Aojs*BCo3)> zmF2d5a=LJ!jRN}gla=ElHsyOyhN;p16bU(ZPAe*J+dL~lzM9!OKOVj04_;JOUeTxu z{%OR0%{)q9i{VPX#_~O|f5<|P*inPG24~O||DCruHkv(J&gRNs^S7HeyxcrL330Ii zG8eqGW%aDsn2zkAUZwGDzwoT>T$gdT%owb+;A5)%9Hk|Tt32`Cx7f2IQ<9{%EVCV* z#5%}E_`t&#UQ?8IAeC}#ndC~&Nl{)T^_!ZHe#T|lQMop+u`m-AyyUFvQf`AVx@q0g zxd6()f>#2efS`6(w~1Zs-NBvpJw3mD=g+E>3{6Vyevy}XJtHnagm-31&mJ5( zUm-p$2lInP7uC?Ly<$5+(E`u4O!LrycUp0aD3w%tTB&=&MCLH0^WrI_#*nRw2>6QZ zMJ6$cvleb7c^AFn9_ylX_m|ud^EG z?pWhcQ<2#y>u@78|PG{1t47kEft(n5-*uEKGt9ha);*9^R1G&(5W)B%NTdWKqZy`Jt2Gxf8 zzU_*5@bIB6PZL&oPKl+j($#}ge#Vz-f9sMY3%q+8l{o8nmv}$GS9BowHzg?2M|)s+ z?v#*lKjD`h^7TD9Nb=VZGPEYu4)ckHSgh^VH!*1P7Z>vJUo7Rn*yOoWD8hZ8Uq_^x z4w(Ugt3nlyy5j|`k@8t56v)C~U_`2f4N>`!~bL8?E8~bwT zx7gKPpxaW{jEp9MImT2hLCR6fGKvzU;{-^PX-7uwgEowrLRw_Yv8KXI3>d;NhVkUe zRVu_J=^pgK&m8eWh4Pd~f7!<^;xjqQAG${XMXH4**ucU(Fk^--o^P633C0tW{zR-pj@>5d0)K!&74WB`6X2Y`qTso64n+^#%lw!42o>>~fHsB3 zCc-%EcWhRr6UUjH>vQw>_7d6GJyPeF!+!CW<$yg8;(u4xKymtw5e7>iJSa3PA^Zaly@9dKTnEv3h3&o7agOl%1?Rks&AA7D+Dsv+yn-ri77z?r>H2Cz(1`dsXcqH=T4!{FRsZ1t74>XG3=!}82jpC`_JOs0+y+YS;a=LaH-)XgtFmIRKRo|}@PBIhQ}X5JU*H1!9! z5EvU4sUA8mO*bv{eDF*9a5Z|(RkJ@$b98mJ1pI3q)>5bAQ~A9Qv2|vKvgEbn3yP zU7$Q)y?H_i`w67;HuPa_*pxb-@kDMN5*i~Wkj{ngVR()#Ul3e9e9UDeL=Y(imVm`Z zqzTVTikfQSuz<@tXC8QDS((p$d44^*qw2NJ=WT^VvH)L%TtZ}7MCMHT1qvdpRJOOF zf%3!j>RX_X|IdnGFBHp;|IIP8Lc6w&>YT^U-h!DLEMT)&DIm55EMYg?2)%i2ig6jU zo`}eUNvDME9Dg}SK`fkk=TqKOQc{Y`!N+zAxYLG=#ez*~px=)-i-yY>!<&IDa=ILJ zngoUp&IdQQku}1VX=1S`K(91LhM$Xmh2r8^zUw?q`1q6=Y#2s5pK>lBuPVHyCQpI= zjbF8Bx$|>;p?@v@P{~=&q$}4W|L?YACRw$_^c*GXlI8s5Cb9RaDDM~D$M6w$FgsO@ zHo}Qq=z=_vt@m!UB^T^gQW5#MD3^9lB%YZV5kvZQwi})8+_}~V_@ZzGxOTcSnMrY`EWQLfZ#ny%4M(Q#vkP8k<@@hWM0y9TsfV zXmnFm>s(|j=Wr9ol$H7JJd2AEGS8>{Qznpm1a+<5_xqVEr*7nn-k`A;rF8LIm2~i; z&u-~&(R{G%_Y;tEDpvPfj1aj>&(BM0svV%oeLxO;n@*-P)QPgch;!eU(?Ou8GfFV# zFti)fddYC28z|LDU4=qZfIgfa>%dacsWMkWCkln}-4LSY`nVtqG`6%FxjyPGg?c>6 z1FZ>syfJXW+Y(bMw$-A6hg*^->TYQI*#IPLj#fh(V^_jCr)XSjUCx|nKCOvTADMU~ zc{s;rUfn=N!S_52bmV|lvzo6HwOZyK{uD7~bRVTlpl68WlPvgAqvs;X8m9S!H7DL@5>;hRVTmgYrZ2 zgmQ@Th~Xa0TnlmeR79P44PEj%3e>v>%ZP$FV73xPqIj`5xdUbiM(jY5zBH6MlA$5p zb3D#GpygDEb@m$p-W9=gTXWM!&VfF&0`2v)g~9?Kd(J7dYu%|@#PyIVCl|Qliz2eT z#a+SmRB;Jo%PijszY>ibiXH^tlaaPzc*T%3VSV^U0#X`eq)_%TI&-z0srna^iqqK~ z0VH&3Tl-K`=!jn{VOA(^+t?Y!S!DP#j6tdzAq&18A~LRMcMR=bY+?DlenF4(qEh6zs%PZjqS+s(~< z`ZQX+HcrY0NrFt)CIQ@FLE#}Txk>nlLCaZkKZ?e`ej5>D8heR~IFd~Pb&m4#E~;vs z#e#e$!|d#(B<(*@$a)!TlG57t`N-N}^I+j=h@Z>DcvCN0SNgGJNg3lrm9;CtD&vPG zO<;{PpcC3gVD27S=H(hKWpl|QwtWWD8gg0*yn1)upF3VOwp=B6y{2PL4VoiX>`309 zN5Qc&drt9ZHTMLnOV4VDEoZldx7q5*-AcKvoUZO>A325pzEvJpDhu8|Q;GX-8Rm@Z zywC7@@})afJ*Aw5Jf|LdJ_?IysdUwRTfVH6!;}rG<&Br3m13GglJzgS#V?cK%kd5H zm#oEhOs9d!Pw0BSt zVEDI-@%jih1(C*r+Y~`GgpIzJp(yJo%~{OV!`XmK{iavqr5dQPo5cgS^O08RENFV3 z&vv}Y3NwkI?`1K+U%b_MjyvL+a! z-%(j^D*Y;ltwQfT=QwnF_7SKc&$$)rY_yeKqUKr_dn% zF}r|#G|Ijzkr9CshPkEVb4VozBs*mgG-T0U+Vqb|0g3DYV@S)gD|a*%uJ9(SrlhwY zwGC%lKx4kV4sm6tGqCp`%nKWHcvkz)hu`PLX?YXmaioj!*HHeM&jptld15C-A-zAu zsDQl*Z--tU-u1U7nxs5&@b4py_V6X08G2FI%Q#A8w#zK%X_janxdp73Y{r({A_=-2 zHy5vGWU_`PxfnXRs+uTKgiMe0>E|)AvxQBKLA%P!bvUgZI2RLSRz(eszSG zHOtwW6QOSHjW0*R!WIxhVVw@fhD&Pj8s3DP{!mDUVUXdjkr;H;Sjf=oIptvtWcXyi z)NwR0r0lmnMb&9p3`gaOxb5vDw_U|hW>IRRt`-n|=DgqhyKaU!Lp{tjoRy!JBIj|* zbP9G#czr$t@74R;@VIfg(FEt!g|*&8rN=o8a=~OHbzY+1_SoPQPWfVntZxC?6IkO} zA26*twBrv*J@h3Ip9qHwrCYK0rv${zfv)K=i%m9Q5TI|3r#GYtO?a6x)dTV{N%CjI z+5s+M4)LyD6a97I!HDSV@A-f`MM__&r7 zJu-}vMt`Jo>@E>ey$?a6XEj39#~AJ3j~VJF7)CFWOpj+;@I87 z(b>);;tMKqq_x3E*>Fy^ z1C?5ZT2lWLK$>s=(=^PPpp(KLL&HNt zfY<>cU;v4ZiBgE)S4A<0I0)&i=}n%NClW*sDkDvDXVLFZKRDr!3{aLB1TiQ;rv?}( zPW4hpqN`SivlgX~RAvT>bwmu!6)zD(7{^amrO|h^`+Gq}`_n|GB2|Oe@0si`U4Ab? z!)`&oLmA1uIwKgv`_le-?T%LWA0mu*z#|px&b~Dv%OG3SpPAB(?$*EPZ+L?V?fevtw_s zeJjUhgYuL%GcAReZ`UM`g2r~5+sQ5pH<*g1U2gxWJM~iy(BtnYn|eu}cDM&InY7Yd zr)C*~Vh{B|VrP-uTDg1#nHtfS=CkDQp}DQNj-`#?^K@7fA`J=9;|>ef3I|_JImzj~ z-D&B;Y^CO8uv!B*J?*Nr-z+jF!L;RGYp5%FOO95YJn_N*uDQkaUR-8LUnHa)_8m3t zl&$X8x5`~=mUnJC#=550WQrQQM2r6&1uYRK z2G+`)SK(>ZS)#CDtYW33jLx2-KQSZ{OrI8`UalbNKocrQbxBs(uo;%Ql)D7EB&7tN z>?hWlTy$BTM4_TO9Iy0_$mb}>?2<3=((Dj_Zh0}rg}WHJG&y&6TLSVFl7e+u#o7z7;XTv`HAc`g0NlGorr=$8}7q2X3%SUji{2V9>-N|eqQ zII*r^azEY^wNb+n?>-I*4Xs&BjnF7aN_*n$uuouInxMjy(Rf16d zsuHq;ji}3n0ShT~#jIx(&K2#nqpyDN>i1mrn>s+^P4qlQ+KJ*0godV#ExvIHkCU>< zRO`UAb<`G-w6jo>4S>Ts7S#G>_r^JpdXS!@_wmtow|o~lWc1D@M^EP(X)5Z$tYDJW zkg(7Y*^PSecQd{zT<5%dxw`glO}!oUT!c5mKl9A{lKa43zb(?FAdbCnD~j58vKT3r zH+%+Y7AOr=6Od0;lq)GKv-@1b$x2+DRV}bzb#q_Wp0`sC(5?v|{+HqsK3KDHN30g?$;fmoGt zPhZPabArv|F{+ge3w$IX1yd%xYwJ(|&;rTO@Vn>zLR*dY)|^x*d>mUP3M4~YmME5J zcCphrziZU|Y18XbuU^7#lhan&!N?V+ckmE?4+FpZ0fp11c)|X1cKlGz-ElY+G!!wl zlTayHF-iMOT2*SPzEEwZoycHI@7GN%qO?njc}gpp%|G^IhF{~wKTC!3N-OsIjbDfl zNgnY&AqpWHF+8yud<9FNj5D|=lZ}_5m!jW+nIJRfPJ3kID$Xe~LM;_fn9te?G$MLJ z+PL()3(v18g8Je@V=*#~aWD$wm&(>*iy<9)#0^zbpNn7^G8%NI4r&W^5;&ygoz z%v#i<#e!#p$ByCjefbM3UZ`crrKq(DLs$%5Dq-c4s5E?oq8~pt)ZoXYh+8TqUA#pY zpH;DEU1NK-1_!j3T61dZPfb=QAxl(u;a|TQvq|%>&qV}C!JZ8?b4-O_9Fl(_i?rV| z(l#3*(aZUr5kkieh((E?XW~)r_19c4x9Ny*!oo2czhi$!pX6q1X?)T$6z>Ofl4rai zc9*AIFKmWg;DjCp3&ysV4Wm23fa;}crqys0`}XkdlW0z~1d1*XY&Sm;A3-k=GfCcP z2^%(W9)n3@ArEVq#fa&xL`*YTacw0^Ww#(|be;%PwmsJs-#E67*UtCkhv5+X^%#cJ za*#B3uw^LmXdee*yzE^GK2@lvS&yqEiI^x**q~Y5Qt$v#BiuvCqeFDRefg-7%%9^1 zZp@t*QRR=~T0Csh6Jt5!dd0Z4b}$A}-5io2Mo(-h9DBPhH; zGzlmqYN3S&Hu#hjBUV@S{8g_Frb>86Es^cXq%;`qi@Hcb zyc^1Po<-e~re+oG845?YOl?c2g|S2X#{QiGl5@8DeoEUmOM18H$;#Qi%r~ot1bo|i z=7GbcJ9fNK1UNC`Indfv`3IK7z&Y$AL*Yoiy_=VU2QM?AF-UFPD?eQgrL+JSGGQ%| z=i0EXM$^4n2<$T!M8ED6$`R@jx_0YenlJMh0rpkEZkm$L(h3^JdiPoS#zIyGUj%(? zU14DrvjdZ8lCr3^6;6}ORb%5w($XE^D_^(>KjjCDM8@ zt%C1CtNGOhv2nuyEsa1RdnXY&`T7kjYJFhpaiztAXmKnaf{d-C?u~AayY2kM0r+C; zd+UiU?S@VlnXm7BY{G3{ql1Yi(?3ElxT4{W>P~8njCA2Am;T7H2@aECG-jr!#ny6D zt87cv?3~{ecAQz4ATtpOktkY;eUj|e(k?gvrV?YBUDMlMkbo9j$3eSB?51h2qfqd} z9;q&8l|`W_D17(sk|Ux@Z3{%zb;_c2NOl`U!Gtc`l11b2uJ00I z${On zY^;zva!@Kdg1$4NCPqYW>y3Gb35H!}oW%+a!V;ni`VlLFU^XvoSB_jJ)RNeV z6UA(QhZS-LMC=o^X%4cXs_Yu!Fh21T*GrAVg~doYS@UV^27+K?90+(=69F4b!%5nH zq=9jFr`j!UwOd(~RhXESl~|b3uyA^M+(=-?3IDNblvM;ew8$3MEz3rF072QR2IvkR zc@a){p}D7b9b%7x*^Lu66r|Hj_`p*4r5;U+N{mZDp}K?1gVTfkv23L+q)nvtB3vab z*cfrtj@8p2^iBf(3s-tI67gib*`g&{i%N?(s?dV9EclQ@QOsfCve_2XrHDd_ByHr? z%)LACBIeF5ao6IhWoPdo6=hnk3gQt!mL308aY}IMF-ZWr?CCWwm7u(~>W zy)iboiGhIM3^lE_y?m_ATVKfnxC-Ldo(p-Oq^r~GbitJ!yY^qTOzR4&f5k0Oo zctSm|*rK$o8_UW+f;rHW`-ef@#$hj2o?~ZElA(PCspolpyr5E-x$U79HeiCq>MxBX zk%;0$UP9^bEs^W<%@@@ABw znHle~$y$XqGB>$@-tN}YK2|){+73nA6~-6-z7zU6{;v7H0d5C>NW-ncy}@ngd`J_n z5xx;_k9<(+ROwVbt(n<8+c?`v>!g=W&!4t4Zew&{e6XOfpeSuGX)isNnN%FpVrZpO zO&LY0fRb*mz*6j8nBN$j7#w*`y2ae3RA%bH+Kz6Ic88%n%u1r%#2%}rx{9QHXxN4^ z;Dh*!Y9Qca{Awvc@&ei79vO|^JE$Yb4{^j!5|br;@|$r{-FFa`4x{+7>gjI*E}>L0ig z=&`fjFAdZ%$LydP(#TKe{+~C_%O@BkD*VCLBZ{?HDt-ojOyMI-+ z$2NpZ9*&+;76g_Y#A&cX(N^Rs+^Y)Ao|Ww4=-#>0GIjkS8C14%GF9Fyc3+2Ow5^tK zuKYxaN@F@Kui}x{v^S}1&;vZHt)!!D?C=t?uwP0dn%}rA6XB7}=&Z?Y)x}@!+elha zh8HGmHb2P1G1p-JaWV65QbUwdxsGh2}$X(xc*+P>fS}?vJY1KH2X5{th2;LippiN`GCXi zusA>c8UVuZ9bAw+V8@W^?nR=VR_Ap&|6Y~y4h&xNn2@UCUK_NEJM}y9bP;M^K2NY5 z@v+m;a`;1!1^UN@gJ$UF$QQi%z8P~((vt*}!KCwDvP5%>`Zv6txQ95Eugy%OOlPbG zb3^K2{=xi0C8b96dCH958st^=Fqy&%DO0)#qj-=*N9drq2#m7j0qWq5bRF6D69UC^ zTjwCQ58q}Xze|gFD5%V@{Upb)G9O-Ve4dpKSI`zO@+S(IRCcnc2lY^s>tuQf?kX&Y z-I?J7@nq*yZ>a$}*Ev^f#STk1%9j?&+Gq?yxelmnVArP3J>k2Cejf$)WL_h7BVIY9 z;E_$LI46owR4<$?(946SV6h4Frb;PNh5)sEh$w-WwB8~}Gagm$pG|(c7WLp*yo+n| zO6Kw0je6yU2AbH5w4N_^kLWcC8rwGcI|t!p54$M@yrwIdgYo^Cs~zQ{xEXlcxOkfo`YdlyZab3_!H zq#sH|2YSwRN#njg-z*1M^dy$u+za$L;vx@p^7In1ol-?X2xMln#@$-$v|n%Z^K z`)K+UM2Drnd&ql2x>LI!Svnx|Nqs;S0mLy|YfWx(dHTfR3pyE5E6lQFOfy4Reh-zp zQjMX2OG{IMUMH8ntBJ>fMf3nVxD0@vBU4lckA;}YNZw7$={||eT!t{DWGn}5U9_j~ zGm1k~FX=%4IA@9|J=fSLmt5Jrd?1zvmGS7+KabWb>KBQbpDb^`ZPc_~9ks{0c|_{zk|9+*~}jlOkm0aKwdpSkmM zTW0Pf_VYZvrVV~ldcuJ;AXcq?ST5EoMJ8xH?t)9_l9jCS={vfQ-57CT1H~ z(1fR7QtqF2G7$Y{&~Uz|BgWQ(fuUDzut(aLSEwc$PY?b&bGaYO+Hx56^Wc}Pq^*C0 zE$A(CM3PlESzqMP1)Eqz*k9xR30jk30i>Qk5*C63tufkqU<;$cw~Q~3%r=yGMOZ~8 z0u^x?3F1X6l|vTP(s*%Of6SAulDLT2|E!yG7+o@w-JaA>@;q6{dT77m6}=nf=yORYvoorJn+QN-MWxIn6Is8`{vzrwwO+T+=P*TPHWJ<=lE zDtjwspipfu4hoe1Gd1P6WhyICv0Q9gv|06?8+5#0TKO`0%u`#{ztA=HR*EBqm0WQpdf<^6d&TB6LWeZ(bBu-DG3Ho+2kp=TD!noRnpS3 z*_|BJ{k@;46y98NV#?Mqftv#z@63ON%rSrCEPs~lEEo6unM zOWN8>Y#_uVouM}vR$L#cEUqXj7+PM;E~zFHA`O(#ebrV57AGyKFV~Y{xB)i8SOzo` zr|*;jn)m_jm?L9+vY)he7sZ?TE#<0o>uj@}+^EK^QPiA4)O24z;e3R+`7dkg!a9?A z&DpnGuol_#X4;lkcB$-x4FDhUIr82TtB!3x6jfiP7_5_mP0T__=h!fI(d(D(HdSyNORBoYgq& zSzveIhVl0B@+|ix5Pa)F&KO1+u%1_NR8-Vbi5kjrD(lG{f zu&BCZ|ItPSUYfF7w7hdk#jJ%>e1{JN!(I39pP6a_N=bLBATJ@Jz2COtMHe2Imx|3s z>MK&dkW`<-y*h#_d|)~sYP%>LPMbwv2gicA)k|cWV3G^L-yRltlzMo02MrCO8YMsf znpcV(Qja245?9-~$j9^J*pV)-$571>{Nu`1Rl&r5{n6G}>`>(5sVrO)IgmCE-pSsl z{7V6wf_94BsoRg-x%-^F=i`IAn@W%wLG_m+95+}Ne_gEXYuaJo@UGJHnT}V7cEmjZ+r^viY;>cImcolx=Ee1qA7-yWzBZIqsiR9cFdDU=rS|+<&h1DFsUM|u z9XeLxEGwz3W9!VuvT+^kL6#*Vh25($)XwVfrk06)^UF4uf67>rYKP^*0U#{%c4Z{f zqYE-f1kwsi+{W6Q*fmDa-&geBDmoqH4?N3apkCv1dw)Dl&z3(K&c`IqK$zuOg;GjRoo#3%>8sA@>JJ2k^fevmSB3ijTR!@|*HR!E-B?v-!nwTybn-I%0rg z!^F28rY%1z-UWsCKLEH801DojwvzZTeq zW1d&LAMD6U@UtFgy9D@iUU}+syToJg1I>{X;jO!qk*(0y?Lqc4k*~Np_X50+<(98n zMY7=^+Tk6)tY0LMy}NeTY@d{)s96*^lJ~1i6c?NxVBRU>h%w`<_=Mm%p+m}cIPp?} zhr^ff=E+eB+%&!7Kkv5Bb+q3COFcsBo4c#CcM?!TpF_fs;NfZf1qL z(Ljk1wf2W>o9N7J0?2+NYpDAf#bWRz?g)wbWr^MBk8J<_J|qQi9CDHNX4$j!8x{$r zB5?&9#T{@(t5qSmNnMP-fY9>$T%o#2jNxbOk(6PuLUFxEWBfzh^-Ic;?oizwweA|d z83m-~34HbRMT+qhOZ{6>5))Jh=AEK~rC*R+=_@spMBtqS5^tke1*$8lNwS;pj$waY z5vUeON8B68yB|i-L)x3*&R}m?DYj42FIC|e!7aT{7tIayjx+QaDvjGyWNqDcWojMzB=nTy(k3J6nAkPaF)L8RHiOxB|EjLBU}gQ2FUApM z9Fxa#B;dWe(I&(7+J9&}rLNIrMfx#28Tuq76CI*ev}p!rwla&^u%V9)QuTsrVj?FC zd4}Bi#N^Znyde^pU3k@o%mxb6-%_?^#*u~dG^Dg;Ca@-ijPrEXItM3boy)buaV6=} zQ%E&h`xd5kM`&S*&dJf@jdTomn~T+JvB2)K zDCJICu7At78qkBw7H^4x(x)9wgA`WQ!aO?}<8ABYy)i3M^Y?ka`}CB!pkhdYbjyV)t;WqFv^U&(>1k3?477Q}3jreQ|Jx9NV{eQE))< z^q=tW1{M9OUH#1IahXQe<_FfDgp@^Uqie-T&5VKTk!0cMYXy^2YzaXNuHmBWNuTLEB?ZuFSl&LJucLV)&z&BaB>s3k|@lvq@mm0 zmYh=FA@H6B5Ou~SQg&~0vWAE(7GSdxY5G(sf6S7UQ-Xcwd{K>-fKqXw*XQT zO6j66Bn?n@56A#nrMFP4I>4MMV&*JtaMVjgrawTnsBqRcXtSKRXgzGIbB%tw>? zJ1wusS(j8RC-?pak`bu%7H79ZkVVp^A9FE;v+U1aVeP0Sc~T~ES_bt`qj~gZtpxJw z3VB**Cv%J{Tu~aSWGze~i%9;Ux&*Ke9ORf_$MP5grmPxxkqows3|j#STOkiykp!k3 z3|T91Zue`m@0Dt&L!6}<^cX#`PL|bwE2iq5R^!2?{cP4@VbD>*ru_`GlE-Z)J5%{u zMsp=!d@axZQueZ@41RZ6S5nT&B-Vo88?#?C=!3i(?t6h@3FtsC({qWD33HSgQ?K(! z*ELG|n4;>9kx2#Wp?}+q3J>&2u_>HjHFxhECExVUY5&9UC@?vB5Z~oZ$8VH3Eb;a5 z$SJfIajYCpmm@9_mxncu_25V?bQbW)PBa^Nj1t$moDa@r$r#YRy^sdtC^)7goR5R^ zqQj31pRAJ;h|$l<3FPKTXXtv*@5=8a)hVi3;071hFwu|d2166#s3oR;SW6gSH;M() zJ1ZAskgIJ6!U;}}p=I4H+RhhEgJ`Q4QWm|&6o}-tbpJenN&}IWixqW5>Z}NE7r@L! zE0h*lOi)xCF76|y1G^mx6)*fEXhI~9BR_OdKKpKNE~XFeuZy5)9`pw<+x#y-u79Fd zFbup9zf?|zpiaKiFWl{+90wUaW&e;B2olkq;4I`cUl$#v`Q9(`MKE51L(Usl8>iQ$ zUmJ&4GhY^N56qwO`+pB965ES1BExy)^PVtu6oeerJ211~(|VlO#Wj?hQEBZh_&Ayy zvOi~gKQ~0biCa_NmzM`UFUJntJvcX%xmlNRZf)=@U3?w^JHbdYM6WfnliiA?hcX_W zZGwKSt0O!!xKGkYI$(ZRN1T{bmTjkA?^}(|5;mgSFgNKO38kla|Fl^v)rWXNr#`2d z{nZa6kGBWbTiE~!nfW>G{=_yedC0a~qtj3*c8r02E_RFEbM4++Ot&00!<*#6H!>QF z;`%!M(f&P5E5;5)mt)OYiDBM2((hr}8R)E5j)yE(GFx@iF`Cut_Sh^mHZ>!|*PCx` zaW%A}3N@FN)aB(L$%{cl>%P~s z51Q~_8ta#8SzAAMef!VXn~B&Pj*!TbHW+I;{5~?~yvkz7s;bweP!Q&2yRoMV!ON@( zURPt$be|s5QIBYcQVs?lrSKUTy)ieovlgiJJh6ZwcA@rEI+= zYYmNSxgoROmfgS_fJfY@u))<~zwPJ+eLNja*F_)f`i1`zmxTCF%#{Br)c9|82NN^n z{~1K&tSD`HNQcnYd#{Q-)LDE z>i%f*WWuEL<+l5&nIF4H7EAEyrxP0zsQY8VVOw9ugs&+#2*>A_R`|{6$(nJiuQ%CH zV?h!ME?I-x8wF&t(kl*{Rs2!X=D79%kn1D=UFFj38%O=*lTSMUkNf*4H1ha6c7ST; z#xA71xBp3@(iDw5MBmIGokU{0!^eX*jj3Kw#XUhHO&o)hBV7lDk>H)q{q49KH%f9x z_Y-hKBfF_}y>4$1?6Y4>{yY%93lmVJf>Cp)ym9_}%^BTJLhbCB9oeoy_?pT^8k)*( zdSyW5*slBbBh$OhrO)7r2x+|8ug8)tO(eU2b8TcW%Ub6;(KByI_nz4pSC-Y|=xjyE z?0`a)j>ImKG#nv{v4AUI92sa81c`APK?-St=!8Out~kRyoU}0G99sS_0VRS~HAGQ{ zEWLkHYXyXsp{pbQ`WD)+1q_`2f&?LvghUImI}(Glu=64_dF@GIw(7y(NV*KT!3&$R ze~DP@sHox9^8Uddb^B*LluhUI`Hb;?q$K+<+>EE^^AA8ZEZFFOI2BC)LE6sn|3UV| zef+JFa_rW@B^v>VhfwrU^Be0 zS|C|1pqvHJOPaFi9dnjz__B1`f(1}#b-nKj#twmJ#|Own0l1j!dQ6?oE~tJ-0C<%L z$bg^8Wi)*8tt>;sK0WQjN}g!bV6rgi#(<#?UZb(?Qk%3O#9P!j_73!c?wvK$%9 zotS@5fCAWC2xd&7PB}NG2S{-#IMqg1!L zdWyoSX44o2)W_kocQ>h>8ozlpeW~bV@O&0d=f0l4kA@-_YdtS%0uRY!UK*CB)O+i$yvA_j<8L;y)%=0{QH&$J}K=_G*Wp$8$NhccoENEINi1~jP$dZY(i75H2S z$kMN0hZz%MxbMb>*wSxq2i65b6PP-fdbTLd-Eq6)PRBit`y7ut9&_w+>~_51c*(KX@v7qu$6JmAj`tiNIzD&w zI(~5csq-6W zpYvPich2L^Q_hr=cd1+km&Iju*tgyT=(F@)nE5RQMr0LEYr z%%8>aZV|^H4B!|b97_<61ggN{Rv!;T)uF~^UN6OLb; zz{xmW&Y{khERLr+S2>>)as0gVMdvHdH=J)f4>><}9?jzTN9PF<#|oF;#ULE#oPp!g zKf&?zKg98D5yueW7^Z(s|C0VG-IM+tC;^uK4D3t)0BX{orjMmROaCX$r+>jg`UnW6 z51@No`a^W}UG$_Ml%UUi2TDN_9etkeL)Jj1zfRwhel2}7z|%LOPo~j64J_&F(YK`1 zhtt=f<6Y?JxAaD!2AcH!>5laJ^oH#AX>q%5VEdc6U7Ov$kiHLXSEtvcze|6g-i>3bp0*&Xj^bT})IM$v%0qWBy(zmCNgK97&eH@#I%}M{5o|^t1 zMA4^iM(ur*u0_3GfWS_w)1!JWB8l+?chqjmj4I;BZ%|O zAfGP>KDdxCfEU7xPz=}v*TR2-`%^!sPJnAt-<|p>^(|PI+LvC%)2W|QCsM!hSnB)K zuju&K)Nd*DWq?hgA*WtW^`yGf1*sQOJ5u+g9!Ncvx;u4$>S44#fVPjMeW}jWqp3Nm z^U`_gAX@$D%CtMpqyuSF+K~374XN+=7x^dnRC+xBDE|oGlWyc6;P30{?493pfA7qm zdwZw$?C71)b60Om&mFzPdT#4&=((l0rst;Ks-A7VWj&jEOL|uKCVKwf+t^#*TiZLN zx4O5gx3agQx4gF$-E(~gdYrxS-ohSJZ?rekBkRrm{P^c5|Mh0Cx7YJ|@4ue=EP3?S zqqiKr|LE4E_Z*#h^!B53j;=Yn1+B}Dj_djPsPHztC)*JudzyJI9|Ls2l@d(7}5uZR|OytQK zmMrjy40jNiLxwB74AKKEB*Q^giEJaoFa&;)dFqv{j~6|B`ZKr=t*hYQge_W^XV$-H zK?GtlI0%aUlZxLg^w999Jv=<9kC`X`w3pyWwx7SHe*(|_m!5w1Kd^QJaV8oRiv87{ z(cfEe>9rG<9Zv7Cnk=I{tJ5e zP-KmtB0l)n^>J9_nMc8izor-9K=If9Kh)2_cwi^cgZY31BtqBe-U-+jxO9d`pPdN` z?D@rk{tC=l$ow)qxt}BVb7LX5J0qJrOAZ=X07S8t-g+|=U_Ic6+BaVaEyZE(Fn() zbrf2M!*x*5bt76EPLCc|ra!~pM-uoJ7J*-2BK_R|7W7Lnq4p(|vEW&EpjZ(T{;)Hy zzxyXG;Rrz_kN`@=f*dGN#-tK*9^|jJ$Y%5i9Y%yPGhl!PSb+`XAn7@fRk;8Q+`xla zfDd6Yfc#-D$O9n|1`(vQDDwIR$nxSK0g4b(QSOB@urg2%DnKQuLRk(6h9J$>f;vzS z8W3iNf+jEwG=t$_1Q-cMffg_ti~(c8I4~X|Y$BKhTES#61)Kxgz*H~|Ob0Vi|7SuB z;$RMxfO*I-FMt$CgY%J41dp~D&W3ZK5?lmTPz^Ou2bM!UxC~qlR)Cez1kK=Y;P2o{ zXaH-#TCfhR2OB^KxC(59MsPKjfLFi(3_>5c4qOlY@b6eLRsud3a1P@zA=c7XBdsUF zU04Myf#s*GDX;?8fJb00tO7gFj0Uh;thWsnEDJWlHE=EH5wX@Q)GffPumz3=`vngT zCkUP!PK1-eK{y4T1KZ$KI8CSk!Ykoi@EM#3S0Y~y$cxZg0<7eCpvM;?4T1Dww0?os zANa{>o=kuUe}DR4cn*@*ZP_*R$D-a|htR$Z$%93b=t4ToL%P7xNJb!O_dpJ55d+(i zY_0<*5CR&(dGILujt7ve-$c*(knV0k=x7A9K|A`qAL(-)8toCJ?aMz9U_>?Lp*PQ@<5A4=DypH06d@>v1N5rorW2!Bh#Md%xDK$_bHeuWb-b=r&a z^Ihq^>F-4AB9R?9(Pt_VW+$PZ&I4DYcDvAsKR_|$Fw*NWsD$Nk2-4{qxE1b&2e5iP zg0I7O5;W;d|AH)uMRt>imKc&(IqLU02-h>vH~$0Ng*?)u;45$fGSG|ARtIm#>{u=~ z7h8mNW8V=taTU>%{z~i#hBQGTNfx1=S0Vcuj#^p>EYoy;#8QOsC~Cb4_3s1t38uzOmj*LLiqR)i#qL8a z?-zmPh&4VMEk>Aw79WftsZ}AZB3pp%a3!+1E0LvLjrw>qd>rnEuOUe`Vw12oYz4L+ zdj@+4JAfU*zQDf0e#T|E9cPh@UnLre$;5-Cnw&;nMy?_skrbzn^GSXpKb3Fi-$+ww zbviHIo?el@CEbOz`n@PgDe`+}`ndq?CB$I!@r&TuLSAc5p;L3qxCg=j>LXhh@ScsL2!GqPql7tV)E;HB^~ zG{&pYay7gLjZ&zyKMG%hU%}(>1X3@FDKRbP!oo;1jo5g!%s@C;h+T%Q!!~2rW81J> zu^rewSSR*4_BxWrw-}Era5e73lW3{OhvOsh8Tb1 z5V!?J$QzNR4gPx;MT@`sw;eo;y@Ayu-t;w+z?I0mz6uK9!=QzjjC^7b%)x%cw!qgA zvYNpx_y{l|Ux$&+z>Zc1zX91bNPmvjKD2(t*C0QL?(xs258)2f`nl;tdK9|?!@a)SZC&+BosE~mquW3yTqv&m@C z>$J2+tx_uFGASh?2^@nUT<>X^?&|bU?76cgso~6LI$hJn z$6RMV$)P9bp7kV`d6FA=64I^;P!S2c>OHQ`H|jjDE;xDgBy{|bI!~*svrjx8As!Qc z@kE7ASQdT4RnN??b9KV$uKLb~_W2$4)9X-syDA_0_G@`040dTvl@BR|mODZ1j1}`4 zv=g4F@3eaA>N_o-I`JOvtDiBuvt{(8`Z^oSwnoC8uy&SbW+(8}bZSD`kBcpLQnj6u zTGX^_fiM7YmFv;)o{o)OG?+O(q@3-UJ>#57o%oDa;VZPEPP3=3(|lg3s*|zwYCQ8h&Yg~AZtds7tjz5HeP!nQyL%MkbKOVLzIn`m)S%E2PZJtNr)!oA zbzqVQ4W&fb%qamKvr5oo=&uzbAuQ;W)lToA%LJK{KHB5z_z562diuUO^U93u6^W1j z2?&RRvIhWz?)M*ehC-crc}V9-KXZ_pP}i!&3q_G|d#BB_n09sAkf=ZllH-ilvM7=} z%L*#Hs*3|N(Mjj3(UUUgE-=%!6L8T`YbQ2cxU;AKj&ZzjXI1~5fsar3AT&NM5?5oV z)IadAp-uYw`DLBZ^j~hz5$|__!f9k3~mCVOIl^djpbotE?s08RMBJ-RWcqcw_#K6cUqv>i8z z?c`c*1BY{3Tgy;8#X>uT*iJ_)YT?{$3+IY0paxRtp=7ujDcRpLY4oJdRdu#buC5g+ z71_rgB-T9$2d%B>sgi-7qWy{mOt#OZsLv&N=%6gq%2?C}YNEBHBXfpOxTmAT)*%>r z=DZ7nvoCPj3td2L0NFwpT-74pTIFGF!UYfOVNthQ>(Ez~Bm3zGXcwsX?-H&YEa58D z+e%Tw)&H9$%>5M!5BW=oD=VVjeB44WLyoDegL1 zMwa}xuM6LIsw9(nkLRz#R+Ezv+bHJbgu*GubXtWTbisDcAb;sr>JbW~yis+v8dF>J z>n~^lAoVwgVk{Vcx;Yayqi?@w?_xk4|~$#I=tp%r_el)toz zx=rYTdV}F%6i@CL+#jUe%u+|J6LhF*Zd@lTjInVpsI z#ulO+jfBijquCVQe<)R&vR3w40i8no6jKnZg7HE`wj>%DK>a96_@gi%7Ih^OMK#+; zIrUX+xm>3A>8>pwQ$1%#NXv$uW%()wT?+U!@}C7aAcl%$#YRo;sC8&eKQya z>!l5{#+>Ej*p$3YB<>i zpM5N-{8-fmALhb(#!M1~-W4=6%vCyqAoZlnsU+ASNoxo{QA{)t3z3>xf_1ae{TN*V z=_75SiV(dA$+dc&IzP06?hXm6X8`?EU+5IH?n9XCgXjTqE7j^S0T-~X8YP`cv?Anr zsG?*Qtj5d+2?fM#PFf>QZKuqG*pCtZ}F%- z4Cy4un1aG7iBfVhDW#>kCX|stlvx7&7Nk!XSjH(NdWqAlH#-TA>B5$BE*3xtr55Q@ zEopZ!mZco9vo0khU7XU+ssJMT_i{AL+3l=@TsqBgs{u2h5g$3!)^`w@kCmZMR2&fx zw7ODIjVz0?rn-;psqRDPG)Y;1wF+ZQ!i;Ho+{n`iW|o081&wTBang*lX4tu?=Hd;P zEFKrh4~@8c!i<}rxM~OA4%dty>t*Mrf*$zV#cg*jJ@>li+!30$TXGLwx%4%3UsgO! z&HsqT=t%eByU1Cf2wcIb2K8Vg#io5E;c!#3E{agGhQ@Y@4!*k)gT!dO=j`H;|_=xeoa0T^P6-1b8g#thg6}* zpz70>1(oelcliS|uDoj|cT;oTh+FRdZens;Ubst3vsQ(E_jUI^3?b`#d3N0;H8qNb zv;VQ=k`eJT$`hNoY5$G4>zpk)`Pn)Rz~4i5AV(J6!O;}ls_D|;N(H4*D%C2L00XN| zE(cPL0;i~}bP^7*BnBl=u2N%~E;y89WTaF|K>$e!rIgEPNZ%@f5{Z_M?ng2SRX~P5 z)GYwNLJ(2~idq%j-6Ty_R9GwbBiV##a|MOA-9!Z~#VTNa#_!;9%7c6KY!W#jBa5#Z z|Lco6`={MLJM~2ndxuJGf;-9ir`D|Dx59-Ru#Qx_EaNxr>2L8qay+nub2&>$?bkLk zt(vWJl3u#h1WhK1(TFTog7p5R>_xlE#2JMV8d#lHC)X-ztyYOEms<2r-02YYdr;7B zWt6GN?5IjBQZWj@eW$8Ly|<-=-kJhgPb?}n;13iD9-VSPL5u98#Z!x%!JA4}f3$Vd zJBKi3j1P@I()ak|{4YAKvnd$9_&-m+HZ2)&!adhM?)mbKPoH=UX{9p#Bauh004m@I zmvUoqi`=ERn06d{Hu%6vv-K&3Je@w3>UmVIbGAgxd zvs1;X?W~`~WZ2Sa2*m#2(Ax~q&>&mt4hE3dO|^HowbR{)+R#`8`$SL`ht(c!7smzK zyt=Gmif$OW9SZ%M2cVj9$iRnH@4oE9$3{o3vA<($F4#G??Oq$zSXr|2gH>Cc%%pNSmg6=d<^`^34HQ0F6+jM@Wa;(tSiT(xC#3 z;M@>!Qaejw_)<>6$_W6KNID9sT+yK7C_5_^<6*|i+7B|_sQ0B&0d3aGZYz`8lQlqC z`Efyv^uPcsn3^8Bl!8szcNkwfbH>P=9XL?viPy=BiF(KwGjAZO(?8%pAiGck8%Tnu zxjA8Tg?XfTedtL)DbFE1%7S4z4Z(&)W6=alYkq5Vo_wxlLH>g1GNRqm0oHkRxyp#l znAA8TjT)K5p^-%hiA}QEP1-icRQ>lfOCz$Sz(G4OM_t6h`0y^M+UYB0(2i4U;?8(9 zJ}SN_PR2`1&>-7V$lnO#Ymc@uDbeqg3M=9Qg4WCkpwVc1l7c7{6{@ioo4+L{^XBA%f)LoZcG!c2(j!)JM zz4Fqs$`dcY@c#B`oALAV>X)4J&lO|FZf!pG{pPjh(*uQb^W4u}v-Misb^E9*r;Xpf zVa=pi?uApQp1b|aed`}x)|8)l7Zgn&#=j%y0UBYvof}J{za)2!q<| z(|H}m+G1V8u}-~4C!=3-6RZO`E934=egviVK#CV>XVhdqdE?;$tO0F%c<5yv3OrPrG z<=S&zCBVt$i<458m8lG68n+TEkD0Cc&VUPJMcWhdBjO2^A1Qzpl%jZi>kXmbl91`^NH!?fnmU5l8?QHAAHGldK=97+6QSSe zA453`on9|@u}-dZsMBUI4YC1yQ18@P6}hh(Wq21`mxrRtUMAf-kjF)M!r)53lmrf^**z~F)bBe;?R zNcCBj6tRWoL8eUPLTvo4ty}-Lpf)O7^Yk+pG`zQpl*H{-JT}ypXFDk)<+bNv+I90D z`p1Y3=N3)l5AmtgYuLZT!(Y2_n#INYQ>_QD-w}7>d`!!#lRAYWtzZf1+a{kwp4aSEghoBdfAPdEdv3Vt?;|c}7xTUMrvd+M?@2h1 z91~+q{Cslxunqhd|KWQMd@wx!68qJ6!nzM&G2%ro(4XUzHz5quAP?Ne%~f+kvHbQ&?EF(eO%eviQc1fG>}oK7-H6loYDEDVX-!Po~vIYWnz7{Pfgk zL*U})UwIx%tn4YHBS#%}WzmycZYm65Y+`i9n9H~G`#<|7lc%0U^3oxBB_PF3yR2YC z0Y=7+@qpbMPuiQLP0U1QfpWR@N#>5UMH>++?F@b6OHzqWd>bH!I zN`IpUx!NX6LaD?O(nTt$Dkh3hZbuMB^5UYbptExBZUfcTqKUULeRN-{hfb~NX6WN> zJ@kpTo~+~)E;^_EJ}v(Q=(qK~h@%yQ zyEv9AGGQ2s!M;$eEVewpBO&qU1?+yG-D~#=#x3iDPjCTFVQ1y?1G3#5`9fZCD9?_Y ze13nvCRmJ{g84Fii>45HI(Zjt;#4J}yihpLrL)^@CCTW1L6B0xeKPd10kapf2?S}{ z`i1ql;5G%DKQ1Pjg9VxqCNP;DFfOK(XE*?{yi^VQrEEm(775hj3W$XU7XPfhp(tP| zOqiW>J#^vJcll3uUh~$T+FNpT7MekWPR_^N^P5bwNBy>$|LCqwpQqsYwj#Mj)SX3-4M=^3$VtD8F9vFrgbci<4Ul*)|tGS== z&bu>oUt~wYj>Mf6PdlCnzvk!x$B1M4V}@Q&ueUeQ8|*FktW+;GNZmupp}L{Qq3-$0 zxuNCCOF~X9HixzpY)M>Sv4gl@d8g_=`wkCdC@d&2>f;HWN~gA~9B!xG z5iW?>!(3$pGNww=nCwR6hC-e^d#EB&Zm&obcnBh1={D*}La7hx<2n+x@}y_42XlKo zF5Od!y$KAps!Jq9J2U9V6~-IlA%|V1#40PoA!z|o6*SVIfG*$)S_(P~_7q488mbGb zr-GrcF3#gV|UrI=bIE zyWefks<~v%ucD&g0?#r)qfiIMS$&2s_G5+WFT{v!QO};fpcgGsEw7)9MaO!pEDN#5 z*>Xt%TLn*!uC$)iV|e}K)Z3$dRo3;Xu6A@6xhR07A};qB%8$$F8EO^%I2G8D#}`?d|L6e@-@Y4%6+PR zY8g;zk=m_D)z*yP)G>C8PD>Jm(i+t2bfN;0+N>674q+3$o=tv!HxnA<^)lJ~pN5q) ztxt%(XGt<+WM_(r8=aAhO3_%)%TLRpz(=yIBHDJUJyt~*x^Y&lX&y#7ehKh_)m%`j zmYV8~_n8Qj)r>-?+F&=TWHzfBHwK}^C$rjYE}crH_I1H`IjhrT#AHFIjI6{~x-BYy zP)(~;e);*w&KTdw<0Dc%f{QsFJV6Fn0(oQ3_AbRj+9dj5flr+2OC*^bIZ+&68Ap-Y z8MDv^NGg@(GcoCyO6!3{`GUu4*vO5OfD%`;szEzNH9X=ELl1Uj39h+zwZnZkr6irFFnXYBbNw?*ppadjxCWZiDR6FuY~r^9gqeZuL|>2z9L7Yk}>LTA@S zbhxfS7n2jT45#t2IQomp3u0%vtAiQQ?Gc<3657G;jPMS2LyM4tS@*YDnf!dwm5C{s zPz}@Sq@|_k9}O!+uaJglrdul3!$>;RCdwCOU;1J}w)!|c(|Sa&15y8RPnHJ=CN;6C z=%jK%IlOogg*^br3udKIHIGZZJ>FYkfh3ipJeN~nQOc;JmS4as=0Ag96!Ag!6oY?} z@i2i3rdV(=l%72Yf8yk|MANBTb1Jk+lUlUD_tKw`ZzCNTx=k2CX?cD87u-5niMf0(V96{9q2Pa1D2jcg=S_;UZnGd{H4y_&9tn z{v=M~c)mQEf2LBha7GJ-1dQaG*=LH6LRA3ipjxO(N(E0QMTA1QkMw{bzh{s5FD!#K zkq?nFP(plWEI_NEi2*%?CJim>!+d&uJRdf}9n=AtJZ7(hTib>tPbw7#KOd2QE>R^M z)%<)`J@a{dvZP$4QcBoUN?hqFQI{B11j`yGzi{gMP+$bncIqKBr!N)4fKWSWPk&6_ zh;l6zuz@zNX^r+u-Fl|O(qSbjD3vn004#!Kva#3{*<{^h(|F4T+L_G7@KWiGx?QGM zb$cy`bRSrH8I)Vta8=MCled_2@D|#h6FqY1_wQnal)sgKpa15L9b}aM;&px-|8KtU z)R(Xsx}hgKCzyC2;mHFEz_Z+(ju4b+gTV&X(BP!-#Mu1s<>4)=9opZ*3X8QM7PEO! z@baigrCsH=v356MGkM5ZBxo@eMEsTlGA4L@;51;K`e(KQZ(weoJ^_aG{)(RgU82m@ygNI^niW%VGtJO<4~)Qf-#Syd)cF z+IQ*Hrzk1k?llKZg?@s?UXtXE{P1t*>U=yYn)fo0ku3z*+1DIX~XU6xgC(T;OTIlRiMuWD6*Q zC0y$`Fi|?*I3;JU^b+YM!ArxJM*g9c%XC)VP|%ojCAmSjLBGwvJ#e4@zQ7^>zx}FU zAmml4W!lgG~fvt ztOS{h1?ZwVEJ2Bv(2~&NcJBk;-QL$xV_n$Pr|Pvsi#W)FE^GlubBViAthr8hm+Wy_ zw~UaX2RK#z^w4b~EM(*Cce6R@v8Ooe-K;GsycwepirLstnd1Aww)Va@M&u9TwIhrb zCB(wD(4Y`l$bAWfW!CLV_JVUD+0lO>Y)dmVMp6Sa`b~*Hcu5D>mglbx6it~WYn(0x=){;v6u0> zR9<)|>5<&@?EQ26c|+nG@7OqI<$Fn=fuB!quDxW*`1+~i_ZDxMa@)_%Znu~dF9Lw1 zkiF%ATR0n)(_o&Um|$KnyUu*K`IQ_gr@EUxPkEIRQzBVCu22lq$pzUaIO*N2OhzhI zQkhIo08*#dk&Of?$AHytlj;RZPU?|IBMH0cz3g&_eUSHhTa0D`sX$>&nvx1{s!B_v z-Q6gBy&cUY#dQ1IQRaA?s15Ort4JF%Z$j`uQYa(90Ry;fim%E7bHCX-$1|R&o$MvY zuUyLaAcr*U{8Xn%35e3h$0=CO?<5X5F@TOVB?S7_8)_43%x*o!ed))b4&)ZU;Jcx>_9pv|gP11V(= zsE+m}zK-43G0&jhRq2a(~u zsZYk%Eh^Yn&3@F2&C1Q!mWoV3-}+MO6yLLKS@*WW2NnvlzA1f}T!m!q1n+S93*-wF z%jC-xYxHZJHyWr$Ih4tya)nAI2U2JU2F7SN=v6wq-f45#on7fgTuf%QxDe=EI;%`! zwaHa-scM{#VJtRAC6kg)gC1b=w~$|#*<4bKRmFC}lTXUqWe;2Ku|5O)fJ^-O3KO!> zCzxZY6PBJ*F<3I$!ed2+xwOA-vrdZSEQmAo-(2^KZd!_EQ$}da0upiF)51p*Mhr`! zU)Y9W?l=ZlE8)$~oM>4CygVGTrHJ4|&d7MOge5dgxl}7XwFY~&sAQVYXO*$6%o;hV zC-R;hE@w0E88@X5Q=C|H+{&G2wv4sRw6t5Uw2&ILMx%(DXp`0&_0xWBG@KhPwUkAh zq7#^9;r9HF=oadF=~net%@*eR{2O8nV`eN;DJ7H3W#MpsD5L-s5;<+w+G%gD&+gSI z)OL->>b83lvA8|al^*jL6UuXSrAKi|X*AzONu@4bR4R|g%nakwg&1oHaUH7w`@Beg zezi^}lQS|YZi(eu%?de1fk+oz#<>y+uZPwE%<^t1njgxO#iB)2zCs!a%gs!lHMh76 z{=lg$?aaem=$^cXUE$U8F8I-tvUcgi`FBU3fnTHOb6SOcNQK9l<5o+GIbjvmDd;o% zqC%U|YeiPGXrB_2VWeR)KScCf5;0iSr*$6s?}eBF07xsO2m+XNkFaJ9&zMPd5DGG^ zI}3|`R0IZWQpkb&v2oF~1cc+I@EpCObOQXM!5YR}D7ajs(B@id;pIhUPdIgLaITWC z$X&tY%|v)13x~+H%2OX>dp3>`(@!;1MR`sgz`n5C@}0#x1-hozJI0-yfZutljqvPm zB1M0fei8ZnN`#>b=;Usd0jXaamnLKtL4PoDW0_Pcl_79gEELkK#qak80w#o$cyV#E zuu$gr2a1am$)rzb(%W?opVRJ$7RKz+3JTaOv@+VRwOexR_WVG^o?o6UvzHh987X0} zKv?tpiYZ?*>C#b@MOIiiPKO#!76yD4lZ=v9MDrbXtfE|}l@461s_Z((h+gh=9nJhZ zk|k&wsT+vi@Gq$B57Z~@)ZNkwN;s`hSDd3ZY;3oYvd65ftlr3!yx8t3Mg1s;f$)xdVU+6j2;Y_zY;&0+tp5@sO&gHV2 znHUzRr??24-{4CY#ALNq3gM(FztJiw8`9uv2wLeH*HRbeay7~u8_uTEnHPycv*e?hK;N8LuM6eA|#3>G8`ia;v9$3r%}j^8m-izu43k| zEcNBuRJ!q1O|^;G=+)1>_%?2>`|ZfBElvOVc;~KMlUF@lC4(wTrqZbSZ5NNTw^ zN;RcdbH+8(5&`eJe8+{wLu|S=m%iK6C+a7d{sMoMybO3jA-IY2Hby5z=S7$KFN$vP zcKFu&?{nPi-{nwH{vv;)ACDTn^2MMF>batz%p-5s7`>rF1JGE45wtfNojR`StoiW3XY(dZpy1wZ zSBr6bTZL`HNX+jOr~U0exWrnyU!eDg+89wZsr|xSzx{3PC?nY4##jYGfPr}? znUP}s7&8yX)NkjP_C;DCQ%TtyhFrLD?T%efy@#*6AQ4UEyR8*6orT~0 z>XKEh^-@!7Z0rNqeg6D&A|@Q^FNk8Y9ocLxgxsUeiu!OveurXz_;BP)oRTX{HJBoy zLOH2CDJxNw`rM&#UVf-3R2)hcyUX&+i^_}3la201PouZFxLvc3y+N_9Xj}0P#f~9| zBVQ4{0YWc7-rByP89?aY#WGHlP41Zk7oi=4$~wXba67@H|qsT}^qc@zJ>Z|}1Y zOgZ+!3zaP=j@*IW={FU@ZCaZ=V8Ad|jLBb?@dT(F88eIB?JV5o}*vth2G-LTHE!FPk9n>a-D68|BTBmucx zRxWqb?f_142G-a}*IE?0sO)L<`fog-zx`3p2I)#bg4>y;i%?uu_@Nb zxa+r^dmFs&d#`86ioznNRuBLE_S-`nH@#t|atnNuW)|9;&7~FRZQ1zjdkg%LMGa$` zgK>dp@Nezx8s#4t@kPL8Qa5h#GVyt$EJ zL?iJ9Od2wtD7M6%U9g!m<`xAbd3p5o+{L-qp-ibU_jFlC$az8)h_4~Qok-CKQ)03d zqJ^Iu^z~1272roA{B;k!S)!U1*rk59a`=)R!{-ohl!WYt0+n2Ld)txP)Fu2$ z{<)DcM6mO7~-tcYPHM8j~RPChL%w)EWarQr)pZX~wIRdx27S{H5#RN-Yt*AP#jN~0`D zBTj*b>gy$C@dp+s zHzl#;h~d#A2itm5rO~!N`e0jsT{U&WYUvYap;<+-!hXXYn8qT!zswdnz@XW}#_Z&u zd?;2ngn9ABNulh4gzN90f@+Q%J^clbKz8D0*4@Hcde_*=|0pY4ykXnTxu$~Lmo(Nb z7gkK-J1$&dah1A;a``3SJUnHkV%=PcP(bo84o^-nQ(C)zdQh zmM{Gk{w}!$RD*-uGRk3g6gwIm6CLv$YaA3+v`)6J80Q?Uv#}a1E}b8&_JC$0FV~eiA-*&aeKGYpH_G0wBb8sTY}t5mtW{2}lSpZeGtb&+ zZ77&HDOijX+b~>1TWa?=#0na=Lk`DEMv;{=1!*QEvQt++J>j~_QThrq&l-Fhtu|zv z!px72FGK`ajSZ7(Wfryi{l%j;o+6%A>7*eoA^6rA>94RHk^>4*0JdWcs2JVT9_V805eWDMOW{7F~KB3Tud@&7w ziuQBV(?4@DQTXXst!C;hk@`IuUIqVIR+gJvUY_fKuc;-Z!Ck_}tn%i95lx{|sbV8e z6pb9)IL~jhZfjF$e}6?a<7(^D zpNzGT{c`8?t(POF!O_~$K4zVhZsCy4$`_JgT7c@R@epq#c9miQgtXC zcIwP#M%sc20;B^)MNxBrGRqBts5z+(@{RNx5xbXA-klX)0xsR#%%&MQsT#H?0z&DhvZ0uQ;3tT{f#pxUC9Lildw zGwX*2iXVR4|;do zceA_QyS;CkUNyhUyl#J;eck=K_f_9}dGCdK^Lj(SdR#IO%*nAwBB6XHyU(}Lv=(in z>g+zM@pu`V#lvXKPBUiKnmlIdVmLoYd8ExMs8KONkJlN=&(YF35$N$2XtXpvMwd@( z@@)_Rrw%ocZ`0C*Sp^ugBpL*AXELA=FBQZzbX0h4jGPe6q~jU)Q&F8Qrkqha5krIS z^nB5b3mPDJnasN_v!h={x%P@~Df+aqp{4zYNX1K#8O5`1CaX&`J}A*9by@b zr6Dn^{M3YkQ(FB0Ue`)Zn0J~qD2buXFh_r)$AD2;*v|@l039)q`Z^k8}nvmYilmOts;E0fo zv8T_3C3`C7K4P@S-Od;rTGVpQqUcEdwMeHro6%q?cglxuZgiR4B)JL4E&AG>H5COz zwm*mAMy7gKd0COGXxmFTQ9NSKXVQxo&00Sn-(X^gyO*!Mzx8&m(KJ5wsmi7^8XQ9z z_1xvOwNm3YVlgExWvs8wKlh(MVtbL*Ft&H6v|cy)K5VbTpa{^on=YjIS$AEq`7~#G z58;G^TJByl;HVA;d}XDE;@op9Zw%ZQ*cEs+@JjiIfdi$7$~8nKQ5DE3t|+Q(tZr3I zsJz6nPIs+Qp(vIm8EC2`xPU9z7_^5Ku#(j&7Rfz|W=*`%VTU?NsaIR733358*c7aB z#0v|ZI)~kX!U)GXb*LYuNt1@+kSFujh81=8+4j|T-0pSMIc7RmI|xT(jom>z95r%C zgPNg3qV4+`fn@NFN3wc5i(_uNoC**mKW6uo+qLWjCgI>&O-QC;DrZ|4 zMMg!+wSF}`7K|$x&-&4eC*J)b;#h~0X52Ir*lIah@r|RRjXlwt(~|`wNZ7W z@^#(ImczRDEWMg9O{y;YGANb9dkk5ZUw&l*1=jl9cH?CcYwRG%$;iHZnPe1qZ>V+o` zJldQ;vG>qh*yE2~dT{^2+phU&-x~t=nZ-}W_uw5M3@SmITU_R7D6g;RE+D8JpEj;7 zwih{?!1D4})bFW3SNG~qgq3=O!Q}SDi@jcL7zds(7n<(P^MurkiM>IUN8L>0UT2sn z)iSZ5+i56_mAVP9p-@{);J830wXs?{#!#gJry?9u7k9zGah}RDmyNScw=K4Ylzq@gVDLg!mfDR#MImtpig;MfZdL zq}}SQ55mvjM*&~4S)zf1D|!CF>EFkI4A+-n{e{+>D=O8zoGggSb#e@hol|foQQNL# z+sVYXlZkC(V%xTDJ8x`EY}>YN+y3XPz0dxyYVTFm>!ACf57txFUC+JNb*W0)9*K#V3Ldp5Os}=;cIvwN9iP>yU zD43W$Xxd`b7L~OF9+b0rT&sN0Yg|Hj#+c4pw4fht=*oQ$hG3RHpT5sLj@&qr0 z%5~;C;t6?dj#i>&LJesXSepwk#u)w*8XI>7bEx+4Nt`B*`&>`_@ z2~I}whMw=|t$`k$%+e?+gx762h)%a+n1pIniGUfWw7wqR35>$pUlv`H${_>kKIVWKb+;OF*bIz71&B(150!JtzLM>f{iw zwIL)VFRi-(<#Z(n6JyKP_-q(Nc6g7gcA}+K9CJClg3Z>5dcqSdcY4C3fCxGYXrn31 z(ra&AeBmtr=}5&G$Msu71gdQ1Sc0}X=dwhc&GLqtwtDTnwZY-DwyH_Y6ygeZQq0(A zHH~8nHM}Pk>_ckWWacauJ(FJZs-7m7L(ELxk(Vi&j3ma9YD@;VrR&<;N}Ef&O!TjS z*4;mb)KLCjtyD$X&hgXG({0nPCpafha^=D0Bt5L(mZTgM7mjR9aG@KFS-nS|vBnOY z(R-dOM(o@!90!XAShn4|Ht|q1?5mbtFtk{M7L~xT<4!?eCiEq_#ZaPrdw65tXlS^g zcmyB*F)g&A9C2D$A}C)QQBrWY#Ph7G-;6? zu*#JzBwn_7#e;2Fu1`MZ?jCR4xPAC^?6`TkeYAIk=<5XRE|1t=kX^`w2}vi!NN>?F zv@Q@#&b|P_R#>eSqHrC~Opjp5m~}g^h?Sf(mS>vldP&t%-)&UlbS?K`EP(AIXTy z=n=9xagjS-B@m3->qwQ_v*sTr`Wam-B|Pe35&TRUb6fR_MZH;;R7ni@RC ziH>k8pZSaXL!F?N?rO%xoJq${mk#IZSHDnUVJ~HWlR2$p%ErplknqG-h19xHCV%@= zhv^JXDR*YhfRg!77+YL{2_?SGb`ADPYv5HBr=~Iu`Xhji!AvZsUzRm z!I6t=wga{ugR}F|Es_Os>^>XmDO9Se?z`577wW$@5s(k~X#+Ud<4o+WCXLPKYK1_j zzRCoG zs|szPivHmE6wJHJc*(d$^ONH)T!_$3le}eQ)hlUBjlVhOYZI`7#gLT<%rJqwLtab)}I3j#sLj_tW{rvv+m3(I8&&b|cah#$nF9LNZE&knEv zEA}QC=Ce>=6cneX9tH35a+qv@`ufkDyqTL9kzWOLIjN@4RX=3UF+EYzO=Q7dF_cTYpyjpmZtrjhQVSeO)GoqrJ?PaIPmf$*QsJ+Do3Uz?h`Q@Ks#F8xX zwd0=pbB6bcovPPpq^38|huNPpJ0I+;_#jTIFnze&g-M72570UX#{j<{g`6SXoAe9>Hg`>~tWB<1;PXbt_siNi zn3qn+`*_a;ujt5keH%m>>+iGTdP2RKZ%v(m3B_^U;)KAql zs%9f+Ld2Drf&}I(C37cSXaGHmBCAlTO365Ul6-CD*tU4gkj+ldpDuJ)uW#lDX18uW zeC-Q*mvQ=&XL}EM#ejxZ4gxV7+d^Tj@93Q)Ml-L6^9$a-NV;GB+q4&d+t8=-&6`^{ zMqpj#k~UPHApG*X3ODmmiwvXaelsSdSL)%jul9ovX;vvt_*pmb_)TEwT}3`=TwH_- z=bRE?{&ex8*?TTMI5;@KF5)2;LP?1Pc{c14Db_H|pdy5HAstGx#~Byi&;xk$YnLSh zI~!)f*&y=&ZHM>Kp_91h*;Yueu@}{b?-md>5#rDY8Hq#S!_!)BbT96=WifA3Ur>)*!R7Li7=EpMzddRbDF)5Z27!!0@r`pSTwFVKn0DmypPJy|He4N%L%A*s8w(+5YM zkF(~8)qZVZnW;A^sjiRPt7$)TQhGd6$vMTPuzY(K8|zlmHJRP;sa`g`df!GLe>l?U zRw}Pcq<=i58Ly$A(7t|!mz~^k#c4gQ54Rre=X?&)PkWV=aq`e^P|Z2C)=K(P{mh5=)}MO#a@&JS{>1&n4NqD`;8?M>snu#-dVC^->>8_}f%Fd6)-gpK%~~J%leh(&JB@5!`~yo9L%C&O;eo zu3@^-U5|tj{*vJmB;8b)`$D?7L4C9V-V%kLtkGx~9Lprj*T@3aWuh4tM*8;*_A*O$ z92Q3T&!O@yxD)PE1}#%1sP()#)-iT7W2G5KoB;{1at2DMEZho{HMsS%Gh-I`YTnvp zZA|rGRa=Sy_X4)zYTrGs+3L{#%|Bv6t?UETfNseH?GVU=GK*;4Sns2mZgK^2f`SJr z@$Z|B7V!kF1Fx5uI4Wq5t5Javnc^pv`}S1{@5^bQ$jwcni4~a={! zo+P50T$Q6&cL_{wCOr9%aV!pY)-}xSu9bfB*8YMTLotpzqUNDh}wD1*B0kl)^YLn&I~XwGw(x&HmrP%^Iua(LYC`P|Kg1<(LgRhFvn z2F_k3$XGp2<^^KqjyD)-r==kalN{2xJ{2tGVF=Pb}W{+SC5y z!On4^A?_^tnp1y@GL=&=0t z*N{lU+TjMQR*iPVZVf)DrndiRmIqvXO}>0zxWi8J|;+{I(t+wFvy$6NhPnR5k}!;CRL1jFmM9PaH*U;6%}lVyuXosWQECX;`PN4<~0Pb3~o`}LR1ED|^SDjt`L8G15@7!jnQ zcobP*gX$iL(JFEiaWauOEC8T<$fs`@xd}U&O#A}GQZl;L3rFP$mCh&H4K5>*(BXea zbuX*mj>ZWklS#Y-G+aCite=6_hKj|}K!7Ee!0n%p1_;;mH!|0JO#fyIK(cX-PjLsk z3VQ#!p#7wV{arWPCqx6=w(v3d%2&7M?-vlr!5I`F4j(yoU4&c&>d-~F!{ z|BPHfmy#+U5kM37cLFd3+$6l3%oJOqvO?E8!Egv*$3+RMrd~DUp&;=%MI9gy^Kxgl z7Dmw0J%Xn5E?tw)>!F+=Z$4Po8 zNj~0=@VS*IN$B47{_@8U2z})0;0^jHaDjxUU~+`^dK^m9J&~#Nx+%hD#WEvx*BAq; zLc#^lD^M6v=ho9iNfLzO{@0N0h^P2@!UArwo)$ z4h6XGf!g?Et|>ki%=rQOP`3MN<+8gC*$Pg3#RChk`4(FVRSWJo{8_(a2scv?9A#OL zSl3idMAtWLQ8rT%f`0pQ`nd+`A0xUwZF(TkFYE}+Nx$O4noZx%o9Mm+&#<<$2M8rn zkphGgsYw|g#u|lU-QLHlWwlD>JzHeMsMPOmFEXn0etPUvSrbw!Hl`C*l5I9TF?<9v zRPBoHPJ^0l@|v^D1*^8eEXx9z6$U60M3#C1NisG9DoJD1f0u=^%M!TbCl`$!A(b;r zEmn~j?4;)DEU}J<>XEc}a9WWb7nv{1FJ?jFw6&4E5a8Tiv3dM>?fCQp@oH+J zVRUjh;EA>B4&IGNe~1{dTN8oRz61X5De(yKL>7!f$p;?Q*Xyv6USJZY zkP>J?Lm?Zg0AuZyNJ1NqeKM&R4^7()pV!(Xr1Y^}iqzQbE|Q8V`{Bm68F9?%HIqrm zXqWQ7#}&_Eu-%NU8jdZb4U-Wxm_ym=+)RMSa2@hGOoFb~$Nj~qZf~CF0tSLo-N%#S z2viE+nZ+cx|4illJ{hfIo)Zo`z+dz!#hA)LWQyK9268Rg8@Vss#_QB3&L$!z zMs~&~FnoM4PR@=d1~xG6>qhBsKLkJj#{0i(ptrAguNMg99S#zU&>tKKyxa|5I%L?{{8yLZ+iFH8XU(g-qvjohab_?I0Uc{_APEYYmVv5Zb$C}R%jyHKCT{M zFOa7CiT51V@2h{j86sVBGf$)cgAzRYggKSDT=xPP*;}Wf3Wl26j3rAO4U^;-Kc(Kg z>B4lI%)a|lCg=hAAiMp%q1RwMNc8Kh+qZ+NawR79K(7w)5}ZKBi3#>4JVIZq8V79P zPueaanzm$wtkI#4+Q<*6M#f)LTI)ca$&*N7hcVLw&*fMDq#e7R=0^oWeemaRqh@=@r_| zDlW!!i~a<33;T)l=NXkjeueuA@fYgN@+&fM#meRB&Dov0IiqBYSqv7!mDHip)Zth$ z1-Yz-@TL#>;tnxo^@6t|rR_Uv2UOUjl-l#!4$8J-BMuAO!{SHG4A{8S+KwbSKBCg+@8Gt+6?mR?|S8!M#AV9bPqQy$odatHwW?!j}QSh z7bO=t7xjmbDv{izDCGr0cy z4xSk=-P_JnkF|ywr`d<y59-cGeEHcuPfj+G-9uP)u? ziwDPT5WAF?xexyrrkexWv2hKS3nNbUxR`picc|RK(x)?T&)lt(XHlOZZ)IoteqJ#-_)-R#`cY^H zs4I~jDo1QDhxl-N-#F;^TcOf`V`&`Ns{3481qbg0kVk~?D0y(={sG@edBEktaT$gt zJ{&qV2Jdt~sdrytKk0T~&3f8HG~6{&wihX^&|SYL99UYtQveJTJ;A z$SG+rGXoxBP`*(lP!21fDXS^JDZ42*TC}qTHrtx%&~vzP z>bdq^_=^4DeIa@gzaF?QytdowX_asFdmxNkI(UM+Z#=f=S!{V$w#$sgSP)`YTZ2Fk~=#k^vwI$W;LOKG$vCsgnV1^u$mU01GdPM((7S9-T`$Mc#yM z{v_m!Xf`H3<@*#aK_`Ac0(jlBZU*@7KXqnPcyMJOM@;Ja(yveJ1vlS0^~UP$d45saeAc5NURoqRPYrU^LLOWZ%k1XVt z`~;*H>CIFg0v_)kBOWDlBz}^=@y>fn?TQ+>(5 z6g;)wC13dCHb z%xyb4pC>uDyp12(7c~Z79be;xgpXKjZpV=X@mLw&`lHckl8V?(h#fLnjVag zIXv%r%+FSSK&?vsS(wTstZTlro^3%Se=6Ah=<6%^t=f9;zeO(sl7LA*j0hAadd?v? zm81Ur6mDm2)x6~KgB$jw`w1iFhF9K6?*{#J^C3j>JV0^X-JF595q-CRhrOTOAYO|= zu%Z?t!Jl$1Z~=dSq)p$S@|y;Nc=;Chbb&ZwF!#{PUmuM6Y^sl5O^+Ra6NXiOK~?fj ztp1GtL;U(aCv79Od88)1)ZOv@2l3nR`pkKcYJ;Q#s|2?E=u*-mpGv70PagRwiIy^u zbK8Yk%?0p)882nFGvV_mS>9BG9J!w9ggnx!{d)H^LPNV8fAA~T@qz4YmbpxOL*%%e zIGTQ>&Fp%5l0V(bE&R&>$+OU5y}Ys_e!Ay347FS7N#gTT9WRNXSc>hp9Ml744TnY zP}Nx2^-w@AKId2Ec$E5~u&+L_V6P$6rvcbeKIOPL)2s$ar!SGZE)%t2vHzdZE<(@> zp(V-)%UPgNN2M|Z)(w9gMN~_ONjRaPIi$tF&;kQ<4x9kZISE#n>@cxHc7auOR)iRF z)Y&(4VoUL)Sz}gk7lDsbLy#nMh}1mTK8BforDQ?OfPa8rSLmErGktW;@Fn%2CiS>f zLr~4oYJrJ480%pSXY_U)ouC+Vbj_IMyrxsGMg*NW+g@^KRK>uWK_c8?!AHzSWT#il zg6J}V9Vm{Gp;i%G=?;uW-B^6m#8wcsTx{7sN$t4i0|*`u$h>VedtppJ_7l{D|eI}+GwDErUCF%D{mrQ;BMSG zSY*K#LQLc?7us5baT1=DflZ!o`tOLDKZ;^IE&5mG(z7=voBf0o~yu6PtvLGl4XHzx;pm|aAHG}%)=^B!Y06Okvqh+o_>5U6vOx?T6hebXNrMv}GQ=k)YkL*(?2l z8UsY#(3|2z006M&p9O7y(q_*4c+4B_#czb?3r#S}!P3!-0^Mr|o~OX!huE(6NIi=h zo&_nu!28O>llquMhV4>9=>$`~61b@3KNoR}=>#lZkw^E@C*2v+9b6jffNeXF1|+n+ z2P(pAkyrG1JGoRavzM2`=6@bFS3EnfH7gdsSGenv1Elw$7T<~c5y53hHYa^PipG|x zo7~3SfY&+Hk%x?PS%;ECy*ez^C^C5F`3jYDZZx|jPZ?93-iKj6K>x}49Uu@K8{WIM zO?r}WF?uC1f0NHw3OQ@_8b%b<&DelUd}lZ71#-y;mBBExW=6NV>7j47Ah_%#;=Kvo z?E)#1@FK7;WDM5PZ+mLSSv`V!d-Mw2iAXTmg96rvgzs(|tS)RFIHS8yct>PGp<|p* zOG=T@l8`wnUJn20?8*0q=^1tid8X-$!=J4;sMp7zYcaN&qSvFOuQ%@ z<25=&9+D|0U2>B680B~WiGd+^UDPQtGN->aIrr{@5rJNxOX)K_HiF%v5+?7?n|9++{Tgfs*P&Y znlbCCOSEdCDiyjT;E#|pX@H_{QjbSuOH$^%q{FvhkYRs!4=93jmW#erDJBA2u9s}7 zTckVSWs^qP{s`Q!l{ux0dC8B=v{N!U`p7Hub9=hw*Vi^ZkHH~F+0N=I!Ohhr^2Na5 zJDs5pyh~~b|1G8 zS6aW+SzqTa%XYu_CmkwQmq>++bdKtzOROD98RVS&C|AwTnC#Svjw&ML*sZN3VwZjrSs4xpD!StlVIC>6tT7lReUX1|Dm#`9KK}`dmuR6(E`7ezPZx3(?f=Zo$^vqCK7JnC`H>eRSD2y^&GiKu2q_ zHym!igFc=1I4~9D^u*l&;M|%X5@AYTieImg|KL)JNS4qw;bD*1 zI`rytQSas3>~z!a66+F(XZ+H^_HZ@>`aKwPr#8a9@gE9lox^^UQ4Nt|D@+9Ypu`Qj$($XN1ikZjwjd0a-v|EpW zwzLu%9F3-gTB$1Mrlo?$4VNX1%&F~=+>J5zIP_fRO30l28rk|p zN)~xp(UD+Le1#-=IMX~cOgVLUBNhx=j3N0LY2hs_znVb=+B)vO(2%XR_8(nUIh3Yz z^@)S3Cg~&4MOf3B@<)!?`6@L;Z5({xTA$_KSq|U8cs+(0Z<2@w7NUU&;bms5t)zR( zCh973y<*jhMrsqrzsrlPR#TqEy3}};xR*|@dwNQd5V&WKJ0kdN9MPVeeA^)IVuMqEBe z=n*aXm4iY=-ZZEYI2K$~mq7)zrijx|HCP`+Ysej!!{j7He~~0wx*LNZKA@83v}p$l z=!cZ|?-X!Pa!)7!ouG|ZoV{HwLJ#!Pv?Nze0#YlQIq{eW+cj@ zs#r$;nfL<{sZ=hRy-p|)=BdOk0oi)iKc0Qb#(7%gCAyopzMj>oAGEE%O-VcZYkZu} z9}>@KPOC^ed{)-e?kFjgPFDJWByKfrYg1URf1XZxEk*ig+-^_a?udW8SCv0j$6b!S zfj;ONF8CR1ErI(?6QK|_=jDjgWUv^k6FS>JT)nCA!tsx@oUEfab3le*I3%vky-Uf1UZzrXc-u z{RIE;LhXq~^zYh(oc*6ZegvD3Uq9GM)|^^=Byf|W)T+b{JEp$F_O8O4DLG`sZA-~6L`p8gW0o^kp%s+H4k2uutG zA9&E6;H@Y42oWEh0F~863KaN#-ttLI5s*i*0$?f{;xS~;z+Ny(1N-D)=9P9c2Ykc3 zLhkDj@zr1mO>k5hLxt50gS%5lQ+^2%#_E=mhb7Ee7I}J<5%tGi3%6NiLSd)EC#s!; zS4fqXgv~fUp{;OQkz(iK3|#6YYuAO0DO0vO)g94MS5I`SE1h8Hu$3f^)y$^noOS%U zAr->3$Oed*EZWnx=5v984;?~_B=7G{E|isDP(p5A~y*srt%=6TUWz`k2aVY zJMk;vu4+=SP&p|Xv8Xn6hIl`!zCgEGrX!P~!TYE+;Em>``tCm`ztyI(?6Life^RVw z^irXP*8jZDXw`-Od^7ZTH>8$U`1!K=Q#WD0T!mC{Or}xhZ+)OHbUiHQrE2*i zd(@LhC)nW+lC5}KI%FSx>9H}LfaB_2&cdrZD-1g*9!sXPSi{g@DhQ8Dpiz>9bkNY? zAD;8X$G&3WN`ns%U>0;nV$1_?Idd=g#xFWgF2}s;#o0W3 zyAVpcJrbFW|Bc0>n5sX^>RW>*XZNpydRlSf0NM zuCvidV5OGwOxlEsi<*q7QY!?S&_F|wRtpS`C(2!Y+hkZ}bN@8XS4xjqBfoMaR`p>T zjU9WeG?5j|6%f>HynnC!Q!gwh?}a>jl=jsBYwA8$`L%g@zJa{8aPxOw2!8YWYIt`> z{&+3V#LRrk&=;5$f~}`ZAv9+*kQD(t5fWD=H%pjFn@yxjrHwVO>%vmfRW4Zy8uL9f zSyqHFmW3ZAB+E}-rx>;I7y$KnTS#5xp<0nxn#N}4Gyn!RVZj2?RAMF+_=t8C)qya8 zge-6PlG#9zWDB~6zuz2@u#&EmREr5|I#isc%DN{1eLPe(QBan0oD|HYf&ZEUpN3C) zR>irlY}Yn-emOOz5V5;$t^3}cx|dUUzaDcFSir<&D6yse#Pc^>{sv@Tly~_GBWODR^qdoi2B3~dbk8%t5bp9Jpt0s6|iiiKGF~})TGNl z1S=QA()+7cAz3L#ov1_zr~As;iNui$v(v35I1iIj0NbJL_?Xw}&n1NX2b11wm_|~P zU0PzyDOe~S6KKwyjo5qLYa{8b5Q)?O5l{ZSo=6sHEW-@O7+kWlxaa^<4r2cA~k^$fh2=s2g&0)<2clduO0cMLK| zR>~8p^oNNOh>RDA12gunEj^tdc{6g;wNf32O5}*ny?xJ~w5QhJPix`jaDF=#6n_2U zmzR}`8_sDvujS(d?!$nB=@5!GqoWfEL4ssY(Cwaq znMVyVtMS;dT~ec$Qs3}$*6>350zZLIzYMQ;a8<5FYbYv3dt+LxG!>_MR_azl6W|l% zYa6v+Qc;8U9=H+aB3B>QWW^dJ2Hz)AhsQ*T88u#ny(iK8+ zz(caaYdbqIyU!r#>R9#pvWa5ohx5Zi?DO1A;2YEwgTs9s(w4jVy3`*EiIb<*x&!)o z*RpE(CWT{C8uj{PE;!yMQz!Q9pPWUeL#aj4r7Aae$c&yiamXqv#2Q{qD`a2d-!WPT zVG!wyRIV={0%mLxj|(SUh!IaV3{J!{-mV;_j|h_fF5#G}v*&6Zm_B8SC(7x;&Ht&{ znLw~Ur+NbUE!$E{p;wGhM6b`@vKZAsr(mx)47Q!)YmH&7@Yv^WR?PN5|exuCX`Rh6s_?u}H{3ic%Vu83*2) z>?g%wa!~5jbilMtFHoeylnwODiN$%up^^9d^r=kaeezU>%)qq03~;u}RP!E9NPhcL z`Zyj6SUT@YleL~qDA~Zt`i@BPM8x3Z!7@TYA*@1ry5w9Y{z+>>A@5W;?xua;^&VuT zG=l}A6MEPzpTliVAfWVGJPu`VaBt2ZshHf}*7aqNc^$b= z6@v=Q<^MiuU9>bGEFZA{UE>^XTCH1c8?GDnz;wzqDs=kwV0}0>YH(~Xf||Bgsj9Uq zZJ4OK9;lj63>%wX*_Y9WQ(^6t-2C^fNg~VzBr1w(B@`Tcpqr!Fg0fs1|ziVJMq&+Vs!Fb}q(b*Zoo(B8@4#&K&cH)G)uXr=;^n zKn}0SltRb;-&?PAPs@?_9f{bT`#Zc(s|j067(lhS;rxHjm-WgMd9|ga7ndeXYfC zW^TFPTPu~%1I(Vw)*2pg89L&+%#R5kvJ7Nu$|#RFts%cGnw{i=c3Mp)*8nr6E<&M- zdW_y)r&(~}tKe(mLAT(3`(oj~Q1%i~55*jSbumlHVX<|}F4Y+h9t6AOXeDGxG7U); zsl=S37TGz|wt&t6O&Tq_9xL0N%0fs|lZ=epONER|z;#5`ZI6wmCJjXN3vaRV930@W^z;&>?i zxNdXSsB$|jc{?K6(Yrx3uXbFP3H8%0fv8oX9ub*UT|ZB4jamjt5e%$PyBIu<6RO+I z&=x8w1t7I%mE8TkYm|%yy2B7gI~n7&*%K_SuX@Xgv?iObz4O4!Z!;K9^;*Yx9AyPj z?3b-zxY$zMWA${n2DyinPb4)DxzGc4gy}8&oTb@WtBnjdPRn#P+xcY|-SUqVm`F$o z`*8JFw0_)+OE0 zSB;X`><_aVQ5Eb^q$Z4+pX7m?h0KDWGA%;5(hz7s9_@Tcl(OdrhoSR$$NWcP+G8r( zC<#|d6ro!(T97bu=o^BGk82_@NQ%ku!-BZW8Ze;``y@0pG6#m8F5g9qI&!QdAN>xB z#lWb1;~tMyCA^X9Qt0&kYM>Evb`w8h7-0-`NrU8{vShk~cOTYQbc~djIqRhAtX;&f z=HCVl6elHmZHBkIG~vJ$jNFBeeK9qD4r@kdXM@r)YCEpm%gG;0L}~nYuP~FS#W{J6 z&r0{lZ!Rd;-D!3<5nMi*w%-;)f&ZXPogaYY9qm!fgg_rmv+g`>q>kFu-Mi#3)RUG} zyMJQ33tAfFQ~+J0a6K`ZQ>~H(X0`O2e%NY1pjfj(6tVs>>Ah7LAIiN5FO!x*W4kH7 zw}h0N^U7QvQ}AXtu)yhfBQEe&gnT{eV0KjZA$w~bHoRfwQT>YWG%LxK%f~XCX4H|Z zv>CVDM}g7ShjZRt<)~FJw!jOc)zqV|k0FslUqu2p5Op~HbK5#{>U|V@$4Z+lvWy71 zT<>mg=ar!Dq(QbKO!g4RjOQET=aGh6kHg=p__5KmCXS)a_R;6_;8XmQ|CLv`(5Q{c z1@q@g1-Eyug>N#3fB{m*Qb5kFBocw+wy>UFBh9Dfzzti6e4#;8oDi3>7D0No<~LF# zhp`zL^2-M(^Vn%x)+%)1+Eh6ka*(b+fD#AT)Z2?-6ibkUQzFEMPwCp#Lt7W~qJZ?z zEPk>>k-Spu5j&+hrebUnf@qHf*gKuj1TB={NHB+L?-aJZ2VFA#ukO$IQ%r=9vunRg zujHTn^7x+p#Nl^v5iBj;Vbru5b37+IxA$Jr?Xe37$m;FJWgYMHk>bDOA%|0GMaG+h zVXt*=_ao#Kyee%x(Of{x$f~)WfD9IrM#@HN5ldY)u7CQzR-uFTf$Q1zTDoImjYFs~ zP_&Eqkgs8GS_I>v<@7m|#dvmk0bBS`G-LkWP7U3RKCBYr&vZMZTaF z2_fOHFm(Eo=9m*e^rMPxj}%!<-Idm=);>l)p9I=Zy-(ubMj&hLmS za7Q&376$H6Q%@$0y8??KDFX^MZpfg-MiZ#~-(1{|8`QKB8%Yj->M z10Tjwv(&!ZrY~l+)z+{DGkT?Dxk#37hCdHC?OfgjU7{ry*=c=D_yuD>{3MkQ6UbO;$J*fK_$hCIg;r#1Xa!p<$>?j|#Jq)&n325bUibVkBh zFeRi}-Vjc{{yU2rjys18KM$Adx$+&*H;TKtN+yXUbMz@@NZ;B#{ss^KumG8=TKb!{ zpFOH6>cRw3d#S*`e>7nqWpv_s&l|ggaCQO(h#pRGt9*Ve*US}gXY)UMK zBPC#5S+yJ!yXqMmMw|MVYOrhMQIUHxqm+D#{D}hf?|&Uy;~g`VD8cbl_F=rlu8+rR zci(NaxG0_UAih(7P(qtp6v2E}IY|?tJZq3fgyNjis)-9vUrtIq#?l9@jK8H3{Xm=$zK!nEniVaU zR1_O9#n!RwJp$bvGjYhA=W}dZhgV8?dJmLOD=CMp8{$se@ zs!6pywve6b_5PB_^m)ZH$$`l+5JjVjYYFcxi3tb%TvC@KTue4bbZQ8e-5_>@VRydX zBZ2Sl;PLv4MPAuw&iQWmH_(4acH~Qr*(6$(dz2jImKlGakbxI%I#>d;ojrEMVKnp~ z^56DNx;+*WavQkZxFuuq@vQeWVB$RjnwBq!MCRzVGw; zz3+u65euZ7@O$-v)ODz9R&k!XrgRniEX)2>Wjmz>IX^lnA^UOCzrfw~(K!*x`+u*( zl*nq%c38VDDEQ|I*zuc_>@1#HKBnw6%H`PzmajWrR!e0&twWaD(g^ z#FLR>LZnn<$vidQ+LTk(_FREvxaTo3p{)mRNSwiL5o(4ijfzlIcZskd-*`Kij^+86 zQr@n!F>8@Cb=^N^qdxiHNEo_htoxPUTI^-CPXQx6L;P`&UCke7U*4EKPJSm9koMqr zmaf)F`&3Kh$BO^M*f|7=5_D;_ZQHhO+pld~ul3rtZQHhO+qP}@^yZ(qmy%zQqPMoluTqYXQ?2LKX31iujv4IQqkd*LAF4V!geK_8bu| z#Su;6FVvMI&6MSU!mg1^B4n3?2uY8wBRuc}H>5RHc(iJe@l%%(eKfein&;n!wiGLo zvfw18PCA>+YOXvt>R*VpF*eWTUo^*@w30)cI2Yzvp)Om;tY|HBqV#-&m4MV;S|&7- zJ5+xTzm@A@{>Mmlxl__dDz>-{$JeTfiW54roQwhS&veUWP_y5}e2pS@I;R%P7q4s} zc#pSxtHW;6;XY@S!}V0iwomqdNey5NM|qtecMx1qf+m1M>9rA1J>|>&t9K!Kx;VgQJ?5^!tQ$tR6WcnK1%2-s-M;weIW ztXRU=MT^d+RHY0d=Q9Ukd`l`71-zV5v@_57JaEoonj-=^b;DSs+1}N`>g?`;FGBQK z*#>L0l~vedD0b>jNBu@_2lDn&wDQlPf&(ow5rQhE(}}~<skfOzkRvuK+#2r=r&f@?9#RK{RlOy*AYKaH7_NsLrY zj(Tv~qs|tT0TgCPLJbQDSx?YOfE4Wh^y5m94lEzsCZ~J z`75sNnvxNXirEGjNy4s0zb@g-!JFa%H8DLHM2SZuw+bzyle+Md@VXvc`XIzGK}tDtu|z#&f1!fnAT$_C0p6 zuuXIbH)ZQfPRaKuIR^Y_&y{gL?9q+{5F;jDAMuv|uKFe-~`9 z@7Y-lV}&8~08_ye%>tRe61 z{9G8wIja{C521}D#70)ux*^87v17R7Mt*zGd-N{zqEH+!5iH4Mp2;MkQId0*Z-Q}x zb)svjX{idRI6tOhladqLwAq1VSr6L z-qHSm`7$sA-K5%8G~s-^Ua#JVxDVnP$pOg%ZJSu^zhD`R))3gFOl{gwBwEp6#BGZR z4d>wJM^XiF9k|)lIAzt7=gN7THV>MAn>~4B9YeP8{HhiWS|llB<}G8nE2Y{{Fq#Qy zI%KP-+Gn#|pa-FD*4$cfCOU3!6W2G+HO>eX@gf2K0jYL|>H7L@CENUI(dzj-V|p>x zkURhpG0!rQ_Vt+Fs!}O(rFzT(v_nk?1)A@dcF>m25H!yp1>}G(?gF6f(3f`nTEb8P zk*G~jB=W^zme&JNQoEy1>2Dq`-*jf+*B&;E@hdp4ru{M zM?n!ung3R&o-IW9r&pJ!nud;&mbM?RQKH!8RJYbXdY9Nu6C7Ff9oJ7Z+Cf~jl_d$k z@uVQXdU8MOK3|C5TB+dbF100hR)>N@F%CWA=;F#bgb6REcUe_anS?_1ccdtwg4$=y zP$Z1wzW0Bg_ve{yzQv2mFD1eNNESdr9X7hBiWtJX?4RoB9F4x3*%LG;jXzLRgrYB3 z|1G1fQdP9yu5hnOlZ8Jdk~F$Wlndo zAlsYJ3~+5(OZ6mglA9l@UZwL_9knyNgU{Zp?Z$;*=>3qJY=@gag>5B;qnCC_cAhR7D#{zP+<>lCWkvCU#Uo357*b zIa__)3I+##ZXn5Tf8Uc=Ff|;y&3BA|(Ae#(2`fg6klZqUjz=TAt9h6$w;4L)LqAq$ zDW1pZyq~D-a$m{*oTXzETCR5`97Ccfi2iVEjAGRVr7?9Y{OI1RA5buDO{5bi!EiWj ziBMi`1V71jx-|;vMaRrg_9Utesib?Y))kAP4*uH?ZOFb*JL?7p^XgO3x&|`+iLnajQCeMC z?e3RVpRHF46%6Sy`+=@K*|V@q*?4@B)X4CR3GB6N{>ax$GLmK=F4e+r5a_w}^_OzT8R!*6ipSj|;_cMv;L@3Z?Y5Q&kxzofqurws%pt(@H zaQj-QFAm(m%EbQ6cnb^aQ^4Ka~&$y5!?!Cj|170YrD` z5v(w{pfc5G*y#P0wfYOT`m(90JDLOcmfybd_oVhlv*k%Q4x4C`lfss=YXo$UGeLtKV8YIhMfS{unL`d&~2q{L%Uea}bofS$Jv90_)5uc|G%AE6Y zP};UhmDyIx#igF0pKRdZ6@#Z*wyblu zQ|PR$XV9w3G=&`&gT*phqXcwfr!yp*6bOkM4Rlp=(C>0vKANm~xS_&z`dos1;N zO1}4qvkYKd1Wdus1WIEgF#?NoFr3FwCWJFAQBY)MVp9l>x%EW}cJ0%1G8rHIcI5>p zEx`_SWYT9nP60-vWP>Ci!AEn9b3jAQZld-l$RVPXVa~x~f_6+^@JkVIiwhAuvYT`0 zu9e8#VF)VQ^)RV8R|dDP^*4AWs$vAo*?9S>V5GWxB!Y*I&;RpT`cL90Gx{*-FM1&M znsSY?%afBsneyjA1C^V4>1x6BSxQopCyg%(S zwW?OL^(M-HeRD!+-vQkItFsiSVxhvYlZrhg3kC&`b;2NJxi9aMDlU@)cDFEmYJax7 z6#xp72x)n}z*;CpcBBY8I~nOYK1`aebL8iKtuv-mMZLTFw=qaPArz9W=k9Dhrvj8vPH$#U^tzY$^e8$*slgFN}uO)xl% zE=xIW|Cvx<1sKz7abh%N5`2Sqj>I!gMCzw7M9nz zdsV-}Y1>1egq;j}!G8y8gEcWbxIQ%RuAghC|L$4}zz=FvJkh>bz72wcBjKVDywmO( zmdfp@Z0Fj#l2ua{y0U4hXe8v(#?h&yUTH~m6h(jIi`t4f397IVLFz>@Qm4X{_9{&7 zTpHW1dU(^wBB_;VEAdbPy!(!n3o#7RMcp@5WQZnhvS`*tNqe93pAK*;!Fd!B$#$(o zicV;&gG&Vvp<+7`I6xX>)oW~V!xA)3IkQXU>*y7^9CCpFS&qKZcar9_d+`u=XR_QB$fR zIDk^tDKn9ffyX12{7au8(4lYB6OuSzluAJzt%-0`^Fsx4=RGWmByf-6Zov}JTiZw& zu>CXLQ~W`row_pR&++F#pb>JkLV@008N9x|Zc55Z0xr!-uGENvqAGBL7VhSu#A_m5 zrjj8Zh6`bv1cj6?7F`vBJ?3E<3ON-sp1k0Spc%TY=5eeXq5zpUW;#0<3m2Z>a3v0W^8+Z#V=*0I^kE!Us|B zGFO1!t7P&qjA?~UL6dqIuVc{$X_7>1uD_$l07qulqH`Uj1rSIpxS2c!VRRV|&$jN^ z;rwAE$G3VKw(*FnY0gvS8-jkuD zHBXA%z*{g^V~KZ78`W1pf=#-(_Lkkw=1WG!8j?@<)}bZj$`ebW+GI@<_t4Lj^6*MR zjW0iQuHe2;wHy5sW2?9Pz|+$TtM$KG$5gr$_El^uY|{jE6&`EPRjL|hhC9JG1#XjL zPdI<<@kP`zgPh37xR}7n1PmpfQDjU+H%cKfuxJWAtC#h{L0_q<)w)c#(^0L~seMb5 zM8&9*h>6aF16EnWhCFDW*m;td%$3np=*gB;_6*7f1mfWsX&jjd=>ENz-MmHy-Z^%z ze+VQkrwz}dK>0{#+7L5X&97A`LahX1UOTbb3?ZOBLa|FZnCrmThd0^>a}I`r=jeHzi>+CNeA zL2schQ!j&8wSBYY(lRDrU7(Y-F46kwjF7T{`t>dz9ru*VEOUH!fNXnpFC0C-<}}Uv zFK+&IAo{Ri-EBA(k|+x>lF>rJGDt!r#e#x;LK+A{xwnTxiXx*>nu}{c5rf@+qf&m+ zJVQIMa&6GHY@E17KpenniLr=x##rSEJ>>xM~G#MboP9+q}=&$)3QSjiGW@ zp^!cQtgu!JUGKL?-_*OJ#Tk#m&{AeMczus&^UhV$@M3Ih$xkE!I?tV?mEB<-P0{8) zW%540Ozb~XR?&%Mccy6*8NsG2ljm$;lzGiL$BHdL;*j$Qf2e3|qB5JszV+x*rC+eL z*IeQ5!V{C`45*I+agYiC6qPW})IAd8I=u6qHV2hH89bu))v|wVbWR;*YRC)bgoCTB z74TWsIxk#plcy_$z`eP6E*SeXmz8DVZ*%yquxsVApo+GzbDD^_f!mf$fGvia{W(J^ zO4*yauttHNk+{)1ad7^01b=Acnq?XRX+rG6 zwh(q_Up}(%MFP(|)mnw7;_rA6NsdTN%0^YPs%!IMg-JbE^HOy6!BVqQHM0I=Zn(@g z)2JRlZu{jvT66~hVI-j3ditWL1~c8m;7VQBT_;EQ#DVIaB}v51`xO>l5-)09cN<-G z`v?5=trW;BR&x;t<3z_;^RN5rqt~vnfRr02{YbLg#M<5P9VoiH8TzJ$VlAIu(e~*> zMx?jixJ!06ElWBSv<&EQe?L%94CwH5TLW34s0~6TNRp_;w1xtnVk*=g&)GdXLLSjd z;_UeBHUefj6}q*81&xfdYf{LIkR1X{dAZ^)jd+ZZO+Sw14!L5|_Y>XzJ7_3s7jMZ& za%ZTO?xg~E!eq58ry1q|CW&9BQ{Ux1ntiWk1npX0!u@7?$|esVYf&+Qsogo5HV1_s z6=B`)5~;FBn^Ni3<~}3a5_yRI$yFjW-J@}dH`yR(G3R#p0?g@er-`ddOUwTyQ!`Le zUn)_ev|e9uSyEl3*pbjn6*ZsR?{dq|D(LELwnF9io5-f=_c$XHVIwFR!e;PzDj!fz znlEb?ttpUzqH9h?WdjW0^71bPiKsVabe*Pd=2W8M-|!qCk9dzHuj$gYsGO^9W~G-J zJ0#gYOH|7i@7A-ONRU_+P&DCTb+E2mOw(E*rzFG|9he(SY?xHtCw1HwXBi9~4JGMQ zW**=f7+b$r4`9A0;sZV84+^48BQduxjVTwJ-rE+)9mjlrpRmeW(*4<`I$@~MJ1Wx3 zlfGzz4ZuAr>kGhAJTj^C9gt8eFWv+DtFQndJqHjolAi>JpIjLoh+5@Otl+;iWEbS6-!^U=|>ghEA706jzeHB39dlK91pXwPt{1dLhuMZ_Jb*vB* z#XY<@%%TORt9IB5OOelQJ(=mx6-BqHbsMFtf)16sDrX#hmbRhs$s$d7QuGP+m12Mu zjUarOQsYj!jk5W&jJf6Uxh4>BHD>+qPp>*n1GkCE$=((2oq~cEXN*`h*_E@(0hM)E zg{7si#`{B_%4)V2zo54;ALX}gkD21Z-)95oeYU9qEQmR`bxMePlnm+(W zb?m#94(?svqh8@4sHO^KQ6v6xz@kzSPwS%q(h}6PbcS8wNEVRjGKunVt=$MuSYTug?pD8bl+q|1_h-cJ3!ub7~-SITT0!TqWptK)!sO(E8C%zbrU-)*nPm?mTZcv}uIY zl#RaluX3Uop>Bi776z6_6+SkKpe#AjDbUwC#)pCA`A3fcC`fXNcoEXH8ee7`XgRNa2|9D3u@1qT_WW88jFIc%MPPL=^T{_!czo6>0lb1*2TH{DDH5Wr1F^>nZyoZ z&fEfMVLvP&N%`L|BGM2(Zj$f4R;gTHKPoSzyY{GN{wg<4JumOSBn{n{3t_L8cX?^c zrs?Fxc*k}u9n+qyw402x6_ux+pPGjncc&FS+bqg&G+WE< zyU9IX7hI3Q8J=L-p^2L=4W5{*o86j4+wVK;yUUmEcPH%?sqt9<=eR!vtcm$=o5fx`*p%(z_)xt={i~vM=SWERI{pCGyOYz8Ap+mEf`4eEEOzzV zllW4+w7OfkYq-N_;*a`a8?Fi8BkS9tb?3X~It3qx@{U?EG{wy%J-Zd>0c_|Ue`Kp* z$~vs-?B>nCZ*9_^&8a=XSrz22N&i%8YE4GrUUw^c0DkO!$7fiM9Zo_NJ@S=g>C1@h zmC#W}7AOZXhXOw7Cu?9Uh9_{+hQwi~)OnOJc5AQMtgEunzW5X9On(W%vNO<&1(}E?nNXwe3-$dq^dQN>C~Lxsc^v z-`^|VzFRijkll(bQC<#4_Zj=d^_-wo4q3|_36yC~hqS6sH+Tv?rd!fE7q&D}mk2Mz z7kC1|U14*eelmp9p-CFj>Jn>!} zEsGC}xp-+ok-;{ag0{vy+q51VK>@S6XEG_Xe01bM6;mH$yHe&L6;n=ja!){?z`$L} zDLe@?FyC;qu~Xl(!)CigKbZq6?*qrOSjX_yy!D>kbwIN<1ITMsFLY@%@$XD|5D2kC z0B$R`j~9hk@fKr9L(J%^%Hnpt;-UnrggAN2p{;*?fvq~OVI-M0Qv>tz?!7{%Yl?Uc zs3GX+P%4RY|U1LW~Dp#; zE|q$cf=M|W6!s%F3^q+hW`_1azucGHfZRmw1Kbeql1n=r6`U%*gBPY3rq*e^DL?H2^qlv|0EI(=M$m%!X}K=7xuQO zOyzZA1dUITZ-mXRTEr|KMg5LxBjZLkfx&Z=dd0(4EQY6{xC3fCYWRc zrpDSj0laCGSXK#ebA)WGQ!24V{s0H+K=v0=PSIes3R)f8r%&#kO=H1a;OYz>gH%yV zyYQ@pRhtBC;7^&xDoyF2#ZozoMj{`i(44Y*sNitM8XobgfglId`_=-F%S_q6;0*LH zZNMC}@vcTV7&57E#Jp)W*z|o1Dd5txZSy>*ioq@S?{97nu>p2jI48bjGx zwcI*mz5=ndm_zZ+ zJ`QS$Au^3YSqE7GL(s4PEFZ4TtNy4?i=UngY~5m5_PuYO#kk2-FB0{S+owu0NSYmp z&4}HI?KnOOt_i+T-X%V8Zw}?wrRWuF72hPjj)DVaFqDsH~KO8-~GZofAEhxree2jFSWBM*PNKnbLx5XB3ML1KUk2nd_(W zw`co_rqfD|uJTi5>flU*l~l(xVSV8gGus{79+E~jE;p* zBrnIyA>47$$D9`g==%%aN1Nt&>+S*`N){*oQ}%I+P~$9+QmRbSfa)|(&S&wMNMXu$ zPGur%7Y50JXJHAQLo=g7>eV@mibk)Ey^cU{;igsWRW~|q<~dlDdK!Acx+vi1Uck99 z!$+zHP{6*yP}8U`9A{Ul7w7UgV&eca0!Ilygcp+)-G#kC7qO?W zj2SWiK(A4N+Fk-R4emxgrsx0;asuPl*Z>G(m+TiU(MNc-lT+9OC#D-7Q$pM%CreF@FSM z5&pTWG*cm4ppXK^bz^%k@Hk%PItl6?Ld4)$fCt%h^9(fmKyfv4(@GOiqFhzc+U0t{Hw7m@JH&?eHv=a?U0`O>h<@= zPYNV0UGlS_0Qz>sy|N9w*hn1Gts@R?2gs|F2~S*bkNUZ|{O53mYalxP9{HMRCj^F> z+^-+iIE9%l?K2`f!N)BsR|rNAqQmu!Nq4h5>y~FAx}oVFh)H7Q#snM;S<7UfIjc+;cxzzJSAB!Z<*^aK5jGyO;8WnY>+g`;?e z<`j8`(%Z($-ye4dl=Y3``8(LBJryHTOlq%7lno;M;B)5i<&%6)?81@pNcTlVBZxYf zf`clyVftl0P!mecrav!U;~{gP#;@d+O9*9#IhgBzq`!$&UeARn2dAhyJQQ!y1p z;JxDHieg2}$&be8awg!J&Rq>lkm?e{Dse@3eChmlPMiuEt83EM1r?6m4_(U zJ{95$i*{~=c+taxQs}O%4NAx}I7a(yxnZZQnlQ)EyNB*&o4SY27_+EmRpBo3{JZ0? zsxt`c3CE^U6Plc-cG0gC84J>T0a{!~R9Nm|Av(KXX(FIkiM%Sg1qJoPe3neBOm%IW`u0loSajO+$8EDN$2deUQYg808 ztEQ&$aR<{NTHi{4N`dJhk?W3{`q&L~ZLUhvsI(9~qY`!8?&m=R-ZRPg0(r;aTq~!K zy@AOX@A|8=QgZK?`XgXgQ`YJ`_;%C^6o(=hqfH0lIxj7r&5ZhI+)yr6)D6+*JGNu< z4S2?mI+b!S!X=&GD{{t6mFUXh%7t8tUAGN!WJRZnzO;2SGydq%TI?R>ulHQ)=u-)d z=0}=y9c_EQ2CC(;!mCsx{1`72iuY{Fu1SlIJUV7x1a(rcn>EQf|B_awQB^)O~G z7*B#3cRr@`?YXc)M{CPyr|x9Pd^JpoQlWw=U0tLYN86zy`H+00>+9<$x8*53J5**9 zpj6TOLY!vRmu#dJ1km_r^oc82DH3Ik+Ea*iWg4@lUr>USr65hYh=>ybd3tK=8=}xn$T_a}%8u}f9h+d&jDZ?q_XtCDMZ1gG&#jl5zQM3T zpngW0rxragZoy%P{-)qB;V2ECMv1(RPjQC|i#qK5V!}7VH&{l#Pi?YoCw4^ejyWR7 zW9aF$Uo>+L%=JJ`qYG$rvF4U8Jhy{hn)JJ8jN@d2V-Xkm{Vjo;AQM5A$xC(R3FSK= zVK4ch&tISV172GX*$MGWS;8_d@%v&VjR84S!m00T<&;5db7o!1ix8KH-^wzhQJL)4F;&$w;j`!V>vj;-tk{Of-hzF{Nf6l31 z!fXT`6gNpe<68q?=--KU4Y!52KGl5T|HkRdH_H#sozCG|;Vne3N}n^YvE3WImuHPr z*|(vD3ty&eRan)s%Ceg$HDlJW{?2UBt<$Y{6^-OMdMY$V;B&oj^E-8sZxQP_#<&}1 zb9ZrjYVP=@52xpRxPYbujrc$Hz0E%C(Tf?zoqX5KT;{mycHDn{^}PRMbYN`fdY5|i z8NvFwEfa0B{la$Jrqe4Q&a*F_j1jt;CgA0vm6!ZV{9r@zR4nUQno-p)IV8cngsmjPcpz8n%ha#pRGJCV2X^xqD?daqqJ)~ z>Z6%AA`W6~jWVPUl7vaWrGiTMATxZ0Mo9+T4E-aYI+zRRBV*tItQ}dWkVQKmncZL5q|6Mjq%L^pBWZxThw%Uuw6r+QPFY#<%1&twpgz_qZ9x3= zLyp3xz>Tl8Dzo(+-M<84ssXhjpOz$vyKLHUJ~#pbkL6YoYiYkisx-E4 ztn`$Fwx`2ybrcJ=7zS_g{RTX`ks&h zT6&;t&~jV{1C!L!3^2d3fdOh2Rv0;Gt^=BQBLuo&*Lpu@3pW@b^k*YPe0aPp9U~p< z-wj5aF2*&`YeR#4vl@lWr5!wc)HTIr`lb#5m<)W>cU=WI*=A5#S@;gdi11JF}v?FZ}r+u~^T-ijaKN_)o?oBQMH zhB<7y0QW3}G7LJPX4J62PE_zLT_YUotXv~{Zk`oIAe#ejhLK4|&>{JlEeXHWH;vod zl!tCpHgDTt38HT`tO+E|zcIyn)OtP*?9;WPL|WOGnHmkBDDJ>xc`B|D|G7-LyBzRZ z8vqwi{>U7x-ZnYdZjdFB_;Z``i?Zm^Z)$&k!6!txpO|S@m@;~wq1d%#W*9IpF#3E< zUtKehS4BOLQ&C@xM-*Wczw4myt1cKuP)Cg20e~B`LwzWD>L@@{7zQI>OyEeEzdq((2UxDedDVHglCBrNEhJvj0_Kpe(zEPgh^BS19-uyfYI{!f7lpP_Rz+%eP^C<^ zf=F^xcyDl{%u!rUM_E5EO!R~;8cP3)0H1&I#(TcHKkuS zrAIfVA2+4j?xUdq0c9|Uu=rrlPX@8*!CG8HS`-VXuOqbb`Mvvns+9kTPH)w`>GT{| zc~2dDrPDuJU0<(kuGic>TG=47vgQvHS{~p2_O0;daOVg?eaU0Y@i>86cRCZLB(na! zuF#hhnO7>L54_C44fei%UiLyOnxS^X2bmly-+7~$hPuIzD?QL<79MADNS?V&lJ)Oe zJ2Zb$>qpKs>j5}#M(-!OZdktq>1UXNH%nweWzv0Ugxmu$TA6L&gE|gNDIZ;S6yJWb2keY?FXWD`*OVO;KvZ={O&2mgw%Cv~ao{xGd1*j%s;jn03 zO-ex``!W#)Q$kV~0_nuE2;1de>ka!2W~Z0@4dCWPr!#>aT1=4M+#s_G1;h{87${Mz zkEi9;fKDpiow~)*{ncP9lojC8?!YdIYTVDsb&t^mPKJS}sMkTeW zYV{UJ2>o;9B%pz@IhlAUIv`^LHV`zmsZIWOE<^5p$s7Pf15y+SBg5fJ;i1MKO&qG^*sldNEJ@}NEoWVE9J4IT{?f9guK=` zW@h1?9Fh99(+k3p|Li8Tl?)*U7v42&UA%zs{? z-qJoi-Cn|`{^*8$k`UTHAN@kpJ;2zqU76HIXKbeb7Nf&-JMQ+~Wn1;P|46AH!0V1Q zfQOkKkW>Qmt`NQ;+yf%1!b4Vs1UeD8_y&v4FEiJ7TitW@ly!Nu$jI5ekt=<`n*47d zeWw2j(q~~||NlE16lJXr7!bNH)Xd@i9(Oez_$d42xglR`|4Q^Ly8kPbR7H>$YleS& zp^mx)S^up^Z)dP)vtvbhU%|~)u#kt6%|u@3F=nA69Q913M~Ybyw?jF8fDs77tsMtl zS0aL&pK;JH2dQ2qAfsz)b3+E4n<;>y7vUrAgha>OB6d}~teN5As({Kiva6I4A45l% zcX93cur89GzJLtv^Xp@qJVGDML^6t7H%`hPiCao+ZsgzTKUOw2cCowP%R&ArRnEUc z^cFpu9!AHvk%d;tn+dK|W!dg9URcL=_aeIMNYYR0p1pJNpz}ICe}~$JG9i&I3vBqP z;Erjj2g3SrgFdS-@A_hR754625#`l=2Ra1#U*`k+|CkSq1dJ?9 z3{dnkCN^e{=1c@^Ol+M0d1EH{?+*m*O#iVy{NMhCC`V9bN>asn*21LgYnwu) zVK#k+=DH|2K@iF_v9oIgK}{i$LOoH@GdK%C(6<>^pPQdAuN}@E7w75blj)Q8n+x|= zPMJv%l|BfwP&A-uL=;q{IN(z3^tDkS5s|^5-$$aYZ7L4~<1i*IaRbR;NEp$s?FThC z+8=;XJp&9OAc|R}7tnp1!MBt{Kv_yoSyN5~gn*2Q=);8oh!lJl;9Z~(K$y@M4+u^$ z!J&dU_M4)#0vl}O_@)Pd4}o7m`8SiktHvd;gzWF{?`!8vh;RUQH|PukXaK+r0s+v`hs_*GX0Rdp;D=3J-ft9fb3oa}VVExZGts1Zw1JdsYj_nnf0N|Aw+qX3K z)Mxx_>{|l@>L&>X#1A1`f?yLL$_2DlzyQIwF4+%Z7@imapudkF*pGn)CMB;4mtv`>zN zs3@R51q85NaI4T>;g9l2Iw4)p%<54=t~UTTBF&2kAQ3N$`S~~@9`wg zhw8jv2LWJh2cvf213P2}U-R}W4u*es2R9NyolY3Q^GAM1S40sc@C)hd2d%dz@dx|H zNA=A|;ireb@CY{G%i-bc{s-SCmmKBzha5oK#1=*sEf}JbWA_Kw67-GML>L0&{OWB- zR|f`x8eXN9I(gqeA4!M^spm?*5p5A3h|SMXz@{hc@rA0ES6`lE5X3L|+ta>QgNW$o zXrLVK(9cU{)|bD-Utjd;raRpZ0q9f7%P?LMfB*{gksP%zUPeX{34tO!i4tS~C1MA_ zfRFm7tzA$af!_8~)0^vi3FR(y2VxK&i0JDU4lv&5bo;gTpi(z!P0h~fasAU!&GA=Em`D~FOdqjwlObQHhJMX1rQRk1l@2J8Ft?I}SG7;`@)=-tan zku{Dbk{$Sw!$CYA9Md8p`?M+7-gS|rr(CU&blZ^TF7hyKz$HHWgHPwRoYz8rvtw;Z zf;V!K<8bQV4Vk)3bfP?i5NVEIOc!X6OM93acN5bVdHzpq^N=C8f^_Y*rG%HkDxWf( zlVDxEb<)_V7ejxNN$g9iUhZj~{prUxLf&ZtD7t@HjhnRWLw|1rmR=j&O2W{Cs3CFK z*YM%==lprv$Cm?FB`xDTSP}hg!h(j0=H^ZcuGbsgs>_2CN`UHT^SYEY(oUZ$nUX+m zrIKb57Tj5tl~|zvQ~seNcn$km?R< zYto+6H}+BmynEf=-%t(9bSD{ebY|FOmqUpIPIjfbpb6QIf#!-qPr#6yS2A%A!A@oL)r;~VgFhQ@2cJ4~Htionrc_(ZzgxpN8jxHT5u=9CtXQ`ohAj?FMU zr5lb*8pzz$^31P@z+=a=eS>*gLnDw*b_@VWp}p+~2FIH?C`H+K-fBn7{dv1Bu?2*Q)Xf2S#5n%6cE-MP z(ICzY41>N=GdH8*Z)WZ{(5s)IPz}jYVP5Q3UHf17V`jQ1wWWV1yzO>NOwYtg;j3s3 z($2zjL77nUq32W-ChwChH2^TBC(98VGr7faG8UQ1rh=lC|Dum>X4+&Yn(AAa5+ef0 zPLR%NCv0LjmK3M#x*S&dGTuaxCh2XC_ah(KJKt-qpSYg=X`I4qB(x?Sikj}&GF^A_ z;~ClcL~RS{qw_9^WuI0_<(_3r!L)`Ji>uB|c7R7{r!|xwY>_P4B1N4H>i*Db<{rs` z(8fpMEV|Bs#(Tg3>%PeqCwP}S6H>xodHD1+Ofi&VAt&5=JC#YO-IMvb_uh6xE)p)BT z{S9h=l+bf%MgYp>h5HrLW;xC*spt>QZ>iQXDUc?IwH!pc%w@MlKvOpXKDxbp{ZR^# zE)G5VY9i!i!qX&_;=3If0j=@klO`qXI?VVRFdOsUc3oG@UMTrer_;Wjux84E<&+Bl z9E4Q|R@!_bD_hM^7ZYx!q+JG^KsN zmtaI_5S72zadc2Rg!@xHy%HmSdwgQsh5375OKjcRfI-1_fMwRQ6md#H-cT)<_43T) zKsRbHS%t%_a$zux!AslKNu( zH+~6Dsr1c9N1)G|DCW}n`6^Mdw8rwhWz+(X1}0qgZx21q3{|r}HrHu5hU||9>aK)& z+VQLan4QAoJF2W0yJ6w5F7kr{ZH&LN?2!rj2-eM0$=~$FoSKy-N&FTQ)UM{c72u=i zL{VN+Px-=@sTv68iAo1PdKFe_yu7JHP7uUYigoRC0a`2V>Glx|G&WTHR$_SLDi5WF zVw)FA;5%)C=0!MH=KTPlGrnXMi((c|Bkwea7R8ySkvZUBF!8E{yVV8{$nzcp_s8p8 ziEXt^r!17W0jIL|aBI-s7Ogw}%0H9WJ}&0U{eDj4qD{C`(5DnhV6GxB0Pe34?wv3W50} zD@&ZV)tgeGkT~uEYddpJPGmwugoPHe;JRwr$(C?PSNcZQHi( z?AW&L{o%(A~HDA@tVORBe_uZ@4b?FwJWRq0sa`%xeN88<}O?U_NKg2iYv2Rd! zND@?*$eN5z4<4LHS}0v!g>|!HV9YV9oH5mjIN6$)(YBLJxW@3BmP~YzQ8pWNPt{UJ zqGhY(Smfa0E60Lcw2$DTC+CN%fBbg@j!R#Y^N4?g)7;}$H`5WCOuwEFQ`afLnS zOm*LQbP*i~B=M0-+(pIq-#>rnjyEbv2v9-92ZAc=kWLHv26K|qJx+zB)Qi^lqdPye zpPVW_q@u(1@Nv+9|9vPf96&YCgG9@uq#q6K;FmIG*9>GF?ReKM0HpVi z-&eveFp<1Z+`sQ9UKZn{zGIZBC5Tv^VCeSfiQRl1M{QT?vhz5`-dkC_G70))5{j}N zY3F3t{*;y4g#;7>4RR47eZQB%=<|y}mU6`52A-RpahjP&y2;81thjZv1oO)+0V7ki zx2MKI`cJ#WH>p)&T%kXZ>(#H3@2{n)>ZoV3@oEG=(cz({9T`TcjWbpA=n_-)xkIb8 z!ZdESWPaSH}?X0hx&q4QDKv{H}IHIzawZ7w>QYO~{n>o#xL6cuKgF>v*t z*O0l_6pp8G*qAs*cbj<^;q{!l`!WY(X%z`&*QS#dM@txLjh@^)T4SfjM77kPnl|Md ziN`U9^@&6`IURiz@nOT6{HMqGWpA(e`qv9=_S$MnuxZyEEPvBi+q1TUZCYAx+&Z1~ zTi8%llAz3^@0(wgfx%F^7nCcOZ6Y}lE^CxbJ^#MatRpW1am69~4l~!eP1T2h1@(!y zTiZ{de{K52NJ^TPtg=1!w0r_QTY4{gBfEH$9DRBJ#Pq9g9PLI1mmT1NV?5iz<;aNu zPepa(`^O8f<#4-$6mt=vmh~(*peSliLdzZ4=NCcwBhTT1m zqHL7>8fF<5;BGjg1W@;OvYD;V-OV!iWJM`RXqnN?D@9u^wBXTlFuP7o@7#qq;*9Ye zM)4h%gLWPnsS9E=#0!0|N~dbEQr7@G$tzSBeC*{oHuzx9Xte5bva$p0S)+Xd4C;60 z2`Ptb;l-9=pC|RDxxdf0s;{_YP7kVcFrNi*(0s%n+E=6HfPBy zBlH91tw`roUY$^*+FGbtOjq8$YTi!jVF$^$BPp%RJx=19K{LweZLG@?EY$2LutDIQ zEv<~cqKOZOw0+J$z=FZ2B~S$2Fu^h?h65HE8CPc|YV@%e#(mRDn(kOhdMIuc(JTZb zw*|>lDpu3WxjNOAiL#54oE6~@&P>pw5@Phd@RT_%^UMAnhwl58p)gEUoWV)cP1KzyK zRk)ViD6HfkBEO#Up6jrpjN#uv(5|UIe=$jscCl{Ebr+20alx@iHnb8B6RIk+^#R$A zs${6#x0y4ev_?)Ke5og@1be)d9P zzczMUW-n^2q~Ia<`YuycwwO2+*mwU21?7m3PURlrL};nirrbO|pcrwc#bKth+W^l&U5HfH7nc5w-#)BI)+YcER*#>@%2 z7a!J!l%4cc{x!u?sGJc#^?5~IW7kNs-fy6|Pl6k0OJpcPwk8&z$cBHwK7vMSe0|@k zod4A6pv_<&3Xyf#RP@^|wqc(l$I|Adl%{Tu#I?JkEja^xe^(r!p_$ZVn(WA&v8t>| z7i(cY05DD?{AV$vhnDv14C2~MiX-tACklO-OX>FfqRaa`ak-=n9gRhMRX|^NGlh1R z-n;`gv31(Cr)H4|!V|5C|Io`;&t)h*B$Q7!w?refT1!+i7uTv}mDgVXA+;yMk;t@p z>NoCkS{Z58N*2>Y6aeibSUZ`5|X7Q@oGpC5Lp+K1Y zNaE5OJq$#f0s93KKyf5fE-I5T1R& zoqSze%-$rwi*}$tP1Bg;jaa=pl9azq7Y`sP+xeWd7?4HGBJ**7ZdkN9(ou;+FFFQC z;0wfo)?rQf2egO1VbAgTlJm9x)RIUJ3m~QVwNq3xF&e*9q8h`JtsQI0LX%dYVd>9) z{;mxi^|4mQ5k+%wvryoxw`LwcP5_zUBXmkHG(LYfbK&e>aLKE{0djc=%_mbUI^rQY z>PY{q4vMTmyP=ZosvazVat3Kp^U+?=qv0m+A1QtM;_kuQ!vZ zQhwm9oR_Z8pxG$?0lTV%C2e^tzHxmpA`tBN&gjhW9Vzr2KiG~PYn%3~7j=A;_qa$t+hd-Y+#UDZ~) z*t{#k*Osx^y8&LO05j?Of$ z`kxLixZN6~*maBo5jD_;9C?d7&xdh25)(g@BirdqFWHP9zWP0)qXI^^TSDO$2HWX7 z3laKpF=SyP(^mI`>c0;rtLz%$tJm!X)&^sZO0XF}?4G80i_sk|xQz|IxFKS)Hi8&# zXP0gTR&=cJI280`KvI%Ex8ZDrJeT436_(_4LiLpP=_i}6h{%M{{(q8lj5;!xe|w%- zlDw2l7s>x>RK`-RgTQyYKgR+=orV<=jO%s^f(v0Madak3L=#Gez3P{C5MEH|Ki1<9 z!tfEbqxctf?&{D}Fn9X)hYg^oO6gS#zeAKFbYefVV5%Zj9(`Po7b42OJRD5i&cCA} z->v0j9)ex=l8;9@A@cSgJg4`qE0wAecszZYyI2Hh4x|VjxvyY-*~o$S=u#3Is&{<= z?04IR`EJ4o9`OV9jy$5^_H|UyeXRspo+nhp%6!CYdh~K6e|_L*byaT0S=sf49W;fI zp3v@2Qd%rutt^PO!WGM|vGLJ_S{q!M@}I)uGT}z^7ehLk4`eKLejAL2D7j@~C?G51 z*i5tuO|+zq(&`D| zZ2m*byUGLAm(Or4emJWknHG@+tr<%bJ<#WM~~&vbU$2 zvHl!+QKMICXTwYQ6TR2vTkO8@w@|*Q3f1`Bi}PDSDyq9n5tqzd0m-La5!1A}I=>#n z|I+NPjLdHV^PBBVZ_wSAWoO;r~*G^8NmA0q=Cy(K1uLv?(r zT=}?CJ4nckHq?|%HuWHr(As)UJI@T1#nE2`u@0I2Q!_`(wopGipJbC)gJ zQS|OfOe!)|%PXl|k*voE#p6<`7MZq}i7;6NF}lnAs4>%G+rn9Gw=>%do#}3W{d6ts z5AlOs^hxXpt)%c(sT#b|=<-Lj%v7;4ci)>r614%!BP&6&>kW&QWxl03jIDb9o2q^3 z6lQf?^=*{{0qUP>0x?(8N#@?;R|SEW8>wX|RHZKZYd8=;T7fRTYvp_GD%7#SbX`c* zB!jlb(_0Y;d_Rw`Q(?oVz8r{=+IUtcH(%)!PRiL#W9=M!=MeUZ_-lKag;k{1SP%Y% zCvQS6*r^4tdQuaqP~t6s?Kf4H<$ z1O!4k)A3j%S}Ad0qaLG+Ry<0J_WoQ1b??{CSmZX^;n+r#9)5ptCH|HDIez@GQmMR7 zaj0}!_gfviim|ws#SeeF#IXnw!RQQ5;ySY6{y?6q9^^nH>4+qg7^)Uq(XOb6eU$2< zSvAAaIX+o>6B_JRNh#xC9z6-dp|+WM9^P=GXeOw_#-X`JQ8zmxmhQdRww!amBuZ`v zbM~JfCY^pJU#7fT%P5SM66Xd~*nRTRSSK zL0szifZ~ooULV2$?CITYtV;N>Y9|?zYr+7YawL3~<``78m>{D4CPOqdu=#Ldpe99F zAOQg&N^sS8QqjHaKR+NlodFa?sU$SKp-OA>w!Cfzk7jP zod1629mEIs)qjl#z#W3L19J)86iFWtP7yv|U4B+5_WfND-5eU6f1R=KDX#Ky4Jk>N zXOkci$Ne>yKwLGpU?9ehzI!R99b~|O#$VG34zKUxGhml5<^tCJ0C1mp(2QWf?aq*b z1!*2Vf1#H3q=5eO9Gm6+=8XRWNB)w4{7M{u|3-eg$Nr3w{rv8p^@~jLY;9Bq-T*N6 z>meZ=UIz(*s{;1yr++8Gw}!5_f8}2GP@fC^nZDc!r77=r{*>&iyDQ(_-+iBi5So- zb$|Fen5izHD6gm(dw*SfQb-%ky`;o*59cxd-5G-gJI)`qhqJN@!5K)#p(F;{$Jj^h z?6&a%0b&wj0{E){5RvW-USJs~p-&+g`>eAN-^!nVe}2q*zi?u3w?Tm4dnq<48F8K3 zwSQf{5Og?sxOS*M&f4}`zYss?l0*va6k&TRp+iDF667Ab(Nw`b&0h;@93^m&ih0%# zL|WzT+6=IYRyeq9lxo(myLdCo&7oYj_J3@*4$RduO={a?r{h}Lkh{{wj%J2;^S(Xe zN~J=a=>pcCK!qmb{7TM#Ixa!V5*hP%Hx~=a(yV^i*l!e*15jswu(}D7W!eZPl2yXP z`7xx_?4`1yt>+`Co!Z7^OuBWYD4K1st{kwa^o7a z2*enjp#M1ud9{{J;3C-_D@zs-3$S{`&L0I|lU^&31GWO37o2h-|0aCiJUgB`cj);=n*sp!n!WKXXlI&Sy}RwGm)M+_Kz*NY$+Oh=*>)Ct`C57aQp zK1r8bQr|~lAFXeDj_)_JW)yV}{7U6{hJae{Lo=BHX3Q4u8x;1YN(F{rHcdnZJ%lXq z%eSE+P7xL7jSBrzP(Gy^MDsdYN%0!z1PG)R@pv8jq!Wfw=X;bzK41w$^n$jp=B>>n z2GX9z`7mB(jMVvnYTk63;6JBTSBbo*@XljVb48IYo1Xm7Q2XJjb{h3f-f1w0N&_=u z6q+&W(LBwjF!vwe4-~)g-qSVz=Wxc?9@VHHb)!O?iJI4UPqC?^xBb3x5iORro;THH zbpG*!U)IUbrMhx@>g_!Ga4zIYHjJ)(N1{Xzg1ivqdp#8NR^cBo%q#*%e@RzedXr;& zY=nqjdH+~636;&3FJoN;PN97ac!PS;j^8Xr_4Z*<6$Pjy!2`t9bEZ=K2{0_>K}nxI z)qF2wfRSqLD+M=Y;2I`@6{}9esI>Y_^pJ9z2@5pH`6=}tX*j}L4=`PuEG$Btq(p?9 z^jIx{tYO@Q!>^%_{#?DJ*{=}26I>k-FVwXEyRH=(ER zE~pBYz|fpjEp4B*_9BL;qTBFtD7RIDXt?Q#W7xC%Cd&jT1t%bqLLcTP{nb52ro5)Wg-$0Gc!i z-<&SehMAV|YM(Y_<|cX}m$eNwtMb1IPCWhH^i2fHPPK`#YkRJZka`v=eK+j1-Hyad zF&JanzSuvc*~dgH$i(FWpLbswkp2_roA}Xd-$!gJw;wm_g9GJ8BRTh@`>PYxa^&9; zvqV!0xe5?{n<14TJgN*e-wQQPBl#}^^HNc81?ZZrZ~!;zMSPR5y3L%0dD%yEFpB>>M^Vlp_v(C zn^eo&8zCa$dJqhWUjN5l)kAN_ydZgF>&!}yNTX(hZ;6)G`UzKx7BR5ZeYl75I-2DIeZ<&x)W%;{+$0 zHJOpz6PkBhOacfhjbgSj7 z`b&|pbqBBGHzBg28+|)5CcvUt%t~}7*mZ-@MT}pt){%pCwwVG-3V$hMZW&4Xfsv#H^x#GbLxIFcS^|JnZ4@~+NanW>yv@`Khz_@iWF4SoCRvxSskFlBj5 zWQGQRI{{urbZ+SoTZ6O%L;898f$>XZzE``M2hbgtjJlZ`Yx&rNL@UX4-UU}g=_RMA zcj<@#U@DQtAmj!2IEgYWA#;)hJ!F`3^hD!}Yt0XSKo zzq{jm09$^8$v~LahnU$q8Z-8cD;%({2w$kEFhICSA>FfKEv?3w3r@HInw&ldHwiTK zXW|f^a=O4lUxgNYz7f1-m;B4}xAet>DN9Eb7Lp~kt6gpP`61nidVPCkTC$6%(8EuQ z?ibV!O$F+m#AWJl?lia3EH0w_!xJySqd-(Ln(U^Q&*MG10?Ou&he*>fbmRaAb_#eW zr_jNTWz7NnsPrRVsWmbzEf{v%-O_ZrBKkfbHxu!pwRT-}c%9oigo&GQR$*I!_06U% z+Ct8Ck`~h4N+PM7F|p8-BOobrhkC4w zAEqVE8{HDyRV3nm;EjA~)!Vu_3ZpJDV&#I(VZ`^7bfrM2SGUa&QR$jGpH|~bt90iK zjnxKfsB1&jBTY{7AA2N)+jwIuGHHd4*>1yh?pIW((|6!p(;j!RyGt}GO3Y($c2+cQ}7Z?Vt)j zJ#_sZd+(=WI$yp8sS2ngly9a?XXV>fxjW^@Ed~e?=h2s|ugc36Os8n@6_QwmtB%_H zHVn%@5^LxVi@wKfoLCa1XQE`HB#p&U;&SSr&4#^$rPqJ&qhO39DS;le#0#{r`3vM=M`5T)BIPI8Lu>HFj3Gt}x0v@Bhg_>k^ z@_)p#1oa2V_m}8`$}U_tUaaDDbcnPOwY2C zPX#777(WwjIHwG_QV07gyd%X1jw?`Oyk~acs<8x@a|N^XF9$MQ;D8L4Nnipn(Pimg zSHT)b;G8&T1g6)O{6?~FSBv^+LO$4~`IWu(^FQU4Cl-0X8n_>7562Bb4A#hYrJLQ( zEC> z%+w#QX>jqrG+T2xO_QvnT7R!@G7NnS{;1Pc6VhLp@GefN{1RiwGi8zF5@82khGS;})Bt zGbH=PTZUMADuXtpbaE($VInSp8nD$^ilm8xhtl ziq$cHY2;8AXFhKlL$(*?ln$pD{L*U%%i-~uGDxxojnv_4k}terlm+`49@Y(f`AIl? zHfvLaJGx)Rx=reDK5~T?*Pk)Uc7ly$`@R*FGa(_{XwTKQc}_|*=+xjey83I*H#JDs zle&YqJQo&4H)}6GX6>6t7vyJJ2%bQ7SsjMak})4Pp$IkSRJ)b*-LN_Q!K1{RGZ~1?>C0nL<{{)xf+g|qY)tKX#zQSP zWpS)2nnn+2>GNPz?o4xw=vp?B0QwQa6ed3tmKW6v`~g|At!{K7_Yb1YvB+JyMQymh zpUrc`gQ%{Rd5agxUH&kd1=bXN;eiY3o@#FEjYVyw2SDXzg!EX%CNi}C!sg)pL!-p9 zl~sM)ndxVU?4|2rtZZ0eG<0$3oUbnY3{XxNG{+UDf(U19YH1e9pXD1B`(4NV#2*

dy3wBHm3YQeG}Is8T9k$^7xT2vOC>%3;y5m=Jq)X zJJl|mh8V0Yn;purri%+$lC-E_7{r?5YDxOjUe5FFIi%y%Ch>5I-I*}rT3VlWm>et= zIi?a$uj_lmmsKIG^Mbp*dmGTRP;TCCPj4BqE4GyMaWb=7yrhDQ4_ zsPi~!`Qa%R7uh;C+hMWT6!$h1V~W`o1y@UqNX@XwM+-gG)B0lxI*%`~p zdcoNF{$J7e{n^1V4wpi#bG+1!j9cCV2(UC4{t9kM@`f%narWff$kjBq>Q%?EEXj^M9) z?Gc8a1@*xWdJL|BEl11R*hcu54HC5Z(;<`qZa9Oh#0c`WraYbl?t5yS*<%aFms ze}A*Cwi$-O(1-IFq0HNJbu)2IUT`-#!siZN_@Pty_dzqRrgI+pa9)(zpC?#US^x=q zI+6KBM}KgF%YCGhtG%Q|ridxB4ry_Pp_%V4Bo}2ser&Lnwg{GYE8yDs1}L?iN1b6V z=HME^)dd{V>BVvYM|{kPpDVOavFnL~LYm&D?9~g)MV3t5ymeS5y$C~Au|A%~SD)7l z6Xp2372ixpYm$4a_8=2Ro6y+<*|fS=T!)V8;uBs&08Q59|3c!S$rn zUIxfMl(&DS>>c!dw}zV$KJO@h%9#AX>}qycT~2FViAp1OJiAV#^mFpC#b-i<=I4a( zg9VD0nDx5Wmn@txEegE{dChR~d`eH3ClWQ4lG=^_?NV`3SsVu~3fF7+Sz0n2ualo( z!y3Kx4tEb_}7frj?Y&oq2g819OhlY;+)_b_$mJ372DEu;I|jw z4-C}<<<&-fMWd{(z9^#Um7-a zJyX44zAp2vWh`DVgG8=z*khUgGTJ{KLYQs2WOGnK6Cfc6tRqi!8Q4#Z=}nhqMNf>s zuC71U*HD1AV>bdeIOi2iGZ3~^>{0zRQPNzI>#jv1T13rTLn^I)5YRj7al(&ke+_!Z z+2yA@c~C;OwPU~iJOgT=rmO)ofU2AW!IO>SsW7wGqN}fk%^WsD~w3k%0st^E^+pZ^bHCv*<)UeUsuZ=E7 zB_oorbwQhK@~sNTdl9LM7%Xip59veJL^Q{YQB`Lw#tO)d^3>Nv>2Awrw4C`XcfdjL z5A2d^r94ifT|xXr{jPj+f`uZh1})wShEt0qVTPoMU~nM8z@*HiInef&y^qvJFW7~s z95{Z)5O|T#D`J&q&#Q9DdpnjKqHFOVA=|Kn$si1##Uh6+-5U_{X|%9!nX+5pW*C3eq$lKae~^>}`ZSmEb_`N@2S(LdJF#>3`mJ|IFw zxOMcEqVeO)I(KY+`>A#5e%9b*J!5xOh#s;de!@V$poI5{0Ny>jkYd&%=3PAZuO+dI zm)V>;d|ndjA+?1Ormy=uyD!)2sU)1F3PF@=4P57&{(VJdd7(}P70emN2inzRAa-4i zZjjKVPdc_1@Yrg+imbH0&gw>$4gU5_QW+{#{ex$v0E(DDZxz4DPmj zv#rKTy}E(R9|0@`z3z;1RR3H{&Wm&-h4iSQoK+k+Kl`{yBY0y5^9>VH?s$(D1i;-# zNY@~7`8pCSX#t{X=kx=Wc$NPN4#lJ|kxz^6^K!mao|w+-<@xY_){EmnZr``IHE1m%vCmlsxzI=Z9O6uUcV-`*Ye>`jPA9UP_af zOBU0Q1bi-gCEM}C78-{JUoIQm$ku1iLH5dDzo45Y#BCP1 zdGc`H(7Gsmr_SPN5k;{~!?psmk8fSNaux2P`eHc#F866SI8f=U-ldEG)J$`ZYK`NV zMf2#tglW~U?%#Ipr0qk06#;QINU$m&cJO|xV7o}U7n62L#kH^38VaDsyW0w@XI{bY z#o#xl3s@V}mn8#&Oj&|n@wWa-T(+7)FZ3@HlajY;L`MHppeHiautjbH-JXZyzNg-yVEhx<~f zWU$^XzzhEkllcdA1c@J=bNKP~>N$c*P{PtC?d)2>O?F=aB?tTjoB7(%Z(_`Lv51IT zjlhKrU#yllguxFCVf5Zbjb8UGW8wayHfwN6G=&Q*=_r-yVc>~>q7)7E5P5-+WQOGt z_PZsQ`>Op#<5kQWTsUoUjemj9>#De0UXOgH7n*i`d{qym4!gwfGTt7_j=q(6C%$-syS%T-{pbdu8X=h;BP!RD9Yz zl{@;9∾UY9z|v_fAk{zpxWswGoIkvto(XC;vu# zhCQUOwYA40HSSFOp5OMtTk?l8V$Dg3I^X2#RlltW15=LR0`7bC2p%9{3Nc>&iyFNuqr~)U)mk`4l?Q;Br7{x_&s2Ixnusc^mpQ7m8 zkNOM?J7VtDSONI(3dxRjE^`JEwyWwvT6){=dX_Mk^J|)7D(1JdgZz)c$BqoSSe96~ zIGAc&H2P2!k^!sH&G#HYXc7x1dC@&!+)APkR-qJyAi|!%;MNgfh5thq$IkRW)d-o` z+5hh>j)Rfwf36X7va|i~`sx37jnLKERX1g8lN~;3Q4(Q?k&tj_hmlosc7A$bo{T7I zk+3u(z=Z=!Bv{ft+$C6Ay7$_5*Z0N!#j}6!ajbdOb>}M1S}!_NA7_{jG)zkb%}5c^ zjvv2(QgG?t$mSBBDY)9%YUsKr2)>XIjg7fEA0(p1xc@?Y2rjPi$pKLCq>d-=gV#$Q z5IhZM00GoKXcS1O82CH>+Q6AV_~3pKkZs3*k0*!)@b>`&xp)QfDZJC8OW+{SK%QM) z+&qEAXnu!haAcMj+$C$6onLA=Ji zw{5(N5s(a@CdDv{4Y<=M@nE;>{+8qbpzw8Ggp;XHg`J!NM*E6i|9@==ig)=WfqZ(N z&xHtfc}VE*zwmw`4Zow;g!n*0gaV3Sz&b+z0)Ofl zH3Di=n|Vm*|6x454ZXOxKLi2s?&l>4{{E)F&Ec^V5CB~p9sxImt@G&*?QIYkMcnki zV5{RhKoSH{B|t(Qg981&{q%ahdlVY}S>AXBILOgpLItcT%E`&A2S4UF|MEzyW4vS} zWdlNi0?-r=yvBbdge_4+DTi;~D1(a-Q!hdg2-`u6qE9l?|ALD(>@S?epy!qtSP;?d44Nlw)&5O zcrOnS6d}I~Bbw2_`Uh)uadUis<3WGnSVO+1zJK$1sqxik5!jg(dCvT&j`!=%_d$WJ zc|6wFf2ZjEE8%_~@q}B9ZV~)@Oo2G^3HJmz`Me=mhfIQzA`pimy!{}|Nia?|QCGdx$V5(N zdf6U&bR>Td|0T-<@vLHb21Az0Bo9_*NbfIoI$T0}-UxY5z^JIgs~+EhQHQQ$?#dH4 z+45$H4%ZU^7Q!eq{PJX;m;V)$9Wi(Gq!663;9g+;T ztdn(f+i78}u#|b6%(uLmZHP|zM&MgjcJnJGiDF_4NPy%H6E8WaXyb@6Xs*iIXT{Y= zNpDBilAn7TKZJ&m3#BpgUe3{0!NTlE5`$}h{B5&WNfsw`ckqpyHQIR};R$i>XCGh+ zjTTDGk!+5MRuE#sCa$<8X15O(OXbSlT4JAN51`>4qXTW_ZK!)PkI=uTnVmH%xl3X@WxM*_d9z{eo1TP8TXRmXsD0p;mLkmx-O>h&F3hRaw`^H5`rsI z4KvrW49IE&K>~d-)k0_nXxYLr=@caimj}D@l_rpy zF8(ISGgrpcVPpwa*?v`5^2*IlOr;fPnH83P;wu;E86r7V(ACS5ZnY4Lo6>005Sle+ zJ48w_YQ1qI_dvC!NGTHo;$zwoC!s=qOC<@$z((GWTR8|RW0p>{hFmber)%H^ja?X@ zFHTtl)W?MKRAlCqI4&huF%x%SJhhH>@Nphy6HTQnr6dcUuOa39NplHzZ02z6QqV`nS)`tiEs8+k0tu_^8ubMqA1j`pk+>9GmpPmq9Madrccv1{ z@|6oUmM>4faxsU-hA(z*Rn3RP=9jzISu5g~Zijc-& zIKmc-d-*}0JZdk>FVj~FxI`uE_wNYhRUM<7kVJV$-shCgDIR?uU?CpZd&93HMH?ZP zM6DzuicKVQ+XZ)T76DovPl_yJ{D}cZ21Y7qJ3G^aBL{B?C2Z|En*YHt;(m&DBt8V~ zK9`nSz-OLs?TFhG9L*_1YCk-$(1%9JvKr5Q+&fFWy@mOPm@>rP6*$xO2Fo;M} zwr&baJun+x&BhK+5ZCCWoXC8dWe!iCxYG_oue_yHGNra9YkW)ZeA@$rVy+vPszy+N z%`eYW7NKJp&@;Y#B*62(+0*~vNLzMUp4J|&foerq$W_1;_$ZJ~cfl=Z&EG)v&G{A- zReC#_q`NQ8Q6BJ~5bY}G*y0Z99l*j`p??f7;oxqF?)T?3yS{QCHcy##RDCw(L)?yMebn$;{W7t+8pNR5~_j1|VML)Y!&c{DOLw62ffPycFYhTfr6}!(>yzurvYPcWBhv5(7HHGtI8V(hCJB==` za@SlEHPW5Faz6)kidp7qUXfFeFyeHh%Pg3YbC9&zU~hReLlt-gQD0X_H1~$8xL7QKQb?Gv`X< zjr1m`*Ph$(6h47x)CLZ=ci%NJT-W_A$RVDq4>SV~#ccdS+RbPFIYtGb1)=GOmUZZVL8x7aSF^PkY+x(I@snucI8w3HH9gp&K9j1# zj^z?}7o{80!6gP8H5S*~H;4G|Cf?VlaWy_Qq^RU3e%K?`&LDzYP9dJ(g3m0=J#xYX znvn!JyJr!F(;u&478tSg54}T!KZNtW8#eYA6x3GN|GY}>9f6IzuH7UwP_y1R4FLG4 z>Ytr;s=bU-mx#gn`!J7?yUTO<`Tb`pz00)M59;cADB*8%ZC8wRrzm=^ut(n=e$zYq zTae)f1}aDT@Mm(WG)FQp#_dXuEifRJa3NXo=5lse8*D#xPtUkTz7@KQrnLXI4&$sP zG1K)W>0w4zSyWF`Sisic{fUcgMY6Cyev%KOq0nwIIpsjx%rZVQPbC48|Lfy|;g$6S zFNwB&eWY`lodN{>Bh_39LXxhwD|5qt1hx(m=hwxuLVj$kV^`t1{AxxTeT-yGVQN`_(Yo3VxEn*2S-&z zy5{4jdqX!SV((3JTuyGo?KHoGtq*6<0ncg76Aj&S&^9~_6~E^PdvL`1-B?io`o$-q zsj22OjrLg8W>c;l`$Mk+;Jpa7(@{|W$3*_=sb&*Wm0CRdLI5kQbklJ*m0|uer!rW+ zi)?h{WN;q_Z7-=94jW}}~8Q8;U!g|&IibubRgBDQ`ewC@-3fEte8 z^8nd$%G=Ie0S5^DyuD#Eq3Axqmr|ch`z!VNrIt{-M;6~}0}H-Zz}0H0iLnaqXvNtm zyxq5#pw%`g8-+z=V;FG4laIO%ILYHW43J%b7|<26TC?F!yt_8_r~Tck!O+;k=;ibi z8lYE09Lhd;x)*Fq-SVS02=ofoP^|LM>2zyfW24EuE`}lHPsXnUui*qPG9V1zGCn@R zGuR-0pv%oTVhqjPa`h5(*EU3zb$vIDDJWI0bH@~irX-Fet?|&~ii5Ge@rX$~CtFYQ zxV|JEpfy(G+w&X_tmPkmO?vn3gDpR8Iw4~F@@Ppd%+bk&}bBaoH2H?8V-YS_^Qd5>$VZ6MNj1j3CVAi4-!J3oa9;Bsn=gX|x znZv*cLb<`AX2pxg&BP+@Jx(mK6mTog*P`mNqwR3*>C0pUhf>#WdSl*D%NNPK7O&XS zVnuTX%NO5_pNYv|+Us7_`~IT0uKR8N30^$b=Sltq1C|fc^s|*=jAa2i>~mNC<3>&u z82p6ukYcwFZzqPd49e+Olkfcg2-M9aJ$Bxz8NLHV^V_NEAF$N^r*imFo1q@Jia4kRsZiigK!=sG8WD8`zqMjYTn0O zRTNmS(WJg@OPcHs!F~)}=CY6YR5)x%IYH6P6Xr8}>I*A2bc-f;;QMXQlf)$mUWs93 zudjyumiNHKeVa#tN|1V#=z>iee&GY@)mUQ3K=3W@=Yop++x0NUE~iYKw1RZJp1V+n z2e3~O%w|7ddjo5-K=FhN447h7j#yx*pI(S^diFVtgqS&f<-clpB8N=CAdQ}QU+uj@ zbTz3RVl;^yUKzK$%>2z@i@{gUg~yvp&T>u~cS-sVyaO66sk!hsZY>=>!{9f`J$z57 zl4uGD?Po8)V-np@=-%uOq5Bp&bLz*PBF#|F;;r!GD}S1%wW(KO9T-_j5h!TZSBDSQ zy!Ndzf4QEEvS(|Ct@EfYu1#EFU~%al*`AcM)enC(SxvF4V_Ic8Qjy4P9Ugx@wZpU- z8R6al)B3@V6@H4wNT=GrlA{?=5L2;gz){#5;?U+b-O3?zvvJ~@azujGZ_b1X-I=-bBHp1A^dYJt?AzeEFayXh0#g_jNqxIjXSa3xpjlKr+o zY4&^u;hG-J61-Tq@(()T5mi;u)azZKN!^)bMpqG4lq@^q6RR^QCCz#oU^%-|#(s%H z|9CdO7E_HDd}Hhiu=Nth@^`R;KDdz$Lj~vjJOLr8sLPpqu+eWK1EvCgt(@5zkxgE0 zmeMGWsi?}P=vrj`Y$Z9%yokogn}{_pKM#eu_|qeDF$t3D1H;Lmv_~HA+PRmvj$mm~ z-NH@?c0k`zs2n(2snS3AGC$K@-(a&HaAoX**gvy!r;WVik;=b&XSOhC&}%*Bi4`^N z>V8iMOy>xhx+hfvBg&*uqg|A+(_xuBSFT_qn7U@r?Ten&ABFK2Z0piv^H~Ja#~@ry zoUNbuMg|@DTt!zeFTHap1%EF5GI$~eAi zcWZ?|Mk`0?ABb&X6U>8p+(Hk_wh1s`rnS%I*F77-_1rH={NUA-pq1^wuXS-9apY#? zBF;^O118!UQHvYZ=8EW0dyIJw&5b$-kU77%D3hb}I081^v{@(lac*&?2Hy(8X`=yw z71+860NZsGQOQiN3Lo9~0HyS&LP0fp4X%3Qbx7MMX{cQM(vbsI$Q1CrEwU#< zEtW#R5N|P+C+w>a!-iPT7Mh;X)so87bYSiw+D>#;$sO^m^ZL2G+XkO5LD4GdF1u)a zN>patcHfUtPN$B#N~0@bRzL1IQ)x9Qm+?;~YLyx#AKNjUc^yXp-kgTyH{jsNzj$OU z<}`83b;)tma1}`k`|s1F4Y{A@RzGw+uXzdQ-+zz7nGD>|Oo;@HbLM?rYVmF#iiK4p z-@|n|C1KR>XjIDk*J=G+$=IJsXV2pjaX`JpZJ5vKkw*;149F|Rs|pZBxy3Zx=*_gQ zhv}pP{p_W4(qX!vV8&n~JRp|}5r5YxcdzPu4{xpyx{P%%ts^k^Dl}Qk<`!mKL(M!B zR_Hg9jacxf#Em_4AWcuk9@jD*g;o^D)SdnpVdvO03=;+C+qP}ncy8OaZQHhO+qP}n zwr%xolQ#X(q`zU4v%6F7Bl`4G zO08ki?z}e8OAbyCy69h9chg?xHgt+_#jYtK9|3*bNc1#P*)3e2ImzyTW z)&OPMw&xUTcu#Y-Z42D#KC+8Eef`8YF+`KJ5bYN4dGDo1^jY~p-z(K z%Xt56X!J;J3uATh^)3YOij>m^z@E{_{&sBprM=7UF1yO3ZP%P;Obf=~OWU-@YR+07 z&Nhl2V(8hdoWWEvysag77da}nskmHp1(&UnJaYMbmX}#&T6u1y+{MH@-A(g~n^GpP z@uk+F-8Q>f&lK2=@NNsxxGJS&I3D*F9%N9SEEoj}aI&>x0~ z_d62+P^smAqnQfB|TDZ=li(Q~MoyD}-DyR~Bn?-bnml`&`h~Yw^aKdoE}mJ zSv*7+M@FyEvLJ7Qx4?krOr!E~k+(WbwqABA9Nb(z^(H{4@?}G;GQGMdD=BKZv-;Dpf}sd(kmP!h%B&ZEQ|@IJ-j2tOW9BUN}t-Q2bokdv)GL#lfmp#K*8LeCP1oRPR+Z7zH>v?gfGtpRUku@)^Gxp;Q zH7xXkHCWU3*yu6vL60J96Pb6J7u)##OZ$&toH4Qh#4qvQ`$qW=cC?4eTFHXWq-4K| z83VM6)dhP0{UCiw@?sCze(k4cFKKVm-C89STm6Rhnkk3+hK0>#XBn){ee^U08}6fy z@a)Up;;90e0zs#m-L{vS^uo-g-yV1~^2%xYvH2P}1#TcA6?DjvcugT-F4fB0`gi#f zRXtrw(AR?ONWZbK0i&6+)milREP+yH8UN%dvBX=GA%G?Meq5^JSxMPVCQUP4Wpq~- z_cP_g*F1)N@5 z!#c!qPHVrvtejg_kYCY1_zM_07p1HE#bFDsA9HzzZ9|o#5D~3WNYkj=M2jYS5dCIV z-z+I>Z$9BmUL4*+-aHb!b~Zil>9##VqK^vEsBJY_mv31W#j~z5B}y0XSDp7<;Wz~v z2ChGroRXV`lt>Q^5m*T!P@yFt&*wZsXfGL3jWI?8^-^boHXD)G7>lA9Hq(o1Dku`a z_p~jYK1#c%r`h0-Lq1D#Xu%}Dqi09sjn<@qo!@fGbn`v$W41S4m5`Tct7zq|vwG-e znSbWrvaZ=is1r-UmgVGoTIk+WH)+6ojSATW9f-#^c>MUw`p7rc-0_v_=$0M?o&~GY ze=%(xy5e}bzENHes9qRn;7TV(cZ2H z8-Urqx4a#D@m%nHRso86qS^VjzG~_!rD5J0rbeAnbaetU3%jw_fDJ!1C#lhZB&}P% z=5FE4_y}-i8=Ca<;339JD%h~TI#224jx(Fjzzv5?k;XYhj0LsLQ$@%j`JpCS8=lms zScX4Pvouy(-cs|AQT^!-$L|&81b^sCzbPgG_l!0w{aS#q`D7W_5@GDLEzG>B z{LADoOw*`1z+2Kfn}Q$)U2KMIQMsW_fV=(?B|ED$i%Ul{4e!H_(r3;ny+6}hO`S}S6N`qjR=pc(m=}% z{fvK$y8X)T|EBD;Q~J>ehgt#qjfQA(?Q*uq39h+7QhVc!Yu;SfJZ$xKDRm5{4{_&M zfGW_6T6Q$)|da1O_Lu{w6=V^1U2_Mm=maz z5t5Imwbrzyomjgu@_>C_8`ax%O#A^vo(Ng|zXbFD;;006hL%v=-2Yo5|DWMirvFPY zXJlq$`#-eZ|2w_P$im9N_P-?a|G#2h<7Ohi{gX&{i$Ks0hqax9pdHjT1ONXV4aCzBx~Qf_)m8X_S^-Vs)c851^`E2M2!ur64pVnsQdxHQn z*EiKWfTj6M#it)zW$#DeyF|QDgVIcY6!^)iVElc`f=3yaZ&Zs2P9Q??QpgJnTyP> z9`^sL@Wpec5i^WTCZ=Xc@d~<4!L#jJ$1L_Szf2 zH4+Q8aV-WJy;N|4I;V;q%Lr^`#We`*%*ym(_gCu$e%`1PPd<6;x+05aMTz*ltpz7G z`aUq@JScN@0S|uQ*5HRo_2Q$Q)Zk{uIaun@xPp5EgP6(;EHJtp4(!~N1uPsC4BSes z@8Sx6UaJdJ`4ZI1JtAr(CUq4Ix%50xUl&2Q;RbJ(H|4ZWM^Y$tOBiW&W?QU)<=BBa z@%f~XE!Blw-nIEvpCqmTKNkvD+eM8eXIH_ST9MuuWP*dCg%e&LEly186@%mTWOjP&Uo^M;tMh)4kr7$9ED@QAm|@S=h8jbu|6Io`~4aw z_hLg{cHi;@A0tB^@PsT#zNz=#jBJ3pCm-SagsqeY^ zm742jsW91gLn8Fjf*vx*@bM$fp#Q{|@R9nYyc0lG>O(WM8+ah06VSDZs5D}i9kFU1 zft=qH=#yP;`PqbK9oSrt-0byFgnomT(qYKTO!3IbaFJw~U5=zy7dd$*9?1jH8VMDe{n$@J0rnT33DK(uFp;XTFcw>6U;_)Y&02P?xR~i96q8JcG{M+ygB`@U&PJl zVAgt03KxGEN#c;_>YA%+j3d*G)cQr1_aUa4vOc{E$xZfnTLGQyRGWdet_LRw6fdBM3Cd}QqoO5l<%N`Hq4cJ9ZpSl&kQV`(gyEVxlX%y7pa+Q zzm!t8ku>hTtAaMEALQ%nj2@U2bL!L&R9U=1H0fBLq`P=NxctN@9-WM0Vi!RT?5Jb6 z+#rWJq@W-C7ptr!%vRv(r0$`v6Sc*^g>lW=3VXjk`PJ9QsUdWjw4m(z53Ew)PoNVs zv_l10A}`ezK1uadVAQG;VTZmELs8cS_(dBG6}X3HNL5@jBWmiOs_an@_Z2m3g!^oG zIJM=9uiM5uYg!QD#$9xfS25>#SwS;zS8*!Kq(s`)I&Pxpxu5n!x2@kr1kY$EC>5g` zj^bvCy;ZeCQbwRzbo@>+J)L>pfay7NrFEoF@M-{PX1)>Y+qdz_#yds1o`V&~jCoY^ z{?i`rKJ+wF@BCeVx2wsVLLZl_v`*OKT{3t$7Bx|dD%-guVEK>r}Q zl_-e?PKAFv(sNU8TGCxwrCJa2D+gt|exkAwye0Kj1!GndV0WbZnOg`iP!_ zLkcU!sDpJjNZc1yldFE9g#1{z93MIYdmC*$SI1Hw`1FT zoRX=}rJt^&Md#@4qBRNa3d7hwP?b&k3F{|~hV);XsLa3B94!CHN8rO?CrO5h%`G{! z|2$d9GMO9yRpO#3Cau~g38j6>tyM+;X_0z&PG{PaZxJ&A$==?}550~%Z^ogWW<17i z;&Uy;6a`(+{RkUbVMLiLoqez+%7~SLz*t*?rzM*(x(G#r^3)=Y`Tjz%LOg>28{0*( zM$a}G!FewX`p|HQb?+kp*Jj&zC5zgq@AlRytYU3Tl1PsZUl|-<{ua`3>$9^6_*KYW zXzCL8w@wtY0O_Y5gc7m3TXJ5(LF)Zeb(M?aJad)ar`n35XQE>Vh)#&|t=~4viYVPX zm|;yLeV(RZ;z0&@qUH%M+@Z{4gSK$RXAn{)A4DKzxDlzE4J$-C3^{q)aa0 zX0vj{z}ZCWy(8#G0SJk0)iaHgG~*b0;8Ly9ljsXIX`qsN7Y&C%6BbGZEB6B^m+~Us zX3_kj{(Y8;#^%7x=;UN|=16EG%V*tx+?EJ{!e7?e!t#K7oCMR&9nPnRnrVFC!-+%feAAJx+NA z6Pm1;rgJSO$Y?aAff7YY2@0Mi_*y}4>QMFFHb0AL?Y3^NHC(gmc9t)6)2Qlhh#VLFM@z8TxFn+O{)CKl@en`r%zFa9pFe z!Z0R7yy0!6IcFIM#C?W-V!l=O6^^+hCjMyt@2);>$yiC#b(lN!mN zq)xl$iaQOuTH93xjUxrSxaN?8s{@r1eh24SYanJLK@C9Q#vd?@*|ou|#K%+)Hi~M6 zO=Lc7YukU+sWnUe)OpjM9x{8vDHav#L3Arcg@ap0=O}hfjHv|zx&r=kglZ6!opYSM z3sf)j=<5D()s$W&XJ{)aLjNNg|Khf?jG;Gwp?SYw+miydc(&yU=EOdma`Utn6|yO9 z-H|Z}dC0n)vM}2)?Gb_4UJfCI|3{D?oT=e^AB)~Yn0tgnjJ{N<<0+LNa6@b^q4!Vm zs?2%{KX&dpV$Y23!nan-K zbv+*M@p0-ikwdL$&QRVcgd*9hET^^Ga6rv>_>_c__{_41B8ah{Nx02Gg&{+`@FM&2 zZH7&g|1iiB>fj`MDDx}=vl>0kmVPj>Jo%K=d%%<@e4PaV3Ovaspd zZmKjLxMlzcv>#JT9-MI2Q4&En;GGTGcojRd5qqmr#Je_J^Jc`e95jQT|t4YF1!oQOGerbx6i?=+=ji7QkAp}(N1S#aoUJHs^ zRc7+3MktRaU4=W2+r%o^RxtJg^Wy2G=4bvC4AvqW#A=R5yrZ~8ocI!&dR`n$6gmaS zs+j0I_}m2hb>WHSc?C`A^GJSlMvtKvd0TI%(R&Gk`#PdV_7u~q*+I@)^ZE_$3Xwq0 zKbgkig4xuu6bp>N8%1_D)P^;b!)Ibgy|becFmzL+<83>z3-FQ4AOO~k6R^IiI*p5e zO}6;#ch!UKQGuzkOcxu!$^7JC0&?ML-AOy`J)ok!mXC`Xh#)RmbM z#RAKu5hfVW;)1f>L5t$&_C3d#S)KL>CCM=i2x~wh+!N8~R#6sq=~5(Pyz-&l#!}Va z%qA0d>%1>s>i8vimeiDZ=g`l+BP~lg{rLL_SllS%_A~i)bG2twtb1yy{)to8BiMjk z8H4=>?uHm}J@d}>@-DY1bF&JvPiyqWC>5l8o_c;7&Lgbqksm1QlT@osz?aH3h_K1&rg_6cdj?FJp%sYX}1>(j+zP-pltwxKlhx< z38YL_2u{Pysx2PzfHS3=z^T*&qe_swtN7vLX;u3?As0lx`?K5%r_7o1OhYgH$7&*W z%q(cXOM}no5nf%Sy-LQ11EamZrJk)K%Hhb+t(N6}BAHSRV66Z){zp`|59BfhYWYx$ zQP=)Zt*v~16()BqmtsqIpj`%N=RH+_BHLzJIzi{#Uk>scNru#nwQhy=4eB}Z0_kM> z5BoGg^L4MB=*&xW3m&qt2_Hqdv(QhmKi#C{ofV!Na_Ux2Wzz;ZWG2svS>W{BGg06P z7K?JRxGIwhiYeH42dV9M@e$nNN=vQ*lkrY-);atr1kXzlKK9BCi;yG2@m(r>t2-jm z*G=pQWGq3 zWo(ls0tMO%!2zM)_lK;^0#$jNg6juN1JY2z4J}*fMLd5fzx44pdD8~T}QYY`Xq>Kpf&kP)oegLYndB5E1nb=W5Jp5!dM(ZZOI%mksR zgH$;c<`I&~mw!kHxeODcA3d8m5StGtv_^{r7M7?FvWw5qm_WOs$!DcxZ9!t~bLHy* zAGhe4bk+uBMLx*^=B!7kT|3gMI4M_rX}h(+J~*@7OmZ-m&dRZk#Rfb|lph zcIlZ66lNz&7H0%Invo6^2z#GNCGu~uaFby|Uot1F)-s|X)r;s1+b!<)l&F##n^~&? zRymq_%vgF!?aa}{MoD!aA`h12T^r?S$&GU z4N9)UlslhTY@V*u>w#3@d*k-uEsdR+nA$F$(xJJ*E6~g6c#@{E>)+Ppa+z_@MMwFv zDM95%W{y+v23yM{J1O$d*?oPJB+U~%A#$w|nLXj(@beqml9Kc|X1uu}>uhXX)5flP zFcZct=+Mza(&skD#i0?fLs9ny18Ov^L(qQjnw+W>L|jBape`*2 z-PElT_Jo2*i1c=G8o3Cin}!=>{3VyV;`E)MPQmg_tY{uKJpHABkw5-xr(qsf?88=S z8>_g;WiulZZp!EyRatV~o#}gSiz1WP*iyWfDu_pc?O54c@mji$hfUaXO2=}%9ng=h zvnc|Qlf2y-^q%}{V*Jc3h=e3P;=)C)4FZY!y{9>Ft^$y=iB3@~1_q}D;%k&((KBE; z>11K1E|1YqSejeE(LBoxF0BJaLj6YYX?BZjuWZ9A*XVQ|>;(rUU@XGjp_FM1-TK1F+z{-T{T(rI?QE$P$;$G0UhyaqjFDN#of(6 z6p;Q-S~PM6xJHaO|Kj3zZxAW5rRQ$ki<$(1=yfJl2SqS`Y;#0>G{$Rml0i3cL?GDl zT3QhH>Lg{$X4}p-Yv{^8af^i=Vvojr!O zA&7Q7y2}?pmsDtV`B1_e?8+*He|v#tum8cAVK}sWsdKU3Jux_hy4l}~Pq(!17%!`t6D1v9qp!v zhmYIS&i4b<);x4D%t7Ds<}Ep=>Id2T%DB!3bO%6Y#r(KK+oTU-3Ej_(2pN&HuO?mJ9GkBMc)0M< z%3Xtd+jP*#al2*5Lm>wLO*}xmM3jcI^2@{EQTmy=<#O{WLHZby{%Z3Dh;bK8-q!e% z#z0-eA$&N+y19PWM2lAv zepJn!2`<{1>()bxra$Cj_9FIyl~MxzHq+}Cb=3A`e?51;daWe-Hmx4-D@P+I3vUy> z%X4)_*sS~u&%Z{=;-2}|xF)gKsh&U360!cIC*LtL8c3vDKm2dWgR9g>`1jwAgtYpf z%nFKBL>D^@6@@v-380=7;n=cy;{hWscC%xVA}9WWlX4(UGrObbtRC*Shu@;};#GAH_0ZsYZQY0-qE0OlH*^!L5v7vn@+ z|Co{!07m2@NL(%zj&Bn=#086XM02T}F(JuY1_v$QYlNPR6>F5z!eN51Lf~)6?=}@2 zsK~pHX>FcXQWsHTonZCM*D~O>A?;1MNx}(mQ+cC@cT@)eYBo`u-U!{!;_`qSfp1>< zXZg};V7W|g)H}z}X8^|>OjaWRBCiR<>9x4p2-Th*yjyca1|a;|_Dy=^3M`R|I^-x` zKeFS~@rMVnmoxQdV#{pE{yKmu>=lAlXrto%Q3c6GV2+l)W+EA)1dCuh5Ukn32Wr6+ z@aCv&If^wkLCiTYVqPIR2JXT4jFQtpxOAV-p@J+-D#02bzCit8Bt>h*cAl z477IdnMnb8HDNB1NR`w|JK;e42c4vNo2+Y<=4AMqD^rN>LbG|#V>i^W8amy`d35?K zgL^lOtV&o))|DtGqe#SO2j`d4Bg%V13yCxA#Nn*XfxA+t?~>jPLTht+)j?LSX8>UsK$S~06#%|n8c|K6$X}aK!H%;6c>hc z0OU9j8^-93_fM0@U3Gibl*SJ~^45&c>b?<5JM*_&G3t@)$Q# zzE-R*b451g5Vps+(_z5D1HLW#U0A9fv-6heQDGkF-7o&dMBm-XCs$r0v!?98dXQ;fDsv)>do;xTOrsXlQbQt* znoAkx_=1H{w1W?v9LcDJCf6hP<0zT?Lr8ESX<+-hy44PLd=%;*i_&nK4p)f^B)8%1 zPwE!F{xH%%ukHF{E@|9K94U#1!UOCY?gl{}WT@0Kj(z#2BxGCM+mvl-+2P`Ec7^;*rwewv-tZQIz zP~sD4TQGE;Zc0Y^mjW#n58^v5sStAd?t?Ho`E5rdVEDS%J{a$SSO@bCJ!S5ZOKF}w z3gPUpX)z8&v|gOS2>;np6)Gl`kQK%FI`J4maa&*&_6kA@ool;#)|5QI$qhsgZm54; z{?|Xs!j+MI>toeC)TiVzwXDpI+-rMCUIx-MUTH?NiezBlP%l~RgnpQSEkjc0m(#?cpdO(7X4nQpi{_n#AC46-)?!S z1Ii}H^FzpW+s*YQn?khVk>DXZn`>@T{>dW4s{tksu}E1aCLrm&?Q*L6CJfShhdxOM zSA;j~mCv6;8;YRO2o>Ju>gVVnh2G&U?5xyW(7czsW&)M_-VV;559V{B&yehR2mW-M zrjr|*O!VkPJ_&<8VeGgtMug`KA{|BAl8o=**SRwz*C!}wBFK^$14GQMiNVVgPgIlG zOTOrWKx7jBSq5Iurtu9St$oDfmUn%2VN?e_4sH;R9g zNOa7%KT;@`W+Z!@Id4yG`*xM-HvzE`Sx$qUT30U7UjhAkObM129nF#dZun7iI78Br z$(ckdp@gNt^R)_A4~wbetXy-SCRYmsJOPr5)VK^O=5vdY*J!!!r#_NdTNL(^UX*(r z-^)`u5Ub^l=VR(tw#S3nF9E5Ap{%|xhBCC)I-*7oJXDN+3W6t^i*_O%X9S@=!x*@UtcdEMq-F|AKVDshG%Z+JFx}1gFa4_992dQ+^|JFzL%-`8yZx4jz$ZJSeOXcT0?ULW@Sq9e00kA` zjJ1ciFJ0GNN)nIewy`t*3JGN^=Mn{Um65;C9U6>z=dh7~jH#g#LEjBmdy%x>PhWgX zV~rn+gCg{L%`JmvCaQC*aMs#yw(m3=F94C)?PiTiedCxrw)tWxwac{x5-D7=CXmDZ zUB00l7Sbrrex$dLm1TA&@NU^!($*t4g(=b7j2fT{=mj)ML3`T_>M2>_<`>gsxW{W4%%klK#so`s=!N81;NmTuK z=uGi5Nn(Dk>m=!vGKiY6sBj+}X;aaee@pmM_l*Vtu1qh4g|=xb;V5KF7nsd`mc$k z#f_>P)oN?bWUxQ64#2^LgIFMgjaqJhMO(ZS>t$&}l7JZ7)yJkTkEbX4MlDZ~YllQK z*Q3V!V5)@1Vg~c12F=X#j;VvbF?f%-ue(?}i)q}bto>`;TaA0aXr(5?v1b`|e3e1W zPvJDB6v`Ygt=M0VoVWK*_JaLeLivr+zN|FylSJ5ub5K}0;xF<--=$v>W`q|`6(q_V zR@96F&*+Il(XPFNKENn~-5MO-X>^@{Zee@exO%?j8dHz_BAQ({s`=|Ea5A*Pf#ssl|INIRmL}G9^HbeD z1-pO!CGWnc5(An-@O2V(HQe$utaRu@0Te0=l=l&iDy#0noas%I#pCd&-h#7@3!=5| z8E&g1?9BEm{6kzRV4Li`Gf zM(ru@RW68-X@{DDWR{MLM*T{~Llts)o5155Vv|`1QAPr;O%dMhF65C01yco8X_aq0 zI#-yrIu^TQ@gCpjQXlk13NtQOQ-a3oO0AmquiX$tBChI%H%>(gX;{lUW(CC}H6^#N9Nm++42x*=L@#TkQna3_r zQ4S9JkaTDP3<+gicco4?nKVBj2$zN3diz5(6k^;T_(0YD_z3uR<&6%P@)Dy1P@ukBV zno^uAYQ;WES%6B$OlwKt4pXB!&M&x&Nz1ix!(l>HPlEK=WggRILMVqzYIM;?S(qrYatDLIEE=L$h z*1z$WW0TVEvVU7zh=K;>Y2|j@TYTUi2e;^u2W~!vrakEvAO=v{l`%Bdi)Dec9@#g_ zMr{}Wq32{=359 z7N?XG2!j6g?DFi3Dge09((BA{lM5V={y^S?YB}hc=q52wUtz0LJ{-0k<{*cA=Z(=8 zMRk+TVQ>s>}lA%Wl8ovjy6bEJkxV zlD|mXd8CAyv~-heCEABP2gffQiy_SnA-`)MR|&+YEi5wY2k=kA&~XLJ)2ufiUis3RnOr( z?+17=?b2q!CjNO9k@ICW_c&)T6x-`x?~bJ+j*#^UMyK8o@XQgj;ZAKa$~!mapdugI zzuwtQRI@L;TJ2jeRazB#!93Q76YQkNMmSgz_;o$afr)L8s!i&cVu6+q>r3<6^Vms^}uzHJR^vV?9*s5k!6qgWx`=gn0KLFwX=Caqj0A9Z9V5 ziR?3YRNzD-HVsEZ@kQ$s3hs-_F~&GG$A-bx`PvgJGZS%6jW;@}TySPqEepiP^2}Z^ zO7{5fKL|lR@w0v?RuzxC+S;N@s=NqlsAU?h)?h{y)7BKn1;vW|h6L#Z+fVBorJy?H zD8E(Lt_Z#)`PP5sZ$R0VGHI>#X^ zN%ABCcFVQJ#&Ahlx$G7=6H<*-B!1c89cmX7Y!K5!xLi(b@IXcTaHa{6(nGK(gxx4hHn*G$beITnl37qGT|ut$V78LN_JK2 zScj`HEtNY_WwQ_>X=B_=WdIkkQ)xIRJE+eH6U1lHe-ipix91Rm?uASaDXo;EZ}74< zZHACmL_|dPLg;0sQMO2(bYiq*rhm5|kFt zoIi9bFxqt+f@UzviihAHrgJOnCt$C4v*anCtZlKhdRe0=Pi6-W^}f<`fw!p3P@(3w zQ#u%=LZu;I-n;q{JOnuVIF*PI8A_|k&to~;Fwd}LxQ;HsY0ddk`{mYgvl zLSE34QV*W;h=vO^lKtwOwjEq=>gt$(2w{uEy2Ire?cg8)QUt&T9CV9sp)bCad+IOOSk`xJyV~ zi#@Ix8j*Q&>hU{mk%R~1*-*NKzVr#l3a6v(O&0uSESW@g@#PJ_D*Al@8TeBx#fsM` z7*C5&5QwbN&;N>*Yd8K;%?CPUua)_BE}*_Wq2B8W-``kaT@mm zW=T23fuZeI=X<7#nfk?O)K?*Rjp!3KtD8~54U2ZBj=OOwWu+yKah4fw*uDN6o^S>|%-U%(#IO@^GI6a!i$g8R^KUgN zgiO9+fjGpo_R~!*da4BRvA@;LJ#B=rY>3VRgyDod_SrQ7ky%}awZ2M=`2})n4`a>T zcOuWow>-jYpR2<9fhYWGMnU5+6{)Yz13xnvn0BKi16rio=#7;PUFaij`ks9nM6-6w z_jQUZX^2XkDPMQcjLh$jxija3Dfe--ih(yHL6YS@r< zz3l`!_SDVGa;IOSPozN{o#do$tjpjHJQUTG3a8Cwbp-f(c%{mvu^7!B1#GL0Y9M9q zj}L@@AZn-+I7s5SV{HMc<>)?KCV`g=E4Vo6UzNE)N*haJ+X1(vQVMOH`Fmmx9y8#= z2o9hvN9rAScXTg`p@%+uu|!dLBxI_KuHFeBFRi%Go7fCf4%`y*p6r*#%!yRxo3{tx zm={JEEwV%a!bdML1duW)_;DXUQXVx=-z?X}(9inC=f%ODh6-Ikuv^PvI9D%O^(D(s z>5}oR^s9<1-AfX$;UuK>mE6WrBKy_05@Qo})BcARU>9BygXa9=x3lU@4k=Loa5!ni z$**k?{-)ML!>@ zir3PAW)X{w9s1Wx$dn@W)r;7PEMTzMWmxqU7y5K@OkEK5srJSZ zU%@iuirXwHSGggDnkE`hB9i} z8@xISe2L~ceIr_39E|QUMKjv8Q&-<;Q?<{8l2KLKQ92mhv9}M(KZp8Khn?A}3s|Ay zGZGLn9AJu>vczgn;X#-evei#XFP9URAEcyV@FDbchUk&+(b9F!W2H~l{}8wfQ`3nH znW*-~Bz&c@Wa zj9w{+_!6LTV>@4H6hRF<+^PfH;dpLy{)8Jss5qUJZ~muJsFK z0w_p)*=3U{=gi-#CE@${QtugGk-SZNF*@6JMO- zTZof0)zmVC_4@629_-jmj za7ZSAc4~fjJf_m}ow%o53d`+5ET;XqzzFNSiGSX+Zh@^KZFsBKaPYoSa?XmUgvWo? zGkup0s6fTb&g8A%=i_pxr|C@l$LTTsd*`kqs$$?3!mCn>8u~YHEA7zLw&o=NO#NG& zg8Yn1xFz?=@|>kOQyOg!z+4TM>s}&wDBqJi;eQnY?24^X53S7%Xh(@_d4AF=(u2uc zR=BlJ4cG0r^`8t%hhr!~sHavBHvW-qoKkFQ`~fhEZXPD2y$K;gxj9Nlk9fx`7+Ler z=#g%f0KX2CdX5|>M3#BQcS@dxDA6E-E^uP&&@k)_o7`>`lUOQ8wjLG__5sG83z{p^ z=-He|w*ZW`E<&CqM3)y_3*p%b3P^7>IdshFEI!6tJ@z_%%{zk;t5B8V@y?f{7Jgsy zX4hP2>MG?!G?oXPMx9^^+Ur6=C32>Mk95=z%|r13rF8KEKX6%2@LQi}Lk3xpAVXN^ zbOOOAseH+bKKFA(M$R(b2K|39_Rhh%MBlb(Y}-3FzSy?y9Va`sZQHhO?AW$#d&kbp zukJmsPQ81o-mUt&zFwx-z716gGch=~m^qQ^%WiGuZm0+=l zG=7d2+VXqzU{rlu-WvV3sKMZb5=5mTYIK1S4oD?+7VFjMuKPlglb*n9ZS7GGU=txr zd=i|{pHYw76&@l}PLRf^)}~>WU1Tc1J#_Z0s9<^E1n{Zt09N5t*+frsDEcE4g>YN*OHO37fMilt^xnP zcHm%s`CT#C+*M?w^FGWrEuc@1ybEU!1Z0gE{pI&L(U~kw*eTkY;ulKPaM1YX^Y(%@ zy@zphI-y6Q)my2shxXd$B<%Y$Jv>CJ>!K>`%l9}N5`#+1{CSGmqh+C}GAx90L3sK* zWc9>-mwAF`F%6n)J|dnUhi6_JpQ!F6QoV?9k>r|MVY$ zR%+==oT}1Edpd2?LMECj_T;-gG_UZ&eO0S|cj5KPNe4UT8op9}8-fGBm@3M*V;#y^ zi{HMtiyN>^*Pz6V&Uyw_V^+NzYn)!dF4zR74xzl&KzM_T5srS@aQ}AivK0lt?!|QI z6pM(uCrO`GYjjMqk&MKKgzYtu?FVd$U4kl9g6s4sK>tYlyMk-qTMLTjMz-6-ri6StGS*1m3{{%u za4aze`7R9c-)@nLD=ZAPRl^jxQ;s06&ME`CsEc;KAyK<#+x{nrB8giD>1{c-?TnP9NWYLfsPeS{sXXD`WcsGvB-6dn;TK{_m7$``ns<%hl@sIBV*%@3N5XXRkz-=M+C_GfaXNU$cATAMfb?x;ZHjoR% zDu<@PZmcU%C`d0-URUJO4H&`E@+?1eUshf$4npi|Av1g#^CtEwSV$jmtRk!N;nz5- zjV16m{s#636lmUZY=)Xb-o#D9mwv1bfnpmt);^qH;O1_Rk!Appn!agTSMa;hrMM}8 z;x?D%cUB)#rmtx)j2+0D28ZU_PlfldEuOb8Z|?6J)Z3kkR=>W>t#1wCZ?jJtNWdnx zvF%h$-_R!(DwJ=OHE4*w-N!YiXc#M6@6c=N<~aD1{3YG!*$g=2*EM#@2q|q<9+Ic4 zANnXxpfEg)FC4V~$4>p^S1!pbzra7=pxtk%u5WgS@98;T-2Lwhp6{*ZtjMmejNkzy z(2r(QpxDS^5D>RbRv`4=8xtejJ+PN9yWZaKk;Q&stD7%=>~9_IUk~cb-?y)>FSSSs zNSWdxt8Fi>W15= zwiGR;#wXGoZX*H`kL1V7;EIvr*q-9RYBZI8Z+{4EK3x3F` z;%8+UANl0szrn>j=Np+}~tV;`*~=$lD$#m5#sVB3?`|$hMLgY~i=F zS0>5yJ;&?;cSV;}?b#YX+$&hO>&ra}+WL!cPsHfT??U6E_xC9b*orc0>$F~F?3Ptm zpeCZt4&t%|A}K^37v#t0Pra^&D|EqL3`>031}p?#KN7ZJ1asIuE#(YPSw&6b&DETP zE*0G-j>oB;>9HLD+NxX76>hQTh%Lw_iL-9muGDBnAI`at+p139e&^5`xvxJdd%VJ$ zE53nzYv^P1&kI_$Lp2V{b{?+{vo(B;Ac`NoR-U>o0418quZ`FL6ige}e`vcT;GaP- zP9)%I4M65!%B1S|dOf1;4t8BpvnpgPp&Zp3ICQ>6Vi;No=DhCM8UnG$=#-amiST^h6d%vS zwWMj^cPmDgyJ_T^B@=?kt(ioNi(t~6Ja282P#QG5NWxcO1(6)%q$`^TG3?sJ zCW6!fR{2<>VbSRcG5FBNDM@RWTL_SF_Lh#oOczT<@|Tivt^ob#-5ZrT(} z#uax+*cobOTtGCZduTOlzleBf3B^)@zwlmN>JTCNLhd;Vv(C`I13zGu-9KAc%@X$G zH`$Gn+C_ax@jL@M-&$BNjt7m+aDM%if%3~JM26kD*BCmWNer5Z-*{Mrv}1){|at%5Zz_Q8~>3J_f;s} zyOEWvsj*VPrW^_8CW|53ootEdd;Jnh<`|9y$ z!lk()ApCsJYkP|#=4_N>k3NT46bN`eZJCLo}+8{O@^zFqI72^tc|Hp%feM%E$rIz`Pw+kTGYp44C^c#!K^mVz+PZ5i#TWgvqEb+i)~=0Npaw&zh6>8>~S$j zw0T(O%2hMgNOP|a&)9NZ&4YC7wktf1OrcNStKHc@0=aP|uhh&;@UNfBlvjc<-UX5V zlueSs-|^<9qtI8IU%n*pPa;GNcGTg58!VrEj?PFilk;)It6dUHz7?ZYzM|zD4BRRG zZV?1uul2tMY`|1iWHVTO@;1%i7`V> zYvBYWx18&(?QiW@G0%by=5j;saoJ+X?qCzs2X+ZPz1{_3)F-}%gLHp^1zZ2?Ep0RM z<#bR1e&&#_eMZ7g@|XhCDr2?=6ADAv`b1y?iHl3}uJn(Oc@#4@OQ*p;iZPQ|8VZG{ z{|xA@%`9R?Y$`sx@&FbjX8;XFJqbG?h#If0k%L!n_NHfg3Qf?nOhZ%&H{M3=BrJE# z?@H6|J9|UnvUu)PYE;%b(R4~wDRuM z_V%?-A}~Ft5PTp_fP9_Ny*0&`QeIdD3t5lX;C!fMs??Ni9ur`Bgoe&&LiZP=C{%|G zm`7sRxV{az-BzO}06TiiKhIhrFR%FZmk()SQA|cyKAYXzt_`tgLb((^9cc3Fl6Dy?82{$YN+u@|J&NQ0p-EfFSuIhq2On)s?R{%(Xq78JbwLDDb-|bq7 zT5<{vQdGWt7;qk6+qC7aGd7^Slahi&Vp55P=Pd1Xs$Pf1KlS5tl?Gxi!ZFvYdzu#SmH>zwq+&Y8T`$AO@WK0{&bTXPAvZ)r@yNW8;L^4tUq_rw`SXB9Uf=V*Z7HL+ zJ*sP_VX0=jDaH~lSx_H~kxMX*trP?!ab|% zl#a2p2O4ib&^OJVR)*ZoE0j5IUn}}lx;R9~McJjOrm2P4ziX!4xgF*tPL#T_Rfb~( z$hUgeykm{)x6x`~rrA+M$5RMs^rFGPTj{5m*Fa`yw@!Z8gV+BA)ZTX8v*|E}j@*&H zza_ju-<=MMao0cdo3L_1txiZXr+0l8RTU=(6xf)OL~I7ooJm?-rJH#jc`a)NJ-pzU zy{Wz_x1_BQXmvhIHx*YAEWb+)OrAuK9WRh@y=Jaqrn-64hf6{7sGd`-9MY>Oblgmt zMR4S|3`p2&s%u^%48&aDVe8AK3&43q;3W_v2ng@xhYfq z1`O4mD65*|;4P8q^`!1}3~mk_n>_vdcKg@0g;-k=mI2_wVOufUI9+SxQIu{A`#V#M zMQ-f6(US0X(Gg0eS}t}aG57%jo?@|d902N#AmiW?Ptm{U*96BvvGD$0a!_s~qwYcp zY$fZv761?g-GMx8XGV7`KuKUA9O}mi?brI5*OrQ+j$x|Y?kO)B5D4WQfEn*H8!!+SfG~N}R(qqCRIhe_! zvRt6ph_=kACG(RnSh*>EQ%ZSIRQBh@EnBCB^;m@i3W znz5bOXNP9uA^zUIOekT_jhkKjRy#K^Y(mjo*i z>$7=mw&m+w#wFfn{$7h6(!nnra)Js2l*Gj-wyRw=OSdySGbPrNv-|jD=7=sqxV5;n zcHs#j?+klO#T$8v1Nsdrob;FnoRjQtOHU9rv&tRGB1v-$jDbq(v^G;}CI{F>sB7Y* z(I*fh`S@du1mTV>hMpxR>i0#G4$#W$VRsMh;w5EZZp&8g=JIlfiQw9*k(?or7q_b= zfq!(nFo2n(4YFdiTf@FHwDL}F#4^yR>uzboIPyWd?t`^e6y5h4c6{OebF%WM)H0v% zNwn9z=tg<`iCrcOe$(5Q;IVVr$w~U*^>x#0sR&F)G=Chm1;_m9{IhOz$vx&>Kb{KQ zaBaJhev+YKPgnjF-m|74&{c_9el%{~95Q^uBfkv6<~~(BIdL|` z>GG1q711KNo;Q-@>|~X=VEW)q0{7r3x3jLd!86ZkcTW;V^#>5vQ!8=`XkJ~FGLyGT ze`|viBSUKjAOX{n;({`& zSM~WecumMT+azoHPBF{_A*6k*Q4m~fD z$SUkWV|XvkOn+p8<@Cfpb67$t97l<*=xrVSV&>|Zl`?zrklrlhd>q8asE)kBFabTDBCLIGoA0}UvU?6nm%V3Xea_f1p6GH}*S9eFoQCX?^+urN}| z+Zw|RRZdNi(u&Wu-c^v{(X zno{+}-xGOOZfn8(O&orls2r#6JOzc8dk1@Qh{z`yiKW6@(d=YFwGUKJUy}PY@7RoR zk$5po+^7VVWH}w_ z|AjfB3L}LNCCvPV4~U&i9|IGSuaOZOE>%z4!A-(tdZkrc&V*GBohp%sl~$fgyd%2@ z8Gx^PVJ-P|FAfOLaI*=ZOvB+Hr+E1~V3t%F|5|epa>^UmVOC5HyWoT9hhSDWdqOIAhGN5CBSIh((rx?`B;*>E|Zfw%EqrU+C~tHA08dSlF}}0 zd#~~}8l?98=lKq%@Yg-qKhTtK&E;Qb_S_d_pJ&2kP;7-*8QRE+DOtfbjq!QGJPHNB z5%V)YQaQ2<<2iG)xbkW}fa=mqz|sf_yq3Ycyd9nXBZkHTiF6Jo>O5N^nFw~p`)J)D zlIGRQCEvWi&NG~+%&-MQ!^%%&l%_;HZoDi5fQP$Eg0$>R#s;{amwTog*VQr$ z@SpQZe{s$R>ZrecM%zM;rTF6$B}yfnRTE~+TvO5H0@ZZ`uG?Y-w_LR-cOBieo)y&N@O|r* zuxRwdiHCn(==Gynll*i(+fzo9byu&%`JD`sctq$~${5A>tJEkgg5Ch*o8IQMcUe*c zHk4Xa{EI!eHia5ccS8c?PaTuTOrPo}NL6#xpR|%DLf6LvJ1w@nby;v;-PhI!qUwo{ zv5#D>tzJ?scsVZ>P_5LdhxAVn$hRy$45#%_h!O(7X+a!$)2_~m7|Ki63S4(%Vr1vy zHr`IQS|N=_R1Z@lv<>7FyLzXi7*rFxAfi=iL)&GQkpB&b$VQChm=l_a7*;jE#9}Ku zq(9r*~lQyQm6OU{1YAC1Lxzt1yNv^^mb~=&cG@xEwvrQ@8eNIY?dGIPMv5r zD636Glvx}m+8}rX@~ZvU6|zwy+KWB@QKE1*rrzDk<8hJ#BaIU^2O^i2Al&=1Vs9b7 z%|V6Ar*2;Io>5y2>EHv{b+K{#Nlpwn0OxAn7B!C$;O1S`<`}Y-daf*PaqztlT zQLv@aDFx@dpFif!gbpomP=iyKd(;s~%G`C-DF&xv^wNmK_!UUVH zW+1=Oh%Wa^MNCAAlsuLiu6qANbIkUY$Z}Rfn{`|zlbi#k8zmMr88UR%MJv{y=^Q( z&n+Ns7VjM9kK!V7>p^-=e*7Dw#e`JzDjz7(2RK;8xwl#{{LAi9-ZujXKBTgP1UX}r zX33K}~O)II0Uj14Ts=vVqn zw-L}81J7l095q1L^6;U>)=Wu5IA+NOxOI+U!;y5;@Tl1`E=OG`wpEDAuM;*8Q5t1CO!u3kbZLpcg|xK$7mN?^ z(SrBRNWR1|u43;i6)qAMB0IlSF>|*AQsabG^%9=BJ<5O^4l5^@JT?W^Yr*I`2Re0} zih@YM9#dA$wRKUVaRTRAu?tDE{4YAHi~1nL*y$aM;f;PojW~a#3)(VPz8czdbcvd~ zn`aC+npEznJHvByO$@4|PX;ePU*Z=I0$<2P`Ds^1|6L{m-p?+p6&f0lOna^Z)S6D0 zaYPI%`|l8(3T@>waf2x^+ZE1ABFzxul{M2vflPUZMSG8}RX6R-uG7cBovAJ*- zs=B0&^J;gNj-;=#d_f~vENre?BS`LYxhtXBwxLD_u;8#``6tVcnn>4?H?uh7E$4Gn ze=(h%4#oKZ!?pxh%rHZ#r}eXp0@6CPik)2y5-iND$ZNet(1-ky)LRY@tdMeZtE3aM zCW!Kkp*jd7^{l~yN3lk1U*b9@N*=Bo9Y<9RMt@#*Ynkxq?n64Ml#9rjqdo7 zd;SsEYy`xw3n3j%J7w^nt(V^$bU%PkNjV12?XQK)L7iw*Wzm1Rhw>$IH@hQ+GL!Za6$7O(xN6&#ls zZKLXt>dM2YkQ>jBBf{RM?LQT)^*q5dzK@;1>ayuRKGwM<Z-yV|dDX*w zn{Vti@in?gtIGH`mWZq8qbkC>lvHU_yH6n;O;a^&1;&aX1S1YWSOe1S{`Rc}=ygkU zcFHr@%uP(m>$QH3)ZbX#iTU^tgYQGzc7xhE@Mu6XL5Fy{q_;Np!J(EI&WG}??6a6Hi|*tCASwjJ zty2Sn+MKhCbKriHcphN0A-EL4O(|IxBHbSdN^K9@=)l1k^^|^z$TGB&IFvJll!_~> zdK4Sxu=f03gmW$oo`y@#cXeQMuk9Gh9j{e@42SI ziNi{$+_6&_%US->O4(;QT~Eaz(J#s4wiTW9H2k^vyEW*A+M~*rx6BG zQjA0y5*X=%NdnMJTt+uxL-xoqX7q-wP0!!MZK%}0F-zXXg(7!DnL#ffw zu9Z(_YIjXlwU^nkf^rHTai`01H!@3=r&oy#hBgzjnyW9D@K3`hmj5Kf9ok5EW1CVC z*uLca3y?+0K^b|Orp{l0x+m2?qq7j!a4*Ud#uM&yHxt^_U&m%t(X<^`1JlcN+3sy2 z;#SkA`qRWNi9;1J@_4V20&Jq(=Ftoz+6$)GwJS?rY>%cS45Sp{7x;LQB$tdPCQ!TQ zF8LlXq^-gsw7EE4>|ULsltPO$E#+p)4NWhoOt&09CO6AI;4)7%x_LuN4I@s05;H7Im*i9*-eIVM!Qeh0xPges79^XwlTLKs@d zTC5@{Cm)dnFS~xG$BI-65b(O@4kXT7{Tzyufu&g+pr^3$cvOKfUAT={PQlAltU^%G zdQF}=ic#8qrYC0w0PM;$mxiftYeBz~TZnamzR*SG&Fzml`$@duaZ?=$lJXl*n#N!l z3;{g@6~Yg?(1esJxr~av4bOz63$>G~T))HVF@r64K)f;s@Dd$KkZ_De1DDD)Sj@)! z)&)qWDZl*0O^3!;R~hGyUAQ8SjdkKixe#S>an``uO8qM{@#SQ!^+iO)`#&V=7T^2F3F>_ z`gI}YT>~l{tK9nGi$UyNg6Dsz2?MbFx0)~&cRLe826;mZ zWhZMG2021TzzA&j22$|Ud|JTJ}TiV*T8)7)#E48~u0YX{t*2uKG z(%EhG3swni*6_TuOxBgjqO~Fv&OVM zMa+^VVqJPiB1LGT3uWMR?y2n)t}h0)LklIrFoEGSp$y1y=#e8b{oz1_u^*flp$Op9 zp#ta5V*c?#pIRl$fRHyMe6~piC9;Hx53GwwhC$HJ)`cGmAtd#N;Q^8X;Ys6#{@NLq zEI^|g2nHGG3)KFapmj9nr))OBl}ZqL8o^9>%#NWUil9S8rXa(!W*Sq#2{;-RfYUQl z2<6kWeXiLpqd#PX2aM&5X~cFYAu_W$%MoIPllt*A0Ao0uT}6XpEu{E*VDe!5@RXla z0u&t)rN6lJJ_LRpbXr_t$6XMTdQ(CCoK7HDaIzT*bebp$LYO4otUntI+?c*`s3Q2T zMJc{DEDkWu?&%SFG@QZBk$}Z-=!6c9T!CHC5cSn0LNm@h4y=B-VX%``9TT22!jUgO zqP}qX{!lUxMn5TmAx6ZS_=XFCi9x)6(Mo?NX*nYURv{P-1i?OXqjV6Yphbd{m_yn+ z*mWQ*94L4vb~|>rM~hy*t({}&jm)VChhDzz?9_iB#!T-%rL>68(+Sjda`;>N~U^lzf!&2Rs37l>+3-rc)DLH6zc z>ExCWYgf@{h<+&dXT>^FW)kJnqHh&$pFBQ^C;dBW<0=_1Z%`n%n9iI_N6SrqP{^;6&L^sUrwg36gnEtCXNd z$1z7s+D5KakPqa9Jkytia<=$&>iX7f)~HHj512O$O}_s%MaT4BQVa$R7kB2~;f}6n zm59Dt2~w*_3N#pf*B^8qrgRTkdcstFfyaV%_p@HQtA$EbYs%nmZcmDXNCc3tbWLC{ zc3^HFTHlCo86OB)fKia-pkoWjEZK9%j>9(o* zs~W!A&`S$L)#_xghNyojuqCHqrQ=p1rZ+55Ma@B( zDc2Pi6Nz#gde-U=L9cGNWuY-Y#eY8RrSUP;Ox!*FiprLctSt5~IoK!Hb^kjgksDH4 z71nM*b%;bS!W#^?3eo-}jCkXab8@gx96M`s0$x58AWLzwV4La-T)aYsPM@-o)Zj57 zNbCY_ZWS?putsSlfU|r%uc_P6<0p&Ffh-~(865(FZ4?hgjMoQxh82?&dUdfy;rf!?)_)QETrVp{B7W2k8R zrRN}_NdQ4O2KtXUt5TQbJaJ=0_$#woQ*Q$O9fSF+vr@_{Pt9oo7DJxeJx@paWKGE0 z)qg1QH#+Xmj6YUdL7NRMBpL$-w0PV#@s4&tJN-D_#?8qbw4_Cwg_o^!5WA&XNUl?} zS^RQTA7nw>8&vCD%J|EUZzv{|7i={y3`9mjJ;#Vm4k>3m$6%;o-V6X3x?5>J4HCG) zx1YxGQcpUWV)P>Mt*NvXs@3n+vW~YG?|Z5-Ob!0 z8#xCskSB!id+6gsC_g8i9^FA#nxV*2)N$ayL2DcDl>PC9;YMT7dO#CHIWU=thwm&; zt|$d)||2IUJ5zxed8zpJ$6q-FMQ%43OHBwtZf;Wc&NxQrO-8N5_1K48=c>o6D~ z^Hi-rpw%2G@Eb&TF4)jyX^GBy>>2MH%|og#9z4!s<3aNhFwf#K2Y_njkXA;1^$neP ziP_y~Lt+t?Sf$OcyCFxPOz}$b_%?xpdGR`(%JCHBk`oXkXRX`?OqfXXGlnBMSlbb; z^T7TM;IvqAYg!0>HRQ?NEYK};`O^@Mk3GJ9`6SEtx1CRMLYi@TQLD1u}xp;O@?h!e1g_ z5qMuhnuo{ZFL=@?opupyt~gRAB)~BjnmS73J({@cnYa-3VR?Xo$@^% zUVGQTeOm(iv)qFlLTQX^O#Lf27fn(YW)Jze&20(>U4+%xr0)3yJr-#nNjDqY(pPwE zkY6fP$LTCAY~oalpH|-kfBKy^11~6fmy4EbtH(w(o0jWEN0u&`zRWc6&s;m$sE>p{ z25Cq*Fqw>+P}-`Q-?qS{amW;(Uk99*enACM!p{WSq>&^iqee^>b^G;t{Q;M!+H)x% z0oUPW8|Gc;Qg;pqPb2lAE>&jriYD7GEG_R|AfK&#Z^#%^EL{p7OjO{}i|C(8J3p(E zCoZ!nDpu$UBdk0tBs7aSsrThCt@C{u*y}jB;ghww8?x(3)3thlZB=ym{al3H;FAHN<)x6-%}X2^SmU_i<_05=}|DtGhta?9AN z>E!`F^4GhS?#Y|9`2;bPG?ASXQ`i=fSIktePKaGqUs)a{cwL<*FwioSAyn`g(sJFivbY3$^apFSQW3_Ob>5i z?M6o~e5y(RJuxfvtVnL&W=U7u`EjFB4(p|$`1vc|7;E` z1coI4ED`#{k^^r$oROLl=RwyS=Gkvp1j0<)xNLBuwK{ zVY9k%!ql=vuTW?!`Fd1ZrN6O<^Xn9=W#6;qcU_~x`YmTKg?&fzGifEsK59q%Xda5N zSXMvsFAkN|WV8M2tne)3rt=Zi5pU^UcRP`F))3YdK5CG)6;56TN< z@5%Og`Pk|^8_^p$9ub9#7!bG-RJ+k^SADTB$;?;!tr_wQ*@^uFF4zVnPR&w25}~G% zGdL6T^23H}O4@~WrjaT4x{+ z<;D3gdn(sN_HS#TD1<#g&0zCKokLah2E%ZyIyzx>4)Pb&+%a3u(QNs3k#)qLwX^1J zcWfom`8sD52`|UPOJFQ7nWKg(0?#yC0_sgr4^(-b9b@$WyYkJ)RYK)^&qL1D*^ZD z?x3j4uE8Bi(ZnGkA^1wYFND8{x}_6B5AS z22Lh~lpzi*E2XM85W50s0M z^T+<@(PYeBObGweN%eoUE$s}P%p6S&2wnd7b=c?`=>b$Qe0=|TvHua;V`k&{Z?E(J zn>%1;{jbyk6Z`)QDI;WLWoG`r#gTbyY{YD^A$dQm-!f_w>inYz7W5+&RLnC+DY{aQ zhtw)q2{IXrGcN7W+rd6>C`zd$3@_fBOlLL~TMVX=W$ITIBqd>)fG$S!0ng7EuoX+A zl*orN%|eT^ zVt$jmkY*QUr6|guC>f;L$RrPGDQw2HvKV3B#Cv7l_{@+uy4b{WQCsv2-hubu)QWjp zXlhm@On85i(@dI2>F!AeQim4nbsaRylQSa$6 zxm??Q!p%A6XmN;;ZK8-%{xmb(6(jOo4@8~rx$hGn zMV9eMi@hbd35m6gjxLHx604BIUSys|J<4M1mdDcdGe3FIo)<=z zsE|6wn2Ttjz?Qo6rArmixP5sstglxO7{t`z;amfVr^(b@LrgMPN72|K$jOghx6>i8 z+4}(3k(}CYIBxglbUR$ zdA9rP8fFWz*T2j%l|2hZiJoHvy8`8c=sg#lt#=w|X{edgP|DU3n8rrOGGa`UwyfCO z=64-1XjA`SS?L5qUP&uwdPkePA-CgAHFN25L-IJmaO#>l?Dw#>- zTUl0>*b5~8G1*9}c^(?=EpI3`r-lSduOYy7v|$uQcK>i103ux-aw@5@jwZ;jXmKy2 z!`S#G>neo9qQRSHW7~z%^;h34?Q1s!m0q>x7guW!LQ8+l?Wl=cJC+7rA2Vt^MI=w` zFwHolA3oLOx^^AmZO*3xy>^xAlQz@1dQ8sf5tQwVmjL!nCp}125a9F#?5^ z839{Ui_SsiBo{1NeOTS}`qf40z$zJee{UldE@XB8`rAlXABO!nN$fQ=$$S3QG!&l8 ziPgn$evszU+WjgndQC?>MhDyVt^zalz;)BChOTS(`fN-Wy|lJ`U~(|=<47h|ECs)s zxNK=iR+L1s&q`-g@*#+H!n&etDP-o_K;7 zIdDmUC^}apf{|OEhg6VFPW@^#PMw?B!6ukSvvil4Y#kQLEpQwNFo1sdfu?RZ>Fq5 zv97yd_A3klHJxtM3@ye$j3%ozUS!^8S>c z>yzRu&kv>8n`loU0=dD`LmL z30NL)S^4sN@#J0iNqIb9pGWUH-a|VgQShIi5sCd;3qB3p##I9Z0=$5pIsORbwX`MLS{n7pH3xPTc@AS|KOjM z{&U73+n?5df`R|EL;9Ol(Ylf(1?h>;H0cg#Yn@R!%?enL*6zCk7DtLAV;5{KtnnI{m~L)-Z1CWXTYFI3aWoE&@Y{qEVS{v-2@-LL@q`8;cgg_j;JH(CczhW680^Fzf#TWp zIY7ESRTGX^kNAD2V~)4$^VT!Zi4_#lhpdo9K=AakfKxkODo!{Y=`O7ns3JI}A7_(C z0h;kHwHCf20!H#4Plt-M7Bt4s*GX%hOrL7qu!*&3qSv8G)r_@BH4n#{b=_hfU7t(~ zew|v$zp@)5x-y)gH_@9e!)9S(EqSHj-=PTZNu<}WjGxc!F^d;JPKQ)RFIN)*i?u87 zsvld#Fecel9=J7}Z0PuNbcE`!g3yNL6J%!WpM63`X&pOWEb|;z~!Klb65W^{a>dL;(~}AKnMZ4c2E!g zK6Y4mz$3l(03e7yBrFJuUP%CmSRWP#A~{_U_1{G0$kbkwd3%7M2Y43wk=;IA9-2Ku z^!|{|EL?jp#4HS(Kb?Tyb{~GD2yUOxJgokB|E-So0dZQ(;gxhT^Dc_MdW|xTR@vYD z%Jp?X8X50MLRON{D>3o-L2`;u`v889)=*}!e((>wxqXbf#_9hP<^dW0UQkMry%CaOv5(VMuC}6?T+Kuaxmt+EaWxL@;%%1R{l*xH zEua$|0P8^?=mqTnCptFs0ZX60(8~ucecz`D!#Rd?4Cffmshgh#2Q7VYQ|mCHPXJ8q znA$P5V`|6Lj;WnC@MD%fFKT~FfTo#za0O6v$Ar#wj9SP1iwURz1(1Ra$Q!anB~bAu zEq(P__cABrRkR|In`jcpqCN$1r%!_!OF#V5ZR@~R%RfF-s!_oBiiZFLFYX5YU^8d~ z8V`)z$c&8))yNc$b)W$p1}iOHc&sNk0)}d=2aND|#L_?aLK#1`(g@8!JZ#ePK>EWdSA(Uj#cA1b=YI1HA5{i+|D{$LuB9Bc_E zmKZ1{=1Zx?@-L6di+@anf3lz0dJ1K3WOH5B_%~qMhX&OcVGYj diff --git a/doc/segment_propagation.png b/doc/segment_propagation.png deleted file mode 100644 index e9d3e1a17a6cae9a7f6a85916c3834eb64678a3d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 75005 zcmbTec|4VW`#oxkoyZPR$dpW(LKHHEjFC(s2_cy?N9H73v!CZv!oK(YzVGXOUF%wFU3av>@niI~Y_w!#Wb|5^YKCNF zl$K;<6#N7VeCMqeM+p9+w$VAJMz%@%--DW*1Tr#iGA%U~BfpgIslI_m>ML^&z9ls| z&H)*GJQ3WZM~`Weg%Un8s}YGisMb#4udJ(7BGNOIS3?99X|GYx5OYjmW2Nc2Bnl8s(g z7NYvk*ZKjr$n7}UU>OAy(Cl50_yS&Hv z>%KlSU!6%CrwbR#%=0z*JS(`#$ovjq*@L2#WMpaTdU~y&Ta#s-`wWbYBijg(4d1^z zC+U8+s&~HGG9~Zx-EXPeG|TXOX0-3@;M%+Wg*VPjOiWx_{$Axd^=|pcz)7LK`=nP# zZJXkb1g_5A+d3p&GS)Pryumh2rM=MF&RxG=-`&wU-GQtE;PT-@ZK-&l&XRM}?M_R!gFI zg?T=+&NmyjKzFuSdzs{>c&D{E-g$p7ry*0 zym#->!d|yerCNF2m)GZ;&Ye3)$Vjvb*m~gxfhVu~_u6#&$O~Tv9=6~yiH7_=i=wq+qanqED$$J z8>^JlYo9I@-ke5woH9069~P6BzclpRd%tPHVA0v5>gwu2{v6D=Yc8+{xn~(hpOhoM zsOK#+btEHOWhZSQgT#i0hQk*>-*_@s=`rEtY)pu%l^b)Jx+9>cPO z>TH?$=FiQIc#X_4ozEXqFKw)^U0RtOTb)YMeQm8(~;9>UJ452x9o zYE$fo_5Z9{u`)DdRQ=M#6&yGg$?E-$dTjqdugIIil)kFPe{UH;W^j7sGY z=~{TZOp%k-T_Q(X3ArL9E`Fvapv*u}jnI1UKsYy{%2JFaW``<8=x8JBnb+ox*?P%@ zFtbv}9jcoA39ac$o2{}VDTgoes|p@G=#9`NWcd8}bXGm0ZF(a7%NNI{-HcT8voC&* zzdo6ByvBc#U)2#;6xa3V(2JE9KVK%_yctSlQw*$>YFk)rQ`#G$#(TogZ*`k^brFt( ziz&oRo7b&PTWjCSmeOY@iA#Ju-=r-`iiYRcLW_99nut~PE}mZq_2A!MbIi&al-3RM zO`4jT_C~~U9tl{TJIfZ!vU8{RpJkVr#<1xN6CYpo?H3H=<~TF6T3_5Fe|g;x|F}A2 zeN(2`3hB5c1PQb*xF6f_dw+@6F+LSyqS$FwLgbYzj8vfqGZ32+yXc-;Jj%(j{ligi z>;1jYG0VE7e=B6G z>8THAkJZ5XIXP#|{eCMNb~>#RD+i^L~xq!ZW@%cvOg+8!NR{ zma~8VtjzbG@66_jkd%^|@BR4EFoKwRTwF}d^;cEV$8wK}H`KwY5@*BmS=1MXd`Ea9 z!otHHKBSN~@E8rYTVbM@2-mfAz}i{`TK2_CG7d)#>Qy@OHiw^!r!+O&6v0IZ9Pc zKF0pzVx&xqTxu1x)O{xS@6xyKyriTgzp~p}3H&>#ii0;Ve}B6EEw9i|fivyGGvA+9 z&Xt}iW6cQy-g0~Vt;7-s?*D#oe=+BH>aB!?X1`-J))u1|3Tt{s3(a2TZRYhmb4uCM zo!ZVAZ8z2s=~Q-~`znsh6G6?uc|CaCvntOdpO?TZE*@_={sQI3>~Fx|rQZ3Z>S~2B z?k*?qzhRyW<}?h77kZw$4n052lFUt@A?4=(t}49SsRotsW^g5T)LKHl^{xXpRH|27 z?NJ|oN3vRJYHHTqr-{|DsC4H!zT9nkh9~0X#_tP(%~4#J%}h*8rdce?o$ICeQkvU2 z`4gJ!CmAg+|NbhVm>&GSub4Xc2!2bF$u34p%srW7F2WLF@Mrl) zW@aXL|l|dtkrRG~y3?rZE8@qDqi3*|$X@@rE zk&`*HF$o<1-j$nRDVEdWSaWIRO^m9A$X(VmfgO(0mr{kV?K zOm`k@TIvsl4HE+ZmK{Ya2k;(le*Vwl42q-fhk`fO9Q^&)+B1$0iS8()+ak z?iFOB4<9{J6HClPd|2tH83{5|goZTRef;WGY<8VE?taKpEU{@~el0M=;ESSv5KCsq zz4Y`m2#8asPFY!58IxXF9gbddsmF7ql5~adyt65OY$QyRG+J?mcuSK{nk}~8`l=dl z1Tnew^SmGL&8sa_3u}P`l-r~B5(#f91JCY1Qu60nKg&iPf+zHY2J!m3?w5leuS=VZa@+8mJi4*x!?ZqesIx$z{WMZwlhHd2f!={@`|4$RFF(cX1Il;nm|$IG%Gd zubVoYoU5y=%dyOSp-zTreU^_gdX?*~ZLX2|bk5wI`}%R5V|tt0xk$spUfjrTCBZw| zk*XQbRC0~j=&0{#pOa<-s5Z$vn54VxwTCmFb@%Rm;5fyg-_@(*F{l(5S{=?4V>OOu zc5u}GVlgx{^u#x3kg01gACZ-tToEA&%|cL7dUecJDrX`0(x>QospUSJxB&Uckxf>RV=6<|~;Q z4yyC+2(Nc6)#0}gjksZ$`+5!Fu%zEISo8V}=WH!PqMbTgE$jv7`p%fp_-2!Bjxq-i z9@P8c(rSP4{Q2|#{{H9A?JEx$wO!cXvxQwcurH3N`HfE7rHNf5v!myOG1pU}aed%Y z8g^$H3*kEmKKx^8_p@SN3yG(Fhg>UDjA>(HVlp2-oI7wAp>y#N$5V5ULmU|ffbgYyLQ zKuE1kO+>4`4j-dTa_r^>v$i*^*ehs1qyW8|Dql4;nCGmsPj3AE zvq#=XZL>6}QfKdaAJBo z7b|No4uk^N`1*B3WaQYU=HDHJ#Fm3>u|TnJvnxylnN9L`vT_%wA`@9ld|3*>KIok( zHZ16NqNk?39PVcqw)m((Z|*3{(mBc9s+z)guHY3BK}ACo6%|E;KO;ylO#pv2-kX|p zDhXaq_wV1w4j3OF$0dJuu%)$iAAwg!rW~LQI6f9IJUTl1>Q%zlt@UMB*E6eV2bs=J zM;qo!C%0CPcXxFOGSjvy-LO?3kh04X6%&iQdeuGO`tp{7^Tox*6HW0yvCn!7P9p#{ z_i1GqFtf6A? zjG`!w+n8@MR)Dtc3|g!&9hvOLR4!)$ zMp^Ru3y)(unl{Of#CI+n39}b(Tz6L;T^Y{7`AVJLjzEtK@c@7 zLe}m3^P-GR>F=+T*hZv!J@^VIXDIq6ytul$`uB6sOxv!IkPzHv?33m1{fi0uieEc( z7}i!}RXc1?7#N5?7M+2V%coK_4W(`)X|47ZpQOr* zfKY_5#l;om=6d8CZBYC0zeg7}*_<#ExTxpvfA}yTTpVICCT3>1MqVaR zO+$m4n!5a1sk02b>50Tc=RIC!83hXa3nFm`B4GqP$ zMw<+SU5tz$Z&_4Op?>V#X>4gZSaZo&c%c^X_Xf{Vpx0Eb>wivs*9BVWONoxPTdl6Q ze%sbon4iDz*si`P*TNK+AUo6tL1v#X?HR$F8)yH$R^!JW>Gj4Sh`r?YB!@~#Nk!4^ z{zxh^e}5mYq+;;?mh$?p0__$TIR#Dzp2@yH?X(KuKITU2u_nFe*M7ac`Gaz6K}x&4UbM2aDZ(!!AeH^dWI_*u( zh$RMr7Onxl$BB7a5=pw$;VJ>NgUCYeo+-B5Qja%L+Eg@&$zC%Oc9F2#?a1j_HMl!d z+?C8?c(?yvvy`zun&*}&BOBYNE;65K zwBGU8uRjvJSjTBoyeoctOG`_t>AM<=P~}k{u7rUt5AGwIH@WO@=bQBOH!!&IB5Tg; zwM}ueoo5B}aBO<7bhO&IpY+UAL$ebn#-^vf^o@UF6k&-meJ{U!K#g~Y32ji}`dZ-2 ziAX7P@=FJJ6ptDh5Y1>$YXbAl&^Y_=ah4hQah%MJMxW8Wg4y*b>Y<4}swuLu(*DTX z&#!$Gc|uXT{ZZD96zf?jKh3tW0gvsYiaw&<-Q635jfb{Iz4G|u*EWzlp|~t-X)p9N zKi_4HL#t3jklD~dsMI{aXx3=vs5$M{FvpK}K5b1*gH2Q_t3LVES?y)m>~3_jZs#IY zc`D;Au3C!G=wyUb=@l_QSLJkzY@1Hd;V}qx8!%E}@_e{e_3c2;zMgu#|3l%mpIOLV zCdS597PKQ7&!2xvf8iu>wxmC9C6ai6QRX=?O&*!rGeB-RF^Hf%-!8 zv~TY-G-XV7(EnO0$}@CW7UMUfP+o`WNY^O(cN zRZnmA3p?G!FgEYk_wm*A;?T>kk_+<@X|I?p`$Aa9296R;$9OKjt!7jD>B3IGk3WvSS{f$7TbjrL3RBRypXpWgqbrU)e)Y!ZDv{VJ8c z(PQhOJZ4%dieM|v8m;6)*`50L_J3=e^Hac8B=7KA%BN@Gxa^ z>$1ZPT|2FGz2O&)EQ4I5=f1)r$8$1W(_#hsyw{mHcBoLZ{-&d&e)U5tcxb?;l%nYR z?F(lmlN4oSGAV9qQZ0XF8J~U?uMsYN-&)9Pg6tfcLBLbr-k}|;=J|J@H($ZJ%wP3H zSr*;5On&1qM-|k#qP_D0^G(gkflNZ8a%M-MGu6OvtSz|Ud@S{OEL@w9QTA?Tx09&2 z_`l--RH|m&c=%puNNh_HO_g;S%74q$?9Ml3eqj!s(|1&rDr74;Jfj8c$kQZ-7kzLs z2KET0ky3-T^@D^x4nVf3u@??%g0r5n?w(odYQ28(Wn3KFL1BJ=d+Y zLa@qW?~8IBrP;p8kzwOLGE~j%pxw5OR0J{%RtGP(fpopidvl!}sPf3$SXUzHgMLl` z_9VCZR4~5`SL2>q}tnD^aCXvVw&cD_GC_jDr1SSkXEi^$ue-{1C%S|K*z8hQ%ZHZUf@B=|7{&LmkTURcr zDc08qz6htL2V&(>f7X9A%aBC-3krh&N7bHcNvwEACVDKdd)ycGKluss>_Ej&a4D;+ zeZ{u;NY9jE509@)zo`YjeYk&c21K3PEo(kkhWqC=S_Zldy~X(xqMzSjRN|ACE`9Xq z^=aR9keuLphP-_r9K6|Vw{z#tX2jtzxnZ7`*fCPhrl$B++81A3aI&EA9*Z}!+*Ko!G-B5s?@J9 z9e|!r%c2~a27-?VL_7R-cw!pa0LVf7fp9ZEeflqmOU3!uOyuO`z_)=@cTCNjasqMh zW7%cjG&Tl-L&bgVuW&na`SQjSftme7X1AXRTP!uhJrZN^*57ulC*NWz;K=dXb1f{f z8VM;WgVygfVlDuxrUx!;E&t;K+69tA`sLq38@YgJDl?ym7e$T@BmYY*Nz=K#$>7yNv$s~xx2f& zx{9qGgrO%#W^SXUWz(~lOybwOFKt~H5>CT_B;f>& z0^Gi6P4b0g$;7Jo@USqVYHYv_6kuKc7Z<1Fl(|V@1sr!7Iz8uYyA}K-j$g#iTV3@7 zW6Q$gGg?RPFk4`fH!(i`wh)NMzQiA~gXZu~v)$O(7+6{<`}eyJxm#zRIyf91#&anR;|}Zcze?-GCQMi^`Y}*}0A-8izp43h?Q?Rr;>!sEC+2*U z3wg|PU+cleK>2&!gq>twVo$I7g`_#vT&hsx<*5iGMHgtltzqs4mR#5@tMe04^z7w! z&9}%Ysn`FEY(kvt58JZMU82sd_HPeKi`pZ1XzftBpXyWg@&>tF=1=>Ln@sN{Cf5F4 z9su`KZ>`C9>+Gv5{Tt@B!sryZ<;${m8=olbJz3$C%f+~3$60B%)cKh`(AeH?;=^mv z2GUSbUEeK5ox?+TSa07n?Eg#uG}sfa%d2O#wY7bGmP09Nu=DWVj)f~-#%0?b4~b+O};QmUiuE8T!o=b08BuziERalk$~B_$9o1a^tKhQ=-aSD<*D`=3qWyg)LT z=lg(Rpi;S=m?=%?sLS4PO9 zDRSPzYuz@*;4iJQu_OKpxXe{+XEj_qftCeH;<@)scIQ`)!4kO*#;}M!YaO&Hj);m9x2z<_BqF9KOS;Bl#9w|v z2fkR~y|XV3as#~sJt$ZzYmpWqFX1J0g5{}n z_&423XF5bh5Y#^oK6eu#J_4_x;1?9rxK2oj@<%Q$<2#`7 zkV`qXPa~D@3&uzO_?V>w-3*24T}un<+-?pIU7{L5Okm(TI3bWVTf;PUbxDHcd@v&u z6ftB2q@L|iXmL>>mZ}kUvasCRS~O6aoox>x59ja~g^Q?4sv>0@+k7We-Bd+PKqtF5CdPMVrm!T1OY61H!*dS<7U z>065}gvz>`?#;nE z6sU+$+rZs6e`ePVbqPy5Wp4hd+E<&XhG<1fVh`lt;0T<2ds9QaAF6M1>xcL6UuQqf zNegg-h6-YUpD^C%}_J(((Ep;u3pbczsyUpX@8)S zc`W`V#2;m4vW-PqN__(~ETi*&piW$*lfn(X2PH{qjWL>!m)8dSZ&Koo%Ko#38@+|s zK74sbNln)W&AC&X${iBS+m6g*$zILrb1^{=E>5?a;aSmzH=Wk6&FF#43Jo2bsThe1(ZJ;**HGw+(0F=!N~dfc@kyGk#h?Au&OGBV!3%fG70uE{(SReHREX1Jze{N06C}`2~|+D z!ngrGMrwnDQ$B<4TA?}|qKg?SJ*hb#>BjtMfSVH@O{9$6W@2m!z72&$QfbFkB#){i@N@;|G? ziEI$5KG@ZVZGjf*G#5u59@=Fq(Ko&X?RFfBvWukT0#relVflRoRX)a>=Y~-{G-j3&s#_@P@e@7bYZNY9TU!vU z5@28ez>pY`c}U7}PKvCXm6_Q+t3qD`OZ-~;!3!+@mth%zdjh}LX!8RX18`#5DwNdc z?qFjh>BvpzITD>RMUQ%WduwQDczSvo7_>jqN}#2seVhKxRQ=LY?PX)HD*|bLHrP*k zHMH)^+S>21J%8BQBs7;g_e-+GEG-YXVGVM%pDy@(Pd)R(N3>vP?fSyDwS*el&PP|e z%E~?*-S{+o93`jRb?DonR)O@u2Ag7zDy% zOQDNb03-gcPQ-6#T#pYQJP;Zh8Y^au9|JXbqqnlE3F!FxYv+EGU$aAhpbmoEAzfbl z-|*x~lIH~E^R;W&`uh4H*hd(E%TH()5f!ZgFu*Oz)=jdJOu_~v)iZFlJQ05Yx_5m>%y4pY#^s_Af3zO z?UGMv875V7AIbP4E)~-Go&BcAQ{~F)m0r|1nVdXntIr@LB$Rtb%l`en`5zS{**Q7B z-@3jgyS?-uj|P60eEZ_i$nC>NK_sDf&SXCBlt`rlgG8ZK%_ooh;B^U9VGy6UEOITk6D(I8PKpI5ECo`!-Tr+iZv24aAK$^_j14827h(7r(iG{kV@m;p=L> zg{G(H0&W>c4&;+Nn>Uh@st}^c$``9FIi9??mVlTQ`XhHa&8st8PnW-BV+~5ZUaH(u zl0#a(@k8hW^miFmh-ZS9j8k_RQ-=fNut}=?HO(~4eeU&DHCta>yW!RCtxWucNY#N` zfCQ3hcP|vO-M)Jl8;@iw@KY}a32tB83Bj2=0swtO4(jvLOs~8q2|xhPBpl%4dv%5W z6?H8DW0J0hqP@L+_Qbd(TWn{Jeg(ZnTzot}o6|1p5suQ)HL2-37GhPwEJXV*V{YZfHhewxT*hSKKx;zRWq*Wv2(h5g08 z;IK*GL1ICQzk2N&8lBr9AhnKu`($2$&a8;JJOvhsgbu~+U`t^piMe{!T4L9Fz^@+z zy;k0y52AGMOP@z4wRC9!Ai|1|aSJu|hh*8pN!nxb-4t!gh1soY0V18}vh z_A6feI+NKugNrA^ObeHfK&0vkE31u(c%_#p#U$a0Kr*PABtxOWCV9qca?Xuk4Km_x zK@IfcsHhe+zO2z|+h1=WcvIrXojDI(>GEK!j$qH_Dg_z@#6|~P*eS`l+)@R(jgaws z+9S;*ZwW0YEFpbYT^MStf)^&W6u$n1c5`)5j8rw%tXV=HJwPQ4*}vQdjURy$Vc`)TS9?AUcazBg$1IZ-3O}F zu`?^O4i%_w-?3K%`MP1qaGjyRDR}Ok5X8)`t6y`_rxbMr{DH-;&s9u4r}KsP%rf>? z-<5>t#6;;wV&9ico|=tq1u^b``_Xn}CB?$^FC3|rj|$IQWxad@#Nbv zck`FF+zHK=#Ws!T1YD%qC@Cosw}TCI7GtxJhQvEH_t+Fy*2jLh9~&-^AAA|r*>7ks zg8IU3OEgMK)MmV*`Uq`oo$Us6uak%JNA9M*@ zUEj>~6d&W^T7BJ_(I?9^;?MNXrxbJXGSjjTVdLba7d(0L9EA>B*~{x!?Z%QgM}XQ6 zvGbaz`(~1K`M=3nE^&bxeIY|Zzirz^Ew_PcZQ~OsU}kvxL!Ir>R&D6!BA`?iYbfv6 z_$OM-4%2FbC3|Qi<#$Qnd^!+Hr_!JAQjk}B^%r!n#l(=@tVnJEVxZV+KB!LV1?4~> zD6cOE&`-**VaJXT6|Y_Q+&0aHe92+Z;f&&ECI6`g`J%$Devn(#E%-@Z-YF3lueJhp zZsH-SzhSNRzYDRmR$*yS+7q@{fZR1m?&a?j+=GMFS z@a^;>J@wH9zBzg$;ho+2O6hF^vacM2t}t+Mf>%LV@ZCB}2a*T5RSCG-&TbN=<`Oo< zt-;&c`_inhf~Xg*V7`0j&X1AW5hj`J5;Kvs2h0n83bH*;+8~!61w8l?H5$!2d-vg5 zsp?UuGIJMpu?PN@CyIUC+$^lDhQBbfK)ir}Dg5+QTD(k?@@-H<#4N1l+ZX3>Mw-k;ca7UOcr3msSz-s*ZlffNZ&!mshZ63#fYCW63!; z)6!;dh0x(N^O&ERJ-yr10`{`1W#5S?sI?lKYx!g`jSDv@JPzYd5e>e0ns0FZ@)u#* zMkQLxp&#SzdXSk`J@c;pN}>DA8e9)gc0yGAk{2h_7Ey2AC-zf>NFao{XDSqAJ$?GV z`CNgJI(#Kr7J&>!%!Z-je`$nZ^a=mdw{LWd!C(3^^P$HBB?zpM?Gq3VyDuI-%n=9m z;1^3sx-v0BN-@lYo144Nl$}1y@RY+T&$@jfyWbxGw1_>K6|5ZCXu z+M_ITq#mDcj#)5_l42>~7xaH=r*~dBpUBotyQ~=!aXhCtDde-1M-!5gpPNa=#4FJ& zR)tZ1!HM9e*GuwzZasX+3tAAgud7ORn%qp+@13z?O`TLE`Q`WZgIQXoaUG3J%lj6& z!otEPJrAz|0{6r=RNjT=U0Hd^S7&~6zT2EW!_8=PELpfJ6>K?dcoNy=98e@Hw9jU? z%p-W778ESE`RiMX^$zipiSbKF6g_)(YyC-wW1xLO{4t?OMd!`sfvgXYDQ~Mb**Q3X zN$D9fT&3`q5uf8q>A#Y>Rr~C@b6LR!y(HUMTpZ-^G$Pe!C6YpCYMO8c(r4RuCho+R zBYez5rmRA01OPHw*K4DBf_L7r%(Bl8pqDJY(EEXO(91K5;8bH>O$Md2+Lex4f@4=ET zB`OMU)h0%(0YT5+ZRjP_$*@JXx~^dvHeybT?he=?2)S$23|<00Mx2t=4o~J20@)HuN+U^!gQ!$+$*6-DQa*_6+c$-- z=-=v5^GI^O?$duZA8hxaoDxmG@p(VagDs*mGRx?HNy{iIV}|hp%E7g3w6HPt&OqX= zY5s1Qds`zDA4VOH^zR;p#uJXZs8{n@SLa}w+;y#zD$6h_H8lvBPg201Mx=TXgc}l(T=8 zkHDOOi*rZ&l*rvK>Zf12co1{mpJn*@`C$Z&f4&Yo!>slA#KcKMLkdbt#K5=GbLa$M zwkk7+rKx9NA*FygFf}sb zkn?(6Sy@?HS_-xQY4@sfVXrLPwa`C&Xh0|C|2YU5qt#%GssE(YbJl@3IQcdFA#1R{ zSIV?~_5geV{d~I8T_wX{^j4|wgAPZw*4RdyX1iyPA3p}?%ZHv3C(qCSrm5+L`YGqm z>{d_!L%&y+&eFlH;s|5_Q%ze-%Sl1zKt&}PnY%4hxI8ebwv}d@}pFM?S>*A6R z$BB9VqnebaiQ;}|#q_o*&-tQLkm*1clDci!>=aoDRFy|w*W&%rYETc&i|N>SV8Vyg zhfdhLf_pd?I-{8g;3Zs5NEC9Z;GN;KYJBqsL=RrDU8x;A26;x~*fE+dTh?wxs*A8p zPfm9A^vIcAy#IOf+55)E#=hH(uOqmT=(3*j2u*fpJ=`c6tEF&*LdJ$I&`BwLri@|-z8Q+BEgod+TAGnE44Q>Ga z5tuj{22OZWR@T=R2^kFys@GVq3JVEEU%&pox%vFsf+qyW@}~UF^MAT~zC`93^#xrI7!w;k@sNR6K>sl>vUa|%a57ai~)W_ z9rnC?tLerLj~|%iVaHpFH61K}{P+uyHN5O9gsJK2i6~E2co{1dFCI#fv%U43#*XHC zR3yjtrXItB&ab}$YE&~2Fo1f^t*t@OVZm%(^z?+E+|SPslnO}{!k`7bQKO@y*nGI< zjM2O3uGEmn|1;(ic-G&|jcVs~Vmcx)ga)buHChYdSdH*tURlleC`PfdSuaKYQVh~9 z{~QTMzl8=Ktexn{lO4W-eLap6yXM$m7w+Q1Mgn$g>Sl}8z;^0(S}=_urnfcyeh2}_ z``ArQK3HQnu|Ma@H`4#(B96K$D$Y`7e%j2;%)r1MXA09CqGBg2>oGO8QU!??6_a95XB zyiJ6pchS*C@=8m0iEi`K;=6@;uYsEn5i2cIs|l}{^HekHByD#lRJRWYLkhg5UOzG4 z@ah#Josp}WLhKN=d`6WI|n%|PAzzZ|8_L5J(|!_x(ny_Z2% zRTZB(e*pfW;72Bp2Qo4;&}Z7s@!m~(T8xBgw}<~sD2&cuy|lAFT1!YfK7mmXf+~Lk zDSkCH;P&zcKZ%X)I`VmY;u9m`tPjBsGRcbz3s9IKBEln>g8PD0B+PsbPNQrM#Ke}y?9hrHPF8GV#P@_vsJtm)R6DCg)^^h=0V~CL2>T1 z4%$&6^GH~0JEwa~cs>ngi41&3Y7Vj#34j#9D$$|2ZlcCU@gxN0tq_}(EM!cci z)nqvKE_^>@fqCWqvw5Ki#n|yxKF&9NpWfm+gn#&ODyzLlOUPSHf)4Xy_qVn{sefn0B&cZ=^ z&ef>6OMM#+@%m+bl1Z^{ll)`#G4Ier`Y_4CO^{r3)%x#M4W(ayq==LvF z={~cB2|s58XJZv|aUR4v2=wjF!f_`AnOBu!l2rz&$2*+EJ5KIWFu$vv1(3^d=I_OQ zhrS*1UznVqXoBZ;`jc~URowU9qGwmxJ}wS}g9WXZjbhnGHLz#?fXuT|+d-@DUh&+z z@aMYF$*4;0T13{l7AHBoETmO`#lK);wMUugb=&#`UFGcBdy2=3;I9wv3}V@C;*ni{IN;Y~HC{;1MJ?ytx9XYQ zdJ|h(B)diMAz}we6jMB22}|!DoZ$mBbA>199zyGLw`f=efKe$Ld1<5 z#45|-TSe@(;QvE~V&qV}yD#0Qyu2l`Ws5D!TMMcHPEoFcazaBvdir@DiPIbu8kLE#{0j*jH zdBK*4M&!cQ#b$8ii6y?UxN&8EJN$h*lw@5z3C$=sgQAhXWCtQ|Qw;mpm6f^+Ns|>F zb9XOl3h2WznPVC!dkjO4p@^O^G#tk|DIdSNO`*aca$yH;=S)Uw=-5Lly=LV9=jr<2^!KvNLfG6pUoo2=w$zW(>)`f{`#v@%}o(q+c+YM*W)E z@R~ArNY)i{RtlYuWXwC5nW<(cSjgjeNV5!G{Wn;PXhbTsgsgkXEw^1CC%ZOh_wN{^ zyB5rpg}gZ7%sLR+MZQ5*SB75P>=yL{ihGV94Hf)DWy9-9L=%7>EYrR%<03E>!58Ek zJ#P%m$9(PX^aMlQcU$ey8+vtRb%8!1yzu!qv7IDcdC4SB`3{ok1FGN;W`&Su2SsB_ zbiuEZ)WF@lMcZg$TcR}ImKE}6m%FG${f-9&eXHmO`P-`cyNX-Wj{L#)@;?4~dK#ot zFW3}ZN!OcC{OLe{O})DV{qnJGeRhW<#xdLsO&&1IF|5i80mrAh7*rQb<$^&gSIFlS4c=OE?aDPC_5dzvzjSs+ute2C3voKV)dA$Ek`zpVrNxwjLk(e<);&gPN$3&`;Z+uL7INKP+}+cI?CTj8qsE)dbD}hxOr=5;W9&%`y{D=@=2?|C=Svw#{rCi# zdWd14p*g3h*ZYWQnV_yDT+I3oI?Lo_XG`w7j_JZ1(cqmRykOQ37VZ5B z&8SL%Aw<=SpWh!~i^Xu){-kGcoNV2?l{)+^V9JM?$h6VH68-~`Ys&!?43slGI z@?dk#Ds|s+qRjk=Q`ckYyk+M*?Lq!RLWXgU#^cC-yk=t02Q^(G*zt1Tz2S9U#)d-ry-DW#* z`Q11TnDp%=`z_pRB8dQ+T6%g%;GqC74-qGZIKH?RZZOJu?|-}ikQR(-cc?<=$3%BP z&9Hw>d%{e`F>)U5+Z0TCWWD3fOdRBeZxZBN$$YnsGK)$VuVXe&J4=#}ZxZ4>TWmIb z?xc}k==@+pVQV^m5tkI>XVGesXB#4*`Hml|0yB%ml)^12sF~9VL;EOrL<~$eAgp|T z4*$gwb8>QmG3>2#SERoA`|0Dv*9hlxlKrT1yu7^7PfYf~26e>d^;L+6q>!eg-n)PQ zh2~rPAR8a{{?>(ESK-`$D>W4(M6xcQB8bmDr?er}u)RN)YlOTmD=+^C$|L8^w*o?QMOzp6WZSLWtdUfFk&G zk+lj=1TKk`d-oiR%?M$FfbG|=nPb}cxWVbusc7Ia+`!B%eg5O4urP`+ZU`Kdq!An3 zHw6_U$XQH$zruL(bFcHt%5@M{ng_|PniK%F~vS0*r8?3CiFL8K(?uY23P&rie?`l4K;DiNk90bEzp z>GizAm+#OA7E4{W0|sDW69|nhc}zLjgo+4P_D^zyi0ObEL-}{))Je73$f!g-9d{mX zmNfs*yVu`>K_!W8D#VfC%^FQUS~@!D`f0~=UPXg7hR^4jhpKA%j79l6?2V_ooKo&h6XS zI8pW1cJ}rKczOX$DXZD=Epkdb-lz%0P}Kz|r?-U=PjKy5%?f*?A|nB!T4nx1G=Fn_ zH=YhL!(f6-rIkgT=(AdVFV(gVGeGRiEg{@jVI_n9gfgGrXu&33w;yv-XcW9Fm_yu< zI1-_0|BH(_8KYdIYmMXfXA}|tYm3ts`8`M}=j`om5{oJSj~MjNs|RJD{rg)C)nW8- z%^RFMa`>_sX1Jg`vCu|hJ|+S79C7i@`6e#XOb|xSuBq{+y!wxbrAdT-HYGa}cyiju zJRc`o>&>73`-ZSMzmyac&o4Zd!q(I;2ujsAI!=x+kt9_le}%(c8M{}^s``toP#{Zn zQOb)fvcL$IWqGpE5X^+&gF);`hxqDBpC5ZOiDb_MFAieuyY6ow3zC7f!;YMM^JD;o zTwK|-@Rq{UFW`fCKAm=HJI`xtiH)VgVDvryP(NTrIiv#9Ey$T@BJK9=u%39PwyXsz zmLHS3uOIYHtppygX!puzqcH#76+7)T^TkWzCza*H50vNBh5SA+;VBXo+#%v*07_kB z#r0bOV9Sg4qdR-Gf3b=cX~L%Q8f55Cj?$nBZm~Jxi6^k?&ig@Gf^X2tM^VfUYK$Kb zyT$jfk(?G)M~G&%_YK{Re-#`NAdla2_YwKy@KcG9IG$?4Y0JdTwL%FMZ`l8hQr$XgY2Q7k0Lum)@jWq z!wjtpwvL)Zu!;IWO;*hjzr^MrYxvf_Bzb5M+P{K`$ah3c!OrG==o59*75`$hA!(xT zn(fGpda>{KCgd&yZeo|xnKR;#I=}jTI@m5adgCFpggIrwK5p&@8wI9h?Vbuf%A9B)Or^!LeTR&QGDn*VC*_x($|tYfqiIoY@HKY03uvhOyERW{BU#bvfE;?L#!F?V$D zz=SOlY_XVb%reBl4m{8ndEKy*gULe=ic$rBhxB9EnrI^%oXVbbJ0&z@1xUrPSs%~2 z5M83nO)N4K!EmX!TBtGvZn#X)+5&(={FbzPi^%S^ZY=BG=C8o%XO==`Lo1ZEZ-a{L zF+*}|LCK3FWR+C$LtEcpxDB;J-hEU#!T|nG7}x>_t$g#M1%{l*Q7U zRqU;=z4au`FcszTPwnx31@rG>B?+k1)9Bnq#-$tvp-Th5PB&V)qi9a|^HXPR^ zV`7yFDXJQ-kX<}p*edYc;UOsDUZ)n;*6-cB2hIpju)+Ldu8}a2Q1lg(od22TWu~2Z z%558ik=%E&KSD7S?o`&tmSZ4T6umti7Z>LJEW@+(ZN*0o(4=9)i8L@iw&j#e$}Y#}B~J$`NlL$ECzcB$8eHl@jr4Y&744OgJPOO-<;J`()QZ zcc=BU`XNN87h%iSnr#NB4@|O%5e4wxxbLVAXwKnPMiMkQPz3tz-%vSbPMhX>8-!Nzui@CE71QIQd1YYLtJ?W zr+nBFTh@(M`vIp*V0Hfun|kM^Kf?fowFKsS!s{RO&KL_D|Lv1spz~?uO|D{HC3|5r zU-&4vrx0V`K%EE2Wxi~~n$L|_7R`_qY$@la5Tq!tx*%jJ#=4P`uYC6G7-oax zX_|ET7vUq1kB;^T)Pji)E<1B|xk4H`hT9YqOz2K`(XGCsxEv`f6bXR=EXB%KID}Dq z?t5t&88Hh+9;djFZE(#%Fk)vg87sgXo=8LjS>Z+f&TU?o0}}r13MqqU&@ z&hS~(?%cVxKt~_U50n%}o;&&YC~nw(@=78e2f<8!AX0S8GR?*7ct(e;t0mmkCMI_; z?bFiH8L0HwyYn!fJazcUks+oHP10m_&~LBBg2*j*bHnSc(NL3-h1gFV)aJp5+=CCw z5;2b_Za^${&xE^gCptoAp(AKWz_&Yt&)%6N8x8jsaFMLOc!cr!$CQ?&eZa$^yaRJ> zQoa=nY+3DG<0>eYw#UT9?h`}`!B-*&Fa zf`q>g(FC^nj9gX>nHg9LQjiV~zsf8N`J5iaEeEm3;|H}fl$^au6QHoZjy(6?7JS5W z@+~^*nfH;TO8==WPsius#f@xw=v3iEPeWOkH#K;c5*`Bq8$rou!n7fPH|Ta)l-i9h z@kA7yj8dZ86WT-7bAw`?-p!CU=;i*uaQ;AQF~V%{|ID8xZG&;J4|6z)Z%5bl*r^5x zB>nq2E29!T!q5>xXP7#;stBQ%0=^EY zd8rk^lin7krzc49T@DWn*}}BtSf56BLNjBe@-pFlEBx;Q)g(9Ek@n z^i<GA#fPY%;Iz#nn(cwf%HQj4>62{ z45~rMdi00`E-IfrO;8abmhFu|j~?>BO|<0G^yrCX3|i_XN$$h6JlyAi0-{1fJZ`A4 zc$gRIz+fBb*=3LrQHLDAU#6@~EfOVUN)(c0N-WEe26MNB zkP1miGGxe@p@dXI%2YI{L?QDOk|rX`G$%g493LwB$ri8OsC}ee2pI*`rT8uNjMKh8aB=mh~t3-t#C$i z`QZ9+XrrGZtWU+ogfbkEFrx-=9!B9jR9JEYAZj4boP$9`Cd;8*wma zFFacS9?s0nK;d+Qi1vuZxA_)l(B9tu)tXnz40H&-eThp?DGDJFeqcL~WScS{-ZgI} zhKGhoF?!T(bYKQx7yPU;7J|`GOFq!qo`85S6d`*{9Q^lFsR4j&P~k_%AO~UX+J0wA zQV-)AOZrs?=Ki-Z-vUY9m zna>K$3G!YoSRP3F*>u_tDLvJPN4iTkF2iI^?8E*2|2#6qmmK#h#!68seds+cLa59E zVkAzAWNC`W#cE2~PgX)PMm*sEXie;eo6`7!JMr}l5n#r{^PTv5)jKzCdwO=0kJBN$ z^tuzzaVZdoFbd;49wuKz@qv7Kk8Wu$@FW0MDIREDduFyp*J_fJmCs!|=!)kJn;Uw) zMLfdrC-{I{fZHSdqB*uj$nvme%Z^p8`X&0GTho;<3esCV4s!fCAS+Q2pP`IdD}E4( zX77RgaMEDfvdw{?9Ih5$bhs#aH^OH{$VaW)ihYiegK!tMu)~@EHMe|?50C0n&Es|S z+rf!F=e)P}zzFMe{PybJgz~w9<>->#>@E+@H-?p6lb`6iFDz|nw_h^5hE9uOVfWai zLBlAeu19oDI9Y`Jl`PRaE-Rqwwpt`EQf}Kxm6Vebv;@_IEFbAlO6YyM<}LVl{#eQP zdPGR+wb+e(U-IkS@A21a{+)PZX|4{%(>D2r0;wde2Avb~eeeV-j?ZDc8$SU54j1s! z{Lh}tfI;8{WmSJ^Cbjj3BO3ee!zgdjfg9XbGFsNc?>Oh<5aJV-2ZaP}Ae2I=%MeaL zI50TEg442~-D2lgl64oJGPv-ydiCu|*#yvh_DDl{)TZJ!W)nz#hF?MN-#d+D0x*vT zMy=@Ugm3Hz@X7r(%f>}^vQQ!tSevATM9ltsQ1!$vT$B$4k+nLM)>HPNc`DQemJR^E zNju7CuHC$eV4@Af5>!@=7M`tC))VdAA0!(qw|%KmiM&MANt6JC7WO0V z2;SR9Ki2WxlG;Xq3A@oX;VKkc+}5|p4hq@H=MgpUKPOI|dSLZ*x-`-vM3aBrFZN4n zEKOJT0Qp5F+uV_zfTwkSv7IHBm&g|Ds|;=2Xu}_vLD7Qy7&7{ttMqvXPl$)IMW!X@ ziOWTvkUIjv;4SY+xHP0ihDN1d9p@@(oBBj`=*u1K&dJFFk7xlM2ey~UpZO$T7FBTZ*kW-E>RQK#q-G|;rr#rqY}5vw%nU=3 zShOu4KK#5jgpds3P*MmW=dDsw3ovW?WkzaMOwgK>s;W;miSSAOa0fED&p zKS$TDk1iLXnsHiGDVw9+`j)m2ja1^hqfB)n%TH48Q>s_1Y!2nt#)ftkK{cSQ4<*J> zGkYzbK|KRaajAc%N?Pv;EJhH7>`W zeG)}3!=e{9G90-06v#=uu^$OiVO4PPAe%^v2e!$T5G8m)=z7xNavE8Nuc}6eCApPAtJE1kA}@07CswUuhZ_u>r@q8x?6{*Ak5L~ zHwst1)-Zrx4b{Nl1P;xxD$dI_EByAG6LK=~m*aa#dKEUZK!VXA6FE7MouGBbMMi>- zyGoDr%ra07f;itZ0hKpkS!A6Qs_xk%pg!h2?n9~32u1x6zyDr^4o`onGr$Rm+xlIV z-m6HCp_xr>5D$kyiBW)yx{#u098Sc7p%rjNA-hWm^pGUMobderKG@4)`3c86x z%d2tPIc9GM+%|Kq?*5%^K_h$T?z3zPp#UUS+A`Lzj369*%(}}hPFHl&8wM)GF z@UplVYz2|?H{jM6co*Mb!~xDioQ6@u1BN>@)3Tx;}4w_e0FZcI?L^Z zq62^E_(D8x_4MJY2>lEf4rtpF4qoc7UA-9^Dhu_fM!U;eX_U||Pi7an!BA#{8Rzkh zx`91UKaZed65bE|&b54`;Lo~_mtPYdG77S`K; z$a@BOF~Bd{*FJ}+yZ~Op0NvSDQNylw*Cn^pR)EPt*=C3w)0tI-bbk-pTl$Ni&{mnQCIJdwF%YtH?Se$r2JOF|IvJGyORnVs>55(>O_oGcQz_ zzGBd0HrU0FJ-?v>X_J#X!Z*1m7W0XFiLQN2H#K^3xhO5I(&bO3%?4pMJ11jTR_6Dc zc`jG|c*#g@pk`)JV(M*kswbMAaLxO|HqCueptYj;el5uYqC%y@)pMUjGFQN>0(<%Ng=B=V;(*gntD4B; z7d-U@j}!+eGw+fO{SOue*zud$_X<|&SL-gq4 z%>xl@Q5r#oOngVy*LZe6bK!FCXGy&ua}Y8n;AhN|=4d6pdK3wZim`-MRczCtr%3xx zd|<%N_`{0BkA+Sd8^jl90wyMY5(;5)s&V`wT8^p`!kOv;Tc0i z5O@Xg)KtbUXT{4kk>|OWhOTqd1V{;Aa}D~C!fY$Y2?-OJpvH?2p2F}<(pA5RHo zS-BO>`3%+4xt(1oRF34@2|vHc2L~UqIyvl{Kvf%aYa3m!?N>(I3tg4k?6^2aI zJYH8N-1(Q&o57RXzvBl1Wt#*35tNnh8dWrxsK!va?Z5Xa=lI@gy4LOXXllSHK=Gxh z7++&6ZC%<`bPi}IyxGcbCypIUHGXgqJRBly!-atUc7c$ByL$mfz+ zaQ%U%nYD3OC!u%XSNHF-%U#VF1l(-?>W>68(v_jCJLTk1Ui^b_W)QL7)?1~e;i31k zv-=LkCH{>O|-BlMRvUwz=Mni-{QnAv#~G0@fL> z2r%LyhJb+>>FK)ljamUuUf(~kcaz`s#6*H;`gk3DG-1YDvuCmye`zMw4%i4i^wh`W z(cpIoXA=Av_yO4suLn655GDS(yp6I-@^9ek_#1HBCuP?|*(V++304)oC5~no`hO#B zr5T%WvxW$*K z?T?Q@D~Hf=vguXC(?LDBbt{dG-V{~UXfKx~rx!@|`Bf3tpWvC)(Yf^A!Rg^J8JnjQ zo^_rGE!PRscsa5UHbJM~W_+%9hoa z#@B>2BRm%AGFXNNOj7n(2&FBhzR*7a167Aw48cL3;x#rJc#L?30RiknC}hp1b=c!e zH&eMa@;6hgZIAxbN$Sfl3j8h)OdTSJ*D5TZ3Px)`4!E)N$=BtNCToez;_##Y4RhfD zthA6L%Mr!M&7>rRmU~~W0r`(O2gvvDBVN>T3JZ#$0P$@=7Wnpp#!tF)N8jksf3-%D z2pT0~iIfIfp)ssl%~gil1actGKS8_Ld=CT+so+MUBPME&Q^H=v}QLI#leN&@^h@_O$p9z>&FkrKmr_}@B&#l!%zZ(m0`~ULkWOZ?)YSfm%lKvatRZ1Ms6k) za@Q(s(5U2G6(QkJ#fQ=f{*?kyjiSONRxaq+`K$0URBcq19hZ9j`TiA(b>%~bV ztt=vcSvcM+Vo5zcTbInY68c-d=5g*C)%JK;wsJ2CGY%Bg2pw`DfcvpZ7DJwkX z>S%Tm>8~9~D8WXhK32|vyd?ZU7&QZR?2p~T93&DY{NucM<27228C%qijGWIS-KTyA z9LBk6{6LG0>|^|%1-m#m3vzoVFD&}8qEN>>;bwrrLxC+4ipkTb29xH%M&!>6`T)!8Hgto(fXh_UHFG08^;@akBRmrYz26dtSk@iS8iDbi7IL_90My z#g(rR+%R+3HvjW@vt9c8A$%UeHSb_EMaRxS@hz|9+YsG#qd23oaz zF+w#AQ$gU`3JIA>r2nIlukpJqVFcy05 zDfB~Ob+_QwpwIw?-j+KVIVvw-N*qH3gQ{;2>W1h?tJjS2&}-1ldWhtVhs zH%m!|sD&~HnI{CGA=DLnHR_B#x=CxT)b>d)Rc!bnJs8Q-Y>Uj=o~40rDMyUSWNxR3 zt4j7cW|co2q1!KMGq3nV6OWq(Y@?&Dg%>Sn77I1ya--D4csQwWH2>iZpZz}9GCK_1J zD12^GU*;d6(2+>h^lb8U(l@L6gwE(6r-j6A3tL|>lEiDw{Q`jk*`^yCXGo-JhDWD@ z3uw6nS`?CNE9Ib%#T9uvk^KGFP3D8#bVtSqFdWL9P)byU)ER zzP}FTVM6f-?OnuqZjwARu=IC5a-=>{4{tMk9(kHmZTKx{OZcL`mlo~HGqFQPHmVsu z&coae@`(^0XXP@J0(dU@=btlsA0koBks7HCt5UxIQ7V)VsfO}d-|$fD@Evl;8AvDG znz8Q~G?J(*6QB6v(g-ItzV08l70@QhA7BCW^VpBgA+Wap%XIEfw?d-ufEM8U>iW}o zLJZUw-K&WWm0g(Ij>-PSAQFHGwH8={)@BAeQh={VQF6n;Yn4b6E*Sj>%4Z9O@ANW` zzXsvH0c^EpOQtvN4;*}u)ke>_TJF7-nE1(k@DmbtXbz~bVt9jY^-s(;X9AAXWzk4q z`S;ipA=g=V;V!gga8byA85_Y-2mJ(;{aJ?|!tUyo-_hEtl>81=34BB-b}+}qa+dcK zD$q8^T{Lo!5JF_R^x0yQAA#3YR#sL>l2%vW^oK^V;OAi0*%Uk^;&DZM@fwY|zqY$| zsx6^52O2cwQ;(Zl_1J|~H~l@NVnQLfXqV2FLc~8fFaX6zh#}&epgKTIEvUj^D`A?@Z7m`qNS#I*nVHO+b1lV)8WjTdJCD7AaWVrBf zlgsdTGg&B_8K%4!bLz;Uu7c;9@T%~&AO0g7#b18OQ8|l8rs?Metb(&7jRnud@=2IgmF4Hx#2*nAykKGYWX#+$Rvj&;UnJKW#5leiM0`vf+dtX--LJ8 zxkG!@21VE@u(%LU0$9Jb>$EBQTeP2`DG=6PglrO=Sg0oR05BzSiiDudp$%chbW6k2 z(rAZNJ5T6>!GpoT1Xd*xEeU#PP8yEW0XLX-O6GqTA&Rj6rnChzfe;rw4cN1=TLY1W-8#w0Spch`it}EYus-^Ww#8Jh1kS> zjf~zCtnibgNpW!#xR$7Q;XO^oi~J+OjecvZjLf@+1|QJN*m?e{FHfioU#;A%IB8kx z@(dC3o;&u`v1%NIu&T>|H-IO)`cnf1r}CtvSY*o&C*Sjj4{VNa(0pq0_JrIS1&iu| zHBii;H(kk>NUz1qhAA#Bj$TA~G@K&1?*hoz#CJlH1vD4v@``cY0ns482~I*7G-vK- z;B+Unt++M#MSSqxA%^fvfla{{26_{be8e;>;OsYUY$hfv;R|7^+Cl8IH2%;7-zR3- zRpLiCUf+K&de7gFxCFnxaOv-s(>C(|vKIRuHu+UL#3p~~qlzon$C1Cy0y5wb^fNCP z_=eJUK(k+lsCrijWXY-jsTmO*JT|pW5}#$!?K!gMU4Bp1iAK+iL?8al>c`#Qd~Wsv z8ae|AfagudO1)E|@h63X*c`FX@H$1LY+kRjG> z>fn9n6BB1|6C>9cObb@MWlbGfPokVv~9@_d(xIkjw ztkO^P%UEit!>7SCqM7^)P+b{AiKdZ}gNNhFUP12@(0T#YLe@_K%Ml6gQlJfZ2>{j> z-rj|Wwhm0p(gzCd*pUM+9SRi(d5}OW1PuLB`FaH>vuA>xQ=emxU0;o&nC%*^`QHOG z|MStKN6)-4hdT6r$M7D(=oJzP01VDLhN2KLX-H>71*Zx%H3-m|n}q!r;1Hg; zN6(*gxj^6ravN$>-qUI5mj$A@>W{gC$ zIWOCt)G=gljX78R5W-s2dktCmgdfD==f&d^l9C#1WD%=H6++YMq%aR&5BYJoATN%q zpLE7Z5*!%7oqz}Zb>_WXgNyd6egJ0$Ck@l2AXWfDPfmrQgR$!zBxiI4R@V(=(MbWM zk6c2RvnP063Js2oB21xzM$tSQWJ;ea1dc)>fIERY>ZBL?FRFB_n>oRxfz+!e8|=3w z6Gt(vFCl#7ssU;{DIO>)MB9x5mYH^iR)}rca`>Nr`U1t>3K!5U6L1%zG{+vHe|A#9 z-z2MF#8eh^K5_LS9YLpC?t=iC|J<`=YGH92S6w`=st?qVO#Fe&mNT8tnOtfIo|!#H z`cYhc>!ZWldvbF-e<2fkH~&>c?DQ_fKVNN^3n?Xb*W3DokhS8zdtR;2$jI1y;3l48 z(9l>sEA>&HxpaKPBGe+lC<5afEOKm_-jPK`;p-NOg}>*^t)!%l&#t-P26op1ALrX? zckr3=e&ojC)EhrOL!~Fx?RE0>MF$ojZ)mcfEvyRk@3~NJ#rY@H@jJMWY1~ z*vkvt2asFw8ODvkT44Hh3LlId$6m%5$o`l&7PIu&sTVs$5>Z)xp$VkqHTdv)f|QI*03 zBkn2eCrp+-&;50DozlU?dT|!fq`KZ|q*HmH$ihU>k^hwvGK$N~5thF6H{g6owZa>y z(0FsURez9@#78gaStrEU6 z7BSfAX`-yVfUvKSMk|K5IHb~_r7zusDs%o^n0ySkjtyvW0I!n*wr~-{PYGx61T?_!`Y)02f|>6`HGp;)dB_1?fStKxuiy~ znaheblkZ0R-3p;exik&V(TiDi9x=^e5&g6Kpq{Pm2bq#;h-Mfry#Hnofocr6;jvr3 zJiTM#3mw(ETEB~KPNNKT@ zF*tbe`n>Le0~neDxiB=sGV%Sp%!{`%ewh2(yY(RC>bad@a6MCbu?ysOy0T|A+O_;} z2jqbhJqVT34|k=b9)$>O#nB~uWYXvM_3N#-W?$2h^ys!zD8mx;0T~qeNa%=Nm2f<{ zgS^K?m$Hj1;LSl@-Sz0<0Ui?o9dN)b;Q4yI$29wHLwX1BwJ&cJk1&i@uU?^>IxP?0 zp3m}V{O#M14JF3P;8fhbd!sr`-*}vr&PL=5lD$rCa5>-}jML2}XedbvoH%~emls+* zcRNp3pFX&EUXMf)vCsc}Hh=900@|wiFSbwP)uopL{om`b_aG|)rHKc2a}a=X3BOXq z1Q7~_GW9snF>EiW(O~Rp<)-UrOGiWTg!sV(!^4LUcQnmH8sO&ke1*oTScaJ?7^x4~ z%b~@F~<Sp>c<|H zjEKjYeGpqF-Mo2e6W-u13U8QJpk+}*eT1F>8kX9AxJ{uDb?JDLjD?O)`N5&dc$;-b z1tZ3pEqzS!CDDkO*Qdk1Kbxs27!83qvKfK~eCzA$Pk3CWTmRd57In%3YP!8-qUIJ3 zPi|ztMx!X(`e?`8Pi?H7dT{6=tY-+Cy*ufARwWr(iWnXTVG4fkL3Tbog(p{jQ5%D> zq*&JYVIAQcqNEJ1L~q(#dI5I_KLrmIp%UsgKW`zpqFg{p14xJ2TbR^aij6NeI79ZJ z-i}F}FPO51zUJzCX`VbWAiHT2Fv*byEcWLLsN6IgA_U_ zPS&p7z@~$Jic2#4_(G(L<`2vhsao{fX%@U%tmtf<9x!2? zwgB@<%iADsbY!6mUwv(!n`TTF+52Zpl7v|FE{5^d#2>G_HcF1+4yw~&GP{*ByFNw- z)1IVm5*(eX$m~_rxZ(WTMz9~Y8#Pl8^6Wx>gnnjjg2DbnBq90uhIPXfQuto6rw%j+ zvj83&!!x&RRm;tfdh4a@7h=y~;X{Uky z_&nnL@2^xiDO9!6k*XQ%*Ls!<2xwjXc9rv6_&JR-$fV?1z6s3^awyvx>xEMQIJy-U&o*tmLHdh?RaC=!74Ll4h_BNVUweg%K}g(Y ze_|G@M%j8MW!>NFH8?KUK$M=iTCT!J(leFs!rKl2D`HSGGVGMx12nmBrM4<3+A5JaPM>V^Z zlJ#%wKaT<+G7V!#36pjHCQQ$OkPf19gvucG#{sRb1QA!sO(P7-MB^(dIg5i>GlbiA zFd#H+Z2-cLkQ}40e`lw?KCb^}u6n#9FqDyOYu{fvwKun8h{Ky30Ery5iS!ME8ZYdE z4K@EW797TTY~P+K;RvNKHrGsvpNP}MwW)c!N@@`A()8|~Ka}Vw@%P{RiEpeadRO5L zA2F{CyW@+DjBm&qMVkT&=GJacnY&HM)i2$RBvdfSA|VUm$P8#`jSs5Y@9ov zZ?0Ze8Ur0hxS|O{wiX*bkoomELW3OzDTTM*)*sw$v|v;K+`Z7U^i+#@m!mpn7t{*+ zNCCwWkLwdaJOWIMV2oMS8J@6d!Rcq8C+!l{Cf`V)ackRMZY8bR8;8mVjl|(*02CxXm z4qR0$D=WyVVA6}EAiT#1Lp2y~rnmfni2IP0RVhHSRlyiSYcH3;$6Z)si{}Vj#hdzi zOcX2%mJAwQi(kLEMWl)7I$>M2j>HGGBSNw08jbS1BFD zGkd!V){q?7ayxIMNEns~3O}jy$<@$A_d2`_NE4MO)*z8FENepz=2&fNq6CLFv!GYG zX9HF=I{%|GyxK-M+8mMa1dHL=0U&UlFi&e6;mk+BivYyhi+|9{VMD8$Z*6OXms<$n zqom|1Knpkl**O0xPuAr2?;*|o9oY@LcEI4ntSl>>7`cFH6VQqv^!OE~1T1iY zBfOyr^tcJwJ5~yWH5kZjZe%q6_44+wez?2x*ouG?`cHrH#7+amND1J}vs~sbWv#wx z_7uQ;xX}cxxOlElrHn|SpH7sQco?lV2gBnF$2KTxh!|o(beOFT#}&FMD1?Z14&GIy zG!r78gamI`er*Qo2L8Zoh7i3FN^~vk7!w>at9Smvhrsj%WZJoqg^|b%^FJhC#fWt! z3j=b6!WF$kKO(`8Aj29S*b=-VJj9HuryRT6qW)V8@Qgfs@sKZC9MKSNmKwk9+etz{ z!%NS#`5OJa)s)+?hKFKk6KkDll~0`oSSJ#H`b4KL0!dLC(^< z3KncR$eADuP{B{^-_H_P?+g70kt~B^58r5XW;0?tjUNz8WsaszA#$kklpqh_0bp6t z5NvA-&Ruo3Ibio>aUlRYLW&!ihJITC789ERFIB;=%FF*?R_x)!iV|+vG&Ww%0T+&^ z9eamMw<`3gHnPQqQ}~oW_efrCoX)5a;fJjimaq9NFQBf_!FZ|OY1W`;D;=V`mtejM z5rz9WoeXG)LUc1$g>IXP=MIbdW{#yQ>nsO_$d)@my|$U>if>r;wrU zpKh7-WK5i0=y?_#Hk5bE2PXD*8**yRfZ?@2d2*M1Ox`MMccE9hcM2Flc+|zJO7B+I zz4aBc*NYGffjWQrc?91T7o*YAD{fVDN)|R$bPGT*5VesI8_RiC`CGV9C3Hi8N9@Ac zN!0B`nHF0dAgWwj$nyiX z^dpE4)sI3_#3DmR=g@I8TCju>|6x|@PF+1cKa4a&cKb)5v~Uzn|7_dV&RKJ*##Y!< zgNbCWa&6}ZH1=YH9cROb&Q4V$)xwLzo^pvXhek>Zv9#_vDH&sAK z3jbxXCADy?khOdKutk zoN~&23TX~FM-E4zg6@+JI(No>$6Cu@IerTu>YH$WDqpOwJD9kY&)QQ-ClI@^Gnk-K zm=Bimim#7l&_%W>>(Fy=E36eF_*kO^W>-3lS1L^a6=2oPuh z>>~5Lqh`#vg`Jy4jkSqPWL;DXNeEjIq<$wE=-P|JRWH$Qee_`FnnlwX47EbRrAO#A8;@<3n35UO3*sXecbnw{H*L9`lYB$$e~AT3L90@Up)V6sNG?~#DA5QQ8>R>*&dIJ5|IJIQ1~y(Xy5@j*Oc}i zdenyhi4dGYP?R{NbkffjT1xS3!@rV=#|egcC-~;L3K+|VTuv~ep)}6Jisy4XEiIsk zdxRp5$d_0FW2Sfb*W1Vng`oOE!ozWa?LTa^MJPuBiLLFMdU6)sS`~N2VJhr?*x%u? z+_&`GK~U-JFVIothb1Fd67%MuK!fWK5?KtPBbXLcvZGyvCIHp*KXcLufsQd8?ht?^ zWuVqr$D(a(oA#DDr{yT_9ROt{pk93M_@X=0KVj1o3%J*3_F(r}NKXTmGb6c_00iIs zh2y$m&sEeeVb!SABgVR3kphQP436CrnPCC09kBJbE26k?$FY9)XPbKd_%Kj1-97PH z<5Angwx5`yWZ4nka-X2%z+>INzbfCd|B@GRa!)jJLX7eoiG^<<2*S{Dp36W=UBeC_ zwq=RrI~pT^X?YX`X9C{a{g^p=3Y{WGc0TR!iBn%mIxPw9CAr39;2x1hgo)j$x&{Wd z33n@*7rg07e38WCiHwhcL>e_5T>>7*_8!4^Lw2L&TXVPdGgl_51K@VfhrQ$W=ROW^ zr8w4|wAqirnhrFJ;yOs;S)a8}ODE`MV_J zH_S-ir>}qYml<@}wMu&x#hT^HKj;a)BD0qf|Ov9GK8H6vuk*pua>`KC)83@rk zgpC<+LladQHd4JV3-6Dw2$ftpHO@g~!SVIF_=X!Xr1TCtPV8#(oRvNSbf1KaomZIO zuudlghTxzhg%E$ap|UoTo>*}~WX^62p6YeFEOewis#nuC?29qsF~MWq&%XtH&$J^o z)}};!9PyBe-$>ZxI(I?WFuBgCvC!)VW~2VRbzH z_;>_8Vjw+=lG08@>6rZU52(1{5h5xLI&3MkXXQor;DX2FrpL)fl4is8cTf{&&U;9r z$zRkvcJ}Nzp1gE3i1G#q7dEiMy6!skME($aNxW|HQb@{&hLhe6kYGZ3ykDvaM%+R|L(2EFtF@Oxi_m~21X8@(ygRSoUg1XVt#NB0dTX|2OaHxYSA=20kR|`h( z#k1%0g|J>jpdF?wBKV}`Tw(WGn?kFd5fCxtCqm}8RK|PLy+wvKP zn-ueM&4nlBE22RZ)@%O$gasEV(2%xaOdzCArM}8A&}V_X#!)1JmHnH> zMp!e5lmJjjs8ETJCv-+I`6DOvzr&AS;v_&l5BM2cctVjpqXlIpaHNTi37pRugDb`$ z)mTC&fqtHdvqN*{m}3S+qsCT)ENxYcQe@2rIY!=f&lvG67#RfyR~%`@vw~1kRY8Mi znnW;=ClfJn0w6m$OCk&h@*H@^@c|XRf-5L#_nZ62yGs3h5SRoc#Bl4485Lk2C0SWe z-8ik_Bw{#GA#p*@{D0Entmcu1wiAgKLwYQjHHK@9@tT?wv`Ux001w?VALG1hckfzT zT7tSF5yO!tCnr&%tpR?&8#3m{=??WUvzJ1+uEQhe+@-WC`0T`Yz3AHGP*J{yTo}WI z`m*!8;V*-f!z*1s^E07{8VX*c2EzIyloo1;NDFDk{!x4ntT3)Gf=$rpKz6B)@_{8Z zC@l*|^P3aTvG8%N6B_MCmvGc*lcmk`hDta0n!kegE8Elu?+Q5r5F28S65Iu2XdMy` zeurZdL}_x#a7B&(psy|PJt3A_lm zHms@Ylg*l_2RB~C0=des%=OEW$dUsZC`2K~nxUK_3+HyKVpbkU*aVi3wz2P z4REFS@6`?05d1_QOOK*tEVt8*i)~Hbpe162GZ2Uv9%&t&*Wgir0=+`x4?I5`l1mtR zyC2kK82`W)%1MJ222KfWGU}@{z~4$o%FtAmLR|)p116?KPyoAtGTuujP(Oes1S}7k zZIAb0GC~5RG3NuX40=2X4+4ZK9e_tL6eL7~Zd%|MY9XRdJOSAPY(lOY8WmWY&?FKY z0qECHBY(lJBXT`pvrq`AU2hC%L3Iy06XXa~WPal{yztGVTxIpD!@b8A0W$nQ>6(gp zD-bJH^a)iVBv9D&&}l;FCM=vKQY-1Xv~!Zh-G`ClMW>^ zp5d0^sfz9tAW+@4D55X{X&G-P5m`AEN?elBUcnXZxaKGWbq1Q>xP8u|(uJ{1jE%J+ z0n23SRR_KCltWtyD?Q*d;=V)B=#5B8EGrl_*2L9+%g;oazyBVB-6{;mdn%#LF4+wW z!~fH~VNE0?!3T~2g_$p&J#DS6M7kVqDLEA`%Ws86 zq~uy|BTK}fS<2+UdIjG(b~sh@%>aS(^4xJgRoEid|35)GTe2@Qp%R7D8&40UOVC^) zxB}LVUATw9o4yLn_$~q32tMivQFHWvSwxDFNsV&z9tQrw=AcQ&BTD3AnB{y$+5j<= zIUbgm-}n}bfMGD5)Y#DQ84y|u?*?HlqgEW(fT{<5x}kfy;KLWvdG(iDAs3f}sSmww z`$Fz!c?>7sx-oq0LdVQmTmwLfP?%l_SaZf?;||+u)ZiPB)D?SA07}7bD$WBqN7DaP zYnBnFfZ-tct?oY5jK$qoi+Fb!U(8>;ng0nRO`+v)q_r%(gOFR9Umw>6;cYF@o;blg zvj5&6T;GXO!h#Xf3^7XZ4Pu-X3@CeGb5p8AtAvmbThupeRL26U`pW0AP2bo*1=|#} z6ftg+ZB;)|ZuIYjvrWwr-YO_I**rcNrRxxu{B5p)pt>9&hQ(4q`H!g-jz!S64k zh=HQc_(2}d0M8=KpWna#PYFYJz5t%EPKa;Vau`VL*D%Ho-xVfHl(3Z;ss%6!7nVxU zn2@r8C>9iwbaEsX8}>O|V{EXeg!Vf8j*yD^*?0; zT%`cRK4EJsB3Pf4lr-!aW>)I!4cadFMu5qN{Lg;adGNV#>;;U2c7Rds8Intml&PTq z^_yr^Of~rFSqJoq2*1Yk(Vn=Q*RRhE9E*Y@4Dt@dPCV=wF4+hKQgiR_e?tT2?v=pT z@9!5w8w`aKjC5bQL@W(#aUcoTJ zDDyw?CGg)nHe)&mF$!CcSF^v$S)WtO2az*#+nO%bv2M{zoP?f9zHK9Ee2cVnHcAsf zB1}g&Q^m;EA2wF6le2F`s@DJIf$=!B-wRdvxLCLz_kX?Qd~fg?==961uJuLlrb~&f zTPXdL2AO-X)#^Tvq9anKpe0Dc&v}pjf{f1nzz_gT0P0A;N9I9GP4mpNBoobaqUMIW zou4CY^tFw*aNo8z`cFj92&}2nhC&#wYO>H-`UEA|?UjWCmf_Z7rD;m?Oe9Dq#?W|oaJKuRII{;l0ktCLCxVJsN5 z_cB#ieaTn$t+5+#>+%ks?D0<*rU%?r&Tgk76rTt?kk(iH;+$V$L-vzGKUaSOScTV- zpC~8p8DmCy4<7897edGVahda7soTFDR)x$y zpzSANyMr>ZJMyg!o>X{CwT;MNA`Zp&Xx|`ObjI1cP81h>&SHE(Tv}o^CsLh`54* z*6k%YcV_}uv=P$%nF2|THyZ}}Mn_$hpi$ll#L`CZBRLfjt~eGDy-Cv(4&U}~LrVG9 z5g4r0=hK8S>BP_W?LCC>1s@S}$pG&5$3tp|n&8vRGuu|s2h5lN6C6t()T#I&>T(=f znW{4nkSyFk?q#v_Y_v+x{ev!l!sfeSpey>~-y^OMT|OcKY(!!ul$8r1et{eu7gMNY z`?2nLA?pZISe2xV4987Q!~>$8LOBCOJrtCgwnL&at=TxX?7G6gJ!pv4Q-cEs0If}y z_mA_nDzeJ!F1DB8ut4N9oOw$sw5wx@d9KFCTp9+>d0%DaOVsyIgb;jK2GXC^L=wTN&mL&(C!V_Zb%p!!E$A5QlAvcz%CryU3GeIR59$IEzDH z-A)?g4Njqlb<0ZcAH?sNjT#?knB%6W#Vl`T4TtyyMCf>(?NdYw1ic|s{`rR3oXm`v zh1WAPbm`G_B=<>}&gah4*kd_GX4>e|A{T1PVXCeSss9$>4Q#=NPq|sh+IKv3)Ml(1 zuaQBi-k|cOqyDL9)TWq}Fat5JFP;>J@bw2pnJYaZ*@rRK>u$HAs8+Kt8(vI@u>F6t z)+7Bqo?~|G$p90{!5H<}_|lF`ah$YW!r?N1iuzk5B|{QvZqz9@A;!rQq+Fzo zCQ*EaqJdvh_~Vl7J}0-iEX-~p;HoA1@;JJXQ;1!6&fgfC%qe0fL#MTIHeVm__3X8P za>|8OB&sNiWH1JW2nkUa)pBj2d#|&!B~pL-{LFhDj;uSYH$l3o7D}paN6q5eaC8RW zEs0|vo>e^m=eeV_iC^9)JUPqcj^Pc*CeOA6)|{s*p$(!M9SWACX@y-#o*DmLG%91B zC(&*s&odk4q{TeQJhyNyc;W7D$O`-T?xP-4ln~WAv4*t(($t0lBnC9rj(9w3`zgM) zxKlcg3@Xo?Hjk}c&K`s}md-*vs{fii2-l_h31Vk$zO+u?O zAB{9RCmmDuND4LtbiU!gK8j;m!=Pz$-M8m*1O+NDeA^`N}qPAmZK9|3Wxrj?njJ5<*nV(u3ln*;~?3tLf`I zM_wDYemjRs8~bk|a>UC{omFt@zCwA^)U;;jcCoVEYPi(4%WD1XZ`m11;mk%qa$eyE zL7iwL3xP5m>tnC~FN;vfkyeyF<7C*gBPp2I8*ZKrMFz4i;X0N}aKb2DF9Z0_o&}2Z zdPnmsUrR`+?;j_!>MV3|$e2E2U$4m+%d9HadvY}rZ@_b_eS8Ec`E65^r%E!WTELQx zwx282d%WnNoWM;^x8&y0^LKuu1lE8cI?rS(}#a@bU zg7vqdWk&4)+8S7C{x=TO4}jqR6`|mXJ1f0)t0SsOi1b-lSsy)oSUiBV(8E^V_t5ep zl?xputaB*K0MLW^v?y>rhQ=7#?l_@JF^s18qN!;+Zo`6(9i!^g3LCG#o%U$srhxFG*)Cd?qi=rRrlAeKt<8H|*d%((1O4|2 z2UpxuIscFagJX1t$x%+n~E$qxumY46gU@@>0*T z$<`a2{3eYs`8~d&X14v`vK_1{X0P4Ha9vE|L{9c?mGhAmHoXjl&JXE*fmF1vK!-3g z8lN}IC>_v4CL!EI753-q4S$R}MWo+S)5WE8IIz7~wT(>fAAE4Jn@C^EY}veJ3p_?$ z4)TQ95_>Vgu$@Vb#FfU#PP`aezUR2%0D90Kq17UKbI^*Lw~NJHoK_mpZU$~x}w7Y(Pk041E_|$qf0>*l3Sr|n51Z6Zb=4lQ9`EYr5 z_`~sF8Kralg*nUCVS#xB``wt+8i+Xpr|2+8325OJu;92afxU7 z#Mjbbh)hsk;X#Gf3OFNbAtl)7P^1BF`hbMaqA#VG6MWBj!!yQ^X2?-+k+*S0030-f z#S+RtVp29S@Zx-}0j9szJm`8sl3_~s?xm~KU1h$p$B0b(EcJ&;j>z%Wr!2K$g( zmNM!HPknY7*~!5r8rvh&7G5GSNm%m|aQe$>S@YjRzZrCCWPU&X=c(Q8fUTcbA!6Px zSD9cw`5uo?a3>xVFO&8;DKLZ- zAta4ePLuIIe{(ig!s~p9`_sNXNj1|03?wD~cc3;cPJ?u2i>4>xa^aXp zFZ&m8+~dc;@CGCj8V4k1Kom?wm1ggTCd z!Ho|2#1ird)6XCTZZ(a$gsltEnoO&y`w_@2iBW5Y7J);F#hAPfx$(Zej&*O}R?7ky z?)6cd8v-?WBR+o6R1RjAch*qC6ZKN3lX%=I45G4>@r1$;&wudImxs3Q@&1$1FaVMA zeM5;>2YIA=0iPd*`2r;KUFcTg8#ej;+F_PiW1E1di)4dTzwBgCDut*3Zb8QPxC~`J zdMBjg60{nvoi6gVW+@;bUHgZS$~UjcEbQvVOdX~LJ+8>cnne@Z2p!qhfhZpLRSNQs_4Zzd-A~{?a0BrQk3SwfZ@K6j@4;!a*hhYX}}Y zuVZM7;^G);b%2pxPf9{M`6zCOX*B7@FBS8fYUML}|h;cvTd`(p37Dc|x8@o1{pFY^r4#ZVi zQ8E98%AS_jUTvW`Dv3A*1|Q|X(1U5RGBQsf>}oX?vPAwCbf4~7*+T5P+^g1!v2a!# zhffp$A-mTL(?{n4{u-)a_AaWQ1A}XNd=x(U$#;hJE7jj@#~u0jU3}CzzKO^|A(3f!ck?l zKioe#DO^7)%}@PVtXE#&oB9&jsd)*%WmPdD1RXbkHj1OBZWAz;;=fm_r|KlU&X*5y zsO#fBsUd6n)IWB>U&F!1qRWH2{)@9BhRj~BLHNf(@ubG@8~Sx5xL@g=gLMjjgh`c* zK^)=_$KQZzg3uvzb3pHQwq<$GV`und1HPqbF)~gb6{3XyhLRRTk}?K)me&m1$2ii#?MQm#)S zN(UV{=zyv|1cu{s47dUL1hee;!m0_V%wy9>ReeCF5$!?b4mu8B97-Q^fP9v4azL&^$gk}_azE^!YCcGff3j@-7 z4Jbi3a&nG?V}tf?=t0oChlo4?L|=XeH^Unn3P35X1O!x~%LUCsW?Xq`mL;*b_P1Va zEU_XkF}#uZIdWumKB`M3bcJuYQ*C<|r!2hIMp;(Hk*$L?<3Xo`t@Pxmlm z-JU~nh>FfP<1oBEhT>oQlbkPtXJ%D2U&{ikD9&Il5leB~_2Lw=E9dIv8d=}5LY6P(i~MB0Wf1P;ll~GapUmMpL{DnKZl1Q3L-*i(%w5@#uaoX z!K>H-KAYof`Aq#}TD~-BOb5FJdIY)7JqJFGczCs8GsF-Th~!XhpU#$biUV$JBtb-g z!J#i4qr*)85Ka@*qKJjo_IoGW22q77Z~Mr&R?D(gJ;SfgKEa4m48+7#-xX+0CbQLX z*V6Wk($dnrJnrA)6NPlO#u)4nTv0B21v6)M@fbS4T6gJ)fq9Wx(UBj!u-TO}vi4YK*ol{4V-p`F@eh78W52uDt^ry6h6HV5uxrfQJ zV>?V*`=?J5Noj*=hqaWRL3eI2C}YHVjs64U({>cx;vPaOaVa{pd+`-qg}$XxG(_+? zg7_Z&F6nL(l3~WD!DkeHDVkF(6k`QAy6|hVwT>?kDTZTSh#89gpo#5gRsBL>)T5=k zzdU&+L_UqezDZOU!G$}L8vot;*strb_HDso40>9mbRo{Ub$u9qZBea?d6GVgya|tJ zNBtMF7PCf`30d76)osM^a|^uvEv7In;k3pS`;$r3vn;_#I$U$;`!>cPa-wR~7!f>$ zd@)rnc5lBB3KovBt`lSBTJB6rppa;W8BYLNnP})sM=Us1lHUbrDY0zrbKb=9hZ{Q+ zV_FM!Z71i_Go*|wonE-3Ml-HkkE7|#ssT0UwZH)NF0cx+t*UZ-ap@C_7B6E2t$Gi( zPjHD0-!|=fjbr+~o#D(Ag}`I$6DN683oK=JG;DoMMc44B2CS`{Y4Zc|LB6FTK_+`-aM?wz1{o&mNaV;LbD`O zGF8&7GL{T0R8qzy2_cmxm5OA}lvIdCl0=9`(`u0vMMarOge0P#*X7>NKKAq5zy4T< zdvB%hb$y2Oe4p>(atPnrE|@w)luI=K5}WdB@<@#o_4!xJUV&pZe%jEiFX3{Orcd(k zA$BH^7I(XDRH6)NKdAS-4X3}eROHO;aAdzfuiBVa-=ErxVkYLvjy8V#ViUnAd3WLt z)ymIzbYy4bpFH~ylR7A-eSL;1#hxqlH6E=df2o=AAk#IR%@4VMt;N!?^=AK7w0p-- zof=$q@j}lreBwhb8#;2kt%eI4L7WjJY;|slcv2ly*(>ddYJK{9$`&#Znt%0E24@GV=CXdLnB|2 zb&aH5L8;-hHd1cy6niLNvgWya)Un_2utb2=Iw5~?_lM8fh+PKPa}WR!x(XQzi(`}0pva= z=d~&2DkEteKd{zlwDwuKvY9_zyEz*DD2?-)T9;F^{dVgU29Kl-*39xT2Pf$r#EAcM zB~B3>M~S3Gsm2TVLhYLJAbP-kM~?XAJO<7t+!l{CkG$;cilsSA7BBwy@VHrqNfYe_ zI}$0;If(3AZ+t;^oLjq1$u{0lJwkdIzq(m3!V7opBA{;E<{|`YPEMMm7Ed~vAZy6L zx_j2gBi^}B&~MNir}MNyDSyCy@*UH!>qmLmbvg*MWQBdepEG96fEIAh{Nv!#OTmVGR2K{)s<_8KKKrZ$E{f8!%EeLuyP-MMw zplL{D*keCDT@HCLm2_I=9o|{+Dc=bzg zadibqe1aqi!=7Ot`KIH?Up;eX+UbX(=x{{gJS3rSUt!^s`x77m2`7q0st2IKplg;i zW&;MKpFh8b!WyJxldta?w*e#~5SwXsd$@)whlujrNvVt3{{brf5|*-A2yAIrJxM|l zBkh<+LnI!{ESoq?eyHifQfWNYRO0X=+yJxy1~Zn>ny5L9f#_S^PzJP~78Cg^>Rz`% zvjElQ8eS5NR-)x{h~`)gf-h(v>b8x2c^q|uZj*$s^1{;WSK_sHH zGfccyTM!mTy$<(obh$aTW83`+its%KTUn9w)rZRHjI{HbTd_Th!`uoN=+`Sw(8+yqlsm>>7p8DDPZZ!G|j!edYe#ol;X@`NeXKH586%1y} zQ>?T^s93Qf{Cs1evw5;b@T1J`|J~5{|8PP?_ zjr5#vs`7fPj*<<#PY_3uTu&4Ongb?J5HK_tJwefcNPYuM=3gk^a5Dq<#52~T|J&#X zgaOnfnE66^3k3#k9Qp_rR{PW7PrX!CiLAsNUeHSjsh7MS+&)NV1Z${;;1bwQY_cDa z20$?2)`XO2&{rrv6C7$`pG}+gFGPxG!JjNdg&aB|{05KS-}&dCpNukEPS<8Hl2VNo ziFi!T26a_GFrGjbMCzyzMxveT?`*SHR=HlePhSE~>sf_#+|RrEGUjHi_CD+fQ16Z5 z{d**p)f3~?GEClt%hWYffy=Zf16))r+_yPwh^(xpSpO`w7wG43o+-(?bLbZ(FQ0QEb+zBJs$05=s#-9pOdWOVYs8O@JF9inU}K(3=y7jhwn#L4$XITD z?Ouk&Y4Ef~;Y?0hfE#<~5g^9nX{egeFM*NKIVexO=Z1}&=~+0PWJTmdOt<*9r?wGl z3HQD5o7PThg6rk}^6(UC9eamS5|{5oK1<0=QWxsL-FrX?g=7>*8%|sfkyNwiPS+wyqxn6v%$;3bG-1ig{{KBk^}8rx%bLT}|`ZjE-=9E>{J2 zkKR5*KZ>bD$9j6@x+40sm=kmU@dv+G`gQ?sG4g%SuWNG2b*;aXWsRgux6=)q+k&tz z4mRwU^sKD9umbQ|U{Nm%;nL0`ibV%Qe&H-WTS)(3Pp2w(Y*Th#^+$ z#k4Z<)LJX&u91Vi%jZDSGaA6kw~*H6&kUQ0!Lr}u#3^QI+{q-95x9V0C)qRpPI zp|9D!Z=d8qh1tG;RA)`n%#5BFa8sXtXuA^9LVd`pOgvoRl26jZ9-ha`p=x@V1U`d% zo}ac~#ehgT+LnN4j7hk;8(j^Q8%iVsZ=0qi$>FRbX9_F*2ctQ`%X!;BwFz^MOJB=e zSr3r?*>qm%?@r&IcU$UWuK)EswB54dvWB(!65Y@@?Ku&0&SAz4o0`^@m>ipXEs8RD z0jOD{>bl-r$5K7!?VfTRa>Jz}4WZ?MuO==+MgD7JgN4M?p@n8%8-Z#~S{7(@j=mZ8 zE6_?TxQ|HmAzqYPURWO4p%S}H#&}G>yI;i;tOFzf+p%U0;Bf_A(;PC+;9Ub=8nS*3 zX45k&e@GvY2~)LARVtxx{twPTsX}+Z<}&PT91;ZY6z&ODQj8adn~9fuW*Ji?7uioTn4I`x|?v=wC>t3!6*3GMu`jBQ4A$IQVoUZvFm?W3D2(xuc#e{q#zIz?rd-&JepM6$-UL&=7o8+8JBGKE`GF3O$PiC*}m%s2s@@aGXO8H<*(>CZ` zcbDdHNUjPFnf3^n#tTr~Q_M7|sl(m5#-<)_@~buDI#0-eEF=*>NiFkMlUEWL^9JWF-p1H7<{;o`Xs zo0M(cXb+#@+>v^_Fj2R=;q7?iWw zMbf!WvLjL=*~yAplQT2p+?%B6s{^(DUL5mVrR_IW{b)?dvfD)+vR2(g9u{e+t-PX} z)y_O3_zKdp#Z`p$Z&kHTupN$M^7~V4R_N^Qx9X<2voB@*(D!hSF=|I1S zOQb2?ZIuNAX*9;zn7NVMvloB$;UH)DBUN&{#ZFsEV!-SX z;*m3_4iH!1+WaUZW2X~8v8R6fi@|3yGrNoOdRzs+hgPBB8F12Qko@R#>jq*zmRJ)k zZ8zRpPF=aar%J5tm=Ja0JA8e88TlXiT5i4$g9~#Qt__)X0aoyazHcnD4p2Dx%tk^T z0%oVpvYng+Tdj`JxI)*<4}Ez;F4wi0DYLlm!aN+Je*fDZ=`97GrCM}aSx+3yNPPQ^ zg)-|6O1DoCpAj2+*ejBUiUyCqF20n{;%Q`_^REt|Mt45Ml|gz+(Qj|{)&B!izp9<(eW~`h`r3#qN}?T9 zh-w{5^}Q}^ms0k3{%7ESew`bG<+V`zubOeP!uxGN*xErtXMp;Lj}8i=^eB1+XY}zp zY0de88rbTY(sXcj*sh%PUEGl#&h0_`%+@A_*bcUpE#{!voX&EhS^TsN!c4lR$dgFY^%WQzpd7bI{17#cAAJvD?t? zB6u}p(U?+>6J_Zxs%Bb!V5|32I&JxvxQeIm>1j8}I-_#amSZ;0RMWl^Pn`TB_OBs( zQL)K;{C3Kim0r={F`Yo-UwcX#(b=rSvNZHj8(}-a4DmY;8$<=JoT;!Shl(8qMfCe# zs&TeaDdZd~ij07q4AbQ%Ys=1e)KvIIOA4+Uu!o0;9aXLg=gsZ6ec)K+FnnpnH)B8-{;|W>E|fYOWG@ z!YNr+WV-IX>fu?dS#$`!0^7031y%2E)m)t=7#US5RPpev(EdVXe#_0o&j&YT?V(Cm{ir}a}D<~+a`hGM!3KOX*10#~_#f%Tt z|CO{+vX}9p1Q#tSMNFOjO(W-Y0Z}kZfzN^#hxq8sOh>Ft?%usyUe5H4qyN_gMGMSt zau?s(g+Y(tcapUIAnFf3CLd^eqZ#?W#6BCo+!&gGT;V@-kwScaQiEyU% z+lQkCl>f+@Q6AGJKD!PQL0Rymv1Y2x)AzsPKVd=%EFZ6x&Ro`RTTb zV@!axL`2*=9yT30J56b$ta4`#GfT}xYz(i)80r=M?E&I#vz`Ll`LC%aTX@?}cj|3G z$AAGD`humf?~(6y)YQyvuizac28S0RwX8F9{#7Oq8-QczUakdxPP0vYi)w|Ni=0LI z`q2LkFFB--96kCg<7T6pO8R=~`euH;XSj$kBs+BQpczmHnkV`qoZdzq03yN}MF{YF z`_`KP;#HXtS}N}DV!d8N;3BZu;Vq=*@=Qe7PI@$%!{i;1Xg&(20Z3_u_XWn0u=B7e zF&i{!8T~HH53~z<1Pe@DB&aTd-#nT==lfjqc;>qM`G3J&feTJ63Uz2X97*^_aPzLT zaKw07?)a2!dO^XT&w(`BS-OXb63S2kZlxO$;ni? zbqDH(#$aJEFVGL>zGfSCQ0maIl)d_iSA4^w^(@$yN`W!a1#QJFIh2?X#*=k*`RG14 z7^mp%Vvq;ASo{TGjsKs&DWfQDgMp zzaXu!1+u`;(EB<{cABB>UhcD2U#`O`GV2D*F9$65-q_5$J3Qz1wwIC~iZdmRk@pw% zwB4Tvg)FrWKbtE^cY@6=_5j4nOQs{n0n!KriD$N-a^S$_pl|Py0*J!r4vQHtCW8LM z0yfOa122Tz#G@8`J#OB-37ZlFQccl)4rlOm# zTT1W>Myz(p_p$hv4cCsG%rz6p!&Dc2`y#jp4qFQzQA3I%=81&+#OSW6A6LJ0zB=Snk7OqT#u(+XEZo_m^Dg-% zCyHanJ8jC8_hW)@7YpQc_`Jg9`~zCH`9!D9Q38O~!QsGQwh_UU;8VJ>G&6-i%u)kU`Uc_`06ge20>n5x6oSYosLbvfV zR;*J{Gh*sm*bmOpp&dJjHDo$n02TAn&u>A$q1nHfZgC`U>~b$J9g!zv8kkd=8Y;eE zNtehEc23r?anMICu7B42*3QWzL#ByO*O{`mk0ms~XC7|Ghc=b(8C1W+@bH=e>PKgu zYQCLbX;~L(h_HaKD|->0@TL6`i0AqyNf11vhQ}5-U{T{d#$1cRyGS&q-%2>KP)RY*l?{GfB4o5!F@d?$b9vR2W*tfw#ls+ zLlob)l8#C-M_b19v{YGV*H5{fHMSST_D<=gtlXF$?rdQ9{?=}%sh2A(cscb$W~%&e z>PyReGT{Cvn5L7R`PHUxPo1R~Rg;fK z>&LEd!H=rgAk}Zj3X4$;Y@b*oj94|pKuH?krGt|3Z>KegKV`XfhsD8}k-{Mk9f*O{ z#^;gGov@=($?%~5RvShQE%_0LSEMZZyNPWU6QP~337Mc2XUBfU2opM>uEIjnZx~ME zYs?|s;}H(8z+SegWWyhR=ri2FKzqA3jsnz^2kySc>PMxCiUzjsrMbhnDRi8CqhBGx zV|~ZAp?P3D1vK#G<>a*Fmi+Zse>12rU3UhHB&r6x@o{bAlkcH{PdkLo^7pqD-9*lA zZgR`M;KYN}wbB3RzxI8LBX2=(#U;Dix~FrWUB#QX2l+)6pb^9^qr#hq4w!+bJio#0 z*{YqfxSF(#gjo243tk?`XF}t3kw`6Oos;&pkKioUY3IhGSu(V&^2z;UWi1LPwxf6~ zS5J{oF4q!#%!Y;ChYT0a?|ZjzBa*4%roti((#O$O0DS0&7-28^epZ4#@C$Uz5-GCKO#HO516iB5T8#_i-)8c0kC* z?v`^@th%y+xMS`XjKm6ly&7@k6;X3$i*CxBjQ?xMLxN{J@$L;ef)CPZzs=k3cQO#G zUt7TFHs7O00g-^{Y<2D@-a+ zBVu_Er;@co6l{V+x0eyNsb}0Tf5Dir^~w`AJi{>KU+Z*KqNT%@`}s+V3YV!v9?5xZ zmb|^lv*kxf{^{^MivrLA9MP99JyGwsWr`405M3+;giSe4)wu&Z?Lc!Bb-|Ddg4Vr{ z?yfatZ@(|y_s&;idf-xW%P>jq#IDVSPG^~X1?^|0mo%~+C`!N-86>Xq^Naa?!6OTU zbHeM<>cIvlJr_7Du=*;2=f`JWUU)-xE&`RYYVu(9upAa~IFah+l(9JjvI1B$C&{s6 zi|JGBlmWu}_v;6*zTU1It-z_-=mh;1M&DhE6~7R2M#)&zL|Cp$++t|D6d2UbF`C2P zmTZlU<%jV(HmoQT8_WD8y@I44?uLBb2y5%~3P43h^hAtfw^vsRHH19-`&4JEj1PH< zH0CPCc;QZmdRHkspU@`U+EI}QwVuTHKf@BOpJj>_iP{iE_U1_%jv6&nTZ!@1*>mT1 zyp7mBNv~yaE2eG$Hc04-0_D_V@K2(0cvMwY7<9DE5jSxO@#mp1e;U62;L?ZHRs3u& zVJLW=8x6yglR3WKJ5l*~3<{w^F0{z|&0;&JS(iRoNjkQ=yH@cz4S^63{&}@ND#X%i z@L-vXgF>|k3)W2RL&7^kXBCU1AY6j>@HzCtk}q&r{4U8!h|5MJsc#LkQ2h={D7s%& zwfIskZbYoPFq`pS8&A#t$v1nDF_U;uf{u+Mo}*Eidw;~O(fLe?nukBL;yq=r=yHUy zW6$a4UYBZF>}QU#%ms=hY(U0Qfay%%s?u4$6l+itsM;_-gbxV17HFR8Y>GE!Qd##! z#aK=KA>^%A43PR!wxCnIa$_**<^@!Jm{n+4&1g9q6}G@Wbw}atkKqq}R{Hn=$>Vjf z>&{}{{&EJ?fB)^Xe0fBolYGhBfFK(iF>H#CAGfH-oH-dVrUXqQ zq$m~~qibSJ>})2dI^#A+yP+A_Al80Urapo4Sg0OK8t3LL?GbJ3l|2{s;*{fU2;f(R zbsPdh@ox~}rIfgwjiI4o;aQ06Y33;jk&=|N?SlZdnAGmKi#9uDtjKlUdLz=R4$fOc zvZRRB$U;)+BS?16ech}%7{_NOHR%=YNOv2a|LbYwsn!Nrn+P{j5MWn5e?CoHiSEaQ zOOtrSbK|2l7TejC(k@X5O-$5uoP825+Ns&S`}9HKdC=P^+IBgz3qwQA(M#o5Tg7<6 z9ORWG8+nO;)aM)~dW*{G>{&m!&P#Lp4jCe{_VxWB=yy4mMHTm7=`udWFhK9q3-*^)Wnh|IBphxm=p>Bnoaq`QlfXV3h z1BMpP(Qp6qDZs1?ouBmqIoRXAA(mm3sqlfR@Mu>!+NOY_|9UWW1atrlE2Hi#jv`bFrxB~V8@P;-Z zaQdu7+AtD0!C0fnFNm#Ju$x#u&mNEm&=a@+S}0%QuuV-&WH5Lz7xw*YVj>~CeM~pY zvKA+cmluqeolZ@Scss7p2#%|mhyz}jdHrVc4vih_{1GEa;;{YRtImpDxAn%FhC;DC zsqZ_W{;!2&iHG{fEs`)NsO^#JCV2$O93c(cXUxIa*w_&Tzs~cqkR3XtD_`neat^H{ zW~<0aT)>#xpd~S*rJ4^s8}IGwo4>AU-yzdQIgeSiyjWn-qc;{tjOZm2othm)0sHgk zPtZg0K9ixj>h7dFn@9u$s&>HcE=?*qkKpF93{7G8Flp=3**OWwaml6H>4!}zk$K0W zaHou0H*X42n!m=GC@e-1Wx`i~7?gz~1iF=NJn@dU^3!YB=V4E~SG zGxw45R|)0YwCOi%I<92$!t8cB5O@FtY1*wOD2A?=+5_>wo$6Y(m{)J!4Bh*W@cUc* z#T^c*jK7MirS$up!1+jToaGxkQ1EjoPzPZ}8>{I)WUnji?HMx~H_rd|?Hd?{AElGe zrcIk5C<%X~V1Z>y*!Pd~kI%$f;e8 z)@Paj^9m|G^1B+1R;*v&210|u#q8NcqgpJ7Ql+e^rD8`2-J5Z1>g7LGq1< z@Zt4j;kU%tXonyM!>?X-E0%A=hu>|ARbo)(P-g6ANPkm8T8Q~c z-1F(swWDp#k~@)AF`hr^&6u=~8qEU`#XX?-H_nN-UI_H__E2o?1jj<68wdtdyiR^6 zOs>oBn##(nL2-WN`>PG4h%ihN!rcmnz74p-q*W-I&YV3<;noo6-(X4%@V6@t(}(r$ zqpBMC@y*vC@QYjPuq=2=a+0V}&?=#ID=5};_12r z27GR2))sVk7F>F-!^Qj=g#@?wm9;B<`|J4^>rD5`%f{9}!68RzbNk!s!$q;T@2-B( zPU0|V48{&;unS5xmZ<0zo0w=7)M{`gS!pGnv`1khufriW&b zzZzet0ru^kIyuiKe*E}4aE{y8`k4osd$f;enxGx>l_+l*Q{$>!qQ*8pvIB;>QqtH` zIq|=~4?}<-qGH*scN)sqnJ+YIl^w~~{xnwi*Wls9zx`JF!z(@y-Z{W#Wm<5kjbEWm zB>n8?wN?9769)?ek#5o^v?;DXD_zv&9onrFZp+1Zy&a6cfOe1yYl0T!vN?rXng4T+ zbZ+bxocWcve;Oh9i?iPcnAu^4dl1vgBS$_i`+aQxD`8l7{$Nah@AfcF( z+^DPrL**ssqlcA`w>JR(w`))S#gWN;d;R~zHZFXmt`b>cw}W38rI$dVrPlsz0Xp$s zeCX6V?FfTyv9aFEmorR??TU;^tfiP-UAA|-v^H(hxP{88uL$F4dR7oNd-k5{>GXYK zqU-jlk9{d=#KxTZ?*S4f8O^2w%Zw`4P1O9(RBV;*VgAkC`i?<(#v@6k(URB;f5gre6Ywh-jJM2*Q>ON(7g0%L^g9(^{(#I8G(vDvkCL9-CexKB$4h| z^93KF4Lf4K@qt#Y5CfOkcl6DaT^ zhkqA^7F0-5nXuz#YS!|ED18Z4huckp%sTNYQfTHlMfw)RV)MeiYu7ijfz z-3@7dRArZ7vW`Z)RB?G_9|_UiD*>l&Ec^jL{_FB$6Us<=?{-spo3wM82x`VOXqxYM z?I59Kl^%~)gB}ouVo$^94`l9ElrPS8h5Qx3$vs##9$qg4Px^mOS}_sXD3C-Y71RT% znt?_}aOP;@BD30L|FQ#kFi`{HnJiRl0s@38Zx_x+DJfz6tc{sC(!XQM;x}}3{ytzE zo-H7W1AIXSSzp24YrMw@MdjICRJ-K5Q4-GGTP?_PKv;T+<`S+odj{H!DUOcOJAdSB z&YV5Fm8M1TnWK1yqC5|JIpeIFs;V*Aii0_9Cxn4f0EM^Ri#W%ZafH>MW!DC+^YCD{ z@C0&K6R%UwxsQdFp9|i4#OM zxX_o~J6XUE6DQ`NpuezuOuxMKoZlPwf!r}~C1IMH2eL*x>MX{8jS(JY-9&u6UHkU6 z0kNN<4cl-BN3Y45E@g|)V-dy0Orauhec@#xK!BCrPP+_f^&PJ*1}V(+gEEDHjIN*6 z1&H+zCnwKG`2Ns`8Sns6A&gay$amj%u-3s{!We(~>%OK&!nTu+&H@x15Fcd`Ejy0h zzoD|S;4|a$32!0;)B`!(ulCil(Yp>FB+4zpfucefdO;Zy6G08(L=zaQ{Lk8--ECMC zPAQhnw>G_0ICK|(al&Dp6c@bfb$wqWr9OMwh&MTMPTbx_cF5ZpI|(I9 zzUk8&nLR>`Z#lIEVk2_PQgNz{5O~pyY!pRc(`YPFA((}A=Ko2V`w8L*ds18k>kyGf zQDWq!&+yMJ%FJn$&cXEEAXtcm+-Y^y6dCp1v)do2 zr#c7Xh0p()f1gP!FOV^TNQBm4Ma{NS{V!Ag7yFyVMYPGtk;+4CW9JULj{8N&b?skG ze&6o3$f(=&9{$O9>Hp;d#Mw2E&+o~FL0Jda@c40if|8%~F74XPtpQC>rb}YtOPqRi z?j}g9e*bqr>ZyCs)D`+{=LDgI_gkS#uU78|U?!x!0xkZAzzQq&D?=Fkpm?89n};$H z_V#|u17UeI=%_2a!46FnE>K-vfcIXlVFpuvq?ul@?nU}9_~X#tjd=s#p9Zhu1kT*c zMq~;^I&>LlC@th_9Cdqd{cFJjMNwc==A{Ooy|AwZb2EN!?plyj+7yfm+PUTg{{+@1 z{kR)>9?s4U-;~N7KwWvq^b_b^IQj;wx6S7?fWj-_CkuOSv>(3;JaCAc41Xd=di^FNzUAW@Jwj5**3)Jg66!0FQooy-;{t+$7*UQ|7?E}pg7xgxD zuW`M@9}VZ8^F;D_|23lR*RNg){w^dz6hPHwn zNw?&ofbm~k|KN`a?Hhm7*p+f6N;6MYfkRLxA@yOLX_$01tSdCBEEkNl#?FItDlZp` z4AR!?*Xd}L>Z4#YGtyd{+a&iBS%hZTuhk?p5eap$$MwFZ{+@_uI<_~fBT&M^q48UP z+o*UPNl9ZIh}>RAPj)k5Q{f{Em14K&;D^etscM=?pH5ocNsJSpB4v6#?*Sxr-mHnq z92)%b%$kWV|NkATNy)3x0(oW!?*o%4UHfC!vj0+r4{XfE0tJ(uUcGyx+{{WsbH>xD z?#H^$I5hb`JoW3-->;wa-;=$#L3i)mS$;`jMZ(2UoBL0ATW?;!zSwaL2cUfaMqBNF z6?^NcSxSl+F{$p=KVw~TYh|=*k@Ri!R>vza4jDe_v4T>4{G$^mxo3qtZuk^(l!RI1 zGOrk-Hcqk?z9MTUZ{F#6 zHgkc}g^QbKsEm}C`+9b_dYZHT?$q{&UiKWkD+XICdw{IkOnDp^4%D7rQFA@OLs~*v zVatoYZ7PN)*BKLk-a_ym?sC-Xhu$wUQez_swnaA<#>K|A{9W>EaOL$c9#e6TIc-$q zb@aohb{!qPz()-Ejrxp!sT6H6#4(TQ12)r2K0|LyOi|PD_4FNws!Rh;C3s=a@0cCp zajKlue?p-7Q`9n**K@Y@8#_dM$&)!xGheb#FZ$L0?Q;6}f^HQqwNExol9q`BCfEBEayljCm9R%fh?+ebs+ zjGM34BxqV(S(_;*3&9f$xohvG`h!Wn)-+rbC=2kzCDRjaLRa@6ezme_=t^$cbicdS zUPAWy)GUvz6hNo2&aYZDHL1=_SVBr?Dhv%UoBt<75Wq#ibt}2wL)U%(VWM z1WWwoVBemd%VybMP#j~OD2-5DeY+#BzTVthl%J;$X_Xk69}Q?{I!ymHk8JZl5iIEF z=l{{vzZ$DOJs^|s-%d&-(D*3elwzL?>0ZE-{lUQ$A3;|V@)|; zoU1>ZA|W?(@X*G;SXF<_=#HMp3=fKWIcPx$VBMh?8Og67 z5D>tT{^!?kDacW-@A2pN_HWNSbH0M8oBQ3orwt4#60MJk~*S zP{X-#@edY=Y*VK~)86tQqino$4fe$A={Ck3HCNYsm%n(CSyx@sgjj8Ch8w-&R zoZIi3iF5;7|BOoT@1EbU{<>`w>7|xZzGXP3AD>sc8%rDQ^%K3*rDB_q*nPWw1OF2iOM=n&1FpNj%dK>W#MLAjxwVpkD zwtQW$?=LaVS;72mc-6ZAEM8APq$>Lp%EM!?@TU0w6IH+{AlEgs@?C4trdhLk{+g3j z-Qkp}FjiM}r}L=ikqnQ(k=)&ljg4b2@7LKviveM6Rz_t}T=!Uik<-xG2CAvXQR?@- z$Lx7+c+yBs{+XhmL~$5a(0GtR*%4YIfkN$YG6HjVNJQYX)m2rnJHA41`K;~)*2E^j zp{vu*K()BhHROKp_mbrse7D}94XyfP*)ZG9EtugdHw_afMvJ1@gam0&8B{zb{sa0uNvsc1+JPd3}A`Lqgp$baj+6#q7NR-e%35$+`OEpy7qbzO=3b zmlk7u1(akM({0%qSt)nSHHe}5A4~~3j8InWf{WU5GGS| zzPVk0+Pz`D%Z49Fu(FoYJF5)2Ll{?$I#B$m#J=AQwf_Cx=Fd;C^J>S20VEZJ9aWzR zx5Zo~m3{Li)g82-Omug52LP*UOO?YwLKq~gvY&?j`3)XTeC2W)SA5$`yIoF4TMlZU z0_q(*^a0{8p3#)!7;L0EUzo5*+g6Sowa z-{gV5v%_vrOnEC>M(R4#0>1&{ZZF7uXG`Qt>)RQ)Ih(&Q7jF#N&MSYm; z-7m7P^~2+yDF>8dc1%{)o$fVPxqttyc3y`HjpQ$%7evoCQ#XQHFm&K&AL*%*|Gt>3 zD?Kk$+Uj$F?Ur804$PA7b1?9?^i3^}V5U%hN_`?qrhDyKZ|@6AeARZ)}`pQO9X z3D}S(z~XqsFb=+Ofqq3$Mb)M4qM*36h?r@x^+k_>dDV1=nVGpFPo6k&sH$x5t-Y*Q z)WUH2HYoh>{zh|$u2!7(Th;XH6=|q$jyfM5S-*?F989!fBd97YzT+7qZ3-!kK^c9A zkP{{*Vsy!R8#|W!gi!>Q0Q@JVE-!p}=-&iMkCpoLs+)hzO+=y-cbR^(*>AnTE0r0O zm2JRKIxi;HZXH#gM#OfiP2Oo5!9oF7gmwBxMpatE`KE+%pQ-)rQjs~Vxt!cv=FY1% zV>>4U^}~7K#OZ3gR8X{F>7qsH*xmI@@KaIh8f@EgCKe-646f7_rqAiYXt5}bODV>S zAv$Yo<|);)KO#qx^Rk@#q^0;#Q-GQN?!IdtKF~*F_B&;k_oxGGJ(_&;k}+i06=Ab2 zDPsQ2xATP`tshWnyGG;S5f)95UY$)bt_?g(TXGG545ETkQ;U}7K&oU3-?(*)e%A^< zHeWhAVu~N1jB74mnnO0`zrEqWewK_F4k}T5U-%Nu8SqnjZTE=AyO_5e&c?t7V|SS( z3%GZ)Qk|oW%ZOFaszzPq6uc0`1e<)OWop%#kgl?`mHozy9gAhhAMiZH+h#f={d}2$ z>@8gi;fMtFK7FJ%_u7)b^Qy`piUT%P_Ycls?YX)JVH_hW#0dLa(58(W$Ay@Y(9`}p z&TB4+w}5lZNt~DYsO#4oE2H(=_|=Px8j=;{cYXM=^`?=$i*&krFmW=ClEcD3UfE7^ zA#%f$sv9AoFwG@%=#NFYAEsc)=m7z_1!v5g7dNqC%WsXkm}{CJ>hU73W#}88YFHmW zp3bl~1I)wn-}Lr(SBQy;Sn&ai5UL&y+l7l3dDX4wpxQafv(NSn<<=Q{IZwQTTZ67m zIUZT-tX}9uo>T9?CEhAUo)<~}+O~CT$%`;X==*OeeOFQLUtw~;L%drO61X7@BCo!O zFJ)knx^&C%hO2{X@6z-kh8ORwn!qz7A@aA+i4qii-{s8Qb6){3bM~9t(Z*R=z)&0kk)GU8p z6I~?dS$oeAg{Q7a5eTULSng~5V&@JE;8g!^>tl7gmhaa16`?q>8#qX~=!ne0@;5y# zuBye@Jd;pAS)@_;U1Gf~Gmh_Zd-m;P1|RYuDy5Dh&RrZ@F@H zovLALWO0-cCyG(SRom3qZQG(gKTDm`@?p4H-}VbvXu&Z<=(Fu7r~^DQ(gnH&o7UV> z?l2EIq*lqWv`93`*^C0zFlnmltGtk+VE2V5)@Buk_6f_4es`zQ3IExN4cL_UtM!>~ zfHrmYE7_}F1bY2g(9r2Q#&bMW0kj*mX~23GtDoC34l(WCg2&#%cnvNU8>i88pl zhFWH6PTT-HIDxVY-af?N+274DNy_}}wL6Qc(qRdu_R#r3KBJHu2E+O+ynGHZ{tH>e z&yS+!78rrNdyd<(P;W_ELRQrs%X9ZoHdP=T3xW7h~FSfazdK8iAndK2&n)t>1( ziGt~|wJpouKB`9*#2U=GQb9v^-XrVE?=@X=S^a&!__} z%a?m(KxzgbP8<_cdbn&%Kj9jC^#$QIU!|pXc@+l?HOX)E5&b#v-7zZukDG(3IZ=iw8-riv1u%&QNBX1nD+dj8i9pQ+;{B$UUAPc~h%BWi$5 z`S-*h-+nBKFyAOHBjfcn?B&=4|BkL6ec;Ex%PdECb7_TS9lJ#;XBL%ug?IQz_4fV= z4o4H3e(bTD~_eN0F3uLB~&e9jgw=6y#IupnZI;f?4lvve3PxBkI z2U#oEtOzuUiTn&t>Pm6;9ysl$rg>-2-czvknm=EoGrpxwI*r)WT0_S#oIrA3vMyQk@wG(O%A zMaI)|x{~LD#=KujQoH3?-W(11Y0SQLynEvKW-Hpd{bMgXGPm$7@%%olhh9WP1jn7J zSnS8lSq7@aRA|oYw}%aTYyLh&y~$^*`$i&UJvx|Q-EV#L$w_+R4j-LLX|Xrm3ZDQ8 zqeVh}Jf2;W^^x5qcgQJAuxeI&+d>iAeMf1?m(DsQNlzYj-(e$i7*e2z_KX$V&R_|~>Y<5l<2E=dk z{mH|JuL34mcR(sW9JXvP;c3h z2WsaFPsYV!{BucLok;D^pFcNzFqRT?a$Mv7Yl7IxhFjJCkNrKmoGt589MZ=*`4Kzd zJj=Aj^Ze1QtgJ&13!DuCt~iocyLOe3e`TGVpwah3*)6ry)YR=;50|}y$CJ`(K6c?G zo8vP-k9D_um9JyyPS)D3<=ru1Jbb;zM56I?U_;%+IJ@UdlT2aMWem-(^Oz-_l-LZ7 zVtqHq)bxm}iOrFR9?LzBdiXocF1nbXz*OW`^BDL*HNSiBh!|vQsus4j)3fwlTL?Om zyS-ms6%)RnY(quI)Cm#&9X~`?t}6A{rRi#yigTAgOC#d7{eP$}?D^4a)?4qRC(d=X293OPzvz9Oxv_Ng=2zk>6IpT^ z8NEccFOvNvj3vY(Lnlp&g=%p{b;(5c#(a}Y3oK7mc>4!l9Pn|2boc;l(9S*l)>BbY zG4fK!m4U!WU3N!OuWl&SR}a$fn|DY}Hu|2pyXL^XQ<9UFuclTvC11CE-d1+&tY@A~ zUmvcP3_Kpzo12^01z(R(oIT{mGwtcK4b-3an=@qZ`}@arEmNK~$8U&DFFMcVd`ooE;8q zmw#mUVcY=^5{1LPyOK#|_A{zh2e&0}zkH(a!afZSJ#UK&8~tNGe94}spDxTe)CS4V zNZrwHv5GSVNZ|Y@&+xR9!ET3vU390;U@j;lEBgd`OI`S*FiOd#E&v(zC#GOB9l3dP zcG@_(bKUy+jMXk({qv(iwbVa%f8VOCTzk81Dy-e7PoK7X`JNlBXYfWySZ|^AR|GB81F&W`z++<2blAwt7PJTk z0oACqe$~+C=H~XW7_NQ68|U}r;>!$yb!UD;=e3E!3e$G)hkyTt{026>W}MwYL>X{Z zq_lUxKKbzdnKQ#uohJ#?teH!fz7{Pz+C?132lMqf)y^_m&GSZ;BlaDD8P?fmEzN=} z07F^EhE&*&PEPtqhK9qvh_kzBxnu(`JaeK|jq7QrWw+n=SC3em6%d5M9STIbK}*B6 zC#TqxAfZOcyFp{wbCxU#zN8qkrZ{Q(H3iF0VId zKn@_jBPHe8B~y(EdaLi(TYEm28o+351n@G)AT=%N9eB5VOjN5}_Mv7keu<~|+1o|4 z&+Hdh!3+U=t4S!sZU!Wm0T`_uNd#&|7lIbDqZ|hyAmydr?e>d-9B z<{v)py%QWZFM*)1(&v<-ruoV0rXsHw$P+?WtY|`+7~Erd?iCDmFEcwj`aNOWHvHA! zgL{fX;YbUEMPT?pk8Rz~@rzfQ?Trb{Y&&-P^i*P6M~~OEb5xG3S(thOrsa+uI|_v_ zYbz_O1*5S#j=p}}P$dq6=eaC)ylc+Vgn|O>>Rs5N&z=WJaKF1-iNEV6G3a!1axx=* zfH!dA=3N7)vcjq#J$k*z@W%G}^MZq~KI}LfD;rIs_@pjOgz~RlL%oG{-LuZNON@93 zg_U?~3)0-J7-PVJ!-j1mokwpMuSIF{{Pc?bDQ7BcYVc`gg&+}s&)#fm8oK+AN6@?D z=+);X{H=IOQG_A0;-fhjn&das)YJ%a9;UzL=3nLY`??KTp4ADOiJoyxveUE6YX{wH z9HzU|8-|K>B=YEQ%*-vipE{Ct;{5q9jPE&*JYNMoI5v5|rH~&AzGJ5ghabw^6W5Sm zD)Ww~r+3SF0_-i3l<4N>#)9uw2Sp!xgmfe<>oBt8D0H^IgzT&!FK;(PHA8S@`&PHM z9GTVzN)H|sj@!tiYvCV7j1=g5eZXckHJ#3xs?pQg2gsC`XI-3qx_vrzK8H^7G=0i7 zWC!@Ue_ar4d;^ZofZncCwGO6joqfKRNwADlKI_~6bJ)1xwlkCTzO=LmNdh)%s`kOn z>xcaq5LQ!gaM}ss#6xGz+cbkYzzFvDuU|i(?R;|jA&VUnoSt5(ibTh8&b_Kfk65Eh zhesPh5W$qR`CqZaucIcEJIq|Lpa3h-srrkJ*DYyh#4>H!GUYvLe4ap*cA1j3RL9r} zvueT@mY0+)V%WWW`EsZh+J%f?2dn{gf!fZPZI}cJQQE!&rNv>(P5taE1k);t zFWAI`)}0nKps1V%spTn>odpx9`cVd)jC`MmLp~r*0!VRV%;|Ia*tMT@JNGN1wrr8= z-re5bo-*-p0ECW$#dlQhClSY54;w%KAoi5H(#0=6J0+X#J#;8YbcgT~ZP8>qo7q7|yYuebMp=ot*U);0Bhz5G;d_Dl4-<|zzCO!>l zhR6r-UpT@gq0Mv*R-yS>S*t=ruM=UJADBH!&vphmwT8on`QlN**>4aJDd|0|EXl6P z*4EaAygb-7BO&|=9%?XnY?Y1ytI3J>Io8(J3dAUye1yun(&s&kA~}-6Ws$dBJYE&o z@K($d^m|h*fG51)6qD~_Lp4|>*Y;J->=8csZ;uoUpN7W`(sh*f=;yGJIxC zX4Um$`qZq~jOr=Ju(jMj9Y7-_#Jnc3;>iCA0xeQ-4vO%=Kr;2OoG}U_x{BpZJ7?yZ`({42*1i8cHkIf=~--VgQJOhXXc$ z>cb^t`muJ9Hcp#%0RcGD=N|ECd_dy(uyFm>(jp2A6}-`Yzw?byrV6+`J7S?ldy!^RF$ zA9t+UL5My+v_YTWm{N}k@x}E^Tqfh%ckkX!VMY96)^qLJwM@xiI4cg@)n)1^rC4c; zM{VT44c!l0g*~O1gj{zt(%iyA(DJI?WQ?uX0 zZv0kV(E0Ge&~yJVTbgfY;GVrO`mcJ$&`j6nzF4lXFOau+r#kOHc+hp$tj)ixat6El zbXgqIRbKz;sYd_)E1G60O+A*8L6WJG;mYR~1Gn}2IBqVNla0|{5#99A(H$Cj{vu8W0 z{qUVo*U}<7=UGxx(oog-hK2@e83`Nasr=%YWN`QLw37Did&CD|m9pO$Ht9&?!Sios zc*Niuz}sv&54`FUZ*@Q*=qn#H^VGJuxW0(}kd7Mj9$5w8T38PpBjvcdN=hglg}hTB zXo9QD$~yQnC0ZW4x2>(sW2(=o&WNQ=yal*yaoe}wD=#Oc@Y)IKGsYa+VT+b&h`o^i znlpb#*QT+&T`HkF<>e{#B1~{`01zT=z{wa!Uw3=iezyT6GZZyHP5-$TqA9%KWu-hj z<@jhEm>~qH#Q1pe-4|h*T5kLzVFMU*f}JN}!-m<&4(0rZxXj(~n_xZt{{;LTfB&;G zrSk$?0D*tT@#EQZ&u|XH^rs@zGd30+`l*M3OZf*V+&^sl`2PL#&w=fODz5fZmF-?o zdCv8;OK=SAz~u=QghAwkIxD|y?Qm4 zDZZOl)c$TBMCTTL>Z;Ck+W@x|`1I1#pcV9F*FJvAuKM>$ZSC+oI>=fq`rpCSrIlTG zk5-LKi;7zqY||5E&yL-�BnBjps^b-3J5X(>T_pBQNCUHcDL${cU#WZYWU3AABXV3UfsO2zY{DkPARfZf;gZJv)ggR}ntdxy^<~h?hl3#; z++dfDMyoS_7O~bA`#C}ubRj;3Li~2zWo(xWo!hyK52UCGTR~Un)#HA*c3CAjOeN0D*H<3ANLqR>4ui{-cgJHui^gExEGkW~^+VXN-vCjqJfi!07U73jrlX<{7OC8HA+8|Veij|eE zV8m1J`!jCCl9Ra+iMP#EW~82@kg!oAOVcPM>+S!k>B{4w&i}t*Fr;!!?lg{NvV|Na zatomdTauzuDMUFcIcA(uL@LKd?ql6WlzbzVtsG$;krGjkBKO_z`Dyp}`C}iZ)-?0^ zypPxWdA(*9VK9Pn9-eHFzfd4!9RcMiYG30q-1?j6nN~-o&qG6Y%K7)p1kzzk!}+au z7oyFu@<+sJROW05$A`nV-DP#4hv?5*$Q{~8*!i6wkT26ng1aU|xZX!*V6V>DuT^k)bbl`wdrii#s zfFFEe>c5pQ?qj|zDk$LbE0?F*^|v2%I8x@!=Yc+bM4U1pXJuvnH4KL%2$_J>8jLUO zbM#0=ZxCVG8XAwQt5-0IX?$z_L+58S_%mv^aZy~JQ(ae2>MMTFxdBsi-HtLv&OE8hp3MrcsdrF3)x5lVwq zt=@)~>RA$j?2-bsl-|8Rc^_7A!3`Ur`Wb5_%1$DQNK31?BVErIY6;Nzdz4CEIsFCc zprcasfoze=remkkt6jx3(Fv+D*-RZ z-u^@EE=U?vS2*D>Xu9QC9?-e6oIochr;i-&r%Q4C*;40!B?o8&B?!FvM0@niri~j{ zh_>kqKVV8_Fc=UmXxH~?=zqeNQ|0#26B#Fg@Yl{jjHE6VXAnm;frlUDz9^CKtEBTM zPIyRIug0NeXeiU?xx&W0YP%!Iw}B?}RL;${XK-e2yttW1hUlh+Os2(679T}7@TJA% z?v)I$L)9dIJJRuz8O8|FIu*Ghn47roAo5nOdpKH+kBxc79+sj6S+}OFVn+A9mlnza z4@jdS?hst&u9<(VYaY0dQ|}HrpOZZv&vp;1!C`y*WfW#}JHtTa-@M76nlLU_Q=%2>E3kUIy9Ahd!?O6WBmQ8WFwURRcM`N4db zbLUc2&scp-2~9BYxDg^9Vy;Fr;lQ&lE(87ji<{Py`vwQueLC$zjWX7H zH1=dke+$fS8=VO@+wtjsmls=(f9be$Pq));E0}0pu=GbDyjF#TI z6EMiaevApx^qPeaF;=Z@ZIG}VW50z87;@G*xrZ3FX_n5#Pe@>YaG+iE*p^)N?0D&- zpHSf_npB<)Xynhl2U-#K$;Gkc5L7}FwhXStI#yU}_l|JdD(&Ej6U?PnDg z6;FS8wjY3$M|slk22!zvT@_8+Vei3o@PQ9{C897Z>&17)vg+#Iy1EPDHo8UJO=l0)*B6d5%B{G_ zjuqgvQ1JDkp;Mk|ZxtTAjEUjx=Pca04&J1sYY)Fa$H@fddz{)HsPtVk(31k&g&@Mf^7>FQ{|bJc7b6c|3SgSVLQi1%}r=+WVo^w(VcxE2;< zRh^`+?_NGWu(;H}SU((w=8GTC6|@bDm$uYzhG$)HW+rzxhr5_!(CCZ8+k)kN1)xjFgE?c6@pVHJGbl-(Nu z2co?p-SF;X|JP|?3K-YsX+Dqx8SItRKG434SE{n9;W7gbwF6nldP1H_R=s|Q#D##- zqTZIX<(71D6Xa(+vc!>LX!{JJcau*V^a@n8JoPcivRYeoC6|*OG6L?cjJAyd#H;Q}< zfSEM+L8<2R8}i~w;QHv)Zgxa;tGe85Jat%&HDr5eM_ztD)Q}_+DcTHc=%YrXa6LU3 zix!5KJ3012ATyps2 z5Nl$I>FCJr97Q7}py?xsj;|;$2Ledl z%||93Ua+LH&k6W=d-EOgM}rGp%FxsTL^nnmBxXeY0ODK_01X|^7C}T&{gvJ*E2MMt^FbA` zox5#%GN7A+H=0@qu5h@6@RWp;Rm5sI8rx@@b)Pu(7w!d8?d1ie5elbiAt1a7SBWBB z%m(9HNe*5L{0pgUpQjeUn_PELxPKk@xoZu)?AB&%C+N=mZ9SjaAr%~P0=-7uG*%`b z^2IP#4!#W%XrXp1x!V8FZzsIqh{wR8Ee~ySSaAYc zuEkNnb_>pSvp;?Slmmg@*(szB)n{9H>Cr?0??{SBN>2WkI$!&_7#Jv?8Ki9Jk%0@3 z-Sfn{siG*z`V!55#b$$^$PFYkiOSGBg(b{*rFti6P9PqcW|rD=B3)+Q3}Rj!oFH(A zeOw6ZSjok2!^4}Y+)T#Ro%k#Z2eJ1HUQ5oe-h@*Pye3SsEf0|aTlVlF1hdmT4Pe7x zOfL8#VwO^3;faXSuk0LUyH2s}c{BX@G-#N(S7_Mu6y6&YT}S8shaCkU?90^UT`3q( z5Zc9>(5XR9q$PP?ioIJQe0Gq(it>c%Y4v)mp~q(>=nO!Q*plGHfSu)RIkHgJHJ~uS z6fqyFW2xii9pfcqGWUHevE7p2wjLR6%850X+*L(!%E7fw%-F)hLdj+U2M-+M78WjF zxv~VTY1ni$ok#S@QAgQ22<$$C=4Z6^sI~RcIqwW z3`;B9EHbFj$SG7GEt+%}Ii_|>1zzGYe^1i7cC?uD=GyEtY>UM1NegFM&<^1(!$U*) z%>R}}4v{)J7HufwyPwP_uu9Zt1luj?j4K&^zJiimS8O!I(D_@;n)tOu(Tiz>r!%l+kl@NMyqMin-TJ zuqAPOfThI3KDCd`Cg2amtab~bq}?Ah2xer4taNwN4p#DQ?PWqx~VtRuP& zK_B931qC3pChApx4HTt%QJ76MfrP;R-+i~O)d-^Lva(d`Z0BJyc5%tK%5x>ocH}Cd z-KTwIS9E^#WLH5!L;y@`_`ZQxP_^sx%~pBMxG6~HZOcr*(eek^-o0ys zXpqxd*BC77f|P^jI*5kkd_UzIJ9Sk~C<*Jz|Lviso9%5rCfZq;$ zx@Zq&2I9kD)<_-IuGOivUpFdy80SbVeA!_`0_r@=5~MixD&Xcdl|g5^vC_-Nc*niO zIuKmO3gH%t@X%*zjuvh2-f@KmLM0vl25t8cu8J>NGp!5R+~5ToHx3EDD@b?=5tHM9s-H}`#g@uK*IFNzJn?m!%a=X;Tl7CFANaT%59;}QInOGB$ z@GC1T2pIwKgrAc&-B{|rgm4Kg3hcpJ0DFXNVDQoQ_rUrgR^T0P&{?{J3^VLPj>0wK zh_Y7IDcE@Y9_pf^>}TRn7*Dqf`z9v~iJ^HTF30OgCLx`p)iJqBa&qL*_Qylrv0!-n zAY1-6gqr`-V;6Atd8kXp{@vWHsHF53P0hdFVVJ%e4*XWTE}^<=pBc)2i`CTZ>>(8G zy0$R(JXgJSXQ-n0RxAL_Ho1~z2!l)iIMlE;*2LrUZ%Dn(hvcHxlH2P^*GNDyOF8oq zTS3|5t67vxG2Ik!c3Xe0v3>yKq zlf<1d?OO7wz9|)xT0?`oV#*DLo5j;k2l1Q-oQ3CjQd#8fM+Voi3V}&R%Ad_&u3Wmr zDBoRTxt&TS^X4Oi1-+zP^L3k9AneuSg$m`FQMardil8aOiJpYNWRlp zSf%eKUn8-f<;E68W<9_4CE^Ew`}TpVVC`&>qFjJD5gYUBvgxi3Sae255xe;Mcqu~r z(6h@m;NvtJlHdh`C++M+EA=GwVvtC~BeosALs0Xr!pT?DW+j9D7pyDny|S{%J>e|q z>S%9=U;>HR45cp9qO40bcC%pTw*PVo#m2<26fGL^8p_`$S~j91W^?m6(ASsE4LIq5 zSVM>)hK7Lw+k~ov_WuB#A-EF-QQRwY6BDR{IF7+Pfn+Ld^w?HGGOjl2ofKElS%4!=4wI5-1c3J4n5YomZf zgUkwx4>gU_P2u{X6tK2}Bw#X`GbNaw;S& z4AO*23?R^n!rS^m9gPw<^^wmvSVPdMh&F+&T;x7$YG^PV&yW1Gxi4_2K?L0zKYR!P zqoNRh_`rd%bZ2AZdU#0Qe)w>1^G(>m6W{CrYC5kk8E^_9@%m--rK--&rSpQqLYqCd zp9lzk3e2vA`U?0)qY9GWlg<&YBJ`;;253x)iHlPK5=`BVE^uN@U963apAM^&LY`?8 z0O4vrwhqvy5UxO8MQ$HCT`V%42su@A5sXZ)?)YoLY^*#JX%=emf5b~?(0zaO??wYw ztGt42irfFsKrk;K-n~0yYkLW`1NH;tCjnQ$9K;5Pj#Uu+X#X_bbsKsJJQ#>{c!hWG z53aTRRB9j8F9q$2UvFB=+?=_BA^xYierpkpUiZQ799$vmC2vR4yJwgUaBm8l*}5Qw`=x zs(G22&v$u4nhDs$fAWghtF7$hU^wwBX*GV2*?cP(a;eM zKL}nBj(^nNgpV8?Tq%_+?iB>F&-_r6F@p`I62p=6kiaIV(ltN{N~Lp0c5IcuYvOw# zV8=ZaVUT>Wb8;%el1@a;poJm~>bmb5&6um>8_wGzdS9GHzNE8Xn$qf0h=JU?t3vA@ zbXG$tG$JABY@|m9OiNhs6^tQ%hFA&iBP?yW2qpki1aCL{^I%XOtihnzqW9&5;~xzR zgG3&yxjBctq9P|w4N~wKQK>|y8PF0hU%WsX2arXmc8@%|@WM(4dOWDTkv3^K337vt zllO0I?DKZ}_mjlu6NmTM+S;P%^7K@hUj%Q2oa6UaGKhNsf(W>|S2_4R7Jn%9u+$;9 zVH9w%J8HV)9eBQIq}S8kty@AwvvfAt`rf7BIkw*n3l9fAe7qfA2SmH*8yFPkg#Wu~ zo!^GsDLACS$jpxn3;_2*z|hjqQLw)O{m;4bCHKmYC4_H@NNJwT@78B`zD!X=xuheDZ6wCVg zb?ZQ{vJNun$HrXynwpwY5Ok1brxq9i>4KvYO_0LVCgi+e8!^-ex99|o;1)qMIu#)=qOSHVK`#)_W)vlg=S; zS<9q_cqjZ<>-e0;?g6b0(l$MPj9fDy#zrbFY-@d^huy2a@t0=t3OjckTp@M;R}?Y3 zpoIE@xDp+d3^*Gh%OwChC#+#jO*T&%RX6_Kw{=}*)z#Deep Storage: push(segment) -Realtime->MySQL: write(metadata) -Master->+MySQL: segments? -MySQL-->-Master: metadata -note over Master: rules select segment owner -Master->Zookeeper: writes(ephemeral node) -Zookeeper->Compute: reads(ephemeral node) -Compute->+Deep Storage: pull(segment) -Deep Storage-->-Compute: transfer(segment) -Compute->Zookeeper: delete(ephemeral node) diff --git a/doc/publications/vldb/.gitignore b/publications/vldb/.gitignore similarity index 100% rename from doc/publications/vldb/.gitignore rename to publications/vldb/.gitignore diff --git a/doc/publications/vldb/Makefile b/publications/vldb/Makefile similarity index 100% rename from doc/publications/vldb/Makefile rename to publications/vldb/Makefile diff --git a/doc/publications/vldb/druid.bib b/publications/vldb/druid.bib similarity index 100% rename from doc/publications/vldb/druid.bib rename to publications/vldb/druid.bib diff --git a/doc/publications/vldb/druid.tex b/publications/vldb/druid.tex similarity index 100% rename from doc/publications/vldb/druid.tex rename to publications/vldb/druid.tex diff --git a/doc/publications/vldb/figures/Druid_cluster.png b/publications/vldb/figures/Druid_cluster.png similarity index 100% rename from doc/publications/vldb/figures/Druid_cluster.png rename to publications/vldb/figures/Druid_cluster.png diff --git a/doc/publications/vldb/figures/cluster_scan_rate.pdf b/publications/vldb/figures/cluster_scan_rate.pdf similarity index 100% rename from doc/publications/vldb/figures/cluster_scan_rate.pdf rename to publications/vldb/figures/cluster_scan_rate.pdf diff --git a/doc/publications/vldb/figures/concise_plot.pdf b/publications/vldb/figures/concise_plot.pdf similarity index 100% rename from doc/publications/vldb/figures/concise_plot.pdf rename to publications/vldb/figures/concise_plot.pdf diff --git a/doc/publications/vldb/figures/core_scan_rate.pdf b/publications/vldb/figures/core_scan_rate.pdf similarity index 100% rename from doc/publications/vldb/figures/core_scan_rate.pdf rename to publications/vldb/figures/core_scan_rate.pdf diff --git a/doc/publications/vldb/figures/data_reading.png b/publications/vldb/figures/data_reading.png similarity index 100% rename from doc/publications/vldb/figures/data_reading.png rename to publications/vldb/figures/data_reading.png diff --git a/doc/publications/vldb/figures/druid_message_bus.png b/publications/vldb/figures/druid_message_bus.png similarity index 100% rename from doc/publications/vldb/figures/druid_message_bus.png rename to publications/vldb/figures/druid_message_bus.png diff --git a/doc/publications/vldb/figures/druid_realtime_flow.png b/publications/vldb/figures/druid_realtime_flow.png similarity index 100% rename from doc/publications/vldb/figures/druid_realtime_flow.png rename to publications/vldb/figures/druid_realtime_flow.png diff --git a/doc/publications/vldb/figures/druid_segment.png b/publications/vldb/figures/druid_segment.png similarity index 100% rename from doc/publications/vldb/figures/druid_segment.png rename to publications/vldb/figures/druid_segment.png diff --git a/doc/publications/vldb/figures/zookeeper.png b/publications/vldb/figures/zookeeper.png similarity index 100% rename from doc/publications/vldb/figures/zookeeper.png rename to publications/vldb/figures/zookeeper.png diff --git a/doc/publications/vldb/src/druid_plot.R b/publications/vldb/src/druid_plot.R similarity index 100% rename from doc/publications/vldb/src/druid_plot.R rename to publications/vldb/src/druid_plot.R diff --git a/doc/publications/vldb/src/druid_tables.R b/publications/vldb/src/druid_tables.R similarity index 100% rename from doc/publications/vldb/src/druid_tables.R rename to publications/vldb/src/druid_tables.R diff --git a/doc/publications/vldb/vldb.cls b/publications/vldb/vldb.cls similarity index 100% rename from doc/publications/vldb/vldb.cls rename to publications/vldb/vldb.cls From 8b480e55db5ed532f9c85fc26e9dcd24528ab123 Mon Sep 17 00:00:00 2001 From: cheddar Date: Fri, 13 Sep 2013 17:20:39 -0500 Subject: [PATCH 75/92] Add docs from github wiki --- .gitignore | 1 + docs/Aggregations.md | 87 ++++++ docs/Batch-ingestion.md | 138 +++++++++ docs/Booting-a-production-cluster.md | 27 ++ docs/Broker.md | 27 ++ docs/Build-from-source.md | 21 ++ docs/Cluster-setup.md | 111 +++++++ docs/Compute.md | 37 +++ docs/Concepts-and-Terminology.md | 12 + docs/Configuration.md | 159 ++++++++++ docs/Contribute.md | 5 + docs/Deep-Storage.md | 39 +++ docs/Design.md | 85 ++++++ docs/Download.md | 11 + docs/Druid-Personal-Demo-Cluster.md | 77 +++++ docs/Druid-vs-Cassandra.md | 5 + docs/Druid-vs-Hadoop.md | 3 + docs/Druid-vs-Impala-or-Shark.md | 42 +++ docs/Druid-vs-redshift.md | 37 +++ docs/Druid-vs-vertica.md | 7 + docs/Examples.md | 65 +++++ docs/Filters.md | 88 ++++++ docs/Firehose.md | 49 ++++ docs/Granularities.md | 46 +++ docs/GroupByQuery.md | 131 +++++++++ docs/Having.md | 90 ++++++ docs/Home.md | 50 ++++ docs/Indexing-Service.md | 190 ++++++++++++ docs/Libraries.md | 17 ++ docs/Loading-Your-Data.md | 367 ++++++++++++++++++++++++ docs/Master.md | 103 +++++++ docs/MySQL.md | 44 +++ docs/OrderBy.md | 24 ++ docs/Plumber.md | 21 ++ docs/Post-aggregations.md | 92 ++++++ docs/Querying-your-data.md | 363 +++++++++++++++++++++++ docs/Querying.md | 111 +++++++ docs/Realtime.md | 154 ++++++++++ docs/Rule-Configuration.md | 77 +++++ docs/SearchQuery.md | 68 +++++ docs/SearchQuerySpec.md | 23 ++ docs/SegmentMetadataQuery.md | 55 ++++ docs/Segments.md | 60 ++++ docs/Spatial-Filters.md | 32 +++ docs/Spatial-Indexing.md | 23 ++ docs/Stand-Alone-With-Riak-CS.md | 223 ++++++++++++++ docs/Support.md | 13 + docs/Tasks.md | 68 +++++ docs/Thanks.md | 8 + docs/TimeBoundaryQuery.md | 26 ++ docs/TimeseriesQuery.md | 115 ++++++++ docs/Tutorial:-A-First-Look-at-Druid.md | 366 +++++++++++++++++++++++ docs/Tutorial:-The-Druid-Cluster.md | 299 +++++++++++++++++++ docs/Tutorial:-Webstream.md | 354 +++++++++++++++++++++++ docs/Twitter-Tutorial.md | 326 +++++++++++++++++++++ docs/Versioning.md | 21 ++ docs/ZooKeeper.md | 63 ++++ docs/contents.md | 68 +++++ 58 files changed, 5224 insertions(+) create mode 100644 docs/Aggregations.md create mode 100644 docs/Batch-ingestion.md create mode 100644 docs/Booting-a-production-cluster.md create mode 100644 docs/Broker.md create mode 100644 docs/Build-from-source.md create mode 100644 docs/Cluster-setup.md create mode 100644 docs/Compute.md create mode 100644 docs/Concepts-and-Terminology.md create mode 100644 docs/Configuration.md create mode 100644 docs/Contribute.md create mode 100644 docs/Deep-Storage.md create mode 100644 docs/Design.md create mode 100644 docs/Download.md create mode 100644 docs/Druid-Personal-Demo-Cluster.md create mode 100644 docs/Druid-vs-Cassandra.md create mode 100644 docs/Druid-vs-Hadoop.md create mode 100644 docs/Druid-vs-Impala-or-Shark.md create mode 100644 docs/Druid-vs-redshift.md create mode 100644 docs/Druid-vs-vertica.md create mode 100644 docs/Examples.md create mode 100644 docs/Filters.md create mode 100644 docs/Firehose.md create mode 100644 docs/Granularities.md create mode 100644 docs/GroupByQuery.md create mode 100644 docs/Having.md create mode 100644 docs/Home.md create mode 100644 docs/Indexing-Service.md create mode 100644 docs/Libraries.md create mode 100644 docs/Loading-Your-Data.md create mode 100644 docs/Master.md create mode 100644 docs/MySQL.md create mode 100644 docs/OrderBy.md create mode 100644 docs/Plumber.md create mode 100644 docs/Post-aggregations.md create mode 100644 docs/Querying-your-data.md create mode 100644 docs/Querying.md create mode 100644 docs/Realtime.md create mode 100644 docs/Rule-Configuration.md create mode 100644 docs/SearchQuery.md create mode 100644 docs/SearchQuerySpec.md create mode 100644 docs/SegmentMetadataQuery.md create mode 100644 docs/Segments.md create mode 100644 docs/Spatial-Filters.md create mode 100644 docs/Spatial-Indexing.md create mode 100644 docs/Stand-Alone-With-Riak-CS.md create mode 100644 docs/Support.md create mode 100644 docs/Tasks.md create mode 100644 docs/Thanks.md create mode 100644 docs/TimeBoundaryQuery.md create mode 100644 docs/TimeseriesQuery.md create mode 100644 docs/Tutorial:-A-First-Look-at-Druid.md create mode 100644 docs/Tutorial:-The-Druid-Cluster.md create mode 100644 docs/Tutorial:-Webstream.md create mode 100644 docs/Twitter-Tutorial.md create mode 100644 docs/Versioning.md create mode 100644 docs/ZooKeeper.md create mode 100644 docs/contents.md diff --git a/.gitignore b/.gitignore index dc61b625f65..c7f1290850b 100644 --- a/.gitignore +++ b/.gitignore @@ -12,3 +12,4 @@ target .settings/ *.log *.DS_Store +_site diff --git a/docs/Aggregations.md b/docs/Aggregations.md new file mode 100644 index 00000000000..ffdbd18a4f2 --- /dev/null +++ b/docs/Aggregations.md @@ -0,0 +1,87 @@ +Aggregations are specifications of processing over metrics available in Druid. +Available aggregations are: + +### Sum aggregators + +#### `longSum` aggregator + +computes the sum of values as a 64-bit, signed integer + + { + "type" : "longSum", + "name" : , + "fieldName" : + } + +`name` โ€“ย output name for the summed value +`fieldName` โ€“ name of the metric column to sum over + +#### `doubleSum` aggregator + +Computes the sum of values as 64-bit floating point value. Similar to `longSum` + + { + "type" : "doubleSum", + "name" : , + "fieldName" : + } + +### Count aggregator + +`count` computes the row count that match the filters + + { + "type" : "count", + "name" : , + } + +### Min / Max aggregators + +#### `min` aggregator + +`min` computes the minimum metric value + + { + "type" : "min", + "name" : , + "fieldName" : + } + +#### `max` aggregator + +`max` computes the maximum metric value + + { + "type" : "max", + "name" : , + "fieldName" : + } + +### JavaScript aggregator + +Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions). + +All JavaScript functions must return numerical values. + + { + "type": "javascript", + "name": "", + "fieldNames" : [ , , ... ], + "fnAggregate" : "function(current, column1, column2, ...) { + + return + }" + "fnCombine" : "function(partialA, partialB) { return ; }" + "fnReset" : "function() { return ; }" + } + +**Example** + + { + "type": "javascript", + "name": "sum(log(x)/y) + 10", + "fieldNames": ["x", "y"], + "fnAggregate" : "function(current, a, b) { return current + (Math.log(a) * b); }" + "fnCombine" : "function(partialA, partialB) { return partialA + partialB; }" + "fnReset" : "function() { return 10; }" + } diff --git a/docs/Batch-ingestion.md b/docs/Batch-ingestion.md new file mode 100644 index 00000000000..97212777bc4 --- /dev/null +++ b/docs/Batch-ingestion.md @@ -0,0 +1,138 @@ +Batch Data Ingestion +==================== + +There are two choices for batch data ingestion to your Druid cluster, you can use the [[Indexing service]] or you can use the `HadoopDruidIndexerMain`. This page describes how to use the `HadoopDruidIndexerMain`. + +Which should I use? +------------------- + +The [[Indexing service]] is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data. + +The `HadoopDruidIndexerMain` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and donโ€™t want to spend the time configuring and deploying the [[Indexing service]] just yet. + +HadoopDruidIndexer +------------------ + +Located at `com.metamx.druid.indexer.HadoopDruidIndexerMain` can be run like + + + java -cp hadoop_config_path:druid_indexer_selfcontained_jar_path com.metamx.druid.indexer.HadoopDruidIndexerMain + + +The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Time_intervals) of the data you are processing. The config\_file is a path to a file (the โ€œspecFileโ€) that contains JSON and an example looks like: + + + { + "dataSource": "the_data_source", + "timestampColumn": "ts", + "timestampFormat": "", + "dataSpec": { + "format": "", + "columns": ["ts", "column_1", "column_2", "column_3", "column_4", "column_5"], + "dimensions": ["column_1", "column_2", "column_3"] + }, + "granularitySpec": { + "type":"uniform", + "intervals":[""], + "gran":"day" + }, + "pathSpec": { "type": "granularity", + "dataGranularity": "hour", + "inputPath": "s3n://billy-bucket/the/data/is/here", + "filePattern": ".*" }, + "rollupSpec": { "aggs": [ + { "type": "count", "name":"event_count" }, + { "type": "doubleSum", "fieldName": "column_4", "name": "revenue" }, + { "type": "longSum", "fieldName" : "column_5", "name": "clicks" } + ], + "rollupGranularity": "minute"}, + "workingPath": "/tmp/path/on/hdfs", + "segmentOutputPath": "s3n://billy-bucket/the/segments/go/here", + "leaveIntermediate": "false", + "partitionsSpec": { + "targetPartitionSize": 5000000 + }, + "updaterJobSpec": { + "type":"db", + "connectURI":"jdbc:mysql://localhost:7980/test_db", + "user":"username", + "password":"passmeup", + "segmentTable":"segments" + } + } + + +### Hadoop indexer config + +|property|description|required?| +|--------|-----------|---------| +|dataSource|name of the dataSource the data will belong to|yes| +|timestampColumn|the column that is to be used as the timestamp column|yes| +|timestampFormat|the format of timestamps; auto = either iso or millis, Joda time formats:http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html|yes| +|dataSpec|a specification of the data format and an array that names all of the columns in the input data|yes| +|dimensions|the columns that are to be used as dimensions|yes| +|granularitySpec|the time granularity and interval to chunk segments up into|yes| +|pathSpec|a specification of where to pull the data in from|yes| +|rollupSpec|a specification of the rollup to perform while processing the data|yes| +|workingPath|the working path to use for intermediate results (results between Hadoop jobs)|yes| +|segmentOutputPath|the path to dump segments into|yes| +|leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool)|no| +|partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur|no| +|updaterJobSpec|a specification of how to update the metadata for the druid cluster these segments belong to|yes| +|registererers|a list of serde handler classnames|no| + +### Path specification + +There are multiple types of path specification: + +##### `granularity` + +Is a type of data loader that expects data to be laid out in a specific path format. Specifically, it expects it to be segregated by day in this directory format `y=XXXX/m=XX/d=XX/H=XX/M=XX/S=XX` (dates are represented by lowercase, time is represented by uppercase). + +|property|description|required?| +|--------|-----------|---------| +|dataGranularity|specifies the granularity to expect the data at, e.g. hour means to expect directories `y=XXXX/m=XX/d=XX/H=XX`|yes| +|inputPath|Base path to append the expected time path to|yes| +|filePattern|Pattern that files should match to be included|yes| + +For example, if the sample config were run with the interval 2012-06-01/2012-06-02, it would expect data at the paths + + s3n://billy-bucket/the/data/is/here/y=2012/m=06/d=01/H=00 + s3n://billy-bucket/the/data/is/here/y=2012/m=06/d=01/H=01 + ... + s3n://billy-bucket/the/data/is/here/y=2012/m=06/d=01/H=23 + +### Rollup specification + +The indexing process has the ability to roll data up as it processes the incoming data. If data has already been summarized, summarizing it again will produce the same results so either way is not a problem. This specifies how that rollup should take place. + +|property|description|required?| +|--------|-----------|---------| +|aggs|specifies a list of aggregators to aggregate for each bucket (a bucket is defined by the tuple of the truncated timestamp and the dimensions). Aggregators available here are the same as available when querying.|yes| +|rollupGranularity|The granularity to use when truncating incoming timestamps for bucketization|yes| + +### Partitioning specification + +Segments are always partitioned based on timestamp (according to the granularitySpec) and may be further partitioned in some other way. For example, data for a day may be split by the dimension โ€œlast\_nameโ€ into two segments: one with all values from A-M and one with all values from N-Z. + +To use this option, the indexer must be given a target partition size. It can then find a good set of partition ranges on its own. + +|property|description|required?| +|--------|-----------|---------| +|targetPartitionSize|target number of rows to include in a partition, should be a number that targets segments of 700MB\~1GB.|yes| +|partitionDimension|the dimension to partition on. Leave blank to select a dimension automatically.|no| +|assumeGrouped|assume input data has already been grouped on time and dimensions. This is faster, but can choose suboptimal partitions if the assumption is violated.|no| + +### Updater job spec + +This is a specification of the properties that tell the job how to update metadata such that the Druid cluster will see the output segments and load them. + +|property|description|required?| +|--------|-----------|---------| +|type|โ€œdbโ€ is the only value available|yes| +|connectURI|a valid JDBC url to MySQL|yes| +|user|username for db|yes| +|password|password for db|yes| +|segmentTable|table to use in DB|yes| + +These properties should parrot what you have configured for your [[Master]]. diff --git a/docs/Booting-a-production-cluster.md b/docs/Booting-a-production-cluster.md new file mode 100644 index 00000000000..32181fce2c1 --- /dev/null +++ b/docs/Booting-a-production-cluster.md @@ -0,0 +1,27 @@ +# Booting a Single Node Cluster # + +[[Loading Your Data]] and [[Querying Your Data]] contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz). + +The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: + +```bash +# Setup environment for ec2-api-tools +export EC2_HOME=/path/to/ec2-api-tools-1.6.7.4/ +export PATH=$PATH:$EC2_HOME/bin +export AWS_ACCESS_KEY= +export AWS_SECRET_KEY= +``` + +Then, booting an ec2 instance running one node of each type is as simple as running the script, run_ec2.sh :) + +# Apache Whirr # + +Apache Whirr is a set of libraries for launching cloud services. You can clone a version of Whirr that includes Druid as a service from git@github.com:rjurney/whirr.git: + +```bash +git clone git@github.com:rjurney/whirr.git +cd whirr +git checkout trunk +mvn clean install -Dmaven.test.failure.ignore=true -Dcheckstyle.skip +sp;bin/whirr launch-cluster --config recipes/druid.properties +``` \ No newline at end of file diff --git a/docs/Broker.md b/docs/Broker.md new file mode 100644 index 00000000000..6d8f3db2ca0 --- /dev/null +++ b/docs/Broker.md @@ -0,0 +1,27 @@ +Broker +====== + +The Broker is the node to route queries to if you want to run a distributed cluster. It understands the metadata published to ZooKeeper about what segments exist on what nodes and routes queries such that they hit the right nodes. This node also merges the result sets from all of the individual nodes together. + +Forwarding Queries +------------------ + +Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [[Segments]] are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple nodes, and hence, the query will likely hit multiple nodes. + +To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [[Compute]] and [[Realtime]] nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker node then forwards down the query to the selected nodes. + +Caching +------- + +Broker nodes employ a distributed cache with a LRU cache invalidation strategy. The broker cache stores per segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it ๏ฌrst maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the +compute nodes. Once the compute nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable. + +Running +------- + +Broker nodes can be run using the `com.metamx.druid.http.BrokerMain` class. + +Configuration +------------- + +See [[Configuration]]. diff --git a/docs/Build-from-source.md b/docs/Build-from-source.md new file mode 100644 index 00000000000..aaa5411368d --- /dev/null +++ b/docs/Build-from-source.md @@ -0,0 +1,21 @@ +### Clone and Build from Source + +The other way to setup Druid is from source via git. To do so, run these commands: + +``` +git clone git@github.com:metamx/druid.git +cd druid +./build.sh +``` + +You should see a bunch of files: + +``` +DruidCorporateCLA.pdf README common examples indexer pom.xml server +DruidIndividualCLA.pdf build.sh doc group_by.body install publications services +LICENSE client eclipse_formatting.xml index-common merger realtime +``` + +You can find the example executables in the examples/bin directory: +* run_example_server.sh +* run_example_client.sh diff --git a/docs/Cluster-setup.md b/docs/Cluster-setup.md new file mode 100644 index 00000000000..23cb806d4fc --- /dev/null +++ b/docs/Cluster-setup.md @@ -0,0 +1,111 @@ +A Druid cluster consists of various node types that need to be set up depending on your use case. See our [[Design]] docs for a description of the different node types. + +Setup Scripts +------------- + +One of our community members, [housejester](https://github.com/housejester/), contributed some scripts to help with setting up a cluster. Checkout the [github](https://github.com/housejester/druid-test-harness) and [wiki](https://github.com/housejester/druid-test-harness/wiki/Druid-Test-Harness). + +Minimum Physical Layout: Absolute Minimum +----------------------------------------- + +As a special case, the absolute minimum setup is one of the standalone examples for realtime ingestion and querying; see [[Examples]] that can easily run on one machine with one core and 1GB RAM. This layout can be set up to try some basic queries with Druid. + +Minimum Physical Layout: Experimental Testing with 4GB of RAM +------------------------------------------------------------- + +This layout can be used to load some data from deep storage onto a Druid compute node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is: + +1. node1: [[Master]] + metadata service + zookeeper + [[Compute]] +2. transient nodes: indexer + +This setup is only reasonable to prove that a configuration works. It would not be worthwhile to use this layout for performance measurement. + +Comfortable Physical Layout: Pilot Project with Multiple Machines +----------------------------------------------------------------- + +*The machine size โ€œflavorsโ€ are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work.* + +A minimal physical layout not constrained by cores that demonstrates parallel querying and realtime, using AWS-EC2 โ€œsmallโ€/m1.small (one core, with 1.7GB of RAM) or larger, no realtime, is: + +1. node1: [[Master]] (m1.small) +2. node2: metadata service (m1.small) +3. node3: zookeeper (m1.small) +4. node4: [[Broker]] (m1.small or m1.medium or m1.large) +5. node5: [[Compute]] (m1.small or m1.medium or m1.large) +6. node6: [[Compute]] (m1.small or m1.medium or m1.large) +7. node7: [[Realtime]] (m1.small or m1.medium or m1.large) +8. transient nodes: indexer + +This layout naturally lends itself to adding more RAM and core to Compute nodes, and to adding many more Compute nodes. Depending on the actual load, the Master, metadata server, and Zookeeper might need to use larger machines. + +High Availability Physical Layout +--------------------------------- + +*The machine size โ€œflavorsโ€ are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work.* + +An HA layout allows full rolling restarts and heavy volume: + +1. node1: [[Master]] (m1.small or m1.medium or m1.large) +2. node2: [[Master]] (m1.small or m1.medium or m1.large) (backup) +3. node3: metadata service (c1.medium or m1.large) +4. node4: metadata service (c1.medium or m1.large) (backup) +5. node5: zookeeper (c1.medium) +6. node6: zookeeper (c1.medium) +7. node7: zookeeper (c1.medium) +8. node8: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +9. node9: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) (backup) +10. node10: [[Compute]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +11. node11: [[Compute]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +12. node12: [[Realtime]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +13. transient nodes: indexer + +Sizing for Cores and RAM +------------------------ + +The Compute and Broker nodes will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Compute daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries. + +The effective utilization of cores by Zookeeper, MySQL, and Master nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB. + +Storage +------- + +Indexed segments should be kept in a permanent store accessible by all nodes like AWS S3 or HDFS or equivalent. Currently Druid supports S3, but this will be extended soon. + +Local disk (โ€œephemeralโ€ on AWS EC2) for caching is recommended over network mounted storage (example of mounted: AWS EBS, Elastic Block Store) in order to avoid network delays during times of heavy usage. If your data center is suitably provisioned for networked storage, perhaps with separate LAN/NICs just for storage, then mounted might work fine. + +Setup +----- + +Setting up a cluster is essentially just firing up all of the nodes you want with the proper [[configuration]]. One thing to be aware of is that there are a few properties in the configuration that potentially need to be set individually for each process: + + + druid.server.type=historical|realtime + druid.host=someHostOrIPaddrWithPort + druid.port=8080 + + +`druid.server.type` should be set to โ€œhistoricalโ€ for your compute nodes and realtime for the realtime nodes. The master will only assign segments to a โ€œhistoricalโ€ node and the broker has some intelligence around its ability to cache results when talking to a realtime node. This does not need to be set for the master or the broker. + +`druid.host` should be set to the hostname and port that can be used to talk to the given server process. Basically, someone should be able to send a request to http://\${druid.host}/ and actually talk to the process. + +`druid.port` should be set to the port that the server should listen on. In the vast majority of cases, this port should be the same as what is on `druid.host`. + +Build/Run +--------- + +The simplest way to build and run from the repository is to run `mvn package` from the base directory and then take `druid-services/target/druid-services-*-selfcontained.jar` and push that around to your machines; the jar does not need to be expanded, and since it contains the main() methods for each kind of service, it is **not** invoked with java ~~jar. It can be run from a normal java command-line by just including it on the classpath and then giving it the main class that you want to run. For example one instance of the Compute node/service can be started like this: +\ + +java~~Duser.timezone=UTC ~~Dfile.encoding=UTF-8~~cp compute/:druid-services/target/druid-services~~\*~~selfcontained.jar com.metamx.druid.http.ComputeMain + + + +The following table shows the possible services and fully qualified class for main(). + +|service|main class| +|-------|----------| +|[[ Realtime ]]|com.metamx.druid.realtime.RealtimeMain| +|[[ Master ]]|com.metamx.druid.http.MasterMain| +|[[ Broker ]]|com.metamx.druid.http.BrokerMain| +|[[ Compute ]]|com.metamx.druid.http.ComputeMain| + diff --git a/docs/Compute.md b/docs/Compute.md new file mode 100644 index 00000000000..755f2475707 --- /dev/null +++ b/docs/Compute.md @@ -0,0 +1,37 @@ +Compute +======= + +Compute nodes are the work horses of a cluster. They load up historical segments and expose them for querying. + +Loading and Serving Segments +---------------------------- + +Each compute node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Compute nodes do not communicate directly with each other or with the master nodes but instead rely on Zookeeper for coordination. + +The [[Master]] node is responsible for assigning new segments to compute nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a compute node. For more information on how the master assigns segments to compute nodes, please see [[Master]]. + +When a compute node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the compute node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [[Segments]]. Once a compute node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying. + +Loading and Serving Segments From Cache +--------------------------------------- + +Recall that when a compute node notices a new segment entry in its load queue path, the compute node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the compute node will directly read the segment binary files from disk and load the segment. + +The segment cache is also leveraged when a compute node is first started. On startup, a compute node will search through its cache directory and immediately load and serve all segments that are found. This feature allows compute nodes to be queried as soon they come online. + +Querying Segments +----------------- + +Please see [[Querying]] for more information on querying compute nodes. + +For every query that a compute node services, it will log the query and report metrics on the time taken to run the query. + +Running +------- + +Compute nodes can be run using the `com.metamx.druid.http.ComputeMain` class. + +Configuration +------------- + +See [[Configuration]]. diff --git a/docs/Concepts-and-Terminology.md b/docs/Concepts-and-Terminology.md new file mode 100644 index 00000000000..a9accabf88a --- /dev/null +++ b/docs/Concepts-and-Terminology.md @@ -0,0 +1,12 @@ +Concepts and Terminology +======================== + +- **Aggregators:** A mechanism for combining records during realtime incremental indexing, Hadoop batch indexing, and in queries. +- **DataSource:** A table-like view of data; specified in a โ€œspecFileโ€ and in a query. +- **Granularity:** The time interval corresponding to aggregation by time. + - The *indexGranularity* setting in a schema is used to aggregate input (ingest) records within an interval into a single output (internal) record. + - The *segmentGranularity* is the interval specifying how internal records are stored together in a single file. + +- **Segment:** A collection of (internal) records that are stored and processed together. +- **Shard:** A unit of partitioning data across machine. TODO: clarify; by time or other dimensions? +- **specFile** is specification for services in JSON format; see [[Realtime]] and [[Batch-ingestion]] diff --git a/docs/Configuration.md b/docs/Configuration.md new file mode 100644 index 00000000000..c3150d44805 --- /dev/null +++ b/docs/Configuration.md @@ -0,0 +1,159 @@ +This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json โ€œspecFileโ€ descriptions in [[Realtime]] and [[Batch-ingestion]]. + +JVM Configuration Best Practices +================================ + +There are three JVM parameters that we set on all of our processes: + +1. `-Duser.timezone=UTC` This sets the default timezone of the JVM to UTC. We always set this and do not test with other default timezones, so local timezones might work, but they also might uncover weird and interesting bugs +2. `-Dfile.encoding=UTF-8` This is similar to timezone, we test assuming UTF-8. Local encodings might work, but they also might result in weird and interesting bugs +3. `-Djava.io.tmpdir=` Various parts of the system that interact with the file system do it via temporary files, these files can get somewhat large. Many production systems are setup to have small (but fast) `/tmp` directories, these can be problematic with Druid so we recommend pointing the JVMโ€™s tmp directory to something with a little more meat. + +Basic Service Configuration +=========================== + +Configuration of the various nodes is done via Java properties. These can either be provided as `-D` system properties on the java command line or they can be passed in via a file called `runtime.properties` that exists on the classpath. Note: as a future item, Iโ€™d like to consolidate all of the various configuration into a yaml/JSON based configuration files. + +The periodic time intervals (like โ€œPT1Mโ€) are [ISO8601 intervals](http://en.wikipedia.org/wiki/ISO_8601#Time_intervals) + +An example runtime.properties is as follows: + + + # S3 access + com.metamx.aws.accessKey= + com.metamx.aws.secretKey= + + # thread pool size for servicing queries + druid.client.http.connections=30 + + # JDBC connection string for metadata database + druid.database.connectURI= + druid.database.user=user + druid.database.password=password + # time between polling for metadata database + druid.database.poll.duration=PT1M + druid.database.segmentTable=prod_segments + + # Path on local FS for storage of segments; dir will be created if needed + druid.paths.indexCache=/tmp/druid/indexCache + # Path on local FS for storage of segment metadata; dir will be created if needed + druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache + + druid.request.logging.dir=/tmp/druid/log + + druid.server.maxSize=300000000000 + + # ZK quorum IPs + druid.zk.service.host= + # ZK path prefix for Druid-usage of zookeeper, Druid will create multiple paths underneath this znode + druid.zk.paths.base=/druid + # ZK path for discovery, the only path not to default to anything + druid.zk.paths.discoveryPath=/druid/discoveryPath + + # the host:port as advertised to clients + druid.host=someHostOrIPaddrWithPort + # the port on which to listen, this port should line up with the druid.host value + druid.port=8080 + + com.metamx.emitter.logging=true + com.metamx.emitter.logging.level=debug + + druid.processing.formatString=processing_%s + druid.processing.numThreads=3 + + + druid.computation.buffer.size=100000000 + + # S3 dest for realtime indexer + druid.pusher.s3.bucket= + druid.pusher.s3.baseKey= + + druid.bard.cache.sizeInBytes=40000000 + druid.master.merger.service=blah_blah + + +Configuration groupings +----------------------- + +### S3 Access + +These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [[Compute]], [[Realtime]] and [[Indexing service]] nodes. + +|Property|Description|Default| +|--------|-----------|-------| +|`com.metamx.aws.accessKey`|The access key to use to access S3.|none| +|`com.metamx.aws.secretKey`|The secret key to use to access S3.|none| +|`druid.pusher.s3.bucket`|The bucket to store segments, this is used by Realtime and the Indexing service.|none| +|`druid.pusher.s3.baseKey`|The base key to use when storing segments, this is used by Realtime and the Indexing service|none| + +### JDBC connection + +These properties specify the jdbc connection and other configuration around the โ€œsegments tableโ€ database. The only processes that connect to the DB with these properties are the [[Master]] and [[Indexing service]]. This is tested on MySQL. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.database.connectURI`|The jdbc connection uri|none| +|`druid.database.user`|The username to connect with|none| +|`druid.database.password`|The password to connect with|none| +|`druid.database.poll.duration`|The duration between polls the Master does for updates to the set of active segments. Generally defines the amount of lag time it can take for the master to notice new segments|PT1M| +|`druid.database.segmentTable`|The table to use to look for segments.|none| +|`druid.database.ruleTable`|The table to use to look for segment load/drop rules.|none| +|`druid.database.configTable`|The table to use to look for configs.|none| + +### Master properties + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.master.period`|The run period for the master. The masterโ€™s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs|PT60S| +|`druid.master.removedSegmentLifetime`|When a node disappears, the master can provide a grace period for how long it waits before deciding that the node really isnโ€™t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the master.|1| +|`druid.master.startDelay`|The operation of the Master works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesnโ€™t allow the Master to know for a fact that itโ€™s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data|PT600S| + +### Zk properties + +See [[ZooKeeper]] for a description of these properties. + +### Service properties + +These are properties that define various service/HTTP server aspects + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.client.http.connections`|Size of connection pool for the Broker to connect to compute nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|none| +|`druid.paths.indexCache`|Segments assigned to a compute node are first stored on the local file system and then served by the compute node. This path defines where that local cache resides. Directory will be created if needed|none| +|`druid.paths.segmentInfoCache`|Compute nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the master to reassign. This path defines where this metadata is kept. Directory will be created if needed|none| +|`druid.http.numThreads`|The number of HTTP worker threads.|10| +|`druid.http.maxIdleTimeMillis`|The amount of time a connection can remain idle before it is terminated|300000 (5 min)| +|`druid.request.logging.dir`|Compute, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs donโ€™t generally capture information about the actual query), this specifies the directory to store the request logs in|none| +|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none| +|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none| +|`druid.processing.formatString`|Realtime and Compute nodes use this format string to name their processing threads.|none| +|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|none| +|`druid.computation.buffer.size`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Compute and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)| +|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none| +|`druid.bard.cache.sizeInBytes`|The Broker (called Bard internally) instance has the ability to store results of queries in an in-memory cache. This specifies the number of bytes to use for that cache|none| + +### Compute Properties + +These are properties that the compute nodes use + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.server.maxSize`|The maximum number of bytes worth of segment that the node wants assigned to it. This is not a limit that the compute nodes actually enforce, they just publish it to the master and trust the master to do the right thing|none| +|`druid.server.type`|Specifies the type of the node. This is published via ZK and depending on the value the node will be treated specially by the Master/Broker. Allowed values are โ€œrealtimeโ€ or โ€œhistoricalโ€. This is a configuration parameter because the plan is to allow for a more configurable cluster composition. At the current time, all realtime nodes should just be โ€œrealtimeโ€ and all compute nodes should just be โ€œcomputeโ€|none| + +### Emitter Properties + +The Druid servers emit various metrics and alerts via something we call an [[Emitter]]. There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. More information can be found on the [[Emitter]] page. The properties for using the logging emitter are described below. + +|Property|Description|Default| +|--------|-----------|-------| +|`com.metamx.emitter.logging`|Set to โ€œtrueโ€ to use the logging emitter|none| +|`com.metamx.emitter.logging.level`|Sets the level to log at|debug| +|`com.metamx.emitter.logging.class`|Sets the class to log at|com.metamx.emiter.core.LoggingEmitter| + +### Realtime Properties + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.realtime.specFile`|The file with realtime specifications in it. See [[Realtime]].|none| + diff --git a/docs/Contribute.md b/docs/Contribute.md new file mode 100644 index 00000000000..8a5bcc75f99 --- /dev/null +++ b/docs/Contribute.md @@ -0,0 +1,5 @@ +If you are interested in contributing to the code, we accept [pull requests](https://help.github.com/articles/using-pull-requests). Note: we have only just completed decoupling our Metamarkets-specific code from the code base and we took some short-cuts in interface design to make it happen. So, there are a number of interfaces that exist right now which are likely to be in flux. If you are embedding Druid in your system, it will be safest for the time being to only extend/implement interfaces that this wiki describes, as those are intended as stable (unless otherwise mentioned). + +For issue tracking, we are using the github issue tracker. Please fill out an issue from the Issues tab on the github screen. + +We also have a [[Libraries]] page that lists external libraries that people have created for working with Druid. diff --git a/docs/Deep-Storage.md b/docs/Deep-Storage.md new file mode 100644 index 00000000000..f30aa50333e --- /dev/null +++ b/docs/Deep-Storage.md @@ -0,0 +1,39 @@ +Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented. + +The currently supported types of deep storage follow. + +## S3-compatible + +S3-compatible deep storage is basically either S3 or something like riak-cs which exposes the same API as S3. This is the default deep storage implementation. + +S3 configuration parameters are + + com.metamx.aws.accessKey= + com.metamx.aws.secretKey= + druid.pusher.s3.bucket= + druid.pusher.s3.baseKey= + +## HDFS + +As of 0.4.0, HDFS can be used for storage of segments as well. + +In order to use hdfs for deep storage, you need to set the following configuration on your realtime nodes. + + druid.pusher.hdfs=true + druid.pusher.hdfs.storageDirectory= + +If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work + + +## Local Mount + +A local mount can be used for storage of segments as well. This allows you to use just your local file system or anything else that can be mount locally like NFS, Ceph, etc. + +In order to use a local mount for deep storage, you need to set the following configuration on your realtime nodes. + + druid.pusher.local=true + druid.pusher.local.storageDirectory= + +Note that you should generally set `druid.pusher.local.storageDirectory` to something different from `druid.paths.indexCache`. + +If you are using the Hadoop indexer in local mode, then just give it a local file as your output directory and it will work. \ No newline at end of file diff --git a/docs/Design.md b/docs/Design.md new file mode 100644 index 00000000000..2d67f1e3139 --- /dev/null +++ b/docs/Design.md @@ -0,0 +1,85 @@ +For a comprehensive look at the architecture of Druid, read the [White Paper](http://static.druid.io/docs/druid.pdf). + +What is Druid? +============== + +Druid is a system built to allow fast (โ€œreal-timeโ€) access to large sets of seldom-changing data. It was designed with the intent of being a service and maintaining 100% uptime in the face of code deployments, machine failures and other eventualities of a production system. It can be useful for back-office use cases as well, but design decisions were made explicitly targetting an always-up service. + +Druid currently allows for single-table queries in a similar manner to [Dremel](http://research.google.com/pubs/pub36632.html) and [PowerDrill](http://www.vldb.org/pvldb/vol5/p1436_alexanderhall_vldb2012.pdf). It adds to the mix + +1. columnar storage format for partially nested data structures +2. hierarchical query distribution with intermediate pruning +3. indexing for quick filtering +4. realtime ingestion (ingested data is immediately available for querying) +5. fault-tolerant distributed architecture that doesnโ€™t lose data + +As far as a comparison of systems is concerned, Druid sits in between PowerDrill and Dremel on the spectrum of functionality. It implements almost everything Dremel offers (Dremel handles arbitrary nested data structures while Druid only allows for a single level of array-based nesting) and gets into some of the interesting data layout and compression methods from PowerDrill. + +Druid is a good fit for products that require real-time data ingestion of a single, large data stream. Especially if you are targetting no-downtime operation and are building your product on top of a time-oriented summarization of the incoming data stream. Druid is probably not the right solution if you care more about query flexibility and raw data access than query speed and no-downtime operation. When talking about query speed it is important to clarify what โ€œfastโ€ means, with Druid it is entirely within the realm of possibility (we have done it) to achieve queries that run in single-digit seconds across a 6TB data set. + +### Architecture + +Druid is architected as a grouping of systems each with a distinct role and together they form a working system. The name comes from the Druid class in many role-playing games: it is a shape-shifter, capable of taking many different forms to fulfill various different roles in a group. + +The node types that currently exist are: +\* **Compute** nodes are the workhorses that handle storage and querying on โ€œhistoricalโ€ data (non-realtime) +\* **Realtime** nodes ingest data in real-time, they are in charge of listening to a stream of incoming data and making it available immediately inside the Druid system. As data they have ingested ages, they hand it off to the compute nodes. +\* **Master** nodes act as coordinators. They look over the grouping of computes and make sure that data is available, replicated and in a generally โ€œoptimalโ€ configuration. +\* **Broker** nodes understand the topology of data across all of the other nodes in the cluster and re-write and route queries accordingly +\* **Indexer** nodes form a cluster of workers to load batch and real-time data into the system as well as allow for alterations to the data stored in the system (also known as the Indexing Service) + +This separation allows each node to only care about what it is best at. By separating Compute and Realtime, we separate the memory concerns of listening on a real-time stream of data and processing it for entry into the system. By separating the Master and Broker, we separate the needs for querying from the needs for maintaining โ€œgoodโ€ data distribution across the cluster. + +All nodes can be run in some highly available fashion. Either as symmetric peers in a share-nothing cluster or as hot-swap failover nodes. + +Aside from these nodes, there are 3 external dependencies to the system: + +1. A running [ZooKeeper](http://zookeeper.apache.org/) cluster for cluster service discovery and maintenance of current data topology +2. A MySQL instance for maintenance of metadata about the data segments that should be served by the system +3. A โ€œdeep storageโ€ LOB store/file system to hold the stored segments + +### Data Storage + +Getting data into the Druid system requires an indexing process. This gives the system a chance to analyze the data, add indexing structures, compress and adjust the layout in an attempt to optimize query speed. A quick list of what happens to the data follows. + +- Converted to columnar format +- Indexed with bitmap indexes +- Compressed using various algorithms + - LZF (switching to Snappy is on the roadmap, not yet implemented) + - Dictionary encoding w/ id storage minimization + - Bitmap compression + - RLE (on the roadmap, but not yet implemented) + +The output of the indexing process is stored in a โ€œdeep storageโ€ LOB store/file system ([[Deep Storage]] for information about potential options). Data is then loaded by compute nodes by first downloading the data to their local disk and then memory mapping it before serving queries. + +If a compute node dies, it will no longer serve its segments, but given that the segments are still available on the โ€œdeep storageโ€ any other node can simply download the segment and start serving it. This means that it is possible to actually remove all compute nodes from the cluster and then re-provision them without any data loss. It also means that if the โ€œdeep storageโ€ is not available, the nodes can continue to serve the segments they have already pulled down (i.e. the cluster goes stale, not down). + +In order for a segment to exist inside of the cluster, an entry has to be added to a table in a MySQL instance. This entry is a self-describing bit of metadata about the segment, it includes things like the schema of the segment, the size, and the location on deep storage. These entries are what the Master uses to know what data **should** be available on the cluster. + +### Fault Tolerance + +- **Compute** As discussed above, if a compute node dies, another compute node can take its place and there is no fear of data loss +- **Master** Can be run in a hot fail-over configuration. If no masters are running, then changes to the data topology will stop happening (no new data and no data balancing decisions), but the system will continue to run. +- **Broker** Can be run in parallel or in hot fail-over. +- **Realtime** Depending on the semantics of the delivery stream, multiple of these can be run in parallel processing the exact same stream. They periodically checkpoint to disk and eventually push out to the Computes. Steps are taken to be able to recover from process death, but loss of access to the local disk can result in data loss if this is the only method of adding data to the system. +- **โ€œdeep storageโ€ file system** If this is not available, new data will not be able to enter the cluster, but the cluster will continue operating as is. +- **MySQL** If this is not available, the master will be unable to find out about new segments in the system, but it will continue with its current view of the segments that should exist in the cluster. +- **ZooKeeper** If this is not available, data topology changes will not be able to be made, but the Brokers will maintain their most recent view of the data topology and continue serving requests accordingly. + +### Query processing + +A query first enters the Broker, where the broker will match the query with the data segments that are known to exist. It will then pick a set of machines that are serving those segments and rewrite the query for each server to specify the segment(s) targetted. The Compute/Realtime nodes will take in the query, process them and return results. The Broker then takes the results and merges them together to get the final answer, which it returns. In this way, the broker can prune all of the data that doesnโ€™t match a query before ever even looking at a single row of data. + +For filters at a more granular level than what the Broker can prune based on, the indexing structures inside each segment allows the compute nodes to figure out which (if any) rows match the filter set before looking at any row of data. It can do all of the boolean algebra of the filter on the bitmap indices and never actually look directly at a row of data. + +Once it knows the rows that match the current query, it can access the columns it cares about for those rows directly without having to load data that it is just going to throw away. + +The following diagram shows the data flow for queries without showing batch indexing: + +![Simple Data Flow](https://raw.github.com/metamx/druid/master/doc/data_flow_simple.png "Simple Data Flow") + +### In-memory? + +Druid is not always and only in-memory. When we first built it, it is true that it was all in-memory all the time, but as time went on the price-performance tradeoff ended up swinging towards keeping all of our customers data in memory all the time a non-starter. We then added the ability to memory map data and allow the OS to handle paging data in and out of memory on demand. Our production cluster is primarily configured to operate with this memory mapping behavior and we are definitely over-subscribed in terms of memory available vs. data a node is serving. + +As you read some of the old blog posts or other literature about the project, you will see โ€œin-memoryโ€ often touted as that is the history of where Druid came from, but the technical reality is that there is a spectrum of price vs. performance and being able to slide along it from all in-memory (high cost, great performance) to mostly on disk (low cost, low performance) is the important knob to be able to adjust. diff --git a/docs/Download.md b/docs/Download.md new file mode 100644 index 00000000000..1bdbe799c50 --- /dev/null +++ b/docs/Download.md @@ -0,0 +1,11 @@ +A version may be declared as a release candidate if it has been deployed to a sizable production cluster. Release candidates are declared as stable after we feel fairly confident there are no major bugs in the version. Check out the [[Versioning]] section for how we describe software versions. + +Release Candidate +----------------- + +There is no release candidate at this time. + +Stable Release +-------------- + +The current stable is tagged at version [0.5.49](https://github.com/metamx/druid/tree/druid-0.5.49). diff --git a/docs/Druid-Personal-Demo-Cluster.md b/docs/Druid-Personal-Demo-Cluster.md new file mode 100644 index 00000000000..81a088226f5 --- /dev/null +++ b/docs/Druid-Personal-Demo-Cluster.md @@ -0,0 +1,77 @@ +# Druid Personal Demo Cluster (DPDC) + +Note, there are currently some issues with the CloudFormation. We are working through them and will update the documentation here when things work properly. In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version. If you just want to get a feel for the types of data and queries that you can issue, check out [[Realtime Examples]] + +## Introduction +To make it easy for you to get started with Druid, we created an AWS (Amazon Web Services) [CloudFormation](http://aws.amazon.com/cloudformation/) Template that allows you to create a small pre-configured Druid cluster using your own AWS account. The cluster contains a pre-loaded sample workload, the Wikipedia edit stream, and a basic query interface that gets you familiar with Druid capabilities like drill-downs and filters. + + +This guide walks you through the steps to create the cluster and then how to create basic queries. (The cluster setup should take you about 15-20 minutes depending on AWS response times). + + +## Whatโ€™s in this Druid Demo Cluster? + +1. A single "Master" node. This node co-locates the [[Master]] process, the [[Broker]] process, Zookeeper, and the MySQL instance. You can read more about Druid architecture [[Design]]. + +1. Three compute nodes; these compute nodes, have been pre-configured to work with the Master node and should automatically load up the Wikipedia edit stream data (no specific setup is required). + +## Setup Instructions +1. Log in to your AWS account: Start by logging into the [Console page](https://console.aws.amazon.com) of your AWS account; if you donโ€™t have one, follow this link to sign up for one [http://aws.amazon.com/](http://aws.amazon.com/). + +![AWS Console Page](images/demo/setup-01-console.png) + +1. If you have a [Key Pair](http://docs.aws.amazon.com/gettingstarted/latest/wah/getting-started-create-key-pair.html) already created you may skip this step. Note: this is required to create the demo cluster and is generally not used unless instances need to be accessed directly (e.g. via SSH). + + 1. Click **EC2** to go to the EC2 Dashboard. From there, click **Key Pairs** under Network & Security. + ![EC2 Dashboard](images/demo/setup-02a-keypair.png) + + 1. Click on the button **Create Key Pair**. A dialog box will appear prompting you to enter a Key Pair name (as long as you remember it, the name is arbitrary, for this example we entered `Druid`). Click **Create**. You will be prompted to download a .pam; store this file in a safe place. + ![Create Key Pair](images/demo/setup-02b-keypair.png) + +1. Unless youโ€™re there already, go back to the Console page, or follow this link: https://console.aws.amazon.com. Click **CloudFormation** under Deployment & Management. +![CloudFormation](images/demo/setup-03-ec2.png) + +1. Click **Create New Stack**, which will bring up the **Create Stack** dialog. +![Create New Stack](images/demo/setup-04-newstack.png) + +1. Enter a **Stack Name** (itโ€™s arbitrary, we chose, `DruidStack`). Click **Provide a Template URL** type in the following template URL: _**https://s3.amazonaws.com/cf-templates-jm2ikmzj3y6x-us-east-1/2013081cA9-Druid04012013.template**_. Press **Continue**, this will take you to the Create Stack dialog. +![Stack Name & URL](images/demo/setup-05-createstack.png) + +1. Enter `Druid` (or the Key Pair name you created in Step 2) in the **KeyPairName** field; click **Continue**. This should bring up another dialog prompting you to enter a **Key** and **Value**. +![Stack Parameters](images/demo/setup-06-parameters.png) + +1. While the inputs are arbitrary, itโ€™s important to remember this information; we chose to enter `version` for **Key** and `1` for **Value**. Press **Continue** to bring up a confirmation dialog. +![Add Tags](images/demo/setup-07a-tags.png) + +1. Click **Continue** to start creating your Druid Demo environment (this will bring up another dialog box indicating your environment is being created; click **Close** to take you to a more detailed view of the Stack creation process). Note: depending on AWS, this step could take over 15 minutes โ€“ initialization continues even after the instances are created. (So yes, now would be a good time to grab that cup of coffee). +![Review](images/demo/setup-07b-review.png) +![Create Stack Complete](images/demo/setup-07c-complete.png) + +1. Click and expand the **Events** tab in the CloudFormation Stacks window to get a more detailed view of the Druid Demo Cluster setup. + +![CloudFormations](images/demo/setup-09-events.png) + +1. Get the IP address of your Druid Master Node: + 1. Go to the following URL: [https://console.aws.amazon.com/ec2](https://console.aws.amazon.com/ec2) + 1. Click **Instances** in the left pane โ€“ you should see something similar to the following figure. + 1. Select the **DruidMaster** instance + 1. Your IP address is right under the heading: **EC2 Instance: DruidMaster**. Select and copy that entire line, which ends with `amazonaws.com`. + +![EC2 Instances](images/demo/setup-10-ip.png) + +## Querying Data + +1. Use the following URL to bring up the Druid Demo Cluster query interface (replace **IPAddressDruidMaster** with the actual druid master IP Address): + +**`http://IPAddressDruidMaster:8082/druid/v3/demoServlet`** + +As you can see from the image below, there are default values in the Dimensions and Granularity fields. Clicking **Execute** will produce a basic query result. +![Demo Query Interface](images/demo/query-1.png) + +1. Note: when the Query is in running the **Execute** button will be disabled and read: **Fetchingโ€ฆ** +![Demo Query](images/demo/query-2.png) + +1. You can add multiple Aggregation values, adjust Granularity, and Dimensions; query results will appear at the bottom of the window. + + +Enjoy! And for sure, please send along your comments and feedback or, aspirations on expanding and developing this demo. https://groups.google.com/d/forum/druid-development. Attention R users: we just open-sourced our R Druid connector: https://github.com/metamx/RDruid. diff --git a/docs/Druid-vs-Cassandra.md b/docs/Druid-vs-Cassandra.md new file mode 100644 index 00000000000..4cac3922324 --- /dev/null +++ b/docs/Druid-vs-Cassandra.md @@ -0,0 +1,5 @@ +We are not experts on Cassandra, if anything is incorrect about our portrayal, please let us know on the mailing list or via some other means. We will fix this page. + +Druid is highly optimized for scans and aggregations, it supports arbitrarily deep drill downs into data sets without the need to pre-compute, and it can ingest event streams in real-time and allow users to query events as they come in. Cassandra is a great key-value store and it has some features that allow you to use it to do more interesting things than what you can do with a pure key-value store. But, it is not built for the same use cases that Druid handles, namely regularly scanning over billions of entries per query. + +Furthermore, Druid is fully read-consistent. Druid breaks down a data set into immutable chunks known as segments. All replicants always present the exact same view for the piece of data they are holding and we donโ€™t have to worry about data synchronization. The tradeoff is that Druid has limited semantics for write and update operations. Cassandra, similar to Amazonโ€™s Dynamo, has an eventually consistent data model. Writes are always supported but updates to data may take some time before all replicas sync up (data reconciliation is done at read time). This model favors availability and scalability over consistency. \ No newline at end of file diff --git a/docs/Druid-vs-Hadoop.md b/docs/Druid-vs-Hadoop.md new file mode 100644 index 00000000000..68744179b1e --- /dev/null +++ b/docs/Druid-vs-Hadoop.md @@ -0,0 +1,3 @@ +Druid is a complementary addition to Hadoop. Hadoop is great at storing and making accessible large amounts of individually low-value data. Unfortunately, Hadoop is not great at providing query speed guarantees on top of that data, nor does it have very good operational characteristics for a customer-facing production system. Druid, on the other hand, excels at taking high-value summaries of the low-value data on Hadoop, making it available in a fast and always-on fashion, such that it could be exposed directly to a customer. + +Druid also requires some infrastructure to exist for โ€œdeep storageโ€. HDFS is one of the implemented options for this โ€œdeep storageโ€. diff --git a/docs/Druid-vs-Impala-or-Shark.md b/docs/Druid-vs-Impala-or-Shark.md new file mode 100644 index 00000000000..e9a0c673b87 --- /dev/null +++ b/docs/Druid-vs-Impala-or-Shark.md @@ -0,0 +1,42 @@ +The question of Druid versus Impala or Shark basically comes down to your product requirements and what the systems were designed to do. + +Druid was designed to + +1. be an always on service +1. ingest data in real-time +1. handle slice-n-dice style ad-hoc queries + +Impala and Shark's primary design concerns (as far as I am aware) were to replace Hadoop MapReduce with another, faster, query layer that is completely generic and plays well with the other ecosystem of Hadoop technologies. I will caveat this discussion with the statement that I am not an expert on Impala or Shark, nor am I intimately familiar with their roadmaps. If anything is incorrect on this page, I'd be happy to change it, please send a note to the mailing list. + +What does this mean? We can talk about it in terms of four general areas + +1. Fault Tolerance +1. Query Speed +1. Data Ingestion +1. Query Flexibility + +## Fault Tolerance + +Druid pulls segments down from [[Deep Storage]] before serving queries on top of it. This means that for the data to exist in the Druid cluster, it must exist as a local copy on a historical node. If deep storage becomes unavailable for any reason, new segments will not be loaded into the system, but the cluster will continue to operate exactly as it was when the backing store disappeared. + +Impala and Shark, on the other hand, pull their data in from HDFS (or some other Hadoop FileSystem) in response to a query. This has implications for the operation of queries if you need to take HDFS down for a bit (say a software upgrade). It's possible that data that has been cached in the nodes is still available when the backing file system goes down, but I'm not sure. + +This is just one example, but Druid was built to continue operating in the face of failures of any one of its various pieces. The [[Design]] describes these design decisions from the Druid side in more detail. + +## Query Speed + +Druid takes control of the data given to it, storing it in a column-oriented fashion, compressing it and adding indexing structures. All of which add to the speed at which queries can be processed. The column orientation means that we only look at the data that a query asks for in order to compute the answer. Compression increases the data storage capacity of RAM and allows us to fit more data into quickly accessible RAM. Indexing structures mean that as you add boolean filters to your queries, we do less processing and you get your result faster, whereas a lot of processing engines do *more* processing when filters are added. + +Impala/Shark can basically be thought of as daemon caching layers on top of HDFS. They are processes that stay on even if there is no query running (eliminating the JVM startup costs from Hadoop MapReduce) and they have facilities to cache data locally so that it can be accessed and updated quicker. But, I do not believe they go beyond caching capabilities to actually speed up queries. So, at the end of the day, they don't change the paradigm from a brute-force, scan-everything query processing paradigm. + +## Data Ingestion + +Druid is built to allow for real-time ingestion of data. You can ingest data and query it immediately upon ingestion, the latency between how quickly the event is reflected in the data is dominated by how long it takes to deliver the event to Druid. + +Impala/Shark, being based on data in HDFS or some other backing store, are limited in their data ingestion rates by the rate at which that backing store can make data available. Generally, the backing store is the biggest bottleneck for how quickly data can become available. + +## Query Flexibility + +Druid supports timeseries and groupBy style queries. It doesn't have support for joins, which makes it a lot less flexible for generic processing. + +Impala/Shark support SQL style queries with full joins. \ No newline at end of file diff --git a/docs/Druid-vs-redshift.md b/docs/Druid-vs-redshift.md new file mode 100644 index 00000000000..2b360a4668b --- /dev/null +++ b/docs/Druid-vs-redshift.md @@ -0,0 +1,37 @@ +###How does Druid compare to Redshift? + +In terms of drawing a differentiation, Redshift is essentially ParAccel (Actian) which Amazon is licensing. + +Aside from potential performance differences, there are some functional differences: + +###Real-time data ingestion + +Because Druid is optimized to provide insight against massive quantities of streaming data; it is able to load and aggregate data in real-time. + +Generally traditional data warehouses including column stores work only with batch ingestion and are not optimal for streaming data in regularly. + +###Druid is a read oriented analytical data store + +Itโ€™s write semantics arenโ€™t as fluid and does not support joins. ParAccel is a full database with SQL support including joins and insert/update statements. + +###Data distribution model + +Druidโ€™s data distribution, is segment based which exists on highly available โ€œdeepโ€ storage, like S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of compute nodes does not result in data loss because new compute nodes can always be brought up by reading data from โ€œdeepโ€ storage. + +To contrast, ParAccelโ€™s data distribution model is hash-based. Expanding the cluster requires re-hashing the data across the nodes, making it difficult to perform without taking downtime. Amazonโ€™s Redshift works around this issue with a multi-step process: + +* set cluster into read-only mode +* copy data from cluster to new cluster that exists in parallel +* redirect traffic to new cluster + +###Replication strategy + +Druid employs segment-level data distribution meaning that more nodes can be added and rebalanced without having to perform a staged swap. The replication strategy also makes all replicas available for querying. + +ParAccelโ€™s hash-based distribution generally means that replication is conducted via hot spares. This puts a numerical limit on the number of nodes you can lose without losing data, and this replication strategy often does not allow the hot spare to help share query load. + +###Indexing strategy + +Along with column oriented structures, Druid uses indexing structures to speed up query execution when a filter is provided. Indexing structures do increase storage overhead (and make it more difficult to allow for mutation), but they can also significantly speed up queries. + +ParAccel does not appear to employ indexing strategies. \ No newline at end of file diff --git a/docs/Druid-vs-vertica.md b/docs/Druid-vs-vertica.md new file mode 100644 index 00000000000..b35f62e9f03 --- /dev/null +++ b/docs/Druid-vs-vertica.md @@ -0,0 +1,7 @@ +How does Druid compare to Vertica? + +Vertica is similar to ParAccel/Redshift ([[Druid-vs-Redshift]]) described above in that it wasnโ€™t built for real-time streaming data ingestion and it supports full SQL. + +The other big difference is that instead of employing indexing, Vertica tries to optimize processing by leveraging run-length encoding (RLE) and other compression techniques along with a โ€œprojectionโ€ system that creates materialized copies of the data in a different sort order (to maximize the effectiveness of RLE). + +We are unclear about how Vertica handles data distribution and replication, so we cannot speak to if/how Druid is different. diff --git a/docs/Examples.md b/docs/Examples.md new file mode 100644 index 00000000000..88ca41fb4fa --- /dev/null +++ b/docs/Examples.md @@ -0,0 +1,65 @@ +Examples +======== + +The examples on this page are setup in order to give you a feel for what Druid does in practice. They are quick demos of Druid based on [RealtimeStandaloneMain](https://github.com/metamx/druid/blob/master/examples/src/main/java/druid/examples/RealtimeStandaloneMain.java). While you wouldnโ€™t run it this way in production you should be able to see how ingestion works and the kind of exploratory queries that are possible. Everything that can be done on your box here can be scaled out to 10โ€™s of billions of events and terabytes of data per day in a production cluster while still giving the snappy responsive exploratory queries. + +Installing Standalone Druid +--------------------------- + +There are two options for installing standalone Druid. Building from source, and downloading the Druid Standalone Kit (DSK). + +### Building from source + +Clone Druid and build it: + + git clone https://github.com/metamx/druid.git druid + cd druid + git fetch --tags + git checkout druid-0.4.30 + ./build.sh + + +### Downloading the DSK (Druid Standalone Kit) + +[Download](http://static.druid.io/data/examples/druid-services-0.4.6.tar.gz) a stand-alone tarball and run it: + + + tar -xzf druid-services-0.X.X-SNAPSHOT-bin.tar.gz + cd druid-services-0.X.X-SNAPSHOT + + +Twitter Example +--------------- + +For a full tutorial based on the twitter example, check out this [[Twitter Tutorial]]. + +This Example uses a feature of Twitter that allows for sampling of itโ€™s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/metamx/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded. + +### What youโ€™ll learn +\* See how large amounts of data gets ingested into Druid in real-time +\* Learn how to do fast, interactive, analytics queries on that real-time data + +### What you need +\* A build of standalone Druid with the Twitter example (see above) +\* A Twitter username and password. + +### What youโ€™ll do + +See [[Tutorial]] + +Rand Example +------------ + +This uses `RandomFirehoseFactory` which emits a stream of random numbers (outColumn, a positive double) with timestamps along with an associated token (target). This provides a timeseries that requires no network access for demonstration, characterization, and testing. The generated tuples can be thought of as asynchronously produced triples (timestamp, outColumn, target) where the timestamp varies depending on speed of processing. + +In a terminal window, (NOTE: If you are using the cloned Github repository these scripts are in ./examples/bin) start the server with: + +`./run_example_server.sh` +`# type rand when prompted` + +In another terminal window: + +`./run_example_client.sh` +`# type rand when prompted` + +The result of the client query is in JSON format. The client makes a REST request using the program `curl` which is usually installed on Linux, Unix, and OSX by default. diff --git a/docs/Filters.md b/docs/Filters.md new file mode 100644 index 00000000000..f655861d5fb --- /dev/null +++ b/docs/Filters.md @@ -0,0 +1,88 @@ +A filter is a JSON object indicating which rows of data should be included in the computation for a query. Itโ€™s essentially the equivalent of the WHERE clause in SQL. Druid supports the following types of filters. + +### Selector filter + +The simplest filter is a selector filter. The selector filter will match a specific dimension with a specific value. Selector filters can be used as the base filters for more complex Boolean expressions of filters. + +The grammar for a SELECTOR filter is as follows: + + "filter": { + "type": "selector", + "dimension": , + "value": + } + + +This is the equivalent of `WHERE = ''`. + +### Regular expression filter + +The regular expression filter is similar to the selector filter, but using regular expressions. It matches the specified dimension with the given pattern. The pattern can be any standard [Java regular expression](http://docs.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html). + + "filter": { + "type": "regex", + "dimension": , + "pattern": + } + + +### Logical expression filters + +#### AND + +The grammar for an AND filter is as follows: + + "filter": { + "type": "and", + "fields": [, , ...] + } + + +The filters in fields can be any other filter defined on this page. + +#### OR + +The grammar for an OR filter is as follows: + + "filter": { + "type": "or", + "fields": [, , ...] + } + + +The filters in fields can be any other filter defined on this page. + +#### NOT + +The grammar for a NOT filter is as follows: + + "filter": { + "type": "not", + "field": + } + + +The filter specified at field can be any other filter defined on this page. + +### JavaScript filter + +The JavaScript filter matches a dimension against the specified JavaScript function predicate. The filter matches values for which the function returns true. + +The function takes a single argument, the dimension value, and returns either true or false. + + { + "type" : "javascript", + "dimension" : , + "function" : "function(value) { <...> }" + } + + +**Example** +The following matches any dimension values for the dimension `name` between `'bar'` and `'foo'` + + { + "type" : "javascript", + "dimension" : "name", + "function" : "function(x) { return(x >= 'bar' && x <= 'foo') }" + } + diff --git a/docs/Firehose.md b/docs/Firehose.md new file mode 100644 index 00000000000..ab9b2ac53d2 --- /dev/null +++ b/docs/Firehose.md @@ -0,0 +1,49 @@ +Firehoses describe the data stream source. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Specifies the type of firehose. Each value will have its own configuration schema, firehoses packaged with Druid are described [here](https://github.com/metamx/druid/wiki/Firehose#available-firehoses)|yes| + +We describe the configuration of the Kafka firehose from the example below, but check [here](https://github.com/metamx/druid/wiki/Firehose#available-firehoses) for more information about the various firehoses that are available in Druid. + +- `consumerProps` is a map of properties for the Kafka consumer. The JSON object is converted into a Properties object and passed along to the Kafka consumer. +- `feed` is the feed that the Kafka consumer should read from. +- `parser` represents a parser that knows how to convert from String representations into the required `InputRow` representation that Druid uses. This is a potentially reusable piece that can be found in many of the firehoses that are based on text streams. The spec in the example describes a JSON feed (new-line delimited objects), with a timestamp column called โ€œtimestampโ€ in ISO8601 format and that it should not include the dimension โ€œvalueโ€ when processing. More information about the options available for the parser are available [here](https://github.com/metamx/druid/wiki/Firehose#parsing-data). + +Available Firehoses +------------------- + +There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment. + +#### KafkaFirehose + +This firehose acts as a Kafka consumer and ingests data from Kafka. + +#### StaticS3Firehose + +This firehose ingests events from a predefined list of S3 objects. + +#### TwitterSpritzerFirehose + +See [[Examples]]. This firehose connects directly to the twitter spritzer data stream. + +#### RandomFirehose + +See [[Examples]]. This firehose creates a stream of random numbers. + +#### RabbitMqFirehouse + +This firehose ingests events from a define rabbit-mq queue. + +Parsing Data +------------ + +There are several ways to parse data. + +#### StringInputRowParser + +This parser converts Strings. + +#### MapInputRowParser + +This parser converts flat, key/value pair maps. diff --git a/docs/Granularities.md b/docs/Granularities.md new file mode 100644 index 00000000000..ea568dd7d62 --- /dev/null +++ b/docs/Granularities.md @@ -0,0 +1,46 @@ +The granularity field determines how data gets bucketed across the time dimension, i.e how it gets aggregated by hour, day, minute, etc. + +It can be specified either as a string for simple granularities or as an object for arbitrary granularities. + +### Simple Granularities + +Simple granularities are specified as a string and bucket timestamps by their UTC time (i.e. days start at 00:00 UTC). + +Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day` +\* **`all`** buckets everything into a single bucket +\* **`none`** does not bucket data (it actually uses the granularity of the index - minimum here is `none` which means millisecond granularity). Using `none` in a [[timeseries query|TimeSeriesQuery]] is currently not recommended (the system will try to generate 0 values for all milliseconds that didnโ€™t exist, which is often a lot). + +### Duration Granularities + +Duration granularities are specified as an exact duration in milliseconds and timestamps are returned as UTC. + +They also support specifying an optional origin, which defines where to start counting time buckets from (defaults to 1970-01-01T00:00:00Z). + + {"type": "duration", "duration": "7200000"} + +This chunks up every 2 hours. + + {"type": "duration", "duration": "3600000", "origin": "2012-01-01T00:30:00Z"} + +This chunks up every hour on the half-hour. + +### Period Granularities + +Period granularities are specified as arbitrary period combinations of years, months, weeks, hours, minutes and seconds (e.g. P2W, P3M, PT1H30M, PT0.750S) in ISO8601 format. + +They support specifying a time zone which determines where period boundaries start and also determines the timezone of the returned timestamps. + +By default years start on the first of January, months start on the first of the month and weeks start on Mondays unless an origin is specified. + +Time zone is optional (defaults to UTC) +Origin is optional (defaults to 1970-01-01T00:00:00 in the given time zone) + + {"type": "period", "period": "P2D", "timeZone": "America/Los_Angeles"} + +This will bucket by two day chunks in the Pacific timezone. + + {"type": "period", "period": "P3M", "timeZone": "America/Los_Angeles", "origin": "2012-02-01T00:00:00-08:00"} + +This will bucket by 3 month chunks in the Pacific timezone where the three-month quarters are defined as starting from February. + +Supported time zones: timezone support is provided by the [Joda Time library](http://www.joda.org), which uses the standard IANA time zones. [Joda Time supported timezones](http://joda-time.sourceforge.net/timezones.html) diff --git a/docs/GroupByQuery.md b/docs/GroupByQuery.md new file mode 100644 index 00000000000..735dd5c393a --- /dev/null +++ b/docs/GroupByQuery.md @@ -0,0 +1,131 @@ +These types of queries take a groupBy query object and return an array of JSON objects where each object represents a grouping asked for by the query. + +An example groupBy query object is shown below: + +

+
+{
+ [queryType]() โ€œgroupByโ€,
+ [dataSource]() โ€œsample\_datasourceโ€,
+ [granularity]() โ€œdayโ€,
+ [dimensions]() [โ€œdim1โ€, โ€œdim2โ€],
+ [limitSpec]() {
+ [type]() โ€œdefaultโ€,
+ [limit]() 5000,
+ [columns]() [โ€œdim1โ€, โ€œmetric1โ€]
+ },
+ [filter]() {
+ [type]() โ€œandโ€,
+ [fields]() [
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension1โ€,
+ [value]() โ€œsample\_value1โ€
+ },
+ {
+ [type]() โ€œorโ€,
+ [fields]() [
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension2โ€,
+ [value]() โ€œsample\_value2โ€
+ },
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension3โ€,
+ [value]() โ€œsample\_value3โ€
+ }
+ ]
+ }
+ ]
+ },
+ [aggregations]() [
+ {
+ [type]() โ€œlongSumโ€,
+ [name]() โ€œsample\_name1โ€,
+ [fieldName]() โ€œsample\_fieldName1โ€
+ },
+ {
+ [type]() โ€œdoubleSumโ€,
+ [name]() โ€œsample\_name2โ€,
+ [fieldName]() โ€œsample\_fieldName2โ€
+ }
+ ],
+ [postAggregations]() [
+ {
+ [type]() โ€œarithmeticโ€,
+ [name]() โ€œsample\_divideโ€,
+ [fn]() โ€œ/โ€,
+ [fields]() [
+ {
+ [type]() โ€œfieldAccessโ€,
+ [name]() โ€œsample\_name1โ€,
+ [fieldName]() โ€œsample\_fieldName1โ€
+ },
+ {
+ [type]() โ€œfieldAccessโ€,
+ [name]() โ€œsample\_name2โ€,
+ [fieldName]() โ€œsample\_fieldName2โ€
+ }
+ ]
+ }
+ ],
+ [intervals]() [
+ โ€œ2012-01-01T00:00:00.000/2012-01-03T00:00:00.000โ€
+ ],
+ [having]() {
+ [type]() โ€œgreaterThanโ€,
+ [aggregation]() โ€œsample\_name1โ€,
+ [value]() 0
+ }
+}
+
+
+ + +There are 9 main parts to a groupBy query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be โ€œgroupByโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|dimensions|A JSON list of dimensions to do the groupBy over|yes| +|orderBy|See [[OrderBy]].|no| +|having|See [[Having]].|no| +|granularity|Defines the granularity of the query. See [[Granularities]]|yes| +|filter|See [[Filters]]|no| +|aggregations|See [[Aggregations]]|yes| +|postAggregations|See [[Post Aggregations]]|no| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +To pull it all together, the above query would return *n\*m* data points, up to a maximum of 5000 points, where n is the cardinality of the โ€œdim1โ€ dimension, m is the cardinality of the โ€œdim2โ€ dimension, each day between 2012-01-01 and 2012-01-03, from the โ€œsample\_datasourceโ€ table. Each data point contains the (long) sum of sample\_fieldName1 if the value of the data point is greater than 0, the (double) sum of sample\_fieldName2 and the (double) the result of sample\_fieldName1 divided by sample\_fieldName2 for the filter set for a particular grouping of โ€œdim1โ€ and โ€œdim2โ€. The output looks like this: + +
+
+[ {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-01-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œdim1โ€ : ,
+ โ€œdim2โ€ : ,
+ โ€œsample\_name1โ€ : ,
+ โ€œsample\_name2โ€ :,
+ โ€œsample\_divideโ€ : 
+ }
+}, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-01-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œdim1โ€ : ,
+ โ€œdim2โ€ : ,
+ โ€œsample\_name1โ€ : ,
+ โ€œsample\_name2โ€ :,
+ โ€œsample\_divideโ€ : 
+ }
+},
+โ€ฆ
+]
+
+
+ diff --git a/docs/Having.md b/docs/Having.md new file mode 100644 index 00000000000..47226f1b88e --- /dev/null +++ b/docs/Having.md @@ -0,0 +1,90 @@ +A having clause is a JSON object identifying which rows from a groupBy query should be returned, by specifying conditions on aggregated values. + +It is essentially the equivalent of the HAVING clause in SQL. + +Druid supports the following types of having clauses. + +### Numeric filters + +The simplest having clause is a numeric filter. +Numeric filters can be used as the base filters for more complex boolean expressions of filters. + +#### Equal To + +The equalTo filter will match rows with a specific aggregate value. +The grammar for an `equalTo` filter is as follows: + + "having": { + "type": "equalTo", + "aggregation": , + "value": + } + + +This is the equivalent of `HAVING = `. + +#### Greater Than + +The greaterThan filter will match rows with aggregate values greater than the given value. +The grammar for a `greaterThan` filter is as follows: + + "having": { + "type": "greaterThan", + "aggregation": , + "value": + } + + +This is the equivalent of `HAVING > `. + +#### Less Than + +The lessThan filter will match rows with aggregate values less than the specified value. +The grammar for a `greaterThan` filter is as follows: + + "having": { + "type": "lessThan", + "aggregation": , + "value": + } + + +This is the equivalent of `HAVING < `. + +### Logical expression filters + +#### AND + +The grammar for an AND filter is as follows: + + "having": { + "type": "and", + "havingSpecs": [, , ...] + } + + +The having clauses in `havingSpecs` can be any other having clause defined on this page. + +#### OR + +The grammar for an OR filter is as follows: + + "having": { + "type": "or", + "havingSpecs": [, , ...] + } + + +The having clauses in `havingSpecs` can be any other having clause defined on this page. + +#### NOT + +The grammar for a NOT filter is as follows: + + "having": { + "type": "not", + "havingSpec": + } + + +The having clause specified at `havingSpec` can be any other having clause defined on this page. diff --git a/docs/Home.md b/docs/Home.md new file mode 100644 index 00000000000..88e1c86b8aa --- /dev/null +++ b/docs/Home.md @@ -0,0 +1,50 @@ +Druid is an open-source analytics datastore designed for realtime, exploratory, queries on large-scale data sets (100โ€™s of Billions entries, 100โ€™s TB data). Druid provides for cost effective, always-on, realtime data ingestion and arbitrary data exploration. + +- Check out some [[Examples]] +- Try out Druid with our Getting Started [Tutorial](https://github.com/metamx/druid/wiki/Tutorial%3A-A-First-Look-at-Druid) +- Learn more by reading the [White Paper](http://static.druid.io/docs/druid.pdf) + +Why Druid? +---------- + +Druid was originally created to resolve query latency issues seen with trying to use Hadoop to power an interactive service. Hadoop has shown the world that itโ€™s possible to house your data warehouse on commodity hardware for a fraction of the price of typical solutions. As people adopt Hadoop for their data warehousing needs, they find two things. + +1. They can now query all of their data in a fairly flexible manner and answer any question they have +2. The queries take a long time + +The first one is the joy that everyone feels the first time they get Hadoop running. The latter is what they realize after they have used Hadoop interactively for a while because Hadoop is optimized for throughput, not latency. Druid is a system that you can set up in your organization next to Hadoop. It provides the ability to access your data in an interactive slice-and-dice fashion. It trades off some query flexibility and takes over the storage format in order to provide the speed. + +Druid is especially useful if you are summarizing your data sets and then querying the summarizations. If you put your summarizations into Druid, you will get quick queryability out of a system that you can be confident will scale up as your data volumes increase. Deployments have scaled up to 2TB of data per hour at peak ingested and aggregated in real-time. + +We have more details about the general design of the system and why you might want to use it in our [White Paper](http://static.druid.io/docs/druid.pdf) or in our [[Design]] doc. + +The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means, we will fix this page. + +#### When Druid? +\* You need to do interactive, fast, exploration of large amounts of data +\* You need analytics (not key value store) +\* You have a lot of data (10s of Billions of events added per day, 10s of TB of data added per day) +\* You want to do your analysis on data as itโ€™s happening (realtime) +\* Your store needs to be always-on, 24x7x365 and years into the future. + +#### Not Druid? +\* The amount of data you have can easily be handled by MySql +\* Your querying for individual entries or doing lookups (Not Analytics) +\* Batch is good enough +\* Canned queries is good enough +\* Downtime is no big deal + +#### Druid vsโ€ฆ +\* [[Druid-vs-Impala-or-Shark]] +\* [[Druid-vs-Redshift]] +\* [[Druid-vs-Vertica]] +\* [[Druid-vs-Cassandra]] +\* [[Druid-vs-Hadoop]] + +Key Features +------------ + +- **Designed for Analytics** - Druid is built for exploratory analytics for OLAP workflows (streamalytics). It supports a variety of filters, aggregators and query types and provides a framework for plugging in new functionality. Users have leveraged Druidโ€™s infrastructure to develop features such as top K queries and histograms. +- **Interactive Queries** - Druidโ€™s low latency data ingestion architecture allows events to be queried milliseconds after they are created. Druidโ€™s query latency is optimized by only reading and scanning exactly what is needed. Aggregate and filter on data without sitting around waiting for results. +- **Highly Available** - Druid is used to back SaaS implementations that need to be up all the time. Your data is still available and queryable during system updates. Scale up or down without data loss. +- **Scalable** - Existing Druid deployments handle billions of events and terabytes of data per day. Druid is designed to be petabyte scale. diff --git a/docs/Indexing-Service.md b/docs/Indexing-Service.md new file mode 100644 index 00000000000..0e4ff939f4a --- /dev/null +++ b/docs/Indexing-Service.md @@ -0,0 +1,190 @@ +Disclaimer: We are still in the process of finalizing the indexing service and these configs are prone to change at any time. We will announce when we feel the indexing service and the configurations described are stable. + +The indexing service is a distributed task/job queue. It accepts requests in the form of [[Tasks]] and executes those tasks across a set of worker nodes. Worker capacity can be automatically adjusted based on the number of tasks pending in the system. The indexing service is highly available, has built in retry logic, and can backup per task logs in deep storage. + +The indexing service is composed of two main components, a coordinator node that manages task distribution and worker capacity, and worker nodes that execute tasks in separate JVMs. + +Preamble +-------- + +The truth is, the indexing service is an experience that is difficult to characterize with words. When they asked me to write this preamble, I was taken aback. I wasnโ€™t quite sure what exactly to write or how to describe thisโ€ฆ entity. I accepted the job, as much for the challenge and inner growth as the money, and took to the mountains for reflection. Six months later, I knew I had it, I was done and had achieved the next euphoric victory in the continuous struggle that plagues my life. But, enough about me. This is about the indexing service. + +The indexing service is philosophical transcendence, an infallible truth that will shape your soul, mold your character, and define your reality. The indexing service is creating world peace, playing with puppies, unwrapping presents on Christmas morning, cradling a loved one, and beating Goro in Mortal Kombat for the first time. The indexing service is sustainable economic growth, global propensity, and a world of transparent financial transactions. The indexing service is a true belieber. The indexing service is panicking because you forgot you signed up for a course and the big exam is in a few minutes, only to wake up and realize it was all a dream. What is the indexing service? More like what isnโ€™t the indexing service. The indexing service is here and it is ready, but are you? + +Indexer Coordinator Node +------------------------ + +The indexer coordinator node exposes HTTP endpoints where tasks can be submitted by posting a JSON blob to specific endpoints. It can be started by launching IndexerCoordinatorMain.java. The indexer coordinator node can operate in local mode or remote mode. In local mode, the coordinator and worker run on the same host and port. In remote mode, worker processes run on separate hosts and ports. + +Tasks can be submitted via POST requests to: + + http://:/druid/indexer/v1/task + +Tasks can cancelled via POST requests to: + + http://:/druid/indexer/v1/task/{taskId}/shutdown + +Issuing the cancel request once sends a graceful shutdown request. Graceful shutdowns may not stop a task right away, but instead issue a safe stop command at a point deemed least impactful to the system. Issuing the cancel request twice in succession will kill โ€“9 the task. + +Task statuses can be retrieved via GET requests to: + + http://:/druid/indexer/v1/task/{taskId}/status + +Task segments can be retrieved via GET requests to: + + http://:/druid/indexer/v1/task/{taskId}/segments + +When a task is submitted, the coordinator creates a lock over the data source and interval of the task. The coordinator also stores the task in a MySQL database table. The database table is read at startup time to bootstrap any tasks that may have been submitted to the coordinator but may not yet have been executed. + +The coordinator also exposes a simple UI to show what tasks are currently running on what nodes at + + http://:/static/console.html + +#### Task Execution + +The coordinator retrieves worker setup metadata from the Druid [[MySQL]] config table. This metadata contains information about the version of workers to create, the maximum and minimum number of workers in the cluster at one time, and additional information required to automatically create workers. + +Tasks are assigned to workers by creating entries under specific /tasks paths associated with a worker, similar to how the Druid master node assigns segments to compute nodes. See [Worker Configuration](Indexing-Service#configuration-1). Once a worker picks up a task, it deletes the task entry and announces a task status under a /status path associated with the worker. Tasks are submitted to a worker until the worker hits capacity. If all workers in a cluster are at capacity, the indexer coordinator node automatically creates new worker resources. + +#### Autoscaling + +The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, worker nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment. + +The Coordinator node controls the number of workers in the cluster according to a worker setup spec that is submitted via a POST request to the indexer at: + + http://:/druid/indexer/v1/worker/setup + +A sample worker setup spec is shown below: + + { + "minVersion":"some_version", + "minNumWorkers":"0", + "maxNumWorkers":"10", + "nodeData": { + "type":"ec2", + "amiId":"ami-someId", + "instanceType":"m1.xlarge", + "minInstances":"1", + "maxInstances":"1", + "securityGroupIds":["securityGroupIds"], + "keyName":"keyName" + }, + "userData":{ + "classType":"galaxy", + "env":"druid", + "version":"druid_version", + "type":"sample_cluster/worker" + } + } + + +Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to write worker setup specs for other deployment environments. A description of the worker setup spec is shown below. + +|Property|Description|Default| +|--------|-----------|-------| +|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none| +|`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0| +|`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0| +|`nodeData`|A JSON object that contains metadata about new nodes to create.|none| +|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none| + +For more information about configuring Auto-scaling, see [Auto-Scaling Configuration](https://github.com/metamx/druid/wiki/Indexing-Service#auto-scaling-configuration). + +#### Running + +Indexer Coordinator nodes can be run using the `com.metamx.druid.indexing.coordinator.http.IndexerCoordinatorMain` class. + +#### Configuration + +Indexer Coordinator nodes require [basic service configuration](https://github.com/metamx/druid/wiki/Configuration#basic-service-configuration). In addition, there are several extra configurations that are required. + + -Ddruid.zk.paths.indexer.announcementsPath=/druid/indexer/announcements + -Ddruid.zk.paths.indexer.leaderLatchPath=/druid/indexer/leaderLatchPath + -Ddruid.zk.paths.indexer.statusPath=/druid/indexer/status + -Ddruid.zk.paths.indexer.tasksPath=/druid/demo/indexer/tasks + + -Ddruid.indexer.runner=remote + -Ddruid.indexer.taskDir=/mnt/persistent/task/ + -Ddruid.indexer.configTable=sample_config + -Ddruid.indexer.workerSetupConfigName=worker_setup + -Ddruid.indexer.strategy=ec2 + -Ddruid.indexer.hadoopWorkingPath=/tmp/druid-indexing + -Ddruid.indexer.logs.s3bucket=some_bucket + -Ddruid.indexer.logs.s3prefix=some_prefix + +The indexing service requires some additional Zookeeper configs. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.zk.paths.indexer.announcementsPath`|The base path where workers announce themselves.|none| +|`druid.zk.paths.indexer.leaderLatchPath`|The base that coordinator nodes use to determine a leader.|none| +|`druid.zk.paths.indexer.statusPath`|The base path where workers announce task statuses.|none| +|`druid.zk.paths.indexer.tasksPath`|The base path where the coordinator assigns new tasks.|none| + +Thereโ€™s several additional configs that are required to run tasks. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.indexer.runner`|Indicates whether tasks should be run locally or in a distributed environment. โ€œlocalโ€ or โ€œremoteโ€.|local| +|`druid.indexer.taskDir`|Intermediate temporary directory that tasks may use.|none| +|`druid.indexer.configTable`|The MySQL config table where misc configs live.|none| +|`druid.indexer.strategy`|The autoscaling strategy to use.|noop| +|`druid.indexer.hadoopWorkingPath`|Intermediate temporary hadoop working directory that certain index tasks may use.|none| +|`druid.indexer.logs.s3bucket`|S3 bucket to store logs.|none| +|`druid.indexer.logs.s3prefix`|S3 key prefix to store logs.|none| + +#### Console + +The indexer console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at: + + http://:8080/static/console.html + +Worker Node +----------- + +The worker node executes submitted tasks. Workers run tasks in separate JVMs. + +#### Running + +Worker nodes can be run using the `com.metamx.druid.indexing.worker.http.WorkerMain` class. Worker nodes can automatically be created by the Indexer Coordinator as part of autoscaling. + +#### Configuration + +Worker nodes require [basic service configuration](https://github.com/metamx/druid/wiki/Configuration#basic-service-configuration). In addition, there are several extra configurations that are required. + + -Ddruid.worker.version=0 + -Ddruid.worker.capacity=3 + + -Ddruid.indexer.threads=3 + -Ddruid.indexer.taskDir=/mnt/persistent/task/ + -Ddruid.indexer.hadoopWorkingPath=/tmp/druid-indexing + + -Ddruid.worker.masterService=druid:sample_cluster:indexer + + -Ddruid.indexer.fork.hostpattern=:%d + -Ddruid.indexer.fork.startport=8080 + -Ddruid.indexer.fork.main=com.metamx.druid.indexing.worker.executor.ExecutorMain + -Ddruid.indexer.fork.opts="-server -Xmx1g -Xms1g -XX:NewSize=256m -XX:MaxNewSize=256m" + -Ddruid.indexer.fork.property.druid.service=druid/sample_cluster/executor + + # These configs are the same configs you would set for basic service configuration, just with a different prefix + -Ddruid.indexer.fork.property.druid.monitoring.monitorSystem=false + -Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456 + -Ddruid.indexer.fork.property.druid.indexer.taskDir=/mnt/persistent/task/ + -Ddruid.indexer.fork.property.druid.processing.formatString=processing-%s + -Ddruid.indexer.fork.property.druid.processing.numThreads=1 + -Ddruid.indexer.fork.property.druid.server.maxSize=0 + -Ddruid.indexer.fork.property.druid.request.logging.dir=request_logs/ + +Many of the configurations for workers are similar to those for basic service configuration":https://github.com/metamx/druid/wiki/Configuration\#basic-service-configuration, but with a different config prefix. Below we describe the unique worker configs. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.worker.version`|Version identifier for the worker.|0| +|`druid.worker.capacity`|Maximum number of tasks the worker can accept.|1| +|`druid.indexer.threads`|Number of processing threads per worker.|1| +|`druid.worker.masterService`|Name of the indexer coordinator used for service discovery.|none| +|`druid.indexer.fork.hostpattern`|The format of the host name.|none| +|`druid.indexer.fork.startport`|Port in which child JVM starts from.|none| +|`druid.indexer.fork.opts`|JVM options for child JVMs.|none| + diff --git a/docs/Libraries.md b/docs/Libraries.md new file mode 100644 index 00000000000..41374e310c1 --- /dev/null +++ b/docs/Libraries.md @@ -0,0 +1,17 @@ +### R + +- [RDruid](https://github.com/metamx/RDruid) - Druid connector for R + +Community Libraries +------------------- + +Some great folks have written their own libraries to interact with Druid + +#### Ruby +\* [madvertise/ruby-druid](https://github.com/madvertise/ruby-druid) - A ruby client for Druid + +#### Helper Libraries + +- [madvertise/druid-dumbo](https://github.com/madvertise/druid-dumbo) - Scripts to help generate batch configs for the ingestion of data into Druid + +- [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness) - A set of scripts to simplify standing up some servers and seeing how things work diff --git a/docs/Loading-Your-Data.md b/docs/Loading-Your-Data.md new file mode 100644 index 00000000000..568a20767ac --- /dev/null +++ b/docs/Loading-Your-Data.md @@ -0,0 +1,367 @@ +Once you have a realtime node working, it is time to load your own data to see how Druid performs. + +Druid can ingest data in three ways: via Kafka and a realtime node, via the indexing service, and via the Hadoop batch loader. Data is ingested in realtime using a [[Firehose]]. + +## Create Config Directories ## +Each type of node needs its own config file and directory, so create them as subdirectories under the druid directory. +```bash +mkdir config +mkdir config/realtime +mkdir config/master +mkdir config/compute +mkdir config/broker +``` + +## Loading Data with Kafka ## + +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/master/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [[Firehose]] with the right configuration, we can import data into Druid in realtime without writing any code. To load data to a realtime node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [[Realtime]] node. + +### Booting Kafka ### + +Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html). + +1. Download Apache Kafka 0.7.2 from [http://kafka.apache.org/downloads.html](http://kafka.apache.org/downloads.html) +```bash +wget http://apache.spinellicreations.com/incubator/kafka/kafka-0.7.2-incubating/kafka-0.7.2-incubating-src.tgz +tar -xvzf kafka-0.7.2-incubating-src.tgz +cd kafka-0.7.2-incubating-src +``` +2. Build Kafka +```bash +./sbt update +./sbt package +``` +3. Boot Kafka +```bash +cat config/zookeeper.properties +bin/zookeeper-server-start.sh config/zookeeper.properties +# in a new console +bin/kafka-server-start.sh config/server.properties +``` +4. Launch the console producer (so you can type in JSON kafka messages in a bit) +```bash +bin/kafka-console-producer.sh --zookeeper localhost:2181 --topic druidtest +``` +### Launching a Realtime Node + +1. Create a valid configuration file similar to this called config/realtime/runtime.properties: +``` +druid.host=0.0.0.0:8080 +druid.port=8080 + +com.metamx.emitter.logging=true + +druid.processing.formatString=processing_%s +druid.processing.numThreads=1 +druid.processing.buffer.sizeBytes=10000000 + +#emitting, opaque marker +druid.service=example + +druid.request.logging.dir=/tmp/example/log +druid.realtime.specFile=realtime.spec +com.metamx.emitter.logging=true +com.metamx.emitter.logging.level=debug + +# below are dummy values when operating a realtime only node +druid.processing.numThreads=3 + +com.metamx.aws.accessKey=dummy_access_key +com.metamx.aws.secretKey=dummy_secret_key +druid.pusher.s3.bucket=dummy_s3_bucket + +druid.zk.service.host=localhost +druid.server.maxSize=300000000000 +druid.zk.paths.base=/druid +druid.database.segmentTable=prod_segments +druid.database.user=user +druid.database.password=diurd +druid.database.connectURI= +druid.host=127.0.0.1:8080 + +``` +2. Create a valid realtime configuration file similar to this called realtime.spec: +```json +[{ + "schema" : { "dataSource":"druidtest", + "aggregators":[ {"type":"count", "name":"impressions"}, + {"type":"doubleSum","name":"wp","fieldName":"wp"}], + "indexGranularity":"minute", + "shardSpec" : { "type": "none" } }, + "config" : { "maxRowsInMemory" : 500000, + "intermediatePersistPeriod" : "PT10m" }, + "firehose" : { "type" : "kafka-0.7.2", + "consumerProps" : { "zk.connect" : "localhost:2181", + "zk.connectiontimeout.ms" : "15000", + "zk.sessiontimeout.ms" : "15000", + "zk.synctime.ms" : "5000", + "groupid" : "topic-pixel-local", + "fetch.size" : "1048586", + "autooffset.reset" : "largest", + "autocommit.enable" : "false" }, + "feed" : "druidtest", + "parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, + "data" : { "format" : "json" }, + "dimensionExclusions" : ["wp"] } }, + "plumber" : { "type" : "realtime", + "windowPeriod" : "PT10m", + "segmentGranularity":"hour", + "basePersistDirectory" : "/tmp/realtime/basePersist", + "rejectionPolicy": {"type": "messageTime"} } + +}] +``` +3. Launch the realtime node +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-Ddruid.realtime.specFile=config/realtime/realtime.spec \ +-classpath lib/*:config/realtime com.metamx.druid.realtime.RealtimeMain +``` +4. Paste data into the Kafka console producer +```json +{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100} +{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50} +{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20} +{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30} +{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40} +``` +5. Watch the events as they are ingested by Druid's realtime node +```bash +... +2013-06-17 21:41:55,569 INFO [Global--0] com.metamx.emitter.core.LoggingEmitter - Event [{"feed":"metrics","timestamp":"2013-06-17T21:41:55.569Z","service":"example","host":"127.0.0.1","metric":"events/processed","value":5,"user2":"druidtest"}] +... +``` +6. In a new console, edit a file called query.body: +```json +{ + "queryType": "groupBy", + "dataSource": "druidtest", + "granularity": "all", + "dimensions": [], + "aggregations": [ + { "type": "count", "name": "rows" }, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} + ], + "intervals": ["2010-01-01T00:00/2020-01-01T00"] +} +``` +7. Submit the query via curl +```bash +curl -X POST "http://localhost:8080/druid/v2/?pretty" \ +-H 'content-type: application/json' -d @query.body +``` +8. View Result! +```json +[ { + "timestamp" : "2010-01-01T01:01:00.000Z", + "result" : { + "imps" : 20, + "wp" : 60000.0, + "rows" : 5 + } +} ] +``` +Now you're ready for [[Querying Your Data]]! + +## Loading Data with the HadoopDruidIndexer ## + +Historical data can be loaded via a Hadoop job. + +The setup for a single node, 'standalone' Hadoop cluster is available at [http://hadoop.apache.org/docs/stable/single_node_setup.html](http://hadoop.apache.org/docs/stable/single_node_setup.html). + +### Setup MySQL ### +1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/) +2. Install MySQL +3. Create a druid user and database +```bash +mysql -u root +``` +```sql +GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; +CREATE database druid; +``` +The [[Master]] node will create the tables it needs based on its configuration. + +### Make sure you have ZooKeeper Running ### + +Make sure that you have a zookeeper instance running. If you followed the instructions for Kafka, it is probably running. If you are unsure if you have zookeeper running, try running + +```bash +ps auxww | grep zoo | grep -v grep +``` + +If you get any result back, then zookeeper is most likely running. If you haven't setup Kafka or do not have zookeeper running, then you can download it and start it up with + +```bash +curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz +tar xzf zookeeper-3.4.5.tar.gz +cd zookeeper-3.4.5 +cp conf/zoo_sample.cfg conf/zoo.cfg +./bin/zkServer.sh start +cd .. +``` + +### Launch a Master Node ### +If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [[Realtime]] node, we use 8081 for the [[Master]]. + +1. Setup a configuration file called config/master/runtime.properties similar to: +```bash +druid.host=0.0.0.0:8081 +druid.port=8081 + +com.metamx.emitter.logging=true + +druid.processing.formatString=processing_%s +druid.processing.numThreads=1 +druid.processing.buffer.sizeBytes=10000000 + +#emitting, opaque marker +druid.service=example + +druid.master.startDelay=PT60s +druid.request.logging.dir=/tmp/example/log +druid.realtime.specFile=realtime.spec +com.metamx.emitter.logging=true +com.metamx.emitter.logging.level=debug + +# below are dummy values when operating a realtime only node +druid.processing.numThreads=3 + +com.metamx.aws.accessKey=dummy_access_key +com.metamx.aws.secretKey=dummy_secret_key +druid.pusher.s3.bucket=dummy_s3_bucket + +druid.zk.service.host=localhost +druid.server.maxSize=300000000000 +druid.zk.paths.base=/druid +druid.database.segmentTable=prod_segments +druid.database.user=druid +druid.database.password=diurd +druid.database.connectURI=jdbc:mysql://localhost:3306/druid +druid.zk.paths.discoveryPath=/druid/discoveryPath +druid.database.ruleTable=rules +druid.database.configTable=config + +# Path on local FS for storage of segments; dir will be created if needed +druid.paths.indexCache=/tmp/druid/indexCache +# Path on local FS for storage of segment metadata; dir will be created if needed +druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache +``` +2. Launch the [[Master]] node +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-classpath lib/*:config/master \ +com.metamx.druid.http.MasterMain +``` + +### Launch a Compute/Historical Node ### +1. Create a configuration file in config/compute/runtime.properties similar to: +```bash +druid.host=0.0.0.0:8082 +druid.port=8082 + +com.metamx.emitter.logging=true + +druid.processing.formatString=processing_%s +druid.processing.numThreads=1 +druid.processing.buffer.sizeBytes=10000000 + +#emitting, opaque marker +druid.service=example + +druid.request.logging.dir=/tmp/example/log +druid.realtime.specFile=realtime.spec +com.metamx.emitter.logging=true +com.metamx.emitter.logging.level=debug + +# below are dummy values when operating a realtime only node +druid.processing.numThreads=3 + +com.metamx.aws.accessKey=dummy_access_key +com.metamx.aws.secretKey=dummy_secret_key +druid.pusher.s3.bucket=dummy_s3_bucket + +druid.zk.service.host=localhost +druid.server.maxSize=300000000000 +druid.zk.paths.base=/druid +druid.database.segmentTable=prod_segments +druid.database.user=druid +druid.database.password=diurd +druid.database.connectURI=jdbc:mysql://localhost:3306/druid +druid.zk.paths.discoveryPath=/druid/discoveryPath +druid.database.ruleTable=rules +druid.database.configTable=config + +# Path on local FS for storage of segments; dir will be created if needed +druid.paths.indexCache=/tmp/druid/indexCache +# Path on local FS for storage of segment metadata; dir will be created if needed +druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache +# Setup local storage mode +druid.pusher.local.storageDirectory=/tmp/druid/localStorage +druid.pusher.local=true +``` +2. Launch the compute node: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-classpath lib/*:config/compute \ +com.metamx.druid.http.ComputeMain +``` + +### Create a File of Records ### + +We can use the same records we have been, in a file called records.json: +```json +{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100} +{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50} +{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20} +{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30} +{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40} +``` + +### Run the Hadoop Job ### + +Now its time to run the Hadoop [[Batch-ingestion]] job, HadoopDruidIndexer, which will fill a historical [[Compute]] node with data. First we'll need to configure the job. + +1. Create a config called batchConfig.json similar to: +```json +{ + "dataSource": "druidtest", + "timestampColumn": "utcdt", + "timestampFormat": "iso", + "dataSpec": { + "format": "json", + "dimensions": ["gender", "age"] + }, + "granularitySpec": { + "type":"uniform", + "intervals":["2010-01-01T01/PT1H"], + "gran":"hour" + }, + "pathSpec": { "type": "static", + "paths": "/Users/rjurney/Software/druid/records.json" }, + "rollupSpec": { "aggs":[ {"type":"count", "name":"impressions"}, + {"type":"doubleSum","name":"wp","fieldName":"wp"} + ], + "rollupGranularity": "minute"}, + "workingPath": "/tmp/working_path", + "segmentOutputPath": "/tmp/segments", + "leaveIntermediate": "false", + "partitionsSpec": { + "targetPartitionSize": 5000000 + }, + "updaterJobSpec": { + "type":"db", + "connectURI":"jdbc:mysql://localhost:3306/druid", + "user":"druid", + "password":"diurd", + "segmentTable":"prod_segments" + } +} +``` +2. Now run the job, with the config pointing at batchConfig.json: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=realtime.spec -classpath lib/* com.metamx.druid.indexer.HadoopDruidIndexerMain batchConfig.json +``` + +You can now move on to [[Querying Your Data]]! \ No newline at end of file diff --git a/docs/Master.md b/docs/Master.md new file mode 100644 index 00000000000..891f6b854ef --- /dev/null +++ b/docs/Master.md @@ -0,0 +1,103 @@ +Master +====== + +The Druid master node is primarily responsible for segment management and distribution. More specifically, the Druid master node communicates to compute nodes to load or drop segments based on configurations. The Druid master is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load. + +The Druid master runs periodically and the time between each run is a configurable parameter. Each time the Druid master runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the broker and compute nodes, the Druid master maintains a connection to a Zookeeper cluster for current cluster information. The master also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled. + +Before any unassigned segments are serviced by compute nodes, the available compute nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The master does not directly communicate with a compute node when assigning it a new segment; instead the master creates some temporary information about the new segment under load queue path of the compute node. Once this request is seen, the compute node will load the segment and begin servicing it. + +Rules +----- + +Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different compute node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The master loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The master will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule + +For more information on rules, see [[Rule Configuration]]. + +Cleaning Up Segments +-------------------- + +Each run, the Druid master compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped. + +Segment Availability +-------------------- + +If a compute node restarts or becomes unavailable for any reason, the Druid master will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other compute nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime represents a period of time in which the master will not reassign a dropped segment. Hence, if a compute node becomes unavailable and available again within a short period of time, the compute node will start up and serve segments from its cache without any those segments being reassigned across the cluster. + +Balancing Segment Load +---------------------- + +To ensure an even distribution of segments across compute nodes in the cluster, the master node will find the total size of all segments being served by every compute node each time the master runs. For every compute node tier in the cluster, the master node will determine the compute node with the highest utilization and the compute node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized node to the lowest utilized node. There is a configurable limit on the number of segments that can be moved from one node to another each time the master runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased. + +HTTP Endpoints +-------------- + +The master node exposes several HTTP endpoints for interactions. + +### GET + +/info/master - returns the current true master of the cluster as a JSON object. E.g. A GET request to :8080/info/master will yield JSON of the form {[host]("IP"}) + +/info/cluster - returns JSON data about every node and segment in the cluster. E.g. A GET request to :8080/info/cluster will yield JSON data organized by nodes. Information about each node and each segment on each node will be returned. + +/info/servers (optional param ?full) - returns all segments in the cluster if the full flag is not set, otherwise returns full metadata about all servers in the cluster + +/info/servers/{serverName} - returns full metadata about a specific server + +/info/servers/{serverName}/segments (optional param ?full) - returns a list of all segments for a server if the full flag is not set, otherwise returns all segment metadata + +/info/servers/{serverName}/segments/{segmentId} - returns full metadata for a specific segment + +/info/segments (optional param ?full)- returns all segments in the cluster as a list if the full flag is not set, otherwise returns all metadata about segments in the cluster + +/info/segments/{segmentId} - returns full metadata for a specific segment + +/info/datasources (optional param ?full) - returns a list of datasources in the cluster if the full flag is not set, otherwise returns all the metadata for every datasource in the cluster + +/info/datasources/{dataSourceName} - returns full metadata for a datasource + +/info/datasources/{dataSourceName}/segments (optional param ?full) - returns a list of all segments for a datasource if the full flag is not set, otherwise returns full segment metadata for a datasource + +/info/datasources/{dataSourceName}/segments/{segmentId} - returns full segment metadata for a specific segment + +/info/rules - returns all rules for all data sources in the cluster including the default datasource. + +/info/rules/{dataSourceName} - returns all rules for a specified datasource + +### POST + +/info/rules/{dataSourceName} - POST with a list of rules in JSON form to update rules. + +The Master Console +------------------ + +The Druid master exposes a web GUI for displaying cluster information and rule configuration. After the master starts, the console can be accessed at http://HOST:PORT/static/. There exists a full cluster view, as well as views for individual compute nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table. + +The master console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited. + +FAQ +--- + +1. **Do clients ever contact the master node?** + +The master is not involved in the lifecycle of a query. + +Compute nodes never directly contact the master node. The Druid master tells the compute nodes to load/drop data via Zookeeper, but the compute nodes are completely unaware of the master. + +Brokers also never contact the master. Brokers base their understanding of the data topology on metadata exposed by the compute nodes via ZK and are completely unaware of the master. + +2. **Does it matter if the master node starts up before or after other processes?** + +No. If the Druid master is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the master node can be started up at any time, and after a configurable delay, will start running master tasks. + +This also means that if you have a working cluster and all of your masters die, the cluster will continue to function, it just wonโ€™t experience any changes to its data topology. + +Running +------- + +Master nodes can be run using the `com.metamx.druid.http.MasterMain` class. + +Configuration +------------- + +See [[Configuration]]. diff --git a/docs/MySQL.md b/docs/MySQL.md new file mode 100644 index 00000000000..79cf6ed6d8b --- /dev/null +++ b/docs/MySQL.md @@ -0,0 +1,44 @@ +MySQL is an external dependency of Druid. We use it to store various metadata about the system, but not to store the actual data. There are a number of tables used for various purposes described below. + +Segments Table +-------------- + +This is dictated by the `druid.database.segmentTable` property (Note that these properties are going to change in the next stable version after 0.4.12). + +This table stores metadata about the segments that are available in the system. The table is polled by the [[Master]] to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes. + +The `used` column is a boolean โ€œtombstoneโ€. A 1 means that the segment should be โ€œusedโ€ by the cluster (i.e. it should be loaded and available for requests). A 0 means that the segment should not be actively loaded into the cluster. We do this as a means of removing segments from the cluster without actually removing their metadata (which allows for simpler rolling back if that is ever an issue). + +The `payload` column stores a JSON blob that has all of the metadata for the segment (some of the data stored in this payload is redundant with some of the columns in the table, that is intentional). This looks something like + + { + "dataSource":"wikipedia", + "interval":"2012-05-23T00:00:00.000Z/2012-05-24T00:00:00.000Z", + "version":"2012-05-24T00:10:00.046Z", + "loadSpec":{"type":"s3_zip", + "bucket":"bucket_for_segment", + "key":"path/to/segment/on/s3"}, + "dimensions":"comma-delimited-list-of-dimension-names", + "metrics":"comma-delimited-list-of-metric-names", + "shardSpec":{"type":"none"}, + "binaryVersion":9, + "size":size_of_segment, + "identifier":"wikipedia_2012-05-23T00:00:00.000Z_2012-05-24T00:00:00.000Z_2012-05-23T00:10:00.046Z" + } + +Note that the format of this blob can and will change from time-to-time. + +Rule Table +---------- + +The rule table is used to store the various rules about where segments should land. These rules are used by the [[Master]] when making segment (re-)allocation decisions about the cluster. + +Config Table +------------ + +The config table is used to store runtime configuration objects. We do not have many of these yet and we are not sure if we will keep this mechanism going forward, but it is the beginnings of a method of changing some configuration parameters across the cluster at runtime. + +Task-related Tables +------------------- + +There are also a number of tables created and used by the [[Indexing Service]] in the course of its work. diff --git a/docs/OrderBy.md b/docs/OrderBy.md new file mode 100644 index 00000000000..993df6f4674 --- /dev/null +++ b/docs/OrderBy.md @@ -0,0 +1,24 @@ +The orderBy field provides the functionality to sort and limit the set of results from a groupBy query. Available options are: + +### DefaultLimitSpec + +The default limit spec takes a limit and the list of columns to do an orderBy operation over. The grammar is: + + + { + "type" : "default", + "limit" : , + "columns" : [list of OrderByColumnSpec], + } + + +#### OrderByColumnSpec + +OrderByColumnSpecs indicate how to do order by operations. Each order by condition can be a String or a map of the following form: + + + { + "dimension" : "", + "direction" : "ASCENDING OR DESCENDING" + } + diff --git a/docs/Plumber.md b/docs/Plumber.md new file mode 100644 index 00000000000..cf650fb6cdd --- /dev/null +++ b/docs/Plumber.md @@ -0,0 +1,21 @@ +The Plumber is the thing that handles generated segments both while they are being generated and when they are โ€œdoneโ€. This is also technically a pluggable interface and there are multiple implementations, but there are a lot of details handled by the plumber such that it is expected that there will only be a few implementations and only more advanced third-parties will implement their own. See [here](https://github.com/metamx/druid/wiki/Plumber#available-plumbers) for a description of the plumbers included with Druid. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|type|String|Specifies the type of plumber. Each value will have its own configuration schema, plumbers packaged with Druid are described [here](https://github.com/metamx/druid/wiki/Plumber#available-plumbers)|yes| + +We provide a brief description of the example to exemplify the types of things that are configured on the plumber. + +- `windowPeriod` is the amount of lag time to allow events. This is configured with a 10 minute window, meaning that any event more than 10 minutes ago will be thrown away and not included in the segment generated by the realtime server. +- `basePersistDirectory` is the directory to put things that need persistence. The plumber is responsible for the actual intermediate persists and this tells it where to store those persists. + +Available Plumbers +------------------ + +#### YeOldePlumber + +This plumber creates single historical segments. + +#### RealtimePlumber + +This plumber creates real-time/mutable segments. diff --git a/docs/Post-aggregations.md b/docs/Post-aggregations.md new file mode 100644 index 00000000000..8ff7a91ecb5 --- /dev/null +++ b/docs/Post-aggregations.md @@ -0,0 +1,92 @@ +Post-aggregations are specifications of processing that should happen on aggregated values as they come out of Druid. If you include a post aggregation as part of a query, make sure to include all aggregators the post-aggregator requires. + +There are several post-aggregators available. + +### Arithmetic post-aggregator + +The arithmetic post-aggregator applies the provided function to the given fields from left to right. The fields can be aggregators or other post aggregators. + +Supported functions are `+`, `-`, `*`, and `/` + +The grammar for an arithmetic post aggregation is: + + postAggregation : { + "type" : "arithmetic", + "name" : , + "fn" : , + "fields": [, , ...] + } + +### Field accessor post-aggregator + +This returns the value produced by the specified [[aggregator|Aggregations]]. + +`fieldName` refers to the output name of the aggregator given in the [[aggregations|Aggregations]] portion of the query. + + field_accessor : { + "type" : "fieldAccess", + "fieldName" : + } + +### Constant post-aggregator + +The constant post-aggregator always returns the specified value. + + constant : { + "type" : "constant", + "name" : , + "value" : , + } + +### Example Usage + +In this example, letโ€™s calculate a simple percentage using post aggregators. Letโ€™s imagine our data set has a metric called โ€œtotalโ€. + +The format of the query JSON is as follows: + + + { + ... + "aggregations" : [ + { + "type" : "count", + "name" : "rows" + }, + { + "type" : "doubleSum", + "name" : "tot", + "fieldName" : "total" + } + ], + "postAggregations" : { + "type" : "arithmetic", + "name" : "average", + "fn" : "*", + "fields" : [ + { + "type" : "arithmetic", + "name" : "div", + "fn" : "/", + "fields" : [ + { + "type" : "fieldAccess", + "name" : "tot", + "fieldName" : "tot" + }, + { + "type" : "fieldAccess", + "name" : "rows", + "fieldName" : "rows" + } + ] + }, + { + "type" : "constant", + "name": "const", + "value" : 100 + } + ] + } + ... + } + diff --git a/docs/Querying-your-data.md b/docs/Querying-your-data.md new file mode 100644 index 00000000000..520edcaf613 --- /dev/null +++ b/docs/Querying-your-data.md @@ -0,0 +1,363 @@ +# Setup # + +Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [[Loading Your Data]] we setup a [[Realtime]], [[Compute]] and [[Master]] node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. + +## Booting a Broker Node ## + +1. Setup a config file at config/broker/runtime.properties that looks like this: +``` +druid.host=0.0.0.0:8083 +druid.port=8083 + +com.metamx.emitter.logging=true + +druid.processing.formatString=processing_%s +druid.processing.numThreads=1 +druid.processing.buffer.sizeBytes=10000000 + +#emitting, opaque marker +druid.service=example + +druid.request.logging.dir=/tmp/example/log +druid.realtime.specFile=realtime.spec +com.metamx.emitter.logging=true +com.metamx.emitter.logging.level=debug + +# below are dummy values when operating a realtime only node +druid.processing.numThreads=3 + +com.metamx.aws.accessKey=dummy_access_key +com.metamx.aws.secretKey=dummy_secret_key +druid.pusher.s3.bucket=dummy_s3_bucket + +druid.zk.service.host=localhost +druid.server.maxSize=300000000000 +druid.zk.paths.base=/druid +druid.database.segmentTable=prod_segments +druid.database.user=druid +druid.database.password=diurd +druid.database.connectURI=jdbc:mysql://localhost:3306/druid +druid.zk.paths.discoveryPath=/druid/discoveryPath +druid.database.ruleTable=rules +druid.database.configTable=config + +# Path on local FS for storage of segments; dir will be created if needed +druid.paths.indexCache=/tmp/druid/indexCache +# Path on local FS for storage of segment metadata; dir will be created if needed +druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache +druid.pusher.local.storageDirectory=/tmp/druid/localStorage +druid.pusher.local=true + +# thread pool size for servicing queries +druid.client.http.connections=30 +``` + +2. Run the broker node: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-Ddruid.realtime.specFile=realtime.spec \ +-classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/broker \ +com.metamx.druid.http.BrokerMain +``` + +## Booting a Master Node ## + +1. Setup a config file at config/master/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818870](https://gist.github.com/rjurney/5818870) +2. Run the master node: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/master \ +com.metamx.druid.http.MasterMain +``` + +## Booting a Realtime Node ## + +1. Setup a config file at config/realtime/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818774](https://gist.github.com/rjurney/5818774) + +2. Setup a realtime.spec file like this: [https://gist.github.com/rjurney/5818779](https://gist.github.com/rjurney/5818779) +3. Run the realtime node: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-Ddruid.realtime.specFile=realtime.spec \ +-classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/realtime \ +com.metamx.druid.realtime.RealtimeMain +``` + +## Booting a Compute Node ## + +1. Setup a config file at config/compute/runtime.properties that looks like this: [https://gist.github.com/rjurney/5818885](https://gist.github.com/rjurney/5818885) +2. Run the compute node: +```bash +java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ +-classpath services/target/druid-services-0.5.50-SNAPSHOT-selfcontained.jar:config/compute \ +com.metamx.druid.http.ComputeMain +``` + +# Querying Your Data # + +Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [[Loading Your Data]]. Having done that, its time to query our data! For a complete specification of queries, see [[Querying]]. + +## Querying Different Nodes ## + +As a shared-nothing system, there are three ways to query druid, against the [[Realtime]], [[Compute]] or [[Broker]] node. Querying a Realtime node returns only realtime data, querying a compute node returns only historical segments. Querying the broker will query both realtime and compute segments and compose an overall result for the query. This is the normal mode of operation for queries in druid. + +### Construct a Query ### + +For constructing this query, see: Querying against the realtime.spec +```json +{ + "queryType": "groupBy", + "dataSource": "druidtest", + "granularity": "all", + "dimensions": [], + "aggregations": [ + {"type": "count", "name": "rows"}, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} + ], + "intervals": ["2010-01-01T00:00/2020-01-01T00"] +} +``` + +### Querying the Realtime Node ### + +Run our query against port 8080: +```bash +curl -X POST "http://localhost:8080/druid/v2/?pretty" \ +-H 'content-type: application/json' -d @query.body +``` +See our result: +```json +[ { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 5, + "wp" : 15000.0, + "rows" : 5 + } +} ] +``` + +### Querying the Compute Node ### +Run the query against port 8082: +```bash +curl -X POST "http://localhost:8082/druid/v2/?pretty" \ +-H 'content-type: application/json' -d @query.body +``` +And get (similar to): +```json +[ { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 27, + "wp" : 77000.0, + "rows" : 9 + } +} ] +``` +### Querying both Nodes via the Broker ### +Run the query against port 8083: +```bash +curl -X POST "http://localhost:8083/druid/v2/?pretty" \ +-H 'content-type: application/json' -d @query.body +``` +And get: +```json +[ { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 5, + "wp" : 15000.0, + "rows" : 5 + } +} ] +``` + +Now that we know what nodes can be queried (although you should usually use the broker node), lets learn how to know what queries are available. + +## Querying Against the realtime.spec ## + +How are we to know what queries we can run? Although [[Querying]] is a helpful index, to get a handle on querying our data we need to look at our [[Realtime]] node's realtime.spec file: + +```json +[{ + "schema" : { "dataSource":"druidtest", + "aggregators":[ {"type":"count", "name":"impressions"}, + {"type":"doubleSum","name":"wp","fieldName":"wp"}], + "indexGranularity":"minute", + "shardSpec" : { "type": "none" } }, + "config" : { "maxRowsInMemory" : 500000, + "intermediatePersistPeriod" : "PT10m" }, + "firehose" : { "type" : "kafka-0.7.2", + "consumerProps" : { "zk.connect" : "localhost:2181", + "zk.connectiontimeout.ms" : "15000", + "zk.sessiontimeout.ms" : "15000", + "zk.synctime.ms" : "5000", + "groupid" : "topic-pixel-local", + "fetch.size" : "1048586", + "autooffset.reset" : "largest", + "autocommit.enable" : "false" }, + "feed" : "druidtest", + "parser" : { "timestampSpec" : { "column" : "utcdt", "format" : "iso" }, + "data" : { "format" : "json" }, + "dimensionExclusions" : ["wp"] } }, + "plumber" : { "type" : "realtime", + "windowPeriod" : "PT10m", + "segmentGranularity":"hour", + "basePersistDirectory" : "/tmp/realtime/basePersist", + "rejectionPolicy": {"type": "messageTime"} } + +}] +``` + +### dataSource ### + +```json +"dataSource":"druidtest" +``` +Our dataSource tells us the name of the relation/table, or 'source of data', to query in both our realtime.spec and query.body! + +### aggregations ### + +Note the [[Aggregations]] in our query: + +```json + "aggregations": [ + {"type": "count", "name": "rows"}, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} + ], +``` + +this matches up to the aggregators in the schema of our realtime.spec! + +```json +"aggregators":[ {"type":"count", "name":"impressions"}, + {"type":"doubleSum","name":"wp","fieldName":"wp"}], +``` + +### dimensions ### + +Lets look back at our actual records (from [[Loading Your Data]]): + +```json +{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100} +{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50} +{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20} +{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30} +{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40} +``` + +Note that we have two dimensions to our data, other than our primary metric, wp. They are 'gender' and 'age'. We can specify these in our query! Note that we have added a dimension: age, below. + +```json +{ + "queryType": "groupBy", + "dataSource": "druidtest", + "granularity": "all", + "dimensions": ["age"], + "aggregations": [ + {"type": "count", "name": "rows"}, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} + ], + "intervals": ["2010-01-01T00:00/2020-01-01T00"] +} +``` + +Which gets us grouped data in return! + +```json +[ { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 1, + "age" : "100", + "wp" : 1000.0, + "rows" : 1 + } +}, { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 1, + "age" : "20", + "wp" : 3000.0, + "rows" : 1 + } +}, { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 1, + "age" : "30", + "wp" : 4000.0, + "rows" : 1 + } +}, { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 1, + "age" : "40", + "wp" : 5000.0, + "rows" : 1 + } +}, { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 1, + "age" : "50", + "wp" : 2000.0, + "rows" : 1 + } +} ] +``` + +### filtering ### + +Now that we've observed our dimensions, we can also filter: + +```json +{ + "queryType": "groupBy", + "dataSource": "druidtest", + "granularity": "all", + "filter": { + "type": "selector", + "dimension": "gender", + "value": "male" + }, + "aggregations": [ + {"type": "count", "name": "rows"}, + {"type": "longSum", "name": "imps", "fieldName": "impressions"}, + {"type": "doubleSum", "name": "wp", "fieldName": "wp"} + ], + "intervals": ["2010-01-01T00:00/2020-01-01T00"] +} +``` + +Which gets us just people aged 40: + +```json +[ { + "version" : "v1", + "timestamp" : "2010-01-01T00:00:00.000Z", + "event" : { + "imps" : 3, + "wp" : 9000.0, + "rows" : 3 + } +} ] +``` + +Check out [[Filters]] for more. + +## Learn More ## + +You can learn more about querying at [[Querying]]! Now check out [[Booting a production cluster]]! \ No newline at end of file diff --git a/docs/Querying.md b/docs/Querying.md new file mode 100644 index 00000000000..21ed93c7bb5 --- /dev/null +++ b/docs/Querying.md @@ -0,0 +1,111 @@ +Querying +======== + +Queries are made using an HTTP REST style request to a [[Broker]], [[Compute]], or [[Realtime]] node. The query is expressed in JSON and each of these node types expose the same REST query interface. + +We start by describing an example query with additional comments that mention possible variations. Query operators are also summarized in a table below. + +Example Query โ€œrandโ€ +-------------------- + +Here is the query in the examples/rand subproject (file is query.body), followed by a commented version of the same. + +\`\`\`javascript +{ + [queryType]() โ€œgroupByโ€, + [dataSource]() โ€œrandSeqโ€, + [granularity]() โ€œallโ€, + [dimensions]() [], + [aggregations]() [ + { [type]() โ€œcountโ€, [name]() โ€œrowsโ€ }, + { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œeventsโ€, [name]() โ€œeโ€ }, + { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œoutColumnโ€, [name]() โ€œrandomNumberSumโ€ } + ], + [postAggregations]() [{ + [type]() โ€œarithmeticโ€, + [name]() โ€œavg\_randomโ€, + [fn]() โ€œ/โ€, + [fields]() [ + { [type]() โ€œfieldAccessโ€, [fieldName]() โ€œrandomNumberSumโ€ }, + { [type]() โ€œfieldAccessโ€, [fieldName]() โ€œrowsโ€ } + ] + }], + [intervals]() [โ€œ2012-10-01T00:00/2020-01-01T00โ€] +} +\`\`\` + +This query could be submitted via curl like so (assuming the query object is in a file โ€œquery.jsonโ€). + + curl -X POST "http://host:port/druid/v2/?pretty" -H 'content-type: application/json' -d @query.json + +The โ€œprettyโ€ query parameter gets the results formatted a bit nicer. + +Details of Example Query โ€œrandโ€ +------------------------------- + +The queryType JSON field identifies which kind of query operator is to be used, in this case it is groupBy, the most frequently used kind (which corresponds to an internal implementation class GroupByQuery registered as โ€œgroupByโ€), and it has a set of required fields that are also part of this query. The queryType can also be โ€œsearchโ€ or โ€œtimeBoundaryโ€ which have similar or different required fields summarized below: +\`\`\`javascript +{ + [queryType]() โ€œgroupByโ€, +\`\`\` +The dataSource JSON field shown next identifies where to apply the query. In this case, randSeq corresponds to the examples/rand/rand\_realtime.spec file schema: +\`\`\`javascript + [dataSource]() โ€œrandSeqโ€, +\`\`\` +The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like โ€œsecondโ€, โ€œminuteโ€, โ€œfifteen\_minuteโ€, โ€œthirty\_minuteโ€, โ€œhourโ€ or โ€œdayโ€. It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning โ€œ6 minute bucketsโ€. See [[Granularities]] for more information on the different options for this field. In this example, it is set to the special value โ€œallโ€ which means [bucket all data points together into the same time bucket]() +\`\`\`javascript + [granularity]() โ€œallโ€, +\`\`\` +The dimensions JSON field value is an array of zero or more fields as defined in the dataSource spec file or defined in the input records and carried forward. These are used to constrain the grouping. If empty, then one value per time granularity bucket is requested in the groupBy: +\`\`\`javascript + [dimensions]() [], +\`\`\` +A groupBy also requires the JSON field โ€œaggregationsโ€ (See [[Aggregations]]), which are applied to the column specified by fieldName and the output of the aggregation will be named according to the value in the โ€œnameโ€ field: +\`\`\`javascript + [aggregations]() [ + { [type]() โ€œcountโ€, [name]() โ€œrowsโ€ }, + { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œeventsโ€, [name]() โ€œeโ€ }, + { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œoutColumnโ€, [name]() โ€œrandomNumberSumโ€ } + ], +\`\`\` +You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [[Post Aggregations]] for a detailed description. In the rand example, an arithmetic type operation (division, as specified by โ€œfnโ€) is performed with the result โ€œnameโ€ of โ€œavg\_randomโ€. The โ€œfieldsโ€ field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to โ€œnameโ€ JSON field inside the type โ€œfieldAccessโ€ are required but not used outside this expression, so they are prefixed with โ€œdummyโ€ for clarity: +\`\`\`javascript + [postAggregations]() [{ + [type]() โ€œarithmeticโ€, + [name]() โ€œavg\_randomโ€, + [fn]() โ€œ/โ€, + [fields]() [ + { [type]() โ€œfieldAccessโ€, [fieldName]() โ€œrandomNumberSumโ€ }, + { [type]() โ€œfieldAccessโ€, [fieldName]() โ€œrowsโ€ } + ] + }], +\`\`\` +The time range(s) of the query; data outside the specified intervals will not be used; this example specifies from October 1, 2012 until January 1, 2020: +\`\`\`javascript + [intervals]() [โ€œ2012-10-01T00:00/2020-01-01T00โ€] +} +\`\`\` + +Query Operators +--------------- + +The following table summarizes query properties. + +|query types|property|description|required?| +|-----------|--------|-----------|---------| +|timeseries, groupBy, search, timeBoundary|dataSource|query is applied to this data source|yes| +|timeseries, groupBy, search|intervals|range of time series to include in query|yes| +|timeseries, groupBy, search, timeBoundary|context|This is a key-value map that can allow the query to alter some of the behavior of a query. It is primarily used for debugging, for example if you include `"bySegment":true` in the map, you will get results associated with the data segment they came from.|no| +|timeseries, groupBy, search|filter|Specifies the filter (the โ€œWHEREโ€ clause in SQL) for the query. See [[Filters]]|no| +|timeseries, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. โ€œhourโ€). See [[Granularities]] for more information.|no| +|groupBy|dimensions|constrains the groupings; if empty, then one value per time granularity bucket|yes| +|timeseries, groupBy|aggregations|aggregations that combine values in a bucket. See [[Aggregations]].|yes| +|timeseries, groupBy|postAggregations|aggregations of aggregations. See [[Post Aggregations]].|yes| +|search|limit|maximum number of results (default is 1000), a system-level maximum can also be set via `com.metamx.query.search.maxSearchLimit`|no| +|search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| +|search|query|The query portion of the search query. This is essentially a predicate that specifies if something matches.|yes| + +Additional Information about Query Types +---------------------------------------- + +[[TimeseriesQuery]] diff --git a/docs/Realtime.md b/docs/Realtime.md new file mode 100644 index 00000000000..1908a469f80 --- /dev/null +++ b/docs/Realtime.md @@ -0,0 +1,154 @@ +Realtime +======== + +Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data theyโ€™ve collected over some span of time and hand these segments off to [[Compute]] nodes. + +Running +------- + +Realtime nodes can be run using the `com.metamx.druid.realtime.RealtimeMain` class. + +Segment Propagation +------------------- + +The segment propagation diagram for real-time data ingestion can be seen below: + +![Segment Propagation](https://raw.github.com/metamx/druid/druid-0.5.4/doc/segment_propagation.png "Segment Propagation") + +Configuration +------------- + +Realtime nodes take a mix of base server configuration and spec files that describe how to connect, process and expose the realtime feed. See [[Configuration]] for information about general server configuration. + +### Realtime โ€œspecFileโ€ + +The property `druid.realtime.specFile` has the path of a file (absolute or relative path and file name) with realtime specifications in it. This โ€œspecFileโ€ should be a JSON Array of JSON objects like the following: + + + [{ + "schema" : { "dataSource":"dataSourceName", + "aggregators":[ {"type":"count", "name":"events"}, + {"type":"doubleSum","name":"outColumn","fieldName":"inColumn"} ], + "indexGranularity":"minute", + "shardSpec" : { "type": "none" } }, + "config" : { "maxRowsInMemory" : 500000, + "intermediatePersistPeriod" : "PT10m" }, + "firehose" : { "type" : "kafka-0.7.2", + "consumerProps" : { "zk.connect" : "zk_connect_string", + "zk.connectiontimeout.ms" : "15000", + "zk.sessiontimeout.ms" : "15000", + "zk.synctime.ms" : "5000", + "groupid" : "consumer-group", + "fetch.size" : "1048586", + "autooffset.reset" : "largest", + "autocommit.enable" : "false" }, + "feed" : "your_kafka_topic", + "parser" : { "timestampSpec" : { "column" : "timestamp", "format" : "iso" }, + "data" : { "format" : "json" }, + "dimensionExclusions" : ["value"] } }, + "plumber" : { "type" : "realtime", + "windowPeriod" : "PT10m", + "segmentGranularity":"hour", + "basePersistDirectory" : "/tmp/realtime/basePersist" } + }] + + +This is a JSON Array so you can give more than one realtime stream to a given node. The number you can put in the same process depends on the exact configuration. In general, it is best to think of each realtime stream handler as requiring 2-threads: 1 thread for data consumption and aggregation, 1 thread for incremental persists and other background tasks. + +There are four parts to a realtime stream specification, `schema`, `config`, `firehose` and `plumber` which we will go into here. + +#### Schema + +This describes the data schema for the output Druid segment. More information about concepts in Druid and querying can be found at [[Concepts-and-Terminology]] and [[Querying]]. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|aggregators|Array of Objects|The list of aggregators to use to aggregate colliding rows together.|yes| +|dataSource|String|The name of the dataSource that the segment belongs to.|yes| +|indexGranularity|String|The granularity of the data inside the segment. E.g. a value of โ€œminuteโ€ will mean that data is aggregated at minutely granularity. That is, if there are collisions in the tuple (minute(timestamp), dimensions), then it will aggregate values together using the aggregators instead of storing individual rows.|yes| +|segmentGranularity|String|The granularity of the segment as a whole. This is generally larger than the index granularity and describes the rate at which the realtime server will push segments out for historical servers to take over.|yes| +|shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a sharded fashion.|no| + +### Config + +This provides configuration for the data processing portion of the realtime stream processor. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|intermediatePersistPeriod|ISO8601 Period String|The period that determines the rate at which intermediate persists occur. These persists determine how often commits happen against the incoming realtime stream. If the realtime data loading process is interrupted at time T, it should be restarted to re-read data that arrived at T minus this period.|yes| +|maxRowsInMemory|Number|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size.|yes| + +### Firehose + +See [[Firehose]]. + +### Plumber + +See [[Plumber]] + +Constraints +----------- + +The following tables summarizes constraints between settings in the spec file for the Realtime subsystem. + +|*. Name |*. Effect |*. Minimum |*. Recommended | +| windowPeriod| when reading an InputRow, events with timestamp older than now minus this window are discarded | time jitter tolerance | use this to reject outliers | +| segmentGranularity| time granularity (minute, hour, day, week, month) for loading data at query time | equal to indexGranularity| more than indexGranularity| +| indexGranularity| time granularity (minute, hour, day, week, month) of indexes | less than segmentGranularity| minute, hour, day, week, month | +| intermediatePersistPeriod| the max real time (ISO8601 Period) between flushes of InputRows from memory to disk | avoid excessive flushing | number of un-persisted rows in memory also constrained by maxRowsInMemory | +| maxRowsInMemory| the max number of InputRows to hold in memory before a flush to disk | number of un-persisted post-aggregation rows in memory is also constrained by intermediatePersistPeriod | use this to avoid running out of heap if too many rows in an intermediatePersistPeriod | + +The normal, expected use cases have the following overall constraints: `indexGranularity < intermediatePersistPeriod =< windowPeriod < segmentGranularity` + +If the RealtimeNode process runs out of heap, try adjusting druid.computation.buffer.size property which specifies a size in bytes that must fit into the heap. + +Requirements +------------ + +Realtime nodes currently require a Kafka cluster to sit in front of them and collect results. Thereโ€™s more configuration required for these as well. + +Extending the code +------------------ + +Realtime integration is intended to be extended in two ways: + +1. Connect to data streams from varied systems ([Firehose](https://github.com/metamx/druid/blob/master/realtime/src/main/java/com/metamx/druid/realtime/FirehoseFactory.java)) +2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/master/realtime/src/main/java/com/metamx/druid/realtime/PlumberSchool.java)) + +The expectations are that the former will be very common and something that users of Druid will do on a fairly regular basis. Most users will probably never have to deal with the latter form of customization. Indeed, we hope that all potential use cases can be packaged up as part of Druid proper without requiring proprietary customization. + +Given those expectations, adding a firehose is straightforward and completely encapsulated inside of the interface. Adding a plumber is more involved and requires understanding of how the system works to get right, itโ€™s not impossible, but itโ€™s not intended that individuals new to Druid will be able to do it immediately. + +We will do our best to accept contributions from the community of new Firehoses and Plumbers, but we also understand the requirement for being able to plug in your own proprietary implementations. The model for doing this is by embedding the druid code in another project and writing your own `main()` method that initializes a RealtimeNode object and registers your proprietary objects with it. + + + public class MyRealtimeMain + { + private static final Logger log = new Logger(MyRealtimeMain.class); + + public static void main(String[] args) throws Exception + { + LogLevelAdjuster.register(); + + Lifecycle lifecycle = new Lifecycle(); + + lifecycle.addManagedInstance( + RealtimeNode.builder() + .build() + .registerJacksonSubtype(foo.bar.MyFirehose.class) + ); + + try { + lifecycle.start(); + } + catch (Throwable t) { + log.info(t, "Throwable caught at startup, committing seppuku"); + System.exit(2); + } + + lifecycle.join(); + } + } + + +Pluggable pieces of the system are either handled by a setter on the RealtimeNode object, or they are configuration driven and need to be setup to allow for [Jackson polymorphic deserialization](http://wiki.fasterxml.com/JacksonPolymorphicDeserialization) and registered via the relevant methods on the RealtimeNode object. diff --git a/docs/Rule-Configuration.md b/docs/Rule-Configuration.md new file mode 100644 index 00000000000..1d2b4c03461 --- /dev/null +++ b/docs/Rule-Configuration.md @@ -0,0 +1,77 @@ +Note: It is recommended that the master console is used to configure rules. However, the master node does have HTTP endpoints to programmatically configure rules. + +Load Rules +---------- + +Load rules indicate how many replicants of a segment should exist in a server tier. + +### Interval Load Rule + +Interval load rules are of the form: + + + { + "type" : "loadByInterval", + "interval" : "2012-01-01/2013-01-01", + "tier" : "hot" + } + + +type - this should always be โ€œloadByIntervalโ€ +interval - A JSON Object representing ISO-8601 Intervals +tier - the configured compute node tier + +### Period Load Rule + +Period load rules are of the form: + + + { + "type" : "loadByInterval", + "period" : "P1M", + "tier" : "hot" + } + + +type - this should always be โ€œloadByPeriodโ€ +period - A JSON Object representing ISO-8601 Periods +tier - the configured compute node tier + +The interval of a segment will be compared against the specified period. The rule matches if the period overlaps the interval. + +Drop Rules +---------- + +Drop rules indicate when segments should be dropped from the cluster. + +### Interval Drop Rule + +Interval drop rules are of the form: + + + { + "type" : "dropByInterval", + "interval" : "2012-01-01/2013-01-01" + } + + +type - this should always be โ€œdropByIntervalโ€ +interval - A JSON Object representing ISO-8601 Periods + +A segment is dropped if the interval contains the interval of the segment. + +### Period Drop Rule + +Period drop rules are of the form: + + + { + "type" : "dropByPeriod", + "period" : "P1M" + } + + +type - this should always be โ€œdropByPeriodโ€ +period - A JSON Object representing ISO-8601 Periods + +The interval of a segment will be compared against the specified period. The period is from some time in the past to the current time. The rule matches if the period contains the interval. diff --git a/docs/SearchQuery.md b/docs/SearchQuery.md new file mode 100644 index 00000000000..af125889c32 --- /dev/null +++ b/docs/SearchQuery.md @@ -0,0 +1,68 @@ +A search query returns dimension values that match the search specification. + + { + "queryType": "search", + "dataSource": "sample_datasource", + "granularity": "day", + "searchDimensions": [ + "dim1", + "dim2" + ], + "query": { + "type": "insensitive_contains", + "value": "Ke" + }, + "sort" : { + "type": "lexicographic" + }, + "intervals": [ + "2013-01-01T00:00:00.000/2013-01-03T00:00:00.000" + ] + } + + +There are several main parts to a search query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be โ€œsearchโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|granularity|Defines the granularity of the query. See [[Granularities]]|yes| +|filter|See [[Filters]]|no| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no| +|query|See [[SearchQuerySpec]].|yes| +|sort|How the results of the search should sorted. Two possible types here are โ€œlexicographicโ€ and โ€œstrlenโ€.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +The format of the result is: + + [ + { + "timestamp": "2012-01-01T00:00:00.000Z", + "result": [ + { + "dimension": "dim1", + "value": "Ke$ha" + }, + { + "dimension": "dim2", + "value": "Ke$haForPresident" + } + ] + }, + { + "timestamp": "2012-01-02T00:00:00.000Z", + "result": [ + { + "dimension": "dim1", + "value": "SomethingThatContainsKe" + }, + { + "dimension": "dim2", + "value": "SomethingElseThatContainsKe" + } + ] + } + ] + diff --git a/docs/SearchQuerySpec.md b/docs/SearchQuerySpec.md new file mode 100644 index 00000000000..48036c65d56 --- /dev/null +++ b/docs/SearchQuerySpec.md @@ -0,0 +1,23 @@ +Search query specs define how a โ€œmatchโ€ is defined between a search value and a dimension value. The available search query specs are: + +InsensitiveContainsSearchQuerySpec +---------------------------------- + +If any part of a dimension value contains the value specified in this search query spec, regardless of case, a โ€œmatchโ€ occurs. The grammar is: + + { + "type" : "insensitive_contains", + "value" : "some_value" + } + + +FragmentSearchQuerySpec +----------------------- + +If any part of a dimension value contains any of the values specified in this search query spec, regardless of case, a โ€œmatchโ€ occurs. The grammar is: + + { + "type" : "fragment", + "values" : ["fragment1", "fragment2"] + } + diff --git a/docs/SegmentMetadataQuery.md b/docs/SegmentMetadataQuery.md new file mode 100644 index 00000000000..606d0800447 --- /dev/null +++ b/docs/SegmentMetadataQuery.md @@ -0,0 +1,55 @@ +Segment metadata queries return per segment information about: +\* Cardinality of all columns in the segment +\* Estimated byte size for the segment columns in TSV format +\* Interval the segment covers +\* Column type of all the columns in the segment +\* Estimated total segment byte size in TSV format +\* Segment id + + { + "queryType":"segmentMetadata", + "dataSource":"sample_datasource", + "intervals":["2013-01-01/2014-01-01"], + } + + +There are several main parts to a segment metadata query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be โ€œsegmentMetadataโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|merge|Merge all individual segment metadata results into a single result|no| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +The format of the result is: + + [ { + "id" : "some_id", + "intervals" : [ "2013-05-13T00:00:00.000Z/2013-05-14T00:00:00.000Z" ], + "columns" : { + "__time" : { + "type" : "LONG", + "size" : 407240380, + "cardinality" : null + }, + "dim1" : { + "type" : "STRING", + "size" : 100000, + "cardinality" : 1944 + }, + "dim2" : { + "type" : "STRING", + "size" : 100000, + "cardinality" : 1504 + }, + "metric1" : { + "type" : "FLOAT", + "size" : 100000, + "cardinality" : null + } + }, + "size" : 300000 + } ] + diff --git a/docs/Segments.md b/docs/Segments.md new file mode 100644 index 00000000000..5bffdd30b10 --- /dev/null +++ b/docs/Segments.md @@ -0,0 +1,60 @@ +Segments +======== + +Segments are the fundamental structure to store data in Druid. [[Compute]] and [[Realtime]] nodes load and serve segments for querying. To construct segments, Druid will always shard data by a time partition. Data may be further sharded based on dimension cardinality and row count. + +The latest Druid segment version is `v9`. + +Naming Convention +----------------- + +Identifiers for segments are typically constructed using the segment datasource, interval start time (in ISO 8601 format), interval end time (in ISO 8601 format), and a version. If data is additionally sharded beyond a time range, the segment identifier will also contain a partition number. + +An example segment identifier may be: +datasource\_intervalStart\_intervalEnd\_version\_partitionNum + +Segment Components +------------------ + +A segment is compromised of several files, listed below. + +### `version.bin` + +4 bytes representing the current segment version as an integer. E.g., for v9 segments, the version is 0x0, 0x0, 0x0, 0x9 + +### `meta.smoosh` + +A file with metadata (filenames and offsets) about the contents of the other `smoosh` files + +### `XXXXX.smoosh` + +There are some number of these files, which are concatenated binary data + +The `smoosh` files represent multiple files โ€œsmooshedโ€ together in order to minimize the number of file descriptors that must be open to house the data. They are files of up to 2GB in size (to match the limit of a memory mapped ByteBuffer in Java). The `smoosh` files house individual files for each of the columns in the data as well as an `index.drd` file with extra metadata about the segment. + +There is also a special column called `__time` that refers to the time column of the segment. This will hopefully become less and less special as the code evolves, but for now itโ€™s as special as my Mommy always told me I am. + +### `index.drd` + +The `index.drd` file houses 3 pieces of data in order + +1. The names of all of the columns of the data +2. The names of the โ€œdimensionsโ€ of the data (these are the dictionary-encoded, string columns. This is here to support some legacy APIs and will be superfluous in the future) +3. The data interval represented by this segment stored as the start and end timestamps as longs + +Format of a column +------------------ + +Each column is stored as two parts: + +1. A Jackson-serialized ColumnDescriptor +2. The rest of the binary for the column + +A ColumnDescriptor is essentially an object that allows us to use jacksonโ€™s polymorphic deserialization to add new and interesting methods of serialization with minimal impact to the code. It consists of some metadata about the column (what type is it, is it multi-valued, etc.) and then a list of serde logic that can deserialize the rest of the binary. + +Sharding Data to Create Segments +-------------------------------- + +### Sharding Data by Dimension + +If the cumulative total number of rows for the different values of a given column exceed some configurable threshold, multiple segments representing the same time interval for the same datasource may be created. These segments will contain some partition number as part of their identifier. Sharding by dimension reduces some of the the costs associated with operations over high cardinality dimensions. diff --git a/docs/Spatial-Filters.md b/docs/Spatial-Filters.md new file mode 100644 index 00000000000..c9ce15d5cc9 --- /dev/null +++ b/docs/Spatial-Filters.md @@ -0,0 +1,32 @@ +Note: This feature is highly experimental and only works with spatially indexed dimensions. + +The grammar for a spatial filter is as follows: + + + { + "dimension": "spatialDim", + "bound": { + "type": "rectangular", + "minCoords": [10.0, 20.0], + "maxCoords": [30.0, 40.0] + } + } + + +Bounds +------ + +### Rectangular + +|property|description|required?| +|--------|-----------|---------| +|minCoords|List of minimum dimension coordinates for coordinates [x, y, z, โ€ฆ]|yes| +|maxCoords|List of maximum dimension coordinates for coordinates [x, y, z, โ€ฆ]|yes| + +### Radius + +|property|description|required?| +|--------|-----------|---------| +|coords|Origin coordinates in the form [x, y, z, โ€ฆ]|yes| +|radius|The float radius value|yes| + diff --git a/docs/Spatial-Indexing.md b/docs/Spatial-Indexing.md new file mode 100644 index 00000000000..5f7dc2b174c --- /dev/null +++ b/docs/Spatial-Indexing.md @@ -0,0 +1,23 @@ +Note: This feature is highly experimental. + +In any of the data specs, there is now the option of providing spatial dimensions. For example, for a JSON data spec, spatial dimensions can be specified as follows: + + + { + "type": "JSON", + "dimensions": , + "spatialDimensions": [ + { + "dimName": "coordinates", + "dims": ["lat", "long"] + }, + ... + ] + } + + +|property|description|required?| +|--------|-----------|---------| +|dimName|The name of the spatial dimension. A spatial dimension may be constructed from multiple other dimensions or it may already exist as part of an event. If a spatial dimension already exists, it must be an array of dimension values.|yes| +|dims|A list of dimension names that comprise a spatial dimension.|no| + diff --git a/docs/Stand-Alone-With-Riak-CS.md b/docs/Stand-Alone-With-Riak-CS.md new file mode 100644 index 00000000000..aaa77b3151c --- /dev/null +++ b/docs/Stand-Alone-With-Riak-CS.md @@ -0,0 +1,223 @@ +This page describes how to use Riak-CS for deep storage instead of S3. We are still setting up some of the peripheral stuff (file downloads, etc.). + +This guide provided by Pablo Nebrera, thanks! + +## The VMWare instance + +A VMWare [image](http://static.druid.io/artifacts/vmware/druid_riak.tgz) based on Druid 0.3.27.2 and built according to the instructions below has also been provided by Pablo Nebrera. + +The provided vmware machine has access with the following credentials: + + username: root + password: riakdruid + + +## The Setup + +We started with a minimal CentOS installation but you can use any other compatible installation. At the end of this setup you will one node that is running: + +1. A Kafka Broker +1. A single-node Zookeeper ensemble +1. A single-node Riak-CS cluster +1. A Druid [[Master]] +1. A Druid [[Broker]] +1. A Druid [[Compute]] +1. A Druid [[Realtime]] + +This just walks through getting the relevant software installed and running. You will then need to configure the [[Realtime]] node to take in your data. + +### Configure System + +1. Install `CentOS-6.4-x86_64-minimal.iso` ("RedHat v6.4" is the name of the AWS AMI) or your favorite Linux OS (if you use a different OS, some of the installation instructions for peripheral services might differ, please adjust them according to the system you are using). The rest of these instructions assume that you have a running instance and are running as the root user. + +1. Configure the network. We used dhcp executing: + + dhclient eth0 + +1. Disable firewall for now + + service iptables stop + chkconfig iptables off + +1. Change the limits on the number of open files a process can have: + + cat >> /etc/security/limits.conf <<- _RBEOF_ + # ulimit settings for Riak CS + root soft nofile 65536 + root hard nofile 65536 + riak soft nofile 65536 + riak hard nofile 65536 + _RBEOF_ + + ulimit -n 65536 + +### Install base software packages + +1. Install necessary software with yum + + yum install -y java-1.7.0-openjdk-devel git wget mysql-server + +1. Install maven + + wget http://apache.rediris.es/maven/maven-3/3.0.5/binaries/apache-maven-3.0.5-bin.tar.gz + tar xzf apache-maven-3.0.5-bin.tar.gz -C /usr/local + pushd /usr/local + sudo ln -s apache-maven-3.0.5 maven + popd + echo 'export M2_HOME=/usr/local/maven' >> /etc/profile.d/maven.sh + echo 'export PATH=${M2_HOME}/bin:${PATH}' >> /etc/profile.d/maven.sh + source /etc/profile.d/maven.sh + +1. Install erlang + + wget http://binaries.erlang-solutions.com/rpm/centos/6/x86_64/esl-erlang-R15B01-1.x86_64.rpm + yum localinstall -y esl-erlang-R15B01-1.x86_64.rpm + +### Install Kafka And Zookeeper + +1. Install kafka and zookeeper: + + wget http://apache.rediris.es/incubator/kafka/kafka-0.7.2-incubating/kafka-0.7.2-incubating-src.tgz + tar zxvf kafka-0.7.2-incubating-src.tgz + pushd kafka-0.7.2-incubating-src/ + ./sbt update + ./sbt package + mkdir -p /var/lib/kafka + rsync -a * /var/lib/kafka/ + popd + +### Install Riak-CS + +1. Install s3cmd to manage riak s3 + + wget http://downloads.sourceforge.net/project/s3tools/s3cmd/1.5.0-alpha3/s3cmd-1.5.0-alpha3.tar.gz + tar xzvf s3cmd-1.5.0-alpha3.tar.gz + cd s3cmd-1.5.0-alpha3 + cp -r s3cmd S3 /usr/local/bin/ + +1. Install riak, riak-cs and stanchion. Note: riak-cs-control is optional + + wget http://s3.amazonaws.com/downloads.basho.com/riak/1.3/1.3.1/rhel/6/riak-1.3.1-1.el6.x86_64.rpm + wget http://s3.amazonaws.com/downloads.basho.com/riak-cs/1.3/1.3.1/rhel/6/riak-cs-1.3.1-1.el6.x86_64.rpm + wget http://s3.amazonaws.com/downloads.basho.com/stanchion/1.3/1.3.1/rhel/6/stanchion-1.3.1-1.el6.x86_64.rpm + wget http://s3.amazonaws.com/downloads.basho.com/riak-cs-control/1.0/1.0.0/rhel/6/riak-cs-control-1.0.0-1.el6.x86_64.rpm + yum localinstall -y riak-*.rpm stanchion-*.rpm + +### Install Druid + +1. Clone the git repository for druid, checkout a "stable" tag and build + + git clone https://github.com/metamx/druid.git druid + pushd druid + git checkout druid-0.4.12 + export LANGUAGE=C + export LC_MESSAGE=C + export LC_ALL=C + export LANG=en_US + ./build.sh + mkdir -p /var/lib/druid/app + cp ./services/target/druid-services-*-selfcontained.jar /var/lib/druid/app + ln -s /var/lib/druid/app/druid-services-*-selfcontained.jar /var/lib/druid/app/druid-services.jar + popd + + +### Configure stuff + +1. Add this line to /etc/hosts + + echo "127.0.0.1 s3.amazonaws.com bucket.s3.amazonaws.com `hostname`" >> /etc/hosts + + NOTE: the bucket name in this case is "bucket", but you might need to update it to your bucket name if you want to use a different bucket name. + +1. Download and extract run scripts and configuration files: + + wget http://static.druid.io/artifacts/scripts/druid_scripts_nebrera.tar / + pushd / + tar xvf ~/druid_scripts_nebrera.tar + popd + + +1. Start Riak in order to create a user: + + /etc/init.d/riak start + /etc/init.d/riak-cs start + /etc/init.d/stanchion start + + You can check riak status using: + + riak-admin member-status + + You should expect results like + + Attempting to restart script through sudo -H -u riak + ================================= Membership ================================== + Status Ring Pending Node + ------------------------------------------------------------------------------- + valid 100.0% -- 'riak@127.0.0.1' + ------------------------------------------------------------------------------- + Valid:1 / Leaving:0 / Exiting:0 / Joining:0 / Down:0 + + +1. Create riak-cs user and yoink out credentials. + + curl -H 'Content-Type: application/json' -X POST http://127.0.0.1:8088/riak-cs/user --data '{"email":"example@domain.com", "name":"admin"}' >> /tmp/riak_user.json + export RIAK_KEY_ID=`sed 's/^.*"key_id":"//' /tmp/riak_user.json | cut -d '"' -f 1` + export RIAK_KEY_SECRET=`sed 's/^.*"key_secret":"//' /tmp/riak_user.json | cut -d '"' -f 1` + sed -i "s/<%=[ ]*@key_id[ ]*%>/${RIAK_KEY_ID}/" /etc/riak-cs/app.config /etc/riak-cs-control/app.config /etc/stanchion/app.config /etc/druid/config.sh /etc/druid/base.properties /root/.s3cfg + sed -i "s/<%=[ ]*@key_secret[ ]*%>/${RIAK_KEY_SECRET}/" /etc/riak-cs/app.config /etc/riak-cs-control/app.config /etc/stanchion/app.config /etc/druid/config.sh /etc/druid/base.properties /root/.s3cfg + + This will store the result of creating the user into `/tmp/riak_user.json`. You can look at it if you are interested. It will look something like this + + {"email":"example@domain.com", + "display_name":"example", + "name":"admin", + "key_id":"DOXKZYR_QM2S-7HSKAEU", + "key_secret":"GtvVJow068RM-_viHIYR9DWMAXsFcL1SmjuNfA==", + "id":"4c5b5468c180f3efafd531b6cd8e2bb24371d99640aad5ced5fbbc0604fc473d", + "status":"enabled"} + +1. Stop riak-cs: + + /etc/init.d/riak-cs stop + /etc/init.d/stanchion stop + /etc/init.d/riak stop + +1. Disable anonymous user creation + + sed 's/{[ ]*anonymous_user_creation[ ]*,[ ]*true[ ]*}/{anonymous_user_creation, false}/' /etc/riak-cs/app.config |grep anonymous_user_creation + +1. Restart riak-cs services: + + /etc/init.d/riak start + /etc/init.d/riak-cs start + /etc/init.d/stanchion start + + +1. Create your bucket. The example name and in config files is "bucket" + + s3cmd mb s3://bucket + + You can verify that the bucket is created with: + + s3cmd ls + +1. Start MySQL server + + service mysqld start + chkconfig mysqld on + /usr/bin/mysqladmin -u root password 'riakdruid' + + NOTE: If you don't like "riakdruid" as your password, feel free to change it around. + NOTE: If you have used root user to connect to database. It should be changed by other user but I have used this one to simplify it + +1. Start zookeeper and kafka + + /etc/init.d/zookeeper start + /etc/init.d/kafka start + +1. Start druid + + /etc/init.d/druid_master start + /etc/init.d/druid_realtime start + /etc/init.d/druid_broker start + /etc/init.d/druid_compute start \ No newline at end of file diff --git a/docs/Support.md b/docs/Support.md new file mode 100644 index 00000000000..1561e935381 --- /dev/null +++ b/docs/Support.md @@ -0,0 +1,13 @@ +Numerous backend engineers at [Metamarkets](http://www.metamarkets.com) work on Druid full-time. If you any questions about usage or code, feel free to contact any of us. + +Google Groups Mailing List +-------------------------- + +The best place for questions is through our mailing list: +[druid-development@googlegroups.com](mailto:druid-development@googlegroups.com) +[https://groups.google.com/d/forum/druid-development](https://groups.google.com/d/forum/druid-development) + +IRC +--- + +Several of us also hang out in the channel \#druid-dev on irc.freenode.net. diff --git a/docs/Tasks.md b/docs/Tasks.md new file mode 100644 index 00000000000..53f441696d9 --- /dev/null +++ b/docs/Tasks.md @@ -0,0 +1,68 @@ +Tasks are run on workers and always operate on a single datasource. Once an indexer coordinator node accepts a task, a lock is created for the datasource and interval specified in the task. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a โ€œtask groupโ€, which is a set of tasks that can share interval locks. + +There are several different types of tasks. + +Append Task +----------- + +Append tasks append a list of segments together into a single segment (one after the other). The grammar is: + + { + "id": , + "dataSource": , + "segments": + } + +Merge Task +---------- + +Merge tasks merge a list of segments together. Any common timestamps are merged. The grammar is: + + { + "id": , + "dataSource": , + "segments": + } + +Delete Task +----------- + +Delete tasks create empty segments with no data. The grammar is: + + { + "id": , + "dataSource": , + "segments": + } + +Kill Task +--------- + +Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is: + + { + "id": , + "dataSource": , + "segments": + } + +Index Task +---------- + +Index Partitions Task +--------------------- + +Index Generator Task +-------------------- + +Index Hadoop Task +----------------- + +Index Realtime Task +------------------- + +Version Converter Task +---------------------- + +Version Converter SubTask +------------------------- diff --git a/docs/Thanks.md b/docs/Thanks.md new file mode 100644 index 00000000000..f84708fb6c8 --- /dev/null +++ b/docs/Thanks.md @@ -0,0 +1,8 @@ +YourKit supports the Druid open source projects with its +full-featured Java Profiler. +YourKit, LLC is the creator of innovative and intelligent tools for profiling +Java and .NET applications. Take a look at YourKit's software products: +
YourKit Java +Profiler and +YourKit .NET +Profiler. \ No newline at end of file diff --git a/docs/TimeBoundaryQuery.md b/docs/TimeBoundaryQuery.md new file mode 100644 index 00000000000..432df69961d --- /dev/null +++ b/docs/TimeBoundaryQuery.md @@ -0,0 +1,26 @@ +Time boundary queries return the earliest and latest data points of a data set. The grammar is: + + { + "queryType" : "timeBoundary", + "dataSource": "sample_datasource" + } + + +There are 3 main parts to a time boundary query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be โ€œtimeBoundaryโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +The format of the result is: + + [ { + "timestamp" : "2013-05-09T18:24:00.000Z", + "result" : { + "minTime" : "2013-05-09T18:24:00.000Z", + "maxTime" : "2013-05-09T18:37:00.000Z" + } + } ] + diff --git a/docs/TimeseriesQuery.md b/docs/TimeseriesQuery.md new file mode 100644 index 00000000000..d189b176a01 --- /dev/null +++ b/docs/TimeseriesQuery.md @@ -0,0 +1,115 @@ +Timeseries queries +================== + +These types of queries take a timeseries query object and return an array of JSON objects where each object represents a value asked for by the timeseries query. + +An example timeseries query object is shown below: + +
+
+{
+ [queryType]() โ€œtimeseriesโ€,
+ [dataSource]() โ€œsample\_datasourceโ€,
+ [granularity]() โ€œdayโ€,
+ [filter]() {
+ [type]() โ€œandโ€,
+ [fields]() [
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension1โ€,
+ [value]() โ€œsample\_value1โ€
+ },
+ {
+ [type]() โ€œorโ€,
+ [fields]() [
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension2โ€,
+ [value]() โ€œsample\_value2โ€
+ },
+ {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œsample\_dimension3โ€,
+ [value]() โ€œsample\_value3โ€
+ }
+ ]
+ }
+ ]
+ },
+ [aggregations]() [
+ {
+ [type]() โ€œlongSumโ€,
+ [name]() โ€œsample\_name1โ€,
+ [fieldName]() โ€œsample\_fieldName1โ€
+ },
+ {
+ [type]() โ€œdoubleSumโ€,
+ [name]() โ€œsample\_name2โ€,
+ [fieldName]() โ€œsample\_fieldName2โ€
+ }
+ ],
+ [postAggregations]() [
+ {
+ [type]() โ€œarithmeticโ€,
+ [name]() โ€œsample\_divideโ€,
+ [fn]() โ€œ/โ€,
+ [fields]() [
+ {
+ [type]() โ€œfieldAccessโ€,
+ [name]() โ€œsample\_name1โ€,
+ [fieldName]() โ€œsample\_fieldName1โ€
+ },
+ {
+ [type]() โ€œfieldAccessโ€,
+ [name]() โ€œsample\_name2โ€,
+ [fieldName]() โ€œsample\_fieldName2โ€
+ }
+ ]
+ }
+ ],
+ [intervals]() [
+ โ€œ2012-01-01T00:00:00.000/2012-01-03T00:00:00.000โ€
+ ]
+}
+
+
+ + +There are 7 main parts to a timeseries query: + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be โ€œtimeseriesโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| +|granularity|Defines the granularity of the query. See [[Granularities]]|yes| +|filter|See [[Filters]]|no| +|aggregations|See [[Aggregations]]|yes| +|postAggregations|See [[Post Aggregations]]|no| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| + +To pull it all together, the above query would return 2 data points, one for each day between 2012-01-01 and 2012-01-03, from the โ€œsample\_datasourceโ€ table. Each data point would be the (long) sum of sample\_fieldName1, the (double) sum of sample\_fieldName2 and the (double) the result of sample\_fieldName1 divided by sample\_fieldName2 for the filter set. The output looks like this: + +
+
+[
+ {
+ [timestamp]() โ€œ2012-01-01T00:00:00.000Zโ€,
+ [result]() {
+ [sample\_name1]() ,
+ [sample\_name2]() ,
+ [sample\_divide]() 
+ }
+ },
+ {
+ [timestamp]() โ€œ2012-01-02T00:00:00.000Zโ€,
+ [result]() {
+ [sample\_name1]() ,
+ [sample\_name2]() ,
+ [sample\_divide]() 
+ }
+ }
+]
+
+
+ diff --git a/docs/Tutorial:-A-First-Look-at-Druid.md b/docs/Tutorial:-A-First-Look-at-Druid.md new file mode 100644 index 00000000000..ef725135aa4 --- /dev/null +++ b/docs/Tutorial:-A-First-Look-at-Druid.md @@ -0,0 +1,366 @@ +Greetings! This tutorial will help clarify some core Druid concepts. We will use a realtime dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on! + +About the data +-------------- + +The data source weโ€™ll be working with is Wikipedia edits. Each time an edit is made in Wikipedia, an event gets pushed to an IRC channel associated with the language of the Wikipedia page. We scrape IRC channels for several different languages and load this data into Druid. + +Each event has a timestamp indicating the time of the edit (in UTC time), a list of dimensions indicating various metadata about the event (such as information about the user editing the page and where the user resides), and a list of metrics associated with the event (such as the number of characters added and deleted). + +Specifically. the data schema looks like so: + +Dimensions (things to filter on): +\`\`\`json +โ€œpageโ€ +โ€œlanguageโ€ +โ€œuserโ€ +โ€œunpatrolledโ€ +โ€œnewPageโ€ +โ€œrobotโ€ +โ€œanonymousโ€ +โ€œnamespaceโ€ +โ€œcontinentโ€ +โ€œcountryโ€ +โ€œregionโ€ +โ€œcityโ€ +\`\`\` + +Metrics (things to aggregate over): +\`\`\`json +โ€œcountโ€ +โ€œaddedโ€ +โ€œdeltaโ€ +โ€œdeletedโ€ +\`\`\` + +These metrics track the number of characters added, deleted, and changed. + +Setting Up +---------- + +There are two ways to setup Druid: download a tarball, or [[Build From Source]]. You only need to do one of these. + +### Download a Tarball + +Weโ€™ve built a tarball that contains everything youโ€™ll need. Youโ€™ll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.5.54-bin.tar.gz) +Download this file to a directory of your choosing. + +You can extract the awesomeness within by issuing: + + tar -zxvf druid-services-*-bin.tar.gz + +Not too lost so far right? Thatโ€™s great! If you cd into the directory: + + cd druid-services-0.5.54 + +You should see a bunch of files: +\* run\_example\_server.sh +\* run\_example\_client.sh +\* LICENSE, config, examples, lib directories + +Running Example Scripts +----------------------- + +Letโ€™s start doing stuff. You can start a Druid [[Realtime]] node by issuing: + + ./run_example_server.sh + +Select โ€œwikipediaโ€. + +Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. + + + 2013-07-19 21:54:05,154 INFO [main] com.metamx.druid.realtime.RealtimeNode - Starting Jetty + 2013-07-19 21:54:05,154 INFO [main] org.mortbay.log - jetty-6.1.x + 2013-07-19 21:54:05,171 INFO [chief-wikipedia] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Expect to run at [2013-07-19T22:03:00.000Z] + 2013-07-19 21:54:05,246 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8083 + + +The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. If you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf). + +Okay, things are about to get real(~~time). To query the real-time node youโ€™ve spun up, you can issue: +\./run\_example\_client.sh\ +Select โ€œwikipediaโ€ once again. This script issues ]s to the data weโ€™ve been ingesting. The query looks like this: +\`\`\`json +{ + [queryType]("groupBy"), + [dataSource]("wikipedia"), + [granularity]("minute"), + [dimensions]([) + โ€œpageโ€ + ], + [aggregations]([) + { + [type]("count"), + [name]("rows") + }, + { + [type]("longSum"), + [fieldName]("edit_count"), + [name]("count") + } + ], + [filter]({) + [type]("selector"), + [dimension]("namespace"), + [value]("article") + }, + [intervals]([) + โ€œ2013-06-01T00:00/2020-01-01T00โ€ + ] +} +\`\`\` +This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: . We are **filtering** via the **โ€œnamespaceโ€** dimension, to only look at edits on **โ€œarticlesโ€**. Our **aggregations** are what we are calculating: a count of the number of data rows, and a count of the number of edits that have occurred. +The result looks something like this: +\`\`\`json +[ + { + [version]() โ€œv1โ€, + [timestamp]() โ€œ2013-09-04T21:44:00.000Zโ€, + [event]() { + [count]() 0, + [page]() โ€œ2013\\u201314\_Brentford\_F.C.*season", + [rows]() 1 + } + }, + { + [version]() "v1", + [timestamp]() "2013-09-04T21:44:00.000Z", + [event]() { + [count]() 0, + [page]() "8e*00e9tape\_du\_Tour\_de\_France\_2013โ€, + [rows]() 1 + } + }, + { + [version]() โ€œv1โ€, + [timestamp]() โ€œ2013-09-04T21:44:00.000Zโ€, + [event]() { + [count]() 0, + [page]() โ€œAgenda\_of\_the\_Tea\_Party\_movementโ€, + [rows]() 1 + } + }, +โ€ฆ +\`\`\` +This groupBy query is a bit complicated and weโ€™ll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script. +h2. Querying Druid +In your favorite editor, create the file: +\time\_boundary\_query.body\ +Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created: +\ +{ + [queryType]() โ€œtimeBoundaryโ€, + [dataSource]() โ€œwikipediaโ€ +} +\ +The ] is one of the simplest Druid queries. To run the query, you can issue: +\ curl~~X POST โ€˜http://localhost:8083/druid/v2/?prettyโ€™ ~~H โ€˜content-type: application/jsonโ€™~~d ```` time_boundary_query.body + +We get something like this JSON back: + +```json +[ { + "timestamp" : "2013-09-04T21:44:00.000Z", + "result" : { + "minTime" : "2013-09-04T21:44:00.000Z", + "maxTime" : "2013-09-04T21:47:00.000Z" + } +} ] +``` +As you can probably tell, the result is indicating the maximum and minimum timestamps we've seen thus far (summarized to a minutely granularity). Let's explore a bit further. + +Return to your favorite editor and create the file: +
timeseries_query.body
+ +We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +

+{
+    "queryType": "timeseries", 
+    "dataSource": "wikipedia", 
+    "intervals": [
+        "2010-01-01/2020-01-01"
+    ], 
+    "granularity": "all", 
+    "aggregations": [
+        {
+            "type": "longSum", 
+            "fieldName": "count",
+            "name": "edit_count"
+        }, 
+        {
+            "type": "doubleSum", 
+            "fieldName": "added", 
+            "name": "chars_added"
+        }
+    ]
+}
+
+ +You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +To issue the query and get some results, run the following in your command line: +
curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
+
+
+Once again, you should get a JSON blob of text back with your results, that looks something like this: + +\`\`\`json +[ { + โ€œtimestampโ€ : โ€œ2013-09-04T21:44:00.000Zโ€, + โ€œresultโ€ : { + โ€œchars\_addedโ€ : 312670.0, + โ€œedit\_countโ€ : 733 + } +} ] +\`\`\` + +If you issue the query again, you should notice your results updating. + +Right now all the results you are getting back are being aggregated into a single timestamp bucket. What if we wanted to see our aggregations on a per minute basis? What field can we change in the query to accomplish this? + +If you loudly exclaimed โ€œwe can change granularity to minuteโ€, you are absolutely correct! We can specify different granularities to bucket our results, like so: + + + { + "queryType": "timeseries", + "dataSource": "wikipedia", + "intervals": [ + "2010-01-01/2020-01-01" + ], + "granularity": "minute", + "aggregations": [ + { + "type": "longSum", + "fieldName": "count", + "name": "edit_count" + }, + { + "type": "doubleSum", + "fieldName": "added", + "name": "chars_added" + } + ] + } + + +This gives us something like the following: + +\`\`\`json +[ + { + โ€œtimestampโ€ : โ€œ2013-09-04T21:44:00.000Zโ€, + โ€œresultโ€ : { + โ€œchars\_addedโ€ : 30665.0, + โ€œedit\_countโ€ : 128 + } + }, { + โ€œtimestampโ€ : โ€œ2013-09-04T21:45:00.000Zโ€, + โ€œresultโ€ : { + โ€œchars\_addedโ€ : 122637.0, + โ€œedit\_countโ€ : 167 + } + }, { + โ€œtimestampโ€ : โ€œ2013-09-04T21:46:00.000Zโ€, + โ€œresultโ€ : { + โ€œchars\_addedโ€ : 78938.0, + โ€œedit\_countโ€ : 159 + } + }, +โ€ฆ +\`\`\` + +Solving a Problem +----------------- + +One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top pages in the US are, ordered by the number of edits over the last few minutes youโ€™ve been going through this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can! + +Letโ€™s create the file: + + group_by_query.body + and put the following in there: +

+    {
+        "queryType": "groupBy", 
+        "dataSource": "wikipedia", 
+        "granularity": "all", 
+        "dimensions": [
+            "page"
+        ], 
+        "orderBy": {
+            "type": "default", 
+            "columns": [
+                {
+                    "dimension": "edit_count", 
+                    "direction": "DESCENDING"
+                }
+            ], 
+            "limit": 10
+        }, 
+        "aggregations": [
+            {
+                "type": "longSum", 
+                "fieldName": "count", 
+                "name": "edit_count"
+            }
+        ], 
+        "filter": {
+            "type": "selector", 
+            "dimension": "country", 
+            "value": "United States"
+        }, 
+        "intervals": [
+            "2012-10-01T00:00/2020-01-01T00"
+        ]
+    }
+    
+
+Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+
+If you issue the query:
+
+    curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d @group_by_query.body
+
+You should see an answer to our question. As an example, some results are shown below:
+
+\`\`\`json
+[
+ {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œpageโ€ : โ€œRTC\_Transitโ€,
+ โ€œedit\_countโ€ : 6
+ }
+ }, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œpageโ€ : โ€œList\_of\_Deadly\_Women\_episodesโ€,
+ โ€œedit\_countโ€ : 4
+ }
+ }, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œpageโ€ : โ€œUser\_talk:David\_Biddulphโ€,
+ โ€œedit\_countโ€ : 4
+ }
+ },
+โ€ฆ
+\`\`\`
+
+Feel free to tweak other query parameters to answer other questions you may have about the data.
+
+Next Steps
+----------
+
+What to know even more information about the Druid Cluster? Check out [[Tutorial: The Druid Cluster]]
+
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [[Loading Your Data]].
+
+Additional Information
+----------------------
+
+This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki.
+
+And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
diff --git a/docs/Tutorial:-The-Druid-Cluster.md b/docs/Tutorial:-The-Druid-Cluster.md
new file mode 100644
index 00000000000..b01824e52a2
--- /dev/null
+++ b/docs/Tutorial:-The-Druid-Cluster.md
@@ -0,0 +1,299 @@
+Welcome back! In our first [tutorial](https://github.com/metamx/druid/wiki/Tutorial%3A-A-First-Look-at-Druid), we introduced you to the most basic Druid setup: a single realtime node. We streamed in some data and queried it. Realtime nodes collect very recent data and periodically hand that data off to the rest of the Druid cluster. Some questions about the architecture must naturally come to mind. What does the rest of Druid cluster look like? How does Druid load available static data?
+
+This tutorial will hopefully answer these questions!
+
+In this tutorial, we will set up other types of Druid nodes as well as and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system.
+
+## Downloading Druid ##
+
+If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first.
+
+You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.5.54-bin.tar.gz)
+
+and untar the contents within by issuing:
+```bash
+tar -zxvf druid-services-*-bin.tar.gz
+cd druid-services-*
+```
+
+You can also [[Build From Source]].
+
+## External Dependencies ##
+
+Druid requires 3 external dependencies. A "deep" storage that acts as a backup data repository, a relational database such as MySQL to hold configuration and metadata information, and [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster.
+
+For deep storage, we have made a public S3 bucket (static.druid.io) available where data for this particular tutorial can be downloaded. More on the data [later](https://github.com/metamx/druid/wiki/Tutorial-Part-2#the-data).
+
+### Setting up MySQL ###
+
+1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
+2. Install MySQL
+3. Create a druid user and database
+```bash
+mysql -u root
+```
+```sql
+GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd';
+CREATE database druid;
+```
+
+### Setting up Zookeeper ###
+```bash
+curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
+tar xzf zookeeper-3.4.5.tar.gz
+cd zookeeper-3.4.5
+cp conf/zoo_sample.cfg conf/zoo.cfg
+./bin/zkServer.sh start
+cd ..
+```
+
+## The Data ##
+
+Similar to the first tutorial, the data we will be loading is based on edits that have occurred on Wikipedia. Every time someone edits a page in Wikipedia, metadata is generated about the editor and edited page. Druid collects each individual event and packages them together in a container known as a [segment](https://github.com/metamx/druid/wiki/Segments). Segments contain data over some span of time. We've prebuilt a segment for this tutorial and will cover making your own segments in other [pages](https://github.com/metamx/druid/wiki/Loading-Your-Data).The segment we are going to work with has the following format:
+
+Dimensions (things to filter on):
+```json
+"page"
+"language"
+"user"
+"unpatrolled"
+"newPage"
+"robot"
+"anonymous"
+"namespace"
+"continent"
+"country"
+"region"
+"city"
+```
+
+Metrics (things to aggregate over):
+```json
+"count"
+"added"
+"delta"
+"deleted"
+```
+
+## The Cluster ##
+
+Let's start up a few nodes and download our data. First things though, let's create a config directory where we will store configs for our various nodes:
+
+```
+mkdir config
+```
+
+If you are interested in learning more about Druid configuration files, check out this [link](https://github.com/metamx/druid/wiki/Configuration). Many aspects of Druid are customizable. For the purposes of this tutorial, we are going to use default values for most things.
+
+### Start a Master Node ###
+
+Master nodes are in charge of load assignment and distribution. Master nodes monitor the status of the cluster and command compute nodes to assign and drop segments.
+
+To create the master config file:
+
+```
+mkdir config/master
+```
+
+Under the directory we just created, create the file ```runtime.properties``` with the following contents:
+
+```
+druid.host=127.0.0.1:8082
+druid.port=8082
+druid.service=master
+
+# logging
+com.metamx.emitter.logging=true
+com.metamx.emitter.logging.level=info
+
+# zk
+druid.zk.service.host=localhost
+druid.zk.paths.base=/druid
+druid.zk.paths.discoveryPath=/druid/discoveryPath
+
+# aws (demo user)
+com.metamx.aws.accessKey=AKIAIMKECRUYKDQGR6YQ
+com.metamx.aws.secretKey=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
+
+# db
+druid.database.segmentTable=segments
+druid.database.user=druid
+druid.database.password=diurd
+druid.database.connectURI=jdbc:mysql://localhost:3306/druid
+druid.database.ruleTable=rules
+druid.database.configTable=config
+
+# master runtime configs
+druid.master.startDelay=PT60S
+```
+
+To start the master node:
+
+```bash
+java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/master com.metamx.druid.http.MasterMain
+```
+
+### Start a Compute Node ###
+
+Compute nodes are the workhorses of a cluster and are in charge of loading historical segments and making them available for queries. Our Wikipedia segment will be downloaded by a compute node.
+
+To create the compute config file:
+
+```
+mkdir config/compute
+```
+
+Under the directory we just created, create the file ```runtime.properties``` with the following contents:
+```
+druid.host=127.0.0.1:8081
+druid.port=8081
+druid.service=compute
+
+# logging
+com.metamx.emitter.logging=true
+com.metamx.emitter.logging.level=info
+
+# zk
+druid.zk.service.host=localhost
+druid.zk.paths.base=/druid
+druid.zk.paths.discoveryPath=/druid/discoveryPath
+
+# processing
+druid.processing.buffer.sizeBytes=10000000
+
+# aws (demo user)
+com.metamx.aws.accessKey=AKIAIMKECRUYKDQGR6YQ
+com.metamx.aws.secretKey=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
+
+# Path on local FS for storage of segments; dir will be created if needed
+druid.paths.indexCache=/tmp/druid/indexCache
+
+# Path on local FS for storage of segment metadata; dir will be created if needed
+druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
+
+# server
+druid.server.maxSize=100000000
+```
+
+To start the compute node:
+
+```bash
+java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/compute com.metamx.druid.http.ComputeMain
+```
+
+### Start a Broker Node ###
+
+Broker nodes are responsible for figuring out which compute and/or realtime nodes correspond to which queries. They also merge partial results from these nodes in a scatter/gather fashion.
+
+To create the broker config file:
+
+```
+mkdir config/broker
+```
+
+Under the directory we just created, create the file ```runtime.properties``` with the following contents:
+
+```
+druid.host=127.0.0.1:8080
+druid.port=8080
+druid.service=broker
+
+# logging
+com.metamx.emitter.logging=true
+com.metamx.emitter.logging.level=info
+
+# zk
+druid.zk.service.host=localhost
+druid.zk.paths.base=/druid
+druid.zk.paths.discoveryPath=/druid/discoveryPath
+
+# thread pool size for servicing queries
+druid.client.http.connections=10
+```
+
+To start the broker node:
+
+```bash
+java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/broker com.metamx.druid.http.BrokerMain
+```
+
+## Loading the Data ##
+
+The MySQL dependency we introduced earlier on contains a 'segments' table that contains entries for segments that should be loaded into our cluster. The Druid master compares this table with segments that already exist in the cluster to determine what should be loaded and dropped. To load our wikipedia segment, we need to create an entry in our MySQL segment table.
+
+Usually, when new segments are created, these MySQL entries are created directly so you never have to do this by hand. For this tutorial, we can do this manually by going back into MySQL and issuing:
+
+```
+use druid;
+```
+
+``
+INSERT INTO segments (id, dataSource, created_date, start, end, partitioned, version, used, payload) VALUES ('wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', 'wikipedia', '2013-08-08T21:26:23.799Z', '2013-08-01T00:00:00.000Z', '2013-08-02T00:00:00.000Z', '0', '2013-08-08T21:22:48.989Z', '1', '{\"dataSource\":\"wikipedia\",\"interval\":\"2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z\",\"version\":\"2013-08-08T21:22:48.989Z\",\"loadSpec\":{\"type\":\"s3_zip\",\"bucket\":\"static.druid.io\",\"key\":\"data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip\"},\"dimensions\":\"dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace,anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup\",\"metrics\":\"count,delta,variation,added,deleted\",\"shardSpec\":{\"type\":\"none\"},\"binaryVersion\":9,\"size\":24664730,\"identifier\":\"wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z\"}');
+``
+
+If you look in your master node logs, you should, after a maximum of a minute or so, see logs of the following form:
+
+```
+2013-08-08 22:48:41,967 INFO [main-EventThread] com.metamx.druid.master.LoadQueuePeon - Server[/druid/loadQueue/127.0.0.1:8081] done processing [/druid/loadQueue/127.0.0.1:8081/wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z]
+2013-08-08 22:48:41,969 INFO [ServerInventoryView-0] com.metamx.druid.client.SingleServerInventoryView - Server[127.0.0.1:8081] added segment[wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z]
+```
+
+When the segment completes downloading and ready for queries, you should see the following message on your compute node logs:
+
+```
+2013-08-08 22:48:41,959 INFO [ZkCoordinator-0] com.metamx.druid.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z] at path[/druid/segments/127.0.0.1:8081/2013-08-08T22:48:41.959Z]
+```
+
+At this point, we can query the segment. For more information on querying, see this[link](https://github.com/metamx/druid/wiki/Querying).
+
+## Next Steps ##
+
+Now that you have an understanding of what the Druid clsuter looks like, why not load some of your own data?
+Check out the [Loading Your Own Data](https://github.com/metamx/druid/wiki/Loading-Your-Data) section for more info!
\ No newline at end of file
diff --git a/docs/Tutorial:-Webstream.md b/docs/Tutorial:-Webstream.md
new file mode 100644
index 00000000000..c8b0bcada8b
--- /dev/null
+++ b/docs/Tutorial:-Webstream.md
@@ -0,0 +1,354 @@
+Greetings! This tutorial will help clarify some core Druid concepts. We will use a realtime dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on!
+
+About the data
+--------------
+
+The data source weโ€™ll be working with is the Bit.ly USA Government website statistics stream. You can see the stream [here](http://developer.usa.gov/1usagov), and read about the stream [here](http://www.usa.gov/About/developer-resources/1usagov.shtml) . This is a feed of json data that gets updated whenever anyone clicks a bit.ly shortened USA.gov website. A typical event might look something like this:
+\`\`\`json
+{
+ [user\_agent]() โ€œMozilla/5.0 (compatible; MSIE 9.0; Windows NT 6.1; WOW64; Trident/5.0)โ€,
+ [country]() โ€œUSโ€,
+ [known\_user]() 1,
+ [timezone]() โ€œAmerica/New\_Yorkโ€,
+ [geo\_region]() โ€œDCโ€,
+ [global\_bitly\_hash]() โ€œ17ctAFsโ€,
+ [encoding\_user\_bitly\_hash]() โ€œ17ctAFrโ€,
+ [encoding\_user\_login]() โ€œsenrubiopressโ€,
+ [aaccept\_language]() โ€œen-USโ€,
+ [short\_url\_cname]() โ€œ1.usa.govโ€,
+ [referring\_url]() โ€œhttp://t.co/4Av4NUFAYqโ€,
+ [long\_url]() โ€œhttp://www.rubio.senate.gov/public/index.cfm/fighting-for-florida?ID=c8357d12-9da8-4e9d-b00d-7168e1bf3599โ€,
+ [timestamp]() 1372190407,
+ [timestamp of time hash was created]() 1372190097,
+ [city]() โ€œWashingtonโ€,
+ [latitude\_longitude]() [
+ 38.893299,
+ ~~77.014603
+ ]
+}
+\`\`\`
+The โ€œknown\_userโ€ field is always 1 or 0. It is 1 if the user is known to the server, and 0 otherwise. We will use this field extensively in this demo.
+h2. Setting Up
+There are two ways to setup Druid: download a tarball, or ]. You only need to do one of these.
+h3. Download a Tarball
+Weโ€™ve built a tarball that contains everything youโ€™ll need. Youโ€™ll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.5.50-bin.tar.gz)
+Download this file to a directory of your choosing.
+You can extract the awesomeness within by issuing:
+\tar~~zxvf druid-services~~**~~bin.tar.gz\
+Not too lost so far right? Thatโ€™s great! If you cd into the directory:
+\cd druid-services-0.5.50\
+You should see a bunch of files:
+\* run\_example\_server.sh
+\* run\_example\_client.sh
+\* LICENSE, config, examples, lib directories
+
+h2. Running Example Scripts
+Letโ€™s start doing stuff. You can start a Druid ] node by issuing:
+\./run\_example\_server.sh\
+Select โ€œwebstreamโ€.
+Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below.
+\
+2013-07-19 21:54:05,154 INFO com.metamx.druid.realtime.RealtimeNode~~ Starting Jetty
+2013-07-19 21:54:05,154 INFO org.mortbay.log - jetty-6.1.x
+2013-07-19 21:54:05,171 INFO com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Expect to run at
+2013-07-19 21:54:05,246 INFO org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8083
+\
+The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. If you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf).
+Okay, things are about to get real. To query the real-time node youโ€™ve spun up, you can issue:
+\./run\_example\_client.sh\
+Select โ€œwebstreamโ€ once again. This script issues ]s to the data weโ€™ve been ingesting. The query looks like this:
+\`\`\`json
+{
+ [queryType]() โ€œgroupByโ€,
+ [dataSource]() โ€œwebstreamโ€,
+ [granularity]() โ€œminuteโ€,
+ [dimensions]() [
+ โ€œtimezoneโ€
+ ],
+ [aggregations]() [
+ {
+ [type]() โ€œcountโ€,
+ [name]() โ€œrowsโ€
+ },
+ {
+ [type]() โ€œdoubleSumโ€,
+ [fieldName]() โ€œknown\_usersโ€,
+ [name]() โ€œknown\_usersโ€
+ }
+ ],
+ [filter]() {
+ [type]() โ€œselectorโ€,
+ [dimension]() โ€œcountryโ€,
+ [value]() โ€œUSโ€
+ },
+ [intervals]() [
+ โ€œ2013-06-01T00:00/2020-01-01T00โ€
+ ]
+}
+\`\`\`
+This is a****groupBy**\* query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: . We are **filtering** via the **โ€œcountryโ€** dimension, to only look at website hits in the US. Our **aggregations** are what we are calculating: a row count, and the sum of the number of known users in our data.
+The result looks something like this:
+\`\`\`json
+[
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2013-07-18T19:39:00.000Zโ€,
+ [event]() {
+ [timezone]() โ€œAmerica/Chicagoโ€,
+ [known\_users]() 10,
+ [rows]() 15
+ }
+ },
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2013-07-18T19:39:00.000Zโ€,
+ [event]() {
+ [timezone]() โ€œAmerica/Los\_Angelesโ€,
+ [known\_users]() 0,
+ [rows]() 3
+ }
+ },
+โ€ฆ
+\`\`\`
+This groupBy query is a bit complicated and weโ€™ll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script.
+h2. Querying Druid
+In your favorite editor, create the file:
+\time\_boundary\_query.body\
+Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created:
+\
+{
+ [queryType]() โ€œtimeBoundaryโ€,
+ [dataSource]() โ€œwebstreamโ€
+}
+\
+The ] is one of the simplest Druid queries. To run the query, you can issue:
+\ curl~~X POST โ€˜http://localhost:8083/druid/v2/?prettyโ€™ ~~H โ€˜content-type: application/jsonโ€™~~d ```` time_boundary_query.body
+ +We get something like this JSON back: + +```json +[ + { + "timestamp": "2013-07-18T19:39:00.000Z", + "result": { + "minTime": "2013-07-18T19:39:00.000Z", + "maxTime": "2013-07-18T19:46:00.000Z" + } + } +] +``` +As you can probably tell, the result is indicating the maximum and minimum timestamps we've seen thus far (summarized to a minutely granularity). Let's explore a bit further. + +Return to your favorite editor and create the file: +
timeseries_query.body
+ +We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +

+{
+    "queryType": "timeseries", 
+    "dataSource": "webstream", 
+    "intervals": [
+        "2010-01-01/2020-01-01"
+    ], 
+    "granularity": "all", 
+    "aggregations": [
+        {
+            "type": "count", 
+            "name": "rows"
+        }, 
+        {
+            "type": "doubleSum", 
+            "fieldName": "known_users", 
+            "name": "known_users"
+        }
+    ]
+}
+
+ +You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +To issue the query and get some results, run the following in your command line: +
curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
+
+
+Once again, you should get a JSON blob of text back with your results, that looks something like this: + +\`\`\`json +[ +{ + โ€œtimestampโ€ : โ€œ2013-07-18T19:39:00.000Zโ€, + โ€œresultโ€ : { + โ€œknown\_usersโ€ : 787.0, + โ€œrowsโ€ : 2004 + } +} +] +\`\`\` + +If you issue the query again, you should notice your results updating. + +Right now all the results you are getting back are being aggregated into a single timestamp bucket. What if we wanted to see our aggregations on a per minute basis? What field can we change in the query to accomplish this? + +If you loudly exclaimed โ€œwe can change granularity to minuteโ€, you are absolutely correct! We can specify different granularities to bucket our results, like so: + + + { + "queryType": "timeseries", + "dataSource": "webstream", + "intervals": [ + "2010-01-01/2020-01-01" + ], + "granularity": "minute", + "aggregations": [ + { + "type": "count", + "name": "rows" + }, + { + "type": "doubleSum", + "fieldName": "known_users", + "name": "known_users" + } + ] + } + + +This gives us something like the following: + +\`\`\`json +[ + { + [timestamp]() โ€œ2013-07-18T19:39:00.000Zโ€, + [result]() { + [known\_users]() 33, + [rows]() 76 + } + }, + { + [timestamp]() โ€œ2013-07-18T19:40:00.000Zโ€, + [result]() { + [known\_users]() 105, + [rows]() 221 + } + }, + { + [timestamp]() โ€œ2013-07-18T19:41:00.000Zโ€, + [result]() { + [known\_users]() 53, + [rows]() 167 + } + }, +โ€ฆ +\`\`\` + +Solving a Problem +----------------- + +One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top states in the US are, ordered by the number of visits by known users over the last few minutes? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can! + +Letโ€™s create the file: + + group_by_query.body + and put the following in there: +

+    {
+        "queryType": "groupBy", 
+        "dataSource": "webstream", 
+        "granularity": "all", 
+        "dimensions": [
+            "geo_region"
+        ], 
+        "orderBy": {
+            "type": "default", 
+            "columns": [
+                {
+                    "dimension": "known_users", 
+                    "direction": "DESCENDING"
+                }
+            ], 
+            "limit": 10
+        }, 
+        "aggregations": [
+            {
+                "type": "count", 
+                "name": "rows"
+            }, 
+            {
+                "type": "doubleSum", 
+                "fieldName": "known_users", 
+                "name": "known_users"
+            }
+        ], 
+        "filter": {
+            "type": "selector", 
+            "dimension": "country", 
+            "value": "US"
+        }, 
+        "intervals": [
+            "2012-10-01T00:00/2020-01-01T00"
+        ]
+    }
+    
+
+Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+
+If you issue the query:
+
+    curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d @group_by_query.body
+
+You should see an answer to our question. For my stream, it looks like this:
+
+\`\`\`json
+[
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2012-10-01T00:00:00.000Zโ€,
+ [event]() {
+ [geo\_region]() โ€œRIโ€,
+ [known\_users]() 359,
+ [rows]() 143
+ }
+ },
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2012-10-01T00:00:00.000Zโ€,
+ [event]() {
+ [geo\_region]() โ€œNYโ€,
+ [known\_users]() 187,
+ [rows]() 322
+ }
+ },
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2012-10-01T00:00:00.000Zโ€,
+ [event]() {
+ [geo\_region]() โ€œCAโ€,
+ [known\_users]() 145,
+ [rows]() 466
+ }
+ },
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2012-10-01T00:00:00.000Zโ€,
+ [event]() {
+ [geo\_region]() โ€œILโ€,
+ [known\_users]() 121,
+ [rows]() 185
+ }
+ },
+โ€ฆ
+\`\`\`
+
+Feel free to tweak other query parameters to answer other questions you may have about the data.
+
+Next Steps
+----------
+
+What to know even more information about the Druid Cluster? Check out [[Tutorial: The Druid Cluster]]
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [[Loading Your Data]].
+
+Additional Information
+----------------------
+
+This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki.
+
+And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
diff --git a/docs/Twitter-Tutorial.md b/docs/Twitter-Tutorial.md
new file mode 100644
index 00000000000..c113282e937
--- /dev/null
+++ b/docs/Twitter-Tutorial.md
@@ -0,0 +1,326 @@
+Greetings! We see youโ€™ve taken an interest in Druid. Thatโ€™s awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [[Examples]], and issue some basic Druid queries. The data source weโ€™ll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on!
+
+Setting Up
+----------
+
+There are two ways to setup Druid: download a tarball, or build it from source.
+
+### Download a Tarball
+
+Weโ€™ve built a tarball that contains everything youโ€™ll need. Youโ€™ll find it [here](http://static.druid.io/data/examples/druid-services-0.4.6.tar.gz).
+Download this bad boy to a directory of your choosing.
+
+You can extract the awesomeness within by issuing:
+
+    tar -zxvf druid-services-0.4.6.tar.gz
+
+Not too lost so far right? Thatโ€™s great! If you cd into the directory:
+
+    cd druid-services-0.4.6-SNAPSHOT
+
+You should see a bunch of files:
+\* run\_example\_server.sh
+\* run\_example\_client.sh
+\* LICENSE, config, examples, lib directories
+
+### Clone and Build from Source
+
+The other way to setup Druid is from source via git. To do so, run these commands:
+
+\`\`\`
+git clone git@github.com:metamx/druid.git
+cd druid
+git checkout druid-0.4.32-branch
+./build.sh
+\`\`\`
+
+You should see a bunch of files:
+
+\`\`\`
+DruidCorporateCLA.pdf README common examples indexer pom.xml server
+DruidIndividualCLA.pdf build.sh doc group\_by.body install publications services
+LICENSE client eclipse\_formatting.xml index-common merger realtime
+\`\`\`
+
+You can find the example executables in the examples/bin directory:
+\* run\_example\_server.sh
+\* run\_example\_client.sh
+
+Running Example Scripts
+-----------------------
+
+Letโ€™s start doing stuff. You can start a Druid [[Realtime]] node by issuing:
+
+    ./run_example_server.sh
+
+Select โ€œtwitterโ€.
+
+Youโ€™ll need to register a new application with the twitter API, which only takes a minute. Go to [https://twitter.com/oauth\_clients/new](https://twitter.com/oauth_clients/new) and fill out the form and submit. Donโ€™t worry, the home page and callback url can be anything. This will generate keys for the Twitter example application. Take note of the values for consumer key/secret and access token/secret.
+
+Enter your credentials when prompted.
+
+Once the node starts up you will see a bunch of logs about setting up properties and connecting to the data source. If everything was successful, you should see messages of the form shown below. If you see crazy exceptions, you probably typed in your login information incorrectly.
+
+    
+    2013-05-17 23:04:40,934 INFO [main] org.mortbay.log - Started SelectChannelConnector@0.0.0.0:8080
+    2013-05-17 23:04:40,935 INFO [main] com.metamx.common.lifecycle.Lifecycle$AnnotationBasedHandler - Invoking start method[public void com.metamx.druid.http.FileRequestLogger.start()] on object[com.metamx.druid.http.FileRequestLogger@42bb0406].
+    2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Connection established.
+    2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] druid.examples.twitter.TwitterSpritzerFirehoseFactory - Connected_to_Twitter
+    2013-05-17 23:04:41,578 INFO [Twitter Stream consumer-1[Establishing connection]] twitter4j.TwitterStreamImpl - Receiving status stream.
+    
+
+Periodically, youโ€™ll also see messages of the form:
+
+    
+    2013-05-17 23:04:59,793 INFO [chief-twitterstream] druid.examples.twitter.TwitterSpritzerFirehoseFactory - nextRow() has returned 1,000 InputRows
+    
+
+These messages indicate you are ingesting events. The Druid real time-node ingests events in an in-memory buffer. Periodically, these events will be persisted to disk. Persisting to disk generates a whole bunch of logs:
+
+    
+    2013-05-17 23:06:40,918 INFO [chief-twitterstream] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - Submitting persist runnable for dataSource[twitterstream]
+    2013-05-17 23:06:40,920 INFO [twitterstream-incremental-persist] com.metamx.druid.realtime.plumber.RealtimePlumberSchool - DataSource[twitterstream], Interval[2013-05-17T23:00:00.000Z/2013-05-18T00:00:00.000Z], persisting Hydrant[FireHydrant{index=com.metamx.druid.index.v1.IncrementalIndex@126212dd, queryable=com.metamx.druid.index.IncrementalIndexSegment@64c47498, count=0}]
+    2013-05-17 23:06:40,937 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting persist for interval[2013-05-17T23:00:00.000Z/2013-05-17T23:07:00.000Z], rows[4,666]
+    2013-05-17 23:06:41,039 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed index.drd in 11 millis.
+    2013-05-17 23:06:41,070 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed dim conversions in 31 millis.
+    2013-05-17 23:06:41,275 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.CompressedPools - Allocating new chunkEncoder[1]
+    2013-05-17 23:06:41,332 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed walk through of 4,666 rows in 262 millis.
+    2013-05-17 23:06:41,334 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting dimension[htags] with cardinality[634]
+    2013-05-17 23:06:41,381 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Completed dimension[htags] in 49 millis.
+    2013-05-17 23:06:41,382 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting dimension[lang] with cardinality[19]
+    2013-05-17 23:06:41,398 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Completed dimension[lang] in 17 millis.
+    2013-05-17 23:06:41,398 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Starting dimension[utc_offset] with cardinality[32]
+    2013-05-17 23:06:41,413 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - Completed dimension[utc_offset] in 15 millis.
+    2013-05-17 23:06:41,413 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexMerger - outDir[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] completed inverted.drd in 81 millis.
+    2013-05-17 23:06:41,425 INFO [twitterstream-incremental-persist] com.metamx.druid.index.v1.IndexIO$DefaultIndexIOHandler - Converting v8[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0/v8-tmp] to v9[/tmp/example/twitter_realtime/basePersist/twitterstream/2013-05-17T23:00:00.000Z_2013-05-18T00:00:00.000Z/0]
+    2013-05-17 23:06:41,426 INFO [twitterstream-incremental-persist] 
+    ... ETC
+    
+
+The logs are about building different columns, probably not the most exciting stuff (they might as well be in Vulcan) if are you learning about Druid for the first time. Nevertheless, if you are interested in the details of our real-time architecture and why we persist indexes to disk, I suggest you read our [White Paper](http://static.druid.io/docs/druid.pdf).
+
+Okay, things are about to get real (~~time). To query the real-time node youโ€™ve spun up, you can issue:
+\./run\_example\_client.sh\
+Select โ€œtwitterโ€ once again. This script issues ]s to the twitter data weโ€™ve been ingesting. The query looks like this:
+\`\`\`json
+{
+ [queryType]() โ€œgroupByโ€,
+ [dataSource]() โ€œtwitterstreamโ€,
+ [granularity]() โ€œallโ€,
+ [dimensions]() ,
+ [aggregations]([)
+ { [type]() โ€œcountโ€, [name]() โ€œrowsโ€},
+ { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œtweetsโ€, [name]() โ€œtweetsโ€}
+ ],
+ [filter]() { [type]() โ€œselectorโ€, [dimension]() โ€œlangโ€, [value]() โ€œenโ€ },
+ [intervals](["2012-10-01T00:00/2020-01-01T00"])
+}
+\`\`\`
+This is a **groupBy** query, which you may be familiar with from SQL. We are grouping, or aggregating, via the **dimensions** field: . We are **filtering** via the **โ€œlangโ€** dimension, to only look at english tweets. Our **aggregations** are what we are calculating: a row count, and the sum of the tweets in our data.
+The result looks something like this:
+\`\`\`json
+[
+ {
+ [version]() โ€œv1โ€,
+ [timestamp]() โ€œ2012-10-01T00:00:00.000Zโ€,
+ [event]() {
+ [utc\_offset]() โ€œ~~10800",
+ [tweets]() 90,
+ [lang]() "en",
+ [rows]() 81
+ }
+ },
+ {
+ [version]() "v1",
+ [timestamp]() "2012-10-01T00:00:00.000Z",
+ [event]() {
+ [utc\_offset]() "~~14400โ€,
+ [tweets]() 177,
+ [lang]() โ€œenโ€,
+ [rows]() 154
+ }
+ },
+โ€ฆ
+\`\`\`
+This data, plotted in a time series/distribution, looks something like this:
+![Timezone / Tweets Scatter Plot](http://metamarkets.com/wp-content/uploads/2013/06/tweets_timezone_offset.png "Timezone / Tweets Scatter Plot")
+This groupBy query is a bit complicated and weโ€™ll return to it later. For the time being, just make sure you are getting some blocks of data back. If you are having problems, make sure you have [curl](http://curl.haxx.se/) installed. Control+C to break out of the client script.
+h2. Querying Druid
+In your favorite editor, create the file:
+\time\_boundary\_query.body\
+Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created:
+\
+\
+The ] is one of the simplest Druid queries. To run the query, you can issue:
+\ curl~~X POST โ€˜http://localhost:8080/druid/v2/?prettyโ€™ ~~H โ€˜content-type: application/jsonโ€™~~d ```` time_boundary_query.body
+ +We get something like this JSON back: + +```json +[ { + "timestamp" : "2013-06-10T19:09:00.000Z", + "result" : { + "minTime" : "2013-06-10T19:09:00.000Z", + "maxTime" : "2013-06-10T20:50:00.000Z" + } +} ] +``` +That's the result. What information do you think the result is conveying? +... +If you said the result is indicating the maximum and minimum timestamps we've seen thus far (summarized to a minutely granularity), you are absolutely correct. I can see you are a person legitimately interested in learning about Druid. Let's explore a bit further. + +Return to your favorite editor and create the file: +
timeseries_query.body
+ +We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +
{
+  "queryType":"timeseries",
+  "dataSource":"twitterstream",
+  "intervals":["2010-01-01/2020-01-01"],
+  "granularity":"all",
+  "aggregations":[
+      { "type": "count", "name": "rows"},
+      { "type": "doubleSum", "fieldName": "tweets", "name": "tweets"}
+  ]
+}
+
+ +You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +To issue the query and get some results, run the following in your command line: +
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
+
+
+Once again, you should get a JSON blob of text back with your results, that looks something like this: + +\`\`\`json +[ { + โ€œtimestampโ€ : โ€œ2013-06-10T19:09:00.000Zโ€, + โ€œresultโ€ : { + โ€œtweetsโ€ : 358562.0, + โ€œrowsโ€ : 272271 + } +} ] +\`\`\` + +If you issue the query again, you should notice your results updating. + +Right now all the results you are getting back are being aggregated into a single timestamp bucket. What if we wanted to see our aggregations on a per minute basis? What field can we change in the query to accomplish this? + +If you loudly exclaimed โ€œwe can change granularity to minuteโ€, you are absolutely correct again! We can specify different granularities to bucket our results, like so: + +\`\`\`json +{ + [queryType]("timeseries"), + [dataSource]("twitterstream"), + [intervals](["2010-01-01/2020-01-01"]), + [granularity]("minute"), + [aggregations]([) + { [type]() โ€œcountโ€, [name]() โ€œrowsโ€}, + { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œtweetsโ€, [name]() โ€œtweetsโ€} + ] +} +\`\`\` + +This gives us something like the following: + +\`\`\`json +[ { + โ€œtimestampโ€ : โ€œ2013-06-10T19:09:00.000Zโ€, + โ€œresultโ€ : { + โ€œtweetsโ€ : 2650.0, + โ€œrowsโ€ : 2120 + } +}, { + โ€œtimestampโ€ : โ€œ2013-06-10T19:10:00.000Zโ€, + โ€œresultโ€ : { + โ€œtweetsโ€ : 3401.0, + โ€œrowsโ€ : 2609 + } +}, { + โ€œtimestampโ€ : โ€œ2013-06-10T19:11:00.000Zโ€, + โ€œresultโ€ : { + โ€œtweetsโ€ : 3472.0, + โ€œrowsโ€ : 2610 + } +}, +โ€ฆ +\`\`\` + +Solving a Problem +----------------- + +One of Druidโ€™s main powers (see what we did there?) is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes youโ€™ve been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can! + +Letโ€™s create the file: + + group_by_query.body + and put the following in there: +
{
+        "queryType": "groupBy",
+        "dataSource": "twitterstream",
+        "granularity": "all",
+        "dimensions": ["htags"],
+        "orderBy": {"type":"default", "columns":[{"dimension": "tweets", "direction":"DESCENDING"}], "limit":5},
+        "aggregations":[
+          { "type": "longSum", "fieldName": "tweets", "name": "tweets"}
+        ],
+        "filter": {"type": "selector", "dimension": "lang", "value": "en" },
+        "intervals":["2012-10-01T00:00/2020-01-01T00"]
+    }
+    
+
+Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+
+If you issue the query:
+
+    curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d @group_by_query.body
+
+You should hopefully see an answer to our question. For my twitter stream, it looks like this:
+
+\`\`\`json
+[ {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œtweetsโ€ : 2660,
+ โ€œhtagsโ€ : โ€œandroidโ€
+ }
+}, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œtweetsโ€ : 1944,
+ โ€œhtagsโ€ : โ€œE3โ€
+ }
+}, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œtweetsโ€ : 1927,
+ โ€œhtagsโ€ : โ€œ15SueรฑosPendientesโ€
+ }
+}, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œtweetsโ€ : 1717,
+ โ€œhtagsโ€ : โ€œipadโ€
+ }
+}, {
+ โ€œversionโ€ : โ€œv1โ€,
+ โ€œtimestampโ€ : โ€œ2012-10-01T00:00:00.000Zโ€,
+ โ€œeventโ€ : {
+ โ€œtweetsโ€ : 1515,
+ โ€œhtagsโ€ : โ€œIDidntTextYouBackBecauseโ€
+ }
+} ]
+\`\`\`
+
+Feel free to tweak other query parameters to answer other questions you may have about the data.
+
+Additional Information
+----------------------
+
+This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to [[Loading Your Data]].
+
+And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
diff --git a/docs/Versioning.md b/docs/Versioning.md
new file mode 100644
index 00000000000..33c665d6542
--- /dev/null
+++ b/docs/Versioning.md
@@ -0,0 +1,21 @@
+This page discusses how we do versioning and provides information on our stable releases.
+
+Versioning Strategy
+-------------------
+
+We generally follow [semantic versioning](http://semver.org/). The general idea is
+
+-   โ€œMajorโ€ version (leftmost): backwards incompatible, no guarantees exist about APIs between the versions
+-   โ€œMinorโ€ version (middle number): you can move forward from a smaller number to a larger number, but moving backwards *might* be incompatible.
+-   โ€œbug-fixโ€ version (โ€œpatchโ€ or the rightmost): Interchangeable. The higher the number, the more things are fixed (hopefully), but the programming interfaces are completely compatible and you should be able to just drop in a new jar and have it work.
+
+Note that this is defined in terms of programming API, **not** in terms of functionality. It is possible that a brand new awesome way of doing something is introduced in a โ€œbug-fixโ€ release version if it doesnโ€™t add to the public API or change it.
+
+One exception for right now, while we are still in major version 0, we are considering the APIs to be in beta and are conflating โ€œmajorโ€ and โ€œminorโ€ so a minor version increase could be backwards incompatible for as long as we are at major version 0. These will be communicated via email on the group.
+
+For external deployments, we recommend running the stable release tag. Releases are considered stable after we have deployed them into our production environment and they have operated bug-free for some time.
+
+Tagging strategy
+----------------
+
+Tags of the codebase are equivalent to release candidates. We tag the code every time we want to take it through our release process, which includes some QA cycles and deployments. So, it is not safe to assume that a tag is a stable release, it is a solidification of the code as it goes through our production QA cycle and deployment. Tags will never change, but we often go through a number of iterations of tags before actually getting a stable release onto production. So, it is recommended that if you are not aware of what is on a tag, to stick to the stable releases listed on the [[Download]] page.
diff --git a/docs/ZooKeeper.md b/docs/ZooKeeper.md
new file mode 100644
index 00000000000..250d0280bcb
--- /dev/null
+++ b/docs/ZooKeeper.md
@@ -0,0 +1,63 @@
+Druid uses ZooKeeper (ZK) for management of current cluster state. The operations that happen over ZK are
+
+1.  [[Master]] leader election
+2.  Segment โ€œpublishingโ€ protocol from [[Compute]] and [[Realtime]]
+3.  Segment load/drop protocol between [[Master]] and [[Compute]]
+
+### Property Configuration
+
+ZooKeeper paths are set via the `runtime.properties` configuration file. Druid will automatically create paths that do not exist, so typos in config files is a very easy way to become split-brained.
+
+There is a prefix path that is required and can be used as the only (well, kinda, see the note below) path-related zookeeper configuration parameter (everything else will be a default based on the prefix):
+
+    druid.zk.paths.base
+
+You can also override each individual path (defaults are shown below):
+
+    druid.zk.paths.propertiesPath=${druid.zk.paths.base}/properties
+    druid.zk.paths.announcementsPath=${druid.zk.paths.base}/announcements
+    druid.zk.paths.servedSegmentsPath=${druid.zk.paths.base}/servedSegments
+    druid.zk.paths.loadQueuePath=${druid.zk.paths.base}/loadQueue
+    druid.zk.paths.masterPath=${druid.zk.paths.base}/master
+    druid.zk.paths.indexer.announcementsPath=${druid.zk.paths.base}/indexer/announcements
+    druid.zk.paths.indexer.tasksPath=${druid.zk.paths.base}/indexer/tasks
+    druid.zk.paths.indexer.statusPath=${druid.zk.paths.base}/indexer/status
+    druid.zk.paths.indexer.leaderLatchPath=${druid.zk.paths.base}/indexer/leaderLatchPath
+
+NOTE: We also use Curatorโ€™s service discovery module to expose some services via zookeeper. This also uses a zookeeper path, but this path is **not** affected by `druid.zk.paths.base` and **must** be specified separately. This property is
+
+    druid.zk.paths.discoveryPath
+
+### Master Leader Election
+
+We use the Curator LeadershipLatch recipe to do leader election at path
+
+    ${druid.zk.paths.masterPath}/_MASTER
+
+### Segment โ€œpublishingโ€ protocol from Compute and Realtime
+
+The `announcementsPath` and `servedSegmentsPath` are used for this.
+
+All [[Compute]] and [[Realtime]] nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
+
+    ${druid.zk.paths.announcementsPath}/${druid.host}
+
+Which signifies that they exist. They will also subsequently create a permanent znode at
+
+    ${druid.zk.paths.servedSegmentsPath}/${druid.host}
+
+And as they load up segments, they will attach ephemeral znodes that look like
+
+    ${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_
+
+Nodes like the [[Master]] and [[Broker]] can then watch these paths to see which nodes are currently serving which segments.
+
+### Segment load/drop protocol between Master and Compute
+
+The `loadQueuePath` is used for this.
+
+When the [[Master]] decides that a [[Compute]] node should load or drop a segment, it writes an ephemeral znode to
+
+    ${druid.zk.paths.loadQueuePath}/_host_of_compute_node/_segment_identifier
+
+This node will contain a payload that indicates to the Compute node what it should do with the given segment. When the Compute node is done with the work, it will delete the znode in order to signify to the Master that it is complete.
diff --git a/docs/contents.md b/docs/contents.md
new file mode 100644
index 00000000000..2298a7c8bdf
--- /dev/null
+++ b/docs/contents.md
@@ -0,0 +1,68 @@
+Contents
+\* [[Introduction|Home]]
+\* [[Download]]
+\* [[Support]]
+\* [[Contribute]]
+========================
+
+Getting Started
+\* [[Tutorial: A First Look at Druid]]
+\* [[Tutorial: The Druid Cluster]]
+\* [[Loading Your Data]]
+\* [[Querying Your Data]]
+\* [[Booting a Production Cluster]]
+\* [[Examples]]
+\* [[Cluster Setup]]
+\* [[Configuration]]
+--------------------------------------
+
+Data Ingestion
+\* [[Realtime]]
+\* [[Batch|Batch Ingestion]]
+\* [[Indexing Service]]
+----------------------------
+
+Querying
+\* [[Querying]]
+**\* ]
+**\* [[Aggregations]]
+**\* ]
+**\* [[Granularities]]
+\* Query Types
+**\* ]
+****\* ]
+****\* ]
+**\* [[SearchQuery]]
+**\* ]
+** [[SegmentMetadataQuery]]
+**\* ]
+**\* [[TimeseriesQuery]]
+---------------------------
+
+Architecture
+\* [[Design]]
+\* [[Segments]]
+\* Node Types
+**\* ]
+**\* [[Broker]]
+**\* ]
+****\* ]
+**\* [[Realtime]]
+**\* ]
+**\* [[Plumber]]
+\* External Dependencies
+**\* ]
+**\* [[MySQL]]
+**\* ]
+** [[Concepts and Terminology]]
+-------------------------------
+
+Development
+\* [[Versioning]]
+\* [[Build From Source]]
+\* [[Libraries]]
+------------------------
+
+Misc
+\* [[Thanks]]
+-------------

From 9b4c3756ede585c001dc0dbba1135891afa3732c Mon Sep 17 00:00:00 2001
From: fjy 
Date: Sun, 15 Sep 2013 10:27:37 -0700
Subject: [PATCH 76/92] fix exception catching logic of moving segments

---
 .../server/master/DruidMasterBalancer.java    | 30 +++++++++++--------
 1 file changed, 18 insertions(+), 12 deletions(-)

diff --git a/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java b/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java
index 19452df4fad..28c966b4162 100644
--- a/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java
+++ b/server/src/main/java/io/druid/server/master/DruidMasterBalancer.java
@@ -156,27 +156,33 @@ public class DruidMasterBalancer implements DruidMasterHelper
         (toServer.getSegment(segmentName) == null) &&
         new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) {
       log.info("Moving [%s] from [%s] to [%s]", segmentName, fromServerName, toServerName);
+
+      LoadPeonCallback callback = null;
       try {
+        currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment);
+        callback = new LoadPeonCallback()
+        {
+          @Override
+          protected void execute()
+          {
+            Map movingSegments = currentlyMovingSegments.get(toServer.getTier());
+            if (movingSegments != null) {
+              movingSegments.remove(segmentName);
+            }
+          }
+        };
         master.moveSegment(
             fromServerName,
             toServerName,
             segmentToMove.getIdentifier(),
-            new LoadPeonCallback()
-            {
-              @Override
-              protected void execute()
-              {
-                Map movingSegments = currentlyMovingSegments.get(toServer.getTier());
-                if (movingSegments != null) {
-                  movingSegments.remove(segmentName);
-                }
-              }
-            }
+            callback
         );
-        currentlyMovingSegments.get(toServer.getTier()).put(segmentName, segment);
       }
       catch (Exception e) {
         log.makeAlert(e, String.format("[%s] : Moving exception", segmentName)).emit();
+        if (callback != null) {
+          callback.execute();
+        }
       }
     } else {
       currentlyMovingSegments.get(toServer.getTier()).remove(segmentName);

From c06b37f36e264f94a1f3612182a170b7b76b0839 Mon Sep 17 00:00:00 2001
From: Russell Jurney 
Date: Mon, 16 Sep 2013 14:08:22 -0700
Subject: [PATCH 77/92] Won't serve html

---
 docs/_config.yml                              |   2 +
 docs/_layouts/default.html                    |  44 +++++
 docs/_layouts/post.html                       |   9 +
 .../2013-09-16-welcome-to-jekyll.markdown     |  24 +++
 docs/css/main.css                             | 160 ++++++++++++++++++
 docs/css/syntax.css                           |  60 +++++++
 docs/index.html                               |  13 ++
 7 files changed, 312 insertions(+)
 create mode 100644 docs/_config.yml
 create mode 100644 docs/_layouts/default.html
 create mode 100644 docs/_layouts/post.html
 create mode 100644 docs/_posts/2013-09-16-welcome-to-jekyll.markdown
 create mode 100755 docs/css/main.css
 create mode 100644 docs/css/syntax.css
 create mode 100644 docs/index.html

diff --git a/docs/_config.yml b/docs/_config.yml
new file mode 100644
index 00000000000..362c8bf5f91
--- /dev/null
+++ b/docs/_config.yml
@@ -0,0 +1,2 @@
+name: Your New Jekyll Site
+pygments: true
diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
new file mode 100644
index 00000000000..22e7e3f2a31
--- /dev/null
+++ b/docs/_layouts/default.html
@@ -0,0 +1,44 @@
+
+
+    
+        
+        
+        {{ page.title }}
+        
+
+        
+        
+
+        
+        
+
+    
+    
+
+        
+ + + {{ content }} + + +
+ + + diff --git a/docs/_layouts/post.html b/docs/_layouts/post.html new file mode 100644 index 00000000000..04e3586b82a --- /dev/null +++ b/docs/_layouts/post.html @@ -0,0 +1,9 @@ +--- +layout: default +--- +

{{ page.title }}

+

{{ page.date | date_to_string }}

+ +
+{{ content }} +
diff --git a/docs/_posts/2013-09-16-welcome-to-jekyll.markdown b/docs/_posts/2013-09-16-welcome-to-jekyll.markdown new file mode 100644 index 00000000000..c50a740d847 --- /dev/null +++ b/docs/_posts/2013-09-16-welcome-to-jekyll.markdown @@ -0,0 +1,24 @@ +--- +layout: post +title: "Welcome to Jekyll!" +date: 2013-09-16 13:06:49 +categories: jekyll update +--- + +You'll find this post in your `_posts` directory - edit this post and re-build (or run with the `-w` switch) to see your changes! +To add new posts, simply add a file in the `_posts` directory that follows the convention: YYYY-MM-DD-name-of-post.ext. + +Jekyll also offers powerful support for code snippets: + +{% highlight ruby %} +def print_hi(name) + puts "Hi, #{name}" +end +print_hi('Tom') +#=> prints 'Hi, Tom' to STDOUT. +{% endhighlight %} + +Check out the [Jekyll docs][jekyll] for more info on how to get the most out of Jekyll. File all bugs/feature requests at [Jekyll's GitHub repo][jekyll-gh]. + +[jekyll-gh]: https://github.com/mojombo/jekyll +[jekyll]: http://jekyllrb.com diff --git a/docs/css/main.css b/docs/css/main.css new file mode 100755 index 00000000000..50a818048ad --- /dev/null +++ b/docs/css/main.css @@ -0,0 +1,160 @@ +/*****************************************************************************/ +/* +/* Common +/* +/*****************************************************************************/ + +/* Global Reset */ +* { + margin: 0; + padding: 0; +} + +html, body { height: 100%; } + +body { + background-color: #FFF; + font: 13.34px Helvetica, Arial, sans-serif; + font-size: small; + text-align: center; +} + +h1, h2, h3, h4, h5, h6 { + font-size: 100%; } + +h1 { margin-bottom: 1em; } +p { margin: 1em 0; } + +a { color: #00a; } +a:hover { color: #000; } +a:visited { color: #a0a; } + +/*****************************************************************************/ +/* +/* Home +/* +/*****************************************************************************/ +ul.posts { + list-style-type: none; + margin-bottom: 2em; +} + +ul.posts li { + line-height: 1.75em; +} + +ul.posts span { + color: #aaa; + font-family: Monaco, "Courier New", monospace; + font-size: 80%; +} + +/*****************************************************************************/ +/* +/* Site +/* +/*****************************************************************************/ + +.site { + font-size: 115%; + text-align: justify; + width: 42em; + margin: 3em auto 2em; + line-height: 1.5em; +} + +.site .header a { + font-weight: bold; + text-decoration: none; +} + +.site .header h1.title { + display: inline-block; + margin-bottom: 2em; +} + +.site .header h1.title a { + color: #a00; +} + +.site .header h1.title a:hover { + color: #000; +} + +.site .header a.extra { + color: #aaa; + margin-left: 1em; +} + +.site .header a.extra:hover { + color: #000; +} + +.site .meta { + color: #aaa; +} + +.site .footer { + font-size: 80%; + color: #666; + border-top: 4px solid #eee; + margin-top: 2em; + overflow: hidden; +} + +.site .footer .contact { + float: left; + margin-right: 3em; +} + +.site .footer .contact a { + color: #8085C1; +} + +.site .footer .rss { + margin-top: 1.1em; + margin-right: -.2em; + float: right; +} + +.site .footer .rss img { + border: 0; +} + +/*****************************************************************************/ +/* +/* Posts +/* +/*****************************************************************************/ + +/* standard */ +.post pre { + border: 1px solid #ddd; + background-color: #eef; + padding: 0 .4em; +} + +.post ul, .post ol { + margin-left: 1.35em; +} + +.post code { + border: 1px solid #ddd; + background-color: #eef; + padding: 0 .2em; +} + +.post pre code { + border: none; +} + +/* terminal */ +.post pre.terminal { + border: 1px solid #000; + background-color: #333; + color: #FFF; +} + +.post pre.terminal code { + background-color: #333; +} diff --git a/docs/css/syntax.css b/docs/css/syntax.css new file mode 100644 index 00000000000..2774b764926 --- /dev/null +++ b/docs/css/syntax.css @@ -0,0 +1,60 @@ +.highlight { background: #ffffff; } +.highlight .c { color: #999988; font-style: italic } /* Comment */ +.highlight .err { color: #a61717; background-color: #e3d2d2 } /* Error */ +.highlight .k { font-weight: bold } /* Keyword */ +.highlight .o { font-weight: bold } /* Operator */ +.highlight .cm { color: #999988; font-style: italic } /* Comment.Multiline */ +.highlight .cp { color: #999999; font-weight: bold } /* Comment.Preproc */ +.highlight .c1 { color: #999988; font-style: italic } /* Comment.Single */ +.highlight .cs { color: #999999; font-weight: bold; font-style: italic } /* Comment.Special */ +.highlight .gd { color: #000000; background-color: #ffdddd } /* Generic.Deleted */ +.highlight .gd .x { color: #000000; background-color: #ffaaaa } /* Generic.Deleted.Specific */ +.highlight .ge { font-style: italic } /* Generic.Emph */ +.highlight .gr { color: #aa0000 } /* Generic.Error */ +.highlight .gh { color: #999999 } /* Generic.Heading */ +.highlight .gi { color: #000000; background-color: #ddffdd } /* Generic.Inserted */ +.highlight .gi .x { color: #000000; background-color: #aaffaa } /* Generic.Inserted.Specific */ +.highlight .go { color: #888888 } /* Generic.Output */ +.highlight .gp { color: #555555 } /* Generic.Prompt */ +.highlight .gs { font-weight: bold } /* Generic.Strong */ +.highlight .gu { color: #aaaaaa } /* Generic.Subheading */ +.highlight .gt { color: #aa0000 } /* Generic.Traceback */ +.highlight .kc { font-weight: bold } /* Keyword.Constant */ +.highlight .kd { font-weight: bold } /* Keyword.Declaration */ +.highlight .kp { font-weight: bold } /* Keyword.Pseudo */ +.highlight .kr { font-weight: bold } /* Keyword.Reserved */ +.highlight .kt { color: #445588; font-weight: bold } /* Keyword.Type */ +.highlight .m { color: #009999 } /* Literal.Number */ +.highlight .s { color: #d14 } /* Literal.String */ +.highlight .na { color: #008080 } /* Name.Attribute */ +.highlight .nb { color: #0086B3 } /* Name.Builtin */ +.highlight .nc { color: #445588; font-weight: bold } /* Name.Class */ +.highlight .no { color: #008080 } /* Name.Constant */ +.highlight .ni { color: #800080 } /* Name.Entity */ +.highlight .ne { color: #990000; font-weight: bold } /* Name.Exception */ +.highlight .nf { color: #990000; font-weight: bold } /* Name.Function */ +.highlight .nn { color: #555555 } /* Name.Namespace */ +.highlight .nt { color: #000080 } /* Name.Tag */ +.highlight .nv { color: #008080 } /* Name.Variable */ +.highlight .ow { font-weight: bold } /* Operator.Word */ +.highlight .w { color: #bbbbbb } /* Text.Whitespace */ +.highlight .mf { color: #009999 } /* Literal.Number.Float */ +.highlight .mh { color: #009999 } /* Literal.Number.Hex */ +.highlight .mi { color: #009999 } /* Literal.Number.Integer */ +.highlight .mo { color: #009999 } /* Literal.Number.Oct */ +.highlight .sb { color: #d14 } /* Literal.String.Backtick */ +.highlight .sc { color: #d14 } /* Literal.String.Char */ +.highlight .sd { color: #d14 } /* Literal.String.Doc */ +.highlight .s2 { color: #d14 } /* Literal.String.Double */ +.highlight .se { color: #d14 } /* Literal.String.Escape */ +.highlight .sh { color: #d14 } /* Literal.String.Heredoc */ +.highlight .si { color: #d14 } /* Literal.String.Interpol */ +.highlight .sx { color: #d14 } /* Literal.String.Other */ +.highlight .sr { color: #009926 } /* Literal.String.Regex */ +.highlight .s1 { color: #d14 } /* Literal.String.Single */ +.highlight .ss { color: #990073 } /* Literal.String.Symbol */ +.highlight .bp { color: #999999 } /* Name.Builtin.Pseudo */ +.highlight .vc { color: #008080 } /* Name.Variable.Class */ +.highlight .vg { color: #008080 } /* Name.Variable.Global */ +.highlight .vi { color: #008080 } /* Name.Variable.Instance */ +.highlight .il { color: #009999 } /* Literal.Number.Integer.Long */ diff --git a/docs/index.html b/docs/index.html new file mode 100644 index 00000000000..c7268192374 --- /dev/null +++ b/docs/index.html @@ -0,0 +1,13 @@ +--- +layout: default +title: Your New Jekyll Site +--- + +
+

Blog Posts

+
    + {% for post in site.posts %} +
  • {{ post.date | date_to_string }} » {{ post.title }}
  • + {% endfor %} +
+
\ No newline at end of file From 248fba683a3cfe8a99c6ebb0d101d3693445e87b Mon Sep 17 00:00:00 2001 From: Russell Jurney Date: Mon, 16 Sep 2013 14:49:36 -0700 Subject: [PATCH 78/92] Added prepend tag to make pages display. --- docs/Aggregations.md | 93 ++----------------------- docs/Batch-ingestion.md | 3 + docs/Booting-a-production-cluster.md | 3 + docs/Broker.md | 3 + docs/Build-from-source.md | 3 + docs/Cluster-setup.md | 3 + docs/Compute.md | 3 + docs/Concepts-and-Terminology.md | 3 + docs/Configuration.md | 3 + docs/Contribute.md | 3 + docs/Deep-Storage.md | 3 + docs/Design.md | 3 + docs/Download.md | 3 + docs/Druid-Personal-Demo-Cluster.md | 3 + docs/Druid-vs-Cassandra.md | 3 + docs/Druid-vs-Hadoop.md | 3 + docs/Druid-vs-Impala-or-Shark.md | 3 + docs/Druid-vs-redshift.md | 3 + docs/Druid-vs-vertica.md | 3 + docs/Examples.md | 3 + docs/Filters.md | 3 + docs/Firehose.md | 3 + docs/Granularities.md | 3 + docs/GroupByQuery.md | 3 + docs/Having.md | 3 + docs/Home.md | 3 + docs/Indexing-Service.md | 3 + docs/Libraries.md | 3 + docs/Loading-Your-Data.md | 3 + docs/Master.md | 5 +- docs/MySQL.md | 3 + docs/OrderBy.md | 3 + docs/Plumber.md | 3 + docs/Post-aggregations.md | 3 + docs/Querying-your-data.md | 3 + docs/Querying.md | 3 + docs/Realtime.md | 3 + docs/Rule-Configuration.md | 3 + docs/SearchQuery.md | 3 + docs/SearchQuerySpec.md | 3 + docs/SegmentMetadataQuery.md | 3 + docs/Segments.md | 3 + docs/Spatial-Filters.md | 3 + docs/Spatial-Indexing.md | 3 + docs/Stand-Alone-With-Riak-CS.md | 3 + docs/Support.md | 3 + docs/Tasks.md | 3 + docs/Thanks.md | 3 + docs/TimeBoundaryQuery.md | 3 + docs/TimeseriesQuery.md | 3 + docs/Tutorial:-A-First-Look-at-Druid.md | 3 + docs/Tutorial:-The-Druid-Cluster.md | 3 + docs/Tutorial:-Webstream.md | 3 + docs/Twitter-Tutorial.md | 3 + docs/Versioning.md | 3 + docs/ZooKeeper.md | 3 + docs/_config.yml | 1 + docs/contents.md | 3 + 58 files changed, 176 insertions(+), 88 deletions(-) diff --git a/docs/Aggregations.md b/docs/Aggregations.md index ffdbd18a4f2..886dac7a572 100644 --- a/docs/Aggregations.md +++ b/docs/Aggregations.md @@ -1,87 +1,6 @@ -Aggregations are specifications of processing over metrics available in Druid. -Available aggregations are: - -### Sum aggregators - -#### `longSum` aggregator - -computes the sum of values as a 64-bit, signed integer - - { - "type" : "longSum", - "name" : , - "fieldName" : - } - -`name` โ€“ย output name for the summed value -`fieldName` โ€“ name of the metric column to sum over - -#### `doubleSum` aggregator - -Computes the sum of values as 64-bit floating point value. Similar to `longSum` - - { - "type" : "doubleSum", - "name" : , - "fieldName" : - } - -### Count aggregator - -`count` computes the row count that match the filters - - { - "type" : "count", - "name" : , - } - -### Min / Max aggregators - -#### `min` aggregator - -`min` computes the minimum metric value - - { - "type" : "min", - "name" : , - "fieldName" : - } - -#### `max` aggregator - -`max` computes the maximum metric value - - { - "type" : "max", - "name" : , - "fieldName" : - } - -### JavaScript aggregator - -Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions). - -All JavaScript functions must return numerical values. - - { - "type": "javascript", - "name": "", - "fieldNames" : [ , , ... ], - "fnAggregate" : "function(current, column1, column2, ...) { - - return - }" - "fnCombine" : "function(partialA, partialB) { return ; }" - "fnReset" : "function() { return ; }" - } - -**Example** - - { - "type": "javascript", - "name": "sum(log(x)/y) + 10", - "fieldNames": ["x", "y"], - "fnAggregate" : "function(current, a, b) { return current + (Math.log(a) * b); }" - "fnCombine" : "function(partialA, partialB) { return partialA + partialB; }" - "fnReset" : "function() { return 10; }" - } +--- +layout: default +--- +--- +layout: default +--- diff --git a/docs/Batch-ingestion.md b/docs/Batch-ingestion.md index 97212777bc4..f91f0dbb081 100644 --- a/docs/Batch-ingestion.md +++ b/docs/Batch-ingestion.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Batch Data Ingestion ==================== diff --git a/docs/Booting-a-production-cluster.md b/docs/Booting-a-production-cluster.md index 32181fce2c1..c25ef25c607 100644 --- a/docs/Booting-a-production-cluster.md +++ b/docs/Booting-a-production-cluster.md @@ -1,3 +1,6 @@ +--- +layout: default +--- # Booting a Single Node Cluster # [[Loading Your Data]] and [[Querying Your Data]] contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz). diff --git a/docs/Broker.md b/docs/Broker.md index 6d8f3db2ca0..e71100e9915 100644 --- a/docs/Broker.md +++ b/docs/Broker.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Broker ====== diff --git a/docs/Build-from-source.md b/docs/Build-from-source.md index aaa5411368d..3f323259b80 100644 --- a/docs/Build-from-source.md +++ b/docs/Build-from-source.md @@ -1,3 +1,6 @@ +--- +layout: default +--- ### Clone and Build from Source The other way to setup Druid is from source via git. To do so, run these commands: diff --git a/docs/Cluster-setup.md b/docs/Cluster-setup.md index 23cb806d4fc..29837e94296 100644 --- a/docs/Cluster-setup.md +++ b/docs/Cluster-setup.md @@ -1,3 +1,6 @@ +--- +layout: default +--- A Druid cluster consists of various node types that need to be set up depending on your use case. See our [[Design]] docs for a description of the different node types. Setup Scripts diff --git a/docs/Compute.md b/docs/Compute.md index 755f2475707..8df11f8ca2f 100644 --- a/docs/Compute.md +++ b/docs/Compute.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Compute ======= diff --git a/docs/Concepts-and-Terminology.md b/docs/Concepts-and-Terminology.md index a9accabf88a..1e7f535388d 100644 --- a/docs/Concepts-and-Terminology.md +++ b/docs/Concepts-and-Terminology.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Concepts and Terminology ======================== diff --git a/docs/Configuration.md b/docs/Configuration.md index c3150d44805..353b8be77b3 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -1,3 +1,6 @@ +--- +layout: default +--- This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json โ€œspecFileโ€ descriptions in [[Realtime]] and [[Batch-ingestion]]. JVM Configuration Best Practices diff --git a/docs/Contribute.md b/docs/Contribute.md index 8a5bcc75f99..58d53a6d224 100644 --- a/docs/Contribute.md +++ b/docs/Contribute.md @@ -1,3 +1,6 @@ +--- +layout: default +--- If you are interested in contributing to the code, we accept [pull requests](https://help.github.com/articles/using-pull-requests). Note: we have only just completed decoupling our Metamarkets-specific code from the code base and we took some short-cuts in interface design to make it happen. So, there are a number of interfaces that exist right now which are likely to be in flux. If you are embedding Druid in your system, it will be safest for the time being to only extend/implement interfaces that this wiki describes, as those are intended as stable (unless otherwise mentioned). For issue tracking, we are using the github issue tracker. Please fill out an issue from the Issues tab on the github screen. diff --git a/docs/Deep-Storage.md b/docs/Deep-Storage.md index f30aa50333e..bd9a0ec8a66 100644 --- a/docs/Deep-Storage.md +++ b/docs/Deep-Storage.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented. The currently supported types of deep storage follow. diff --git a/docs/Design.md b/docs/Design.md index 2d67f1e3139..888d0b871b3 100644 --- a/docs/Design.md +++ b/docs/Design.md @@ -1,3 +1,6 @@ +--- +layout: default +--- For a comprehensive look at the architecture of Druid, read the [White Paper](http://static.druid.io/docs/druid.pdf). What is Druid? diff --git a/docs/Download.md b/docs/Download.md index 1bdbe799c50..00de8597f11 100644 --- a/docs/Download.md +++ b/docs/Download.md @@ -1,3 +1,6 @@ +--- +layout: default +--- A version may be declared as a release candidate if it has been deployed to a sizable production cluster. Release candidates are declared as stable after we feel fairly confident there are no major bugs in the version. Check out the [[Versioning]] section for how we describe software versions. Release Candidate diff --git a/docs/Druid-Personal-Demo-Cluster.md b/docs/Druid-Personal-Demo-Cluster.md index 81a088226f5..ab49d828dbc 100644 --- a/docs/Druid-Personal-Demo-Cluster.md +++ b/docs/Druid-Personal-Demo-Cluster.md @@ -1,3 +1,6 @@ +--- +layout: default +--- # Druid Personal Demo Cluster (DPDC) Note, there are currently some issues with the CloudFormation. We are working through them and will update the documentation here when things work properly. In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version. If you just want to get a feel for the types of data and queries that you can issue, check out [[Realtime Examples]] diff --git a/docs/Druid-vs-Cassandra.md b/docs/Druid-vs-Cassandra.md index 4cac3922324..e191dde2af7 100644 --- a/docs/Druid-vs-Cassandra.md +++ b/docs/Druid-vs-Cassandra.md @@ -1,3 +1,6 @@ +--- +layout: default +--- We are not experts on Cassandra, if anything is incorrect about our portrayal, please let us know on the mailing list or via some other means. We will fix this page. Druid is highly optimized for scans and aggregations, it supports arbitrarily deep drill downs into data sets without the need to pre-compute, and it can ingest event streams in real-time and allow users to query events as they come in. Cassandra is a great key-value store and it has some features that allow you to use it to do more interesting things than what you can do with a pure key-value store. But, it is not built for the same use cases that Druid handles, namely regularly scanning over billions of entries per query. diff --git a/docs/Druid-vs-Hadoop.md b/docs/Druid-vs-Hadoop.md index 68744179b1e..37559b1da8f 100644 --- a/docs/Druid-vs-Hadoop.md +++ b/docs/Druid-vs-Hadoop.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Druid is a complementary addition to Hadoop. Hadoop is great at storing and making accessible large amounts of individually low-value data. Unfortunately, Hadoop is not great at providing query speed guarantees on top of that data, nor does it have very good operational characteristics for a customer-facing production system. Druid, on the other hand, excels at taking high-value summaries of the low-value data on Hadoop, making it available in a fast and always-on fashion, such that it could be exposed directly to a customer. Druid also requires some infrastructure to exist for โ€œdeep storageโ€. HDFS is one of the implemented options for this โ€œdeep storageโ€. diff --git a/docs/Druid-vs-Impala-or-Shark.md b/docs/Druid-vs-Impala-or-Shark.md index e9a0c673b87..3174fbbea5f 100644 --- a/docs/Druid-vs-Impala-or-Shark.md +++ b/docs/Druid-vs-Impala-or-Shark.md @@ -1,3 +1,6 @@ +--- +layout: default +--- The question of Druid versus Impala or Shark basically comes down to your product requirements and what the systems were designed to do. Druid was designed to diff --git a/docs/Druid-vs-redshift.md b/docs/Druid-vs-redshift.md index 2b360a4668b..8469209b10b 100644 --- a/docs/Druid-vs-redshift.md +++ b/docs/Druid-vs-redshift.md @@ -1,3 +1,6 @@ +--- +layout: default +--- ###How does Druid compare to Redshift? In terms of drawing a differentiation, Redshift is essentially ParAccel (Actian) which Amazon is licensing. diff --git a/docs/Druid-vs-vertica.md b/docs/Druid-vs-vertica.md index b35f62e9f03..b20976b74a6 100644 --- a/docs/Druid-vs-vertica.md +++ b/docs/Druid-vs-vertica.md @@ -1,3 +1,6 @@ +--- +layout: default +--- How does Druid compare to Vertica? Vertica is similar to ParAccel/Redshift ([[Druid-vs-Redshift]]) described above in that it wasnโ€™t built for real-time streaming data ingestion and it supports full SQL. diff --git a/docs/Examples.md b/docs/Examples.md index 88ca41fb4fa..9ab10466e56 100644 --- a/docs/Examples.md +++ b/docs/Examples.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Examples ======== diff --git a/docs/Filters.md b/docs/Filters.md index f655861d5fb..41ae91f93e2 100644 --- a/docs/Filters.md +++ b/docs/Filters.md @@ -1,3 +1,6 @@ +--- +layout: default +--- A filter is a JSON object indicating which rows of data should be included in the computation for a query. Itโ€™s essentially the equivalent of the WHERE clause in SQL. Druid supports the following types of filters. ### Selector filter diff --git a/docs/Firehose.md b/docs/Firehose.md index ab9b2ac53d2..c571f035a10 100644 --- a/docs/Firehose.md +++ b/docs/Firehose.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Firehoses describe the data stream source. They are pluggable and thus the configuration schema can and will vary based on the `type` of the firehose. |Field|Type|Description|Required| diff --git a/docs/Granularities.md b/docs/Granularities.md index ea568dd7d62..cf5283841c0 100644 --- a/docs/Granularities.md +++ b/docs/Granularities.md @@ -1,3 +1,6 @@ +--- +layout: default +--- The granularity field determines how data gets bucketed across the time dimension, i.e how it gets aggregated by hour, day, minute, etc. It can be specified either as a string for simple granularities or as an object for arbitrary granularities. diff --git a/docs/GroupByQuery.md b/docs/GroupByQuery.md index 735dd5c393a..656ff1a41a1 100644 --- a/docs/GroupByQuery.md +++ b/docs/GroupByQuery.md @@ -1,3 +1,6 @@ +--- +layout: default +--- These types of queries take a groupBy query object and return an array of JSON objects where each object represents a grouping asked for by the query. An example groupBy query object is shown below: diff --git a/docs/Having.md b/docs/Having.md index 47226f1b88e..62ab4644451 100644 --- a/docs/Having.md +++ b/docs/Having.md @@ -1,3 +1,6 @@ +--- +layout: default +--- A having clause is a JSON object identifying which rows from a groupBy query should be returned, by specifying conditions on aggregated values. It is essentially the equivalent of the HAVING clause in SQL. diff --git a/docs/Home.md b/docs/Home.md index 88e1c86b8aa..934f11b8c92 100644 --- a/docs/Home.md +++ b/docs/Home.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Druid is an open-source analytics datastore designed for realtime, exploratory, queries on large-scale data sets (100โ€™s of Billions entries, 100โ€™s TB data). Druid provides for cost effective, always-on, realtime data ingestion and arbitrary data exploration. - Check out some [[Examples]] diff --git a/docs/Indexing-Service.md b/docs/Indexing-Service.md index 0e4ff939f4a..60abbd73b9f 100644 --- a/docs/Indexing-Service.md +++ b/docs/Indexing-Service.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Disclaimer: We are still in the process of finalizing the indexing service and these configs are prone to change at any time. We will announce when we feel the indexing service and the configurations described are stable. The indexing service is a distributed task/job queue. It accepts requests in the form of [[Tasks]] and executes those tasks across a set of worker nodes. Worker capacity can be automatically adjusted based on the number of tasks pending in the system. The indexing service is highly available, has built in retry logic, and can backup per task logs in deep storage. diff --git a/docs/Libraries.md b/docs/Libraries.md index 41374e310c1..75bc17c633c 100644 --- a/docs/Libraries.md +++ b/docs/Libraries.md @@ -1,3 +1,6 @@ +--- +layout: default +--- ### R - [RDruid](https://github.com/metamx/RDruid) - Druid connector for R diff --git a/docs/Loading-Your-Data.md b/docs/Loading-Your-Data.md index 568a20767ac..dd4b0f8a7fb 100644 --- a/docs/Loading-Your-Data.md +++ b/docs/Loading-Your-Data.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Once you have a realtime node working, it is time to load your own data to see how Druid performs. Druid can ingest data in three ways: via Kafka and a realtime node, via the indexing service, and via the Hadoop batch loader. Data is ingested in realtime using a [[Firehose]]. diff --git a/docs/Master.md b/docs/Master.md index 891f6b854ef..f7345524980 100644 --- a/docs/Master.md +++ b/docs/Master.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Master ====== @@ -12,7 +15,7 @@ Rules Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different compute node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The master loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The master will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule -For more information on rules, see [[Rule Configuration]]. +For more information on rules, see [[Rule Configuration.md]]. Cleaning Up Segments -------------------- diff --git a/docs/MySQL.md b/docs/MySQL.md index 79cf6ed6d8b..f7ee2ec4db1 100644 --- a/docs/MySQL.md +++ b/docs/MySQL.md @@ -1,3 +1,6 @@ +--- +layout: default +--- MySQL is an external dependency of Druid. We use it to store various metadata about the system, but not to store the actual data. There are a number of tables used for various purposes described below. Segments Table diff --git a/docs/OrderBy.md b/docs/OrderBy.md index 993df6f4674..9dcffff7886 100644 --- a/docs/OrderBy.md +++ b/docs/OrderBy.md @@ -1,3 +1,6 @@ +--- +layout: default +--- The orderBy field provides the functionality to sort and limit the set of results from a groupBy query. Available options are: ### DefaultLimitSpec diff --git a/docs/Plumber.md b/docs/Plumber.md index cf650fb6cdd..b2123e94393 100644 --- a/docs/Plumber.md +++ b/docs/Plumber.md @@ -1,3 +1,6 @@ +--- +layout: default +--- The Plumber is the thing that handles generated segments both while they are being generated and when they are โ€œdoneโ€. This is also technically a pluggable interface and there are multiple implementations, but there are a lot of details handled by the plumber such that it is expected that there will only be a few implementations and only more advanced third-parties will implement their own. See [here](https://github.com/metamx/druid/wiki/Plumber#available-plumbers) for a description of the plumbers included with Druid. |Field|Type|Description|Required| diff --git a/docs/Post-aggregations.md b/docs/Post-aggregations.md index 8ff7a91ecb5..4aa6c7f8db7 100644 --- a/docs/Post-aggregations.md +++ b/docs/Post-aggregations.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Post-aggregations are specifications of processing that should happen on aggregated values as they come out of Druid. If you include a post aggregation as part of a query, make sure to include all aggregators the post-aggregator requires. There are several post-aggregators available. diff --git a/docs/Querying-your-data.md b/docs/Querying-your-data.md index 520edcaf613..39d22ab3a32 100644 --- a/docs/Querying-your-data.md +++ b/docs/Querying-your-data.md @@ -1,3 +1,6 @@ +--- +layout: default +--- # Setup # Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [[Loading Your Data]] we setup a [[Realtime]], [[Compute]] and [[Master]] node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. diff --git a/docs/Querying.md b/docs/Querying.md index 21ed93c7bb5..db845bc694f 100644 --- a/docs/Querying.md +++ b/docs/Querying.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Querying ======== diff --git a/docs/Realtime.md b/docs/Realtime.md index 1908a469f80..c92cc7f7175 100644 --- a/docs/Realtime.md +++ b/docs/Realtime.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Realtime ======== diff --git a/docs/Rule-Configuration.md b/docs/Rule-Configuration.md index 1d2b4c03461..2695da646ab 100644 --- a/docs/Rule-Configuration.md +++ b/docs/Rule-Configuration.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Note: It is recommended that the master console is used to configure rules. However, the master node does have HTTP endpoints to programmatically configure rules. Load Rules diff --git a/docs/SearchQuery.md b/docs/SearchQuery.md index af125889c32..7acf04419fa 100644 --- a/docs/SearchQuery.md +++ b/docs/SearchQuery.md @@ -1,3 +1,6 @@ +--- +layout: default +--- A search query returns dimension values that match the search specification. { diff --git a/docs/SearchQuerySpec.md b/docs/SearchQuerySpec.md index 48036c65d56..9b9db04b8e6 100644 --- a/docs/SearchQuerySpec.md +++ b/docs/SearchQuerySpec.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Search query specs define how a โ€œmatchโ€ is defined between a search value and a dimension value. The available search query specs are: InsensitiveContainsSearchQuerySpec diff --git a/docs/SegmentMetadataQuery.md b/docs/SegmentMetadataQuery.md index 606d0800447..0e6eefb78e1 100644 --- a/docs/SegmentMetadataQuery.md +++ b/docs/SegmentMetadataQuery.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Segment metadata queries return per segment information about: \* Cardinality of all columns in the segment \* Estimated byte size for the segment columns in TSV format diff --git a/docs/Segments.md b/docs/Segments.md index 5bffdd30b10..7da12950d15 100644 --- a/docs/Segments.md +++ b/docs/Segments.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Segments ======== diff --git a/docs/Spatial-Filters.md b/docs/Spatial-Filters.md index c9ce15d5cc9..2ca83b9a3f9 100644 --- a/docs/Spatial-Filters.md +++ b/docs/Spatial-Filters.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Note: This feature is highly experimental and only works with spatially indexed dimensions. The grammar for a spatial filter is as follows: diff --git a/docs/Spatial-Indexing.md b/docs/Spatial-Indexing.md index 5f7dc2b174c..1df36593433 100644 --- a/docs/Spatial-Indexing.md +++ b/docs/Spatial-Indexing.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Note: This feature is highly experimental. In any of the data specs, there is now the option of providing spatial dimensions. For example, for a JSON data spec, spatial dimensions can be specified as follows: diff --git a/docs/Stand-Alone-With-Riak-CS.md b/docs/Stand-Alone-With-Riak-CS.md index aaa77b3151c..505b59f9283 100644 --- a/docs/Stand-Alone-With-Riak-CS.md +++ b/docs/Stand-Alone-With-Riak-CS.md @@ -1,3 +1,6 @@ +--- +layout: default +--- This page describes how to use Riak-CS for deep storage instead of S3. We are still setting up some of the peripheral stuff (file downloads, etc.). This guide provided by Pablo Nebrera, thanks! diff --git a/docs/Support.md b/docs/Support.md index 1561e935381..3dd512e050f 100644 --- a/docs/Support.md +++ b/docs/Support.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Numerous backend engineers at [Metamarkets](http://www.metamarkets.com) work on Druid full-time. If you any questions about usage or code, feel free to contact any of us. Google Groups Mailing List diff --git a/docs/Tasks.md b/docs/Tasks.md index 53f441696d9..95341f581ec 100644 --- a/docs/Tasks.md +++ b/docs/Tasks.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Tasks are run on workers and always operate on a single datasource. Once an indexer coordinator node accepts a task, a lock is created for the datasource and interval specified in the task. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a โ€œtask groupโ€, which is a set of tasks that can share interval locks. There are several different types of tasks. diff --git a/docs/Thanks.md b/docs/Thanks.md index f84708fb6c8..cb1c873cca0 100644 --- a/docs/Thanks.md +++ b/docs/Thanks.md @@ -1,3 +1,6 @@ +--- +layout: default +--- YourKit supports the Druid open source projects with its full-featured Java Profiler. YourKit, LLC is the creator of innovative and intelligent tools for profiling diff --git a/docs/TimeBoundaryQuery.md b/docs/TimeBoundaryQuery.md index 432df69961d..bde4ca1c812 100644 --- a/docs/TimeBoundaryQuery.md +++ b/docs/TimeBoundaryQuery.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Time boundary queries return the earliest and latest data points of a data set. The grammar is: { diff --git a/docs/TimeseriesQuery.md b/docs/TimeseriesQuery.md index d189b176a01..56f2ce733b9 100644 --- a/docs/TimeseriesQuery.md +++ b/docs/TimeseriesQuery.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Timeseries queries ================== diff --git a/docs/Tutorial:-A-First-Look-at-Druid.md b/docs/Tutorial:-A-First-Look-at-Druid.md index ef725135aa4..4722dd173c0 100644 --- a/docs/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/Tutorial:-A-First-Look-at-Druid.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Greetings! This tutorial will help clarify some core Druid concepts. We will use a realtime dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on! About the data diff --git a/docs/Tutorial:-The-Druid-Cluster.md b/docs/Tutorial:-The-Druid-Cluster.md index b01824e52a2..e2eff84f505 100644 --- a/docs/Tutorial:-The-Druid-Cluster.md +++ b/docs/Tutorial:-The-Druid-Cluster.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Welcome back! In our first [tutorial](https://github.com/metamx/druid/wiki/Tutorial%3A-A-First-Look-at-Druid), we introduced you to the most basic Druid setup: a single realtime node. We streamed in some data and queried it. Realtime nodes collect very recent data and periodically hand that data off to the rest of the Druid cluster. Some questions about the architecture must naturally come to mind. What does the rest of Druid cluster look like? How does Druid load available static data? This tutorial will hopefully answer these questions! diff --git a/docs/Tutorial:-Webstream.md b/docs/Tutorial:-Webstream.md index c8b0bcada8b..973204f31d4 100644 --- a/docs/Tutorial:-Webstream.md +++ b/docs/Tutorial:-Webstream.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Greetings! This tutorial will help clarify some core Druid concepts. We will use a realtime dataset and issue some basic Druid queries. If you are ready to explore Druid, and learn a thing or two, read on! About the data diff --git a/docs/Twitter-Tutorial.md b/docs/Twitter-Tutorial.md index c113282e937..cedd26b9250 100644 --- a/docs/Twitter-Tutorial.md +++ b/docs/Twitter-Tutorial.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Greetings! We see youโ€™ve taken an interest in Druid. Thatโ€™s awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [[Examples]], and issue some basic Druid queries. The data source weโ€™ll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on! Setting Up diff --git a/docs/Versioning.md b/docs/Versioning.md index 33c665d6542..7b9fa24045c 100644 --- a/docs/Versioning.md +++ b/docs/Versioning.md @@ -1,3 +1,6 @@ +--- +layout: default +--- This page discusses how we do versioning and provides information on our stable releases. Versioning Strategy diff --git a/docs/ZooKeeper.md b/docs/ZooKeeper.md index 250d0280bcb..03f2b1b8e0c 100644 --- a/docs/ZooKeeper.md +++ b/docs/ZooKeeper.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Druid uses ZooKeeper (ZK) for management of current cluster state. The operations that happen over ZK are 1. [[Master]] leader election diff --git a/docs/_config.yml b/docs/_config.yml index 362c8bf5f91..1ba74937d8b 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,2 +1,3 @@ name: Your New Jekyll Site pygments: true +markdown: redcarpet diff --git a/docs/contents.md b/docs/contents.md index 2298a7c8bdf..23b56bc33a5 100644 --- a/docs/contents.md +++ b/docs/contents.md @@ -1,3 +1,6 @@ +--- +layout: default +--- Contents \* [[Introduction|Home]] \* [[Download]] From 063a068ab238f2412c090c33c756a73540a77b17 Mon Sep 17 00:00:00 2001 From: Russell Jurney Date: Mon, 16 Sep 2013 16:01:14 -0700 Subject: [PATCH 79/92] Converted links, sans space to slash --- docs/Aggregations.md | 90 ++++++++++++++++++++++++- docs/Batch-ingestion.md | 8 +-- docs/Booting-a-production-cluster.md | 2 +- docs/Broker.md | 6 +- docs/Cluster-setup.md | 40 +++++------ docs/Compute.md | 8 +-- docs/Concepts-and-Terminology.md | 2 +- docs/Configuration.md | 12 ++-- docs/Contribute.md | 2 +- docs/Design.md | 2 +- docs/Download.md | 2 +- docs/Druid-Personal-Demo-Cluster.md | 4 +- docs/Druid-vs-Impala-or-Shark.md | 4 +- docs/Druid-vs-vertica.md | 2 +- docs/Examples.md | 4 +- docs/Firehose.md | 4 +- docs/Granularities.md | 2 +- docs/GroupByQuery.md | 12 ++-- docs/Home.md | 14 ++-- docs/Indexing-Service.md | 4 +- docs/Loading-Your-Data.md | 16 ++--- docs/Master.md | 4 +- docs/MySQL.md | 6 +- docs/Post-aggregations.md | 4 +- docs/Querying-your-data.md | 16 ++--- docs/Querying.md | 18 ++--- docs/Realtime.md | 10 +-- docs/SearchQuery.md | 6 +- docs/Segments.md | 2 +- docs/Stand-Alone-With-Riak-CS.md | 10 +-- docs/TimeseriesQuery.md | 8 +-- docs/Tutorial:-A-First-Look-at-Druid.md | 16 ++--- docs/Tutorial:-The-Druid-Cluster.md | 2 +- docs/Tutorial:-Webstream.md | 12 ++-- docs/Twitter-Tutorial.md | 14 ++-- docs/Versioning.md | 2 +- docs/ZooKeeper.md | 12 ++-- docs/contents.md | 64 +++++++++--------- 38 files changed, 265 insertions(+), 181 deletions(-) diff --git a/docs/Aggregations.md b/docs/Aggregations.md index 886dac7a572..37b99aeffc2 100644 --- a/docs/Aggregations.md +++ b/docs/Aggregations.md @@ -1,6 +1,90 @@ --- layout: default --- ---- -layout: default ---- +Aggregations are specifications of processing over metrics available in Druid. +Available aggregations are: + +### Sum aggregators + +#### `longSum` aggregator + +computes the sum of values as a 64-bit, signed integer + + { + "type" : "longSum", + "name" : , + "fieldName" : + } + +`name` โ€“ output name for the summed value +`fieldName` โ€“ name of the metric column to sum over + +#### `doubleSum` aggregator + +Computes the sum of values as 64-bit floating point value. Similar to `longSum` + + { + "type" : "doubleSum", + "name" : , + "fieldName" : + } + +### Count aggregator + +`count` computes the row count that match the filters + + { + "type" : "count", + "name" : , + } + +### Min / Max aggregators + +#### `min` aggregator + +`min` computes the minimum metric value + + { + "type" : "min", + "name" : , + "fieldName" : + } + +#### `max` aggregator + +`max` computes the maximum metric value + + { + "type" : "max", + "name" : , + "fieldName" : + } + +### JavaScript aggregator + +Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions). + +All JavaScript functions must return numerical values. + + { + "type": "javascript", + "name": "", + "fieldNames" : [ , , ... ], + "fnAggregate" : "function(current, column1, column2, ...) { + + return + }" + "fnCombine" : "function(partialA, partialB) { return ; }" + "fnReset" : "function() { return ; }" + } + +**Example** + + { + "type": "javascript", + "name": "sum(log(x)/y) + 10", + "fieldNames": ["x", "y"], + "fnAggregate" : "function(current, a, b) { return current + (Math.log(a) * b); }" + "fnCombine" : "function(partialA, partialB) { return partialA + partialB; }" + "fnReset" : "function() { return 10; }" + } \ No newline at end of file diff --git a/docs/Batch-ingestion.md b/docs/Batch-ingestion.md index f91f0dbb081..6511b85b452 100644 --- a/docs/Batch-ingestion.md +++ b/docs/Batch-ingestion.md @@ -4,14 +4,14 @@ layout: default Batch Data Ingestion ==================== -There are two choices for batch data ingestion to your Druid cluster, you can use the [[Indexing service]] or you can use the `HadoopDruidIndexerMain`. This page describes how to use the `HadoopDruidIndexerMain`. +There are two choices for batch data ingestion to your Druid cluster, you can use the [Indexing service](Indexing-service.html) or you can use the `HadoopDruidIndexerMain`. This page describes how to use the `HadoopDruidIndexerMain`. Which should I use? ------------------- -The [[Indexing service]] is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data. +The [Indexing service](Indexing service.html) is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data. -The `HadoopDruidIndexerMain` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and donโ€™t want to spend the time configuring and deploying the [[Indexing service]] just yet. +The `HadoopDruidIndexerMain` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and donโ€™t want to spend the time configuring and deploying the [Indexing service](Indexing service.html) just yet. HadoopDruidIndexer ------------------ @@ -138,4 +138,4 @@ This is a specification of the properties that tell the job how to update metada |password|password for db|yes| |segmentTable|table to use in DB|yes| -These properties should parrot what you have configured for your [[Master]]. +These properties should parrot what you have configured for your [Master](Master.html). diff --git a/docs/Booting-a-production-cluster.md b/docs/Booting-a-production-cluster.md index c25ef25c607..d5fc38c8ce5 100644 --- a/docs/Booting-a-production-cluster.md +++ b/docs/Booting-a-production-cluster.md @@ -3,7 +3,7 @@ layout: default --- # Booting a Single Node Cluster # -[[Loading Your Data]] and [[Querying Your Data]] contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz). +[Loading Your Data](Loading Your Data.html) and [Querying Your Data](Querying Your Data.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz). The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables: diff --git a/docs/Broker.md b/docs/Broker.md index e71100e9915..fee33bedc83 100644 --- a/docs/Broker.md +++ b/docs/Broker.md @@ -9,9 +9,9 @@ The Broker is the node to route queries to if you want to run a distributed clus Forwarding Queries ------------------ -Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [[Segments]] are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple nodes, and hence, the query will likely hit multiple nodes. +Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](Segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple nodes, and hence, the query will likely hit multiple nodes. -To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [[Compute]] and [[Realtime]] nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker node then forwards down the query to the selected nodes. +To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Compute](Compute.html) and [Realtime](Realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker node then forwards down the query to the selected nodes. Caching ------- @@ -27,4 +27,4 @@ Broker nodes can be run using the `com.metamx.druid.http.BrokerMain` class. Configuration ------------- -See [[Configuration]]. +See [Configuration](Configuration.html). diff --git a/docs/Cluster-setup.md b/docs/Cluster-setup.md index 29837e94296..b8281e99468 100644 --- a/docs/Cluster-setup.md +++ b/docs/Cluster-setup.md @@ -1,7 +1,7 @@ --- layout: default --- -A Druid cluster consists of various node types that need to be set up depending on your use case. See our [[Design]] docs for a description of the different node types. +A Druid cluster consists of various node types that need to be set up depending on your use case. See our [Design](Design.html) docs for a description of the different node types. Setup Scripts ------------- @@ -11,14 +11,14 @@ One of our community members, [housejester](https://github.com/housejester/), co Minimum Physical Layout: Absolute Minimum ----------------------------------------- -As a special case, the absolute minimum setup is one of the standalone examples for realtime ingestion and querying; see [[Examples]] that can easily run on one machine with one core and 1GB RAM. This layout can be set up to try some basic queries with Druid. +As a special case, the absolute minimum setup is one of the standalone examples for realtime ingestion and querying; see [Examples](Examples.html) that can easily run on one machine with one core and 1GB RAM. This layout can be set up to try some basic queries with Druid. Minimum Physical Layout: Experimental Testing with 4GB of RAM ------------------------------------------------------------- This layout can be used to load some data from deep storage onto a Druid compute node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is: -1. node1: [[Master]] + metadata service + zookeeper + [[Compute]] +1. node1: [Master](Master.html) + metadata service + zookeeper + [Compute](Compute.html) 2. transient nodes: indexer This setup is only reasonable to prove that a configuration works. It would not be worthwhile to use this layout for performance measurement. @@ -30,13 +30,13 @@ Comfortable Physical Layout: Pilot Project with Multiple Machines A minimal physical layout not constrained by cores that demonstrates parallel querying and realtime, using AWS-EC2 โ€œsmallโ€/m1.small (one core, with 1.7GB of RAM) or larger, no realtime, is: -1. node1: [[Master]] (m1.small) +1. node1: [Master](Master.html) (m1.small) 2. node2: metadata service (m1.small) 3. node3: zookeeper (m1.small) -4. node4: [[Broker]] (m1.small or m1.medium or m1.large) -5. node5: [[Compute]] (m1.small or m1.medium or m1.large) -6. node6: [[Compute]] (m1.small or m1.medium or m1.large) -7. node7: [[Realtime]] (m1.small or m1.medium or m1.large) +4. node4: [Broker](Broker.html) (m1.small or m1.medium or m1.large) +5. node5: [Compute](Compute.html) (m1.small or m1.medium or m1.large) +6. node6: [Compute](Compute.html) (m1.small or m1.medium or m1.large) +7. node7: [Realtime](Realtime.html) (m1.small or m1.medium or m1.large) 8. transient nodes: indexer This layout naturally lends itself to adding more RAM and core to Compute nodes, and to adding many more Compute nodes. Depending on the actual load, the Master, metadata server, and Zookeeper might need to use larger machines. @@ -48,18 +48,18 @@ High Availability Physical Layout An HA layout allows full rolling restarts and heavy volume: -1. node1: [[Master]] (m1.small or m1.medium or m1.large) -2. node2: [[Master]] (m1.small or m1.medium or m1.large) (backup) +1. node1: [Master](Master.html) (m1.small or m1.medium or m1.large) +2. node2: [Master](Master.html) (m1.small or m1.medium or m1.large) (backup) 3. node3: metadata service (c1.medium or m1.large) 4. node4: metadata service (c1.medium or m1.large) (backup) 5. node5: zookeeper (c1.medium) 6. node6: zookeeper (c1.medium) 7. node7: zookeeper (c1.medium) -8. node8: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -9. node9: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) (backup) -10. node10: [[Compute]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -11. node11: [[Compute]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) -12. node12: [[Realtime]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +8. node8: [Broker](Broker.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +9. node9: [Broker](Broker.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) (backup) +10. node10: [Compute](Compute.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +11. node11: [Compute](Compute.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) +12. node12: [Realtime](Realtime.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) 13. transient nodes: indexer Sizing for Cores and RAM @@ -79,7 +79,7 @@ Local disk (โ€œephemeralโ€ on AWS EC2) for caching is recommended over network Setup ----- -Setting up a cluster is essentially just firing up all of the nodes you want with the proper [[configuration]]. One thing to be aware of is that there are a few properties in the configuration that potentially need to be set individually for each process: +Setting up a cluster is essentially just firing up all of the nodes you want with the proper [configuration](configuration.html). One thing to be aware of is that there are a few properties in the configuration that potentially need to be set individually for each process: druid.server.type=historical|realtime @@ -107,8 +107,8 @@ The following table shows the possible services and fully qualified class for ma |service|main class| |-------|----------| -|[[ Realtime ]]|com.metamx.druid.realtime.RealtimeMain| -|[[ Master ]]|com.metamx.druid.http.MasterMain| -|[[ Broker ]]|com.metamx.druid.http.BrokerMain| -|[[ Compute ]]|com.metamx.druid.http.ComputeMain| +|[ Realtime ]( Realtime .html)|com.metamx.druid.realtime.RealtimeMain| +|[ Master ]( Master .html)|com.metamx.druid.http.MasterMain| +|[ Broker ]( Broker .html)|com.metamx.druid.http.BrokerMain| +|[ Compute ]( Compute .html)|com.metamx.druid.http.ComputeMain| diff --git a/docs/Compute.md b/docs/Compute.md index 8df11f8ca2f..e7df17ebbd5 100644 --- a/docs/Compute.md +++ b/docs/Compute.md @@ -11,9 +11,9 @@ Loading and Serving Segments Each compute node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Compute nodes do not communicate directly with each other or with the master nodes but instead rely on Zookeeper for coordination. -The [[Master]] node is responsible for assigning new segments to compute nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a compute node. For more information on how the master assigns segments to compute nodes, please see [[Master]]. +The [Master](Master.html) node is responsible for assigning new segments to compute nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a compute node. For more information on how the master assigns segments to compute nodes, please see [Master](Master.html). -When a compute node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the compute node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [[Segments]]. Once a compute node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying. +When a compute node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the compute node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [Segments](Segments.html). Once a compute node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying. Loading and Serving Segments From Cache --------------------------------------- @@ -25,7 +25,7 @@ The segment cache is also leveraged when a compute node is first started. On sta Querying Segments ----------------- -Please see [[Querying]] for more information on querying compute nodes. +Please see [Querying](Querying.html) for more information on querying compute nodes. For every query that a compute node services, it will log the query and report metrics on the time taken to run the query. @@ -37,4 +37,4 @@ Compute nodes can be run using the `com.metamx.druid.http.ComputeMain` class. Configuration ------------- -See [[Configuration]]. +See [Configuration](Configuration.html). diff --git a/docs/Concepts-and-Terminology.md b/docs/Concepts-and-Terminology.md index 1e7f535388d..925941dd8c5 100644 --- a/docs/Concepts-and-Terminology.md +++ b/docs/Concepts-and-Terminology.md @@ -12,4 +12,4 @@ Concepts and Terminology - **Segment:** A collection of (internal) records that are stored and processed together. - **Shard:** A unit of partitioning data across machine. TODO: clarify; by time or other dimensions? -- **specFile** is specification for services in JSON format; see [[Realtime]] and [[Batch-ingestion]] +- **specFile** is specification for services in JSON format; see [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html) diff --git a/docs/Configuration.md b/docs/Configuration.md index 353b8be77b3..544b9ea4f55 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -1,7 +1,7 @@ --- layout: default --- -This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json โ€œspecFileโ€ descriptions in [[Realtime]] and [[Batch-ingestion]]. +This describes the basic server configuration that is loaded by all the server processes; the same file is loaded by all. See also the json โ€œspecFileโ€ descriptions in [Realtime](Realtime.html) and [Batch-ingestion](Batch-ingestion.html). JVM Configuration Best Practices ================================ @@ -80,7 +80,7 @@ Configuration groupings ### S3 Access -These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [[Compute]], [[Realtime]] and [[Indexing service]] nodes. +These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [Compute](Compute.html), [Realtime](Realtime.html) and [Indexing service](Indexing service.html) nodes. |Property|Description|Default| |--------|-----------|-------| @@ -91,7 +91,7 @@ These properties are for connecting with S3 and using it to pull down segments. ### JDBC connection -These properties specify the jdbc connection and other configuration around the โ€œsegments tableโ€ database. The only processes that connect to the DB with these properties are the [[Master]] and [[Indexing service]]. This is tested on MySQL. +These properties specify the jdbc connection and other configuration around the โ€œsegments tableโ€ database. The only processes that connect to the DB with these properties are the [Master](Master.html) and [Indexing service](Indexing service.html). This is tested on MySQL. |Property|Description|Default| |--------|-----------|-------| @@ -113,7 +113,7 @@ These properties specify the jdbc connection and other configuration around the ### Zk properties -See [[ZooKeeper]] for a description of these properties. +See [ZooKeeper](ZooKeeper.html) for a description of these properties. ### Service properties @@ -146,7 +146,7 @@ These are properties that the compute nodes use ### Emitter Properties -The Druid servers emit various metrics and alerts via something we call an [[Emitter]]. There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. More information can be found on the [[Emitter]] page. The properties for using the logging emitter are described below. +The Druid servers emit various metrics and alerts via something we call an [Emitter](Emitter.html). There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. More information can be found on the [Emitter](Emitter.html) page. The properties for using the logging emitter are described below. |Property|Description|Default| |--------|-----------|-------| @@ -158,5 +158,5 @@ The Druid servers emit various metrics and alerts via something we call an [[Emi |Property|Description|Default| |--------|-----------|-------| -|`druid.realtime.specFile`|The file with realtime specifications in it. See [[Realtime]].|none| +|`druid.realtime.specFile`|The file with realtime specifications in it. See [Realtime](Realtime.html).|none| diff --git a/docs/Contribute.md b/docs/Contribute.md index 58d53a6d224..a853eb430d2 100644 --- a/docs/Contribute.md +++ b/docs/Contribute.md @@ -5,4 +5,4 @@ If you are interested in contributing to the code, we accept [pull requests](htt For issue tracking, we are using the github issue tracker. Please fill out an issue from the Issues tab on the github screen. -We also have a [[Libraries]] page that lists external libraries that people have created for working with Druid. +We also have a [Libraries](Libraries.html) page that lists external libraries that people have created for working with Druid. diff --git a/docs/Design.md b/docs/Design.md index 888d0b871b3..25d69d95fe8 100644 --- a/docs/Design.md +++ b/docs/Design.md @@ -53,7 +53,7 @@ Getting data into the Druid system requires an indexing process. This gives the - Bitmap compression - RLE (on the roadmap, but not yet implemented) -The output of the indexing process is stored in a โ€œdeep storageโ€ LOB store/file system ([[Deep Storage]] for information about potential options). Data is then loaded by compute nodes by first downloading the data to their local disk and then memory mapping it before serving queries. +The output of the indexing process is stored in a โ€œdeep storageโ€ LOB store/file system ([Deep Storage](Deep Storage.html) for information about potential options). Data is then loaded by compute nodes by first downloading the data to their local disk and then memory mapping it before serving queries. If a compute node dies, it will no longer serve its segments, but given that the segments are still available on the โ€œdeep storageโ€ any other node can simply download the segment and start serving it. This means that it is possible to actually remove all compute nodes from the cluster and then re-provision them without any data loss. It also means that if the โ€œdeep storageโ€ is not available, the nodes can continue to serve the segments they have already pulled down (i.e. the cluster goes stale, not down). diff --git a/docs/Download.md b/docs/Download.md index 00de8597f11..1bf1352de58 100644 --- a/docs/Download.md +++ b/docs/Download.md @@ -1,7 +1,7 @@ --- layout: default --- -A version may be declared as a release candidate if it has been deployed to a sizable production cluster. Release candidates are declared as stable after we feel fairly confident there are no major bugs in the version. Check out the [[Versioning]] section for how we describe software versions. +A version may be declared as a release candidate if it has been deployed to a sizable production cluster. Release candidates are declared as stable after we feel fairly confident there are no major bugs in the version. Check out the [Versioning](Versioning.html) section for how we describe software versions. Release Candidate ----------------- diff --git a/docs/Druid-Personal-Demo-Cluster.md b/docs/Druid-Personal-Demo-Cluster.md index ab49d828dbc..498f8ff8e14 100644 --- a/docs/Druid-Personal-Demo-Cluster.md +++ b/docs/Druid-Personal-Demo-Cluster.md @@ -3,7 +3,7 @@ layout: default --- # Druid Personal Demo Cluster (DPDC) -Note, there are currently some issues with the CloudFormation. We are working through them and will update the documentation here when things work properly. In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version. If you just want to get a feel for the types of data and queries that you can issue, check out [[Realtime Examples]] +Note, there are currently some issues with the CloudFormation. We are working through them and will update the documentation here when things work properly. In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version. If you just want to get a feel for the types of data and queries that you can issue, check out [Realtime Examples](Realtime Examples.html) ## Introduction To make it easy for you to get started with Druid, we created an AWS (Amazon Web Services) [CloudFormation](http://aws.amazon.com/cloudformation/) Template that allows you to create a small pre-configured Druid cluster using your own AWS account. The cluster contains a pre-loaded sample workload, the Wikipedia edit stream, and a basic query interface that gets you familiar with Druid capabilities like drill-downs and filters. @@ -14,7 +14,7 @@ This guide walks you through the steps to create the cluster and then how to cre ## Whatโ€™s in this Druid Demo Cluster? -1. A single "Master" node. This node co-locates the [[Master]] process, the [[Broker]] process, Zookeeper, and the MySQL instance. You can read more about Druid architecture [[Design]]. +1. A single "Master" node. This node co-locates the [Master](Master.html) process, the [Broker](Broker.html) process, Zookeeper, and the MySQL instance. You can read more about Druid architecture [Design](Design.html). 1. Three compute nodes; these compute nodes, have been pre-configured to work with the Master node and should automatically load up the Wikipedia edit stream data (no specific setup is required). diff --git a/docs/Druid-vs-Impala-or-Shark.md b/docs/Druid-vs-Impala-or-Shark.md index 3174fbbea5f..ee59b3def0c 100644 --- a/docs/Druid-vs-Impala-or-Shark.md +++ b/docs/Druid-vs-Impala-or-Shark.md @@ -20,11 +20,11 @@ What does this mean? We can talk about it in terms of four general areas ## Fault Tolerance -Druid pulls segments down from [[Deep Storage]] before serving queries on top of it. This means that for the data to exist in the Druid cluster, it must exist as a local copy on a historical node. If deep storage becomes unavailable for any reason, new segments will not be loaded into the system, but the cluster will continue to operate exactly as it was when the backing store disappeared. +Druid pulls segments down from [Deep Storage](Deep Storage.html) before serving queries on top of it. This means that for the data to exist in the Druid cluster, it must exist as a local copy on a historical node. If deep storage becomes unavailable for any reason, new segments will not be loaded into the system, but the cluster will continue to operate exactly as it was when the backing store disappeared. Impala and Shark, on the other hand, pull their data in from HDFS (or some other Hadoop FileSystem) in response to a query. This has implications for the operation of queries if you need to take HDFS down for a bit (say a software upgrade). It's possible that data that has been cached in the nodes is still available when the backing file system goes down, but I'm not sure. -This is just one example, but Druid was built to continue operating in the face of failures of any one of its various pieces. The [[Design]] describes these design decisions from the Druid side in more detail. +This is just one example, but Druid was built to continue operating in the face of failures of any one of its various pieces. The [Design](Design.html) describes these design decisions from the Druid side in more detail. ## Query Speed diff --git a/docs/Druid-vs-vertica.md b/docs/Druid-vs-vertica.md index b20976b74a6..535e5e06300 100644 --- a/docs/Druid-vs-vertica.md +++ b/docs/Druid-vs-vertica.md @@ -3,7 +3,7 @@ layout: default --- How does Druid compare to Vertica? -Vertica is similar to ParAccel/Redshift ([[Druid-vs-Redshift]]) described above in that it wasnโ€™t built for real-time streaming data ingestion and it supports full SQL. +Vertica is similar to ParAccel/Redshift ([Druid-vs-Redshift](Druid-vs-Redshift.html)) described above in that it wasnโ€™t built for real-time streaming data ingestion and it supports full SQL. The other big difference is that instead of employing indexing, Vertica tries to optimize processing by leveraging run-length encoding (RLE) and other compression techniques along with a โ€œprojectionโ€ system that creates materialized copies of the data in a different sort order (to maximize the effectiveness of RLE). diff --git a/docs/Examples.md b/docs/Examples.md index 9ab10466e56..2f48f60b1b5 100644 --- a/docs/Examples.md +++ b/docs/Examples.md @@ -34,7 +34,7 @@ Clone Druid and build it: Twitter Example --------------- -For a full tutorial based on the twitter example, check out this [[Twitter Tutorial]]. +For a full tutorial based on the twitter example, check out this [Twitter Tutorial](Twitter Tutorial.html). This Example uses a feature of Twitter that allows for sampling of itโ€™s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/metamx/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded. @@ -48,7 +48,7 @@ This Example uses a feature of Twitter that allows for sampling of itโ€™s stream ### What youโ€™ll do -See [[Tutorial]] +See [Tutorial](Tutorial.html) Rand Example ------------ diff --git a/docs/Firehose.md b/docs/Firehose.md index c571f035a10..92c5caa2386 100644 --- a/docs/Firehose.md +++ b/docs/Firehose.md @@ -28,11 +28,11 @@ This firehose ingests events from a predefined list of S3 objects. #### TwitterSpritzerFirehose -See [[Examples]]. This firehose connects directly to the twitter spritzer data stream. +See [Examples](Examples.html). This firehose connects directly to the twitter spritzer data stream. #### RandomFirehose -See [[Examples]]. This firehose creates a stream of random numbers. +See [Examples](Examples.html). This firehose creates a stream of random numbers. #### RabbitMqFirehouse diff --git a/docs/Granularities.md b/docs/Granularities.md index cf5283841c0..0cb25a7a5df 100644 --- a/docs/Granularities.md +++ b/docs/Granularities.md @@ -11,7 +11,7 @@ Simple granularities are specified as a string and bucket timestamps by their UT Supported granularity strings are: `all`, `none`, `minute`, `fifteen_minute`, `thirty_minute`, `hour` and `day` \* **`all`** buckets everything into a single bucket -\* **`none`** does not bucket data (it actually uses the granularity of the index - minimum here is `none` which means millisecond granularity). Using `none` in a [[timeseries query|TimeSeriesQuery]] is currently not recommended (the system will try to generate 0 values for all milliseconds that didnโ€™t exist, which is often a lot). +\* **`none`** does not bucket data (it actually uses the granularity of the index - minimum here is `none` which means millisecond granularity). Using `none` in a [timeseries query|TimeSeriesQuery](timeseries query|TimeSeriesQuery.html) is currently not recommended (the system will try to generate 0 values for all milliseconds that didnโ€™t exist, which is often a lot). ### Duration Granularities diff --git a/docs/GroupByQuery.md b/docs/GroupByQuery.md index 656ff1a41a1..7e95ebcbdee 100644 --- a/docs/GroupByQuery.md +++ b/docs/GroupByQuery.md @@ -93,12 +93,12 @@ There are 9 main parts to a groupBy query: |queryType|This String should always be โ€œgroupByโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| |dimensions|A JSON list of dimensions to do the groupBy over|yes| -|orderBy|See [[OrderBy]].|no| -|having|See [[Having]].|no| -|granularity|Defines the granularity of the query. See [[Granularities]]|yes| -|filter|See [[Filters]]|no| -|aggregations|See [[Aggregations]]|yes| -|postAggregations|See [[Post Aggregations]]|no| +|orderBy|See [OrderBy](OrderBy.html).|no| +|having|See [Having](Having.html).|no| +|granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes| +|filter|See [Filters](Filters.html)|no| +|aggregations|See [Aggregations](Aggregations.html)|yes| +|postAggregations|See [Post Aggregations](Post Aggregations.html)|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/docs/Home.md b/docs/Home.md index 934f11b8c92..8587aae4749 100644 --- a/docs/Home.md +++ b/docs/Home.md @@ -3,7 +3,7 @@ layout: default --- Druid is an open-source analytics datastore designed for realtime, exploratory, queries on large-scale data sets (100โ€™s of Billions entries, 100โ€™s TB data). Druid provides for cost effective, always-on, realtime data ingestion and arbitrary data exploration. -- Check out some [[Examples]] +- Check out some [Examples](Examples.html) - Try out Druid with our Getting Started [Tutorial](https://github.com/metamx/druid/wiki/Tutorial%3A-A-First-Look-at-Druid) - Learn more by reading the [White Paper](http://static.druid.io/docs/druid.pdf) @@ -19,7 +19,7 @@ The first one is the joy that everyone feels the first time they get Hadoop runn Druid is especially useful if you are summarizing your data sets and then querying the summarizations. If you put your summarizations into Druid, you will get quick queryability out of a system that you can be confident will scale up as your data volumes increase. Deployments have scaled up to 2TB of data per hour at peak ingested and aggregated in real-time. -We have more details about the general design of the system and why you might want to use it in our [White Paper](http://static.druid.io/docs/druid.pdf) or in our [[Design]] doc. +We have more details about the general design of the system and why you might want to use it in our [White Paper](http://static.druid.io/docs/druid.pdf) or in our [Design](Design.html) doc. The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means, we will fix this page. @@ -38,11 +38,11 @@ The data store world is vast, confusing and constantly in flux. This page is mea \* Downtime is no big deal #### Druid vsโ€ฆ -\* [[Druid-vs-Impala-or-Shark]] -\* [[Druid-vs-Redshift]] -\* [[Druid-vs-Vertica]] -\* [[Druid-vs-Cassandra]] -\* [[Druid-vs-Hadoop]] +\* [Druid-vs-Impala-or-Shark](Druid-vs-Impala-or-Shark.html) +\* [Druid-vs-Redshift](Druid-vs-Redshift.html) +\* [Druid-vs-Vertica](Druid-vs-Vertica.html) +\* [Druid-vs-Cassandra](Druid-vs-Cassandra.html) +\* [Druid-vs-Hadoop](Druid-vs-Hadoop.html) Key Features ------------ diff --git a/docs/Indexing-Service.md b/docs/Indexing-Service.md index 60abbd73b9f..d878e1b4176 100644 --- a/docs/Indexing-Service.md +++ b/docs/Indexing-Service.md @@ -3,7 +3,7 @@ layout: default --- Disclaimer: We are still in the process of finalizing the indexing service and these configs are prone to change at any time. We will announce when we feel the indexing service and the configurations described are stable. -The indexing service is a distributed task/job queue. It accepts requests in the form of [[Tasks]] and executes those tasks across a set of worker nodes. Worker capacity can be automatically adjusted based on the number of tasks pending in the system. The indexing service is highly available, has built in retry logic, and can backup per task logs in deep storage. +The indexing service is a distributed task/job queue. It accepts requests in the form of [Tasks](Tasks.html) and executes those tasks across a set of worker nodes. Worker capacity can be automatically adjusted based on the number of tasks pending in the system. The indexing service is highly available, has built in retry logic, and can backup per task logs in deep storage. The indexing service is composed of two main components, a coordinator node that manages task distribution and worker capacity, and worker nodes that execute tasks in separate JVMs. @@ -45,7 +45,7 @@ The coordinator also exposes a simple UI to show what tasks are currently runnin #### Task Execution -The coordinator retrieves worker setup metadata from the Druid [[MySQL]] config table. This metadata contains information about the version of workers to create, the maximum and minimum number of workers in the cluster at one time, and additional information required to automatically create workers. +The coordinator retrieves worker setup metadata from the Druid [MySQL](MySQL.html) config table. This metadata contains information about the version of workers to create, the maximum and minimum number of workers in the cluster at one time, and additional information required to automatically create workers. Tasks are assigned to workers by creating entries under specific /tasks paths associated with a worker, similar to how the Druid master node assigns segments to compute nodes. See [Worker Configuration](Indexing-Service#configuration-1). Once a worker picks up a task, it deletes the task entry and announces a task status under a /status path associated with the worker. Tasks are submitted to a worker until the worker hits capacity. If all workers in a cluster are at capacity, the indexer coordinator node automatically creates new worker resources. diff --git a/docs/Loading-Your-Data.md b/docs/Loading-Your-Data.md index dd4b0f8a7fb..a5edd9d65ea 100644 --- a/docs/Loading-Your-Data.md +++ b/docs/Loading-Your-Data.md @@ -3,7 +3,7 @@ layout: default --- Once you have a realtime node working, it is time to load your own data to see how Druid performs. -Druid can ingest data in three ways: via Kafka and a realtime node, via the indexing service, and via the Hadoop batch loader. Data is ingested in realtime using a [[Firehose]]. +Druid can ingest data in three ways: via Kafka and a realtime node, via the indexing service, and via the Hadoop batch loader. Data is ingested in realtime using a [Firehose](Firehose.html). ## Create Config Directories ## Each type of node needs its own config file and directory, so create them as subdirectories under the druid directory. @@ -17,7 +17,7 @@ mkdir config/broker ## Loading Data with Kafka ## -[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/master/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [[Firehose]] with the right configuration, we can import data into Druid in realtime without writing any code. To load data to a realtime node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [[Realtime]] node. +[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/master/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in realtime without writing any code. To load data to a realtime node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node. ### Booting Kafka ### @@ -165,7 +165,7 @@ curl -X POST "http://localhost:8080/druid/v2/?pretty" \ } } ] ``` -Now you're ready for [[Querying Your Data]]! +Now you're ready for [Querying Your Data](Querying Your Data.html)! ## Loading Data with the HadoopDruidIndexer ## @@ -184,7 +184,7 @@ mysql -u root GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd'; CREATE database druid; ``` -The [[Master]] node will create the tables it needs based on its configuration. +The [Master](Master.html) node will create the tables it needs based on its configuration. ### Make sure you have ZooKeeper Running ### @@ -206,7 +206,7 @@ cd .. ``` ### Launch a Master Node ### -If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [[Realtime]] node, we use 8081 for the [[Master]]. +If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [Realtime](Realtime.html) node, we use 8081 for the [Master](Master.html). 1. Setup a configuration file called config/master/runtime.properties similar to: ```bash @@ -251,7 +251,7 @@ druid.paths.indexCache=/tmp/druid/indexCache # Path on local FS for storage of segment metadata; dir will be created if needed druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache ``` -2. Launch the [[Master]] node +2. Launch the [Master](Master.html) node ```bash java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \ -classpath lib/*:config/master \ @@ -324,7 +324,7 @@ We can use the same records we have been, in a file called records.json: ### Run the Hadoop Job ### -Now its time to run the Hadoop [[Batch-ingestion]] job, HadoopDruidIndexer, which will fill a historical [[Compute]] node with data. First we'll need to configure the job. +Now its time to run the Hadoop [Batch-ingestion](Batch-ingestion.html) job, HadoopDruidIndexer, which will fill a historical [Compute](Compute.html) node with data. First we'll need to configure the job. 1. Create a config called batchConfig.json similar to: ```json @@ -367,4 +367,4 @@ Now its time to run the Hadoop [[Batch-ingestion]] job, HadoopDruidIndexer, whic java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=realtime.spec -classpath lib/* com.metamx.druid.indexer.HadoopDruidIndexerMain batchConfig.json ``` -You can now move on to [[Querying Your Data]]! \ No newline at end of file +You can now move on to [Querying Your Data](Querying Your Data.html)! \ No newline at end of file diff --git a/docs/Master.md b/docs/Master.md index f7345524980..c96af56dea9 100644 --- a/docs/Master.md +++ b/docs/Master.md @@ -15,7 +15,7 @@ Rules Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different compute node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The master loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The master will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule -For more information on rules, see [[Rule Configuration.md]]. +For more information on rules, see [Rule Configuration](Rule Configuration.html). Cleaning Up Segments -------------------- @@ -103,4 +103,4 @@ Master nodes can be run using the `com.metamx.druid.http.MasterMain` class. Configuration ------------- -See [[Configuration]]. +See [Configuration](Configuration.html). diff --git a/docs/MySQL.md b/docs/MySQL.md index f7ee2ec4db1..88ef75006cf 100644 --- a/docs/MySQL.md +++ b/docs/MySQL.md @@ -8,7 +8,7 @@ Segments Table This is dictated by the `druid.database.segmentTable` property (Note that these properties are going to change in the next stable version after 0.4.12). -This table stores metadata about the segments that are available in the system. The table is polled by the [[Master]] to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes. +This table stores metadata about the segments that are available in the system. The table is polled by the [Master](Master.html) to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes. The `used` column is a boolean โ€œtombstoneโ€. A 1 means that the segment should be โ€œusedโ€ by the cluster (i.e. it should be loaded and available for requests). A 0 means that the segment should not be actively loaded into the cluster. We do this as a means of removing segments from the cluster without actually removing their metadata (which allows for simpler rolling back if that is ever an issue). @@ -34,7 +34,7 @@ Note that the format of this blob can and will change from time-to-time. Rule Table ---------- -The rule table is used to store the various rules about where segments should land. These rules are used by the [[Master]] when making segment (re-)allocation decisions about the cluster. +The rule table is used to store the various rules about where segments should land. These rules are used by the [Master](Master.html) when making segment (re-)allocation decisions about the cluster. Config Table ------------ @@ -44,4 +44,4 @@ The config table is used to store runtime configuration objects. We do not have Task-related Tables ------------------- -There are also a number of tables created and used by the [[Indexing Service]] in the course of its work. +There are also a number of tables created and used by the [Indexing Service](Indexing Service.html) in the course of its work. diff --git a/docs/Post-aggregations.md b/docs/Post-aggregations.md index 4aa6c7f8db7..2e11f98d0e0 100644 --- a/docs/Post-aggregations.md +++ b/docs/Post-aggregations.md @@ -22,9 +22,9 @@ The grammar for an arithmetic post aggregation is: ### Field accessor post-aggregator -This returns the value produced by the specified [[aggregator|Aggregations]]. +This returns the value produced by the specified [aggregator|Aggregations](aggregator|Aggregations.html). -`fieldName` refers to the output name of the aggregator given in the [[aggregations|Aggregations]] portion of the query. +`fieldName` refers to the output name of the aggregator given in the [aggregations|Aggregations](aggregations|Aggregations.html) portion of the query. field_accessor : { "type" : "fieldAccess", diff --git a/docs/Querying-your-data.md b/docs/Querying-your-data.md index 39d22ab3a32..5bf72a6fa54 100644 --- a/docs/Querying-your-data.md +++ b/docs/Querying-your-data.md @@ -3,7 +3,7 @@ layout: default --- # Setup # -Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [[Loading Your Data]] we setup a [[Realtime]], [[Compute]] and [[Master]] node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. +Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading Your Data.html) we setup a [Realtime](Realtime.html), [Compute](Compute.html) and [Master](Master.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'. ## Booting a Broker Node ## @@ -98,11 +98,11 @@ com.metamx.druid.http.ComputeMain # Querying Your Data # -Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [[Loading Your Data]]. Having done that, its time to query our data! For a complete specification of queries, see [[Querying]]. +Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [Loading Your Data](Loading Your Data.html). Having done that, its time to query our data! For a complete specification of queries, see [Querying](Querying.html). ## Querying Different Nodes ## -As a shared-nothing system, there are three ways to query druid, against the [[Realtime]], [[Compute]] or [[Broker]] node. Querying a Realtime node returns only realtime data, querying a compute node returns only historical segments. Querying the broker will query both realtime and compute segments and compose an overall result for the query. This is the normal mode of operation for queries in druid. +As a shared-nothing system, there are three ways to query druid, against the [Realtime](Realtime.html), [Compute](Compute.html) or [Broker](Broker.html) node. Querying a Realtime node returns only realtime data, querying a compute node returns only historical segments. Querying the broker will query both realtime and compute segments and compose an overall result for the query. This is the normal mode of operation for queries in druid. ### Construct a Query ### @@ -183,7 +183,7 @@ Now that we know what nodes can be queried (although you should usually use the ## Querying Against the realtime.spec ## -How are we to know what queries we can run? Although [[Querying]] is a helpful index, to get a handle on querying our data we need to look at our [[Realtime]] node's realtime.spec file: +How are we to know what queries we can run? Although [Querying](Querying.html) is a helpful index, to get a handle on querying our data we need to look at our [Realtime](Realtime.html) node's realtime.spec file: ```json [{ @@ -225,7 +225,7 @@ Our dataSource tells us the name of the relation/table, or 'source of data', to ### aggregations ### -Note the [[Aggregations]] in our query: +Note the [Aggregations](Aggregations.html) in our query: ```json "aggregations": [ @@ -244,7 +244,7 @@ this matches up to the aggregators in the schema of our realtime.spec! ### dimensions ### -Lets look back at our actual records (from [[Loading Your Data]]): +Lets look back at our actual records (from [Loading Your Data](Loading Your Data.html)): ```json {"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100} @@ -359,8 +359,8 @@ Which gets us just people aged 40: } ] ``` -Check out [[Filters]] for more. +Check out [Filters](Filters.html) for more. ## Learn More ## -You can learn more about querying at [[Querying]]! Now check out [[Booting a production cluster]]! \ No newline at end of file +You can learn more about querying at [Querying](Querying.html)! Now check out [Booting a production cluster](Booting a production cluster.html)! \ No newline at end of file diff --git a/docs/Querying.md b/docs/Querying.md index db845bc694f..7e613a074ec 100644 --- a/docs/Querying.md +++ b/docs/Querying.md @@ -4,7 +4,7 @@ layout: default Querying ======== -Queries are made using an HTTP REST style request to a [[Broker]], [[Compute]], or [[Realtime]] node. The query is expressed in JSON and each of these node types expose the same REST query interface. +Queries are made using an HTTP REST style request to a [Broker](Broker.html), [Compute](Compute.html), or [Realtime](Realtime.html) node. The query is expressed in JSON and each of these node types expose the same REST query interface. We start by describing an example query with additional comments that mention possible variations. Query operators are also summarized in a table below. @@ -55,7 +55,7 @@ The dataSource JSON field shown next identifies where to apply the query. In thi \`\`\`javascript [dataSource]() โ€œrandSeqโ€, \`\`\` -The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like โ€œsecondโ€, โ€œminuteโ€, โ€œfifteen\_minuteโ€, โ€œthirty\_minuteโ€, โ€œhourโ€ or โ€œdayโ€. It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning โ€œ6 minute bucketsโ€. See [[Granularities]] for more information on the different options for this field. In this example, it is set to the special value โ€œallโ€ which means [bucket all data points together into the same time bucket]() +The granularity JSON field specifies the bucket size for values. It could be a built-in time interval like โ€œsecondโ€, โ€œminuteโ€, โ€œfifteen\_minuteโ€, โ€œthirty\_minuteโ€, โ€œhourโ€ or โ€œdayโ€. It can also be an expression like `{"type": "period", "period":"PT6m"}` meaning โ€œ6 minute bucketsโ€. See [Granularities](Granularities.html) for more information on the different options for this field. In this example, it is set to the special value โ€œallโ€ which means [bucket all data points together into the same time bucket]() \`\`\`javascript [granularity]() โ€œallโ€, \`\`\` @@ -63,7 +63,7 @@ The dimensions JSON field value is an array of zero or more fields as defined in \`\`\`javascript [dimensions]() [], \`\`\` -A groupBy also requires the JSON field โ€œaggregationsโ€ (See [[Aggregations]]), which are applied to the column specified by fieldName and the output of the aggregation will be named according to the value in the โ€œnameโ€ field: +A groupBy also requires the JSON field โ€œaggregationsโ€ (See [Aggregations](Aggregations.html)), which are applied to the column specified by fieldName and the output of the aggregation will be named according to the value in the โ€œnameโ€ field: \`\`\`javascript [aggregations]() [ { [type]() โ€œcountโ€, [name]() โ€œrowsโ€ }, @@ -71,7 +71,7 @@ A groupBy also requires the JSON field โ€œaggregationsโ€ (See [[Aggregations]]) { [type]() โ€œdoubleSumโ€, [fieldName]() โ€œoutColumnโ€, [name]() โ€œrandomNumberSumโ€ } ], \`\`\` -You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [[Post Aggregations]] for a detailed description. In the rand example, an arithmetic type operation (division, as specified by โ€œfnโ€) is performed with the result โ€œnameโ€ of โ€œavg\_randomโ€. The โ€œfieldsโ€ field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to โ€œnameโ€ JSON field inside the type โ€œfieldAccessโ€ are required but not used outside this expression, so they are prefixed with โ€œdummyโ€ for clarity: +You can also specify postAggregations, which are applied after data has been aggregated for the current granularity and dimensions bucket. See [Post Aggregations](Post Aggregations.html) for a detailed description. In the rand example, an arithmetic type operation (division, as specified by โ€œfnโ€) is performed with the result โ€œnameโ€ of โ€œavg\_randomโ€. The โ€œfieldsโ€ field specifies the inputs from the aggregation stage to this expression. Note that identifiers corresponding to โ€œnameโ€ JSON field inside the type โ€œfieldAccessโ€ are required but not used outside this expression, so they are prefixed with โ€œdummyโ€ for clarity: \`\`\`javascript [postAggregations]() [{ [type]() โ€œarithmeticโ€, @@ -99,11 +99,11 @@ The following table summarizes query properties. |timeseries, groupBy, search, timeBoundary|dataSource|query is applied to this data source|yes| |timeseries, groupBy, search|intervals|range of time series to include in query|yes| |timeseries, groupBy, search, timeBoundary|context|This is a key-value map that can allow the query to alter some of the behavior of a query. It is primarily used for debugging, for example if you include `"bySegment":true` in the map, you will get results associated with the data segment they came from.|no| -|timeseries, groupBy, search|filter|Specifies the filter (the โ€œWHEREโ€ clause in SQL) for the query. See [[Filters]]|no| -|timeseries, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. โ€œhourโ€). See [[Granularities]] for more information.|no| +|timeseries, groupBy, search|filter|Specifies the filter (the โ€œWHEREโ€ clause in SQL) for the query. See [Filters](Filters.html)|no| +|timeseries, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. โ€œhourโ€). See [Granularities](Granularities.html) for more information.|no| |groupBy|dimensions|constrains the groupings; if empty, then one value per time granularity bucket|yes| -|timeseries, groupBy|aggregations|aggregations that combine values in a bucket. See [[Aggregations]].|yes| -|timeseries, groupBy|postAggregations|aggregations of aggregations. See [[Post Aggregations]].|yes| +|timeseries, groupBy|aggregations|aggregations that combine values in a bucket. See [Aggregations](Aggregations.html).|yes| +|timeseries, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post Aggregations.html).|yes| |search|limit|maximum number of results (default is 1000), a system-level maximum can also be set via `com.metamx.query.search.maxSearchLimit`|no| |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| |search|query|The query portion of the search query. This is essentially a predicate that specifies if something matches.|yes| @@ -111,4 +111,4 @@ The following table summarizes query properties. Additional Information about Query Types ---------------------------------------- -[[TimeseriesQuery]] +[TimeseriesQuery](TimeseriesQuery.html) diff --git a/docs/Realtime.md b/docs/Realtime.md index c92cc7f7175..855607d7eb5 100644 --- a/docs/Realtime.md +++ b/docs/Realtime.md @@ -4,7 +4,7 @@ layout: default Realtime ======== -Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data theyโ€™ve collected over some span of time and hand these segments off to [[Compute]] nodes. +Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data theyโ€™ve collected over some span of time and hand these segments off to [Compute](Compute.html) nodes. Running ------- @@ -21,7 +21,7 @@ The segment propagation diagram for real-time data ingestion can be seen below: Configuration ------------- -Realtime nodes take a mix of base server configuration and spec files that describe how to connect, process and expose the realtime feed. See [[Configuration]] for information about general server configuration. +Realtime nodes take a mix of base server configuration and spec files that describe how to connect, process and expose the realtime feed. See [Configuration](Configuration.html) for information about general server configuration. ### Realtime โ€œspecFileโ€ @@ -62,7 +62,7 @@ There are four parts to a realtime stream specification, `schema`, `config`, `fi #### Schema -This describes the data schema for the output Druid segment. More information about concepts in Druid and querying can be found at [[Concepts-and-Terminology]] and [[Querying]]. +This describes the data schema for the output Druid segment. More information about concepts in Druid and querying can be found at [Concepts-and-Terminology](Concepts-and-Terminology.html) and [Querying](Querying.html). |Field|Type|Description|Required| |-----|----|-----------|--------| @@ -83,11 +83,11 @@ This provides configuration for the data processing portion of the realtime stre ### Firehose -See [[Firehose]]. +See [Firehose](Firehose.html). ### Plumber -See [[Plumber]] +See [Plumber](Plumber.html) Constraints ----------- diff --git a/docs/SearchQuery.md b/docs/SearchQuery.md index 7acf04419fa..b206f652c6e 100644 --- a/docs/SearchQuery.md +++ b/docs/SearchQuery.md @@ -30,11 +30,11 @@ There are several main parts to a search query: |--------|-----------|---------| |queryType|This String should always be โ€œsearchโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| -|granularity|Defines the granularity of the query. See [[Granularities]]|yes| -|filter|See [[Filters]]|no| +|granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes| +|filter|See [Filters](Filters.html)|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no| -|query|See [[SearchQuerySpec]].|yes| +|query|See [SearchQuerySpec](SearchQuerySpec.html).|yes| |sort|How the results of the search should sorted. Two possible types here are โ€œlexicographicโ€ and โ€œstrlenโ€.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/docs/Segments.md b/docs/Segments.md index 7da12950d15..5af50cd8e48 100644 --- a/docs/Segments.md +++ b/docs/Segments.md @@ -4,7 +4,7 @@ layout: default Segments ======== -Segments are the fundamental structure to store data in Druid. [[Compute]] and [[Realtime]] nodes load and serve segments for querying. To construct segments, Druid will always shard data by a time partition. Data may be further sharded based on dimension cardinality and row count. +Segments are the fundamental structure to store data in Druid. [Compute](Compute.html) and [Realtime](Realtime.html) nodes load and serve segments for querying. To construct segments, Druid will always shard data by a time partition. Data may be further sharded based on dimension cardinality and row count. The latest Druid segment version is `v9`. diff --git a/docs/Stand-Alone-With-Riak-CS.md b/docs/Stand-Alone-With-Riak-CS.md index 505b59f9283..d1dc8f780d8 100644 --- a/docs/Stand-Alone-With-Riak-CS.md +++ b/docs/Stand-Alone-With-Riak-CS.md @@ -22,12 +22,12 @@ We started with a minimal CentOS installation but you can use any other compatib 1. A Kafka Broker 1. A single-node Zookeeper ensemble 1. A single-node Riak-CS cluster -1. A Druid [[Master]] -1. A Druid [[Broker]] -1. A Druid [[Compute]] -1. A Druid [[Realtime]] +1. A Druid [Master](Master.html) +1. A Druid [Broker](Broker.html) +1. A Druid [Compute](Compute.html) +1. A Druid [Realtime](Realtime.html) -This just walks through getting the relevant software installed and running. You will then need to configure the [[Realtime]] node to take in your data. +This just walks through getting the relevant software installed and running. You will then need to configure the [Realtime](Realtime.html) node to take in your data. ### Configure System diff --git a/docs/TimeseriesQuery.md b/docs/TimeseriesQuery.md index 56f2ce733b9..9ea79fcfa75 100644 --- a/docs/TimeseriesQuery.md +++ b/docs/TimeseriesQuery.md @@ -84,10 +84,10 @@ There are 7 main parts to a timeseries query: |--------|-----------|---------| |queryType|This String should always be โ€œtimeseriesโ€; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String defining the data source to query, very similar to a table in a relational database|yes| -|granularity|Defines the granularity of the query. See [[Granularities]]|yes| -|filter|See [[Filters]]|no| -|aggregations|See [[Aggregations]]|yes| -|postAggregations|See [[Post Aggregations]]|no| +|granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes| +|filter|See [Filters](Filters.html)|no| +|aggregations|See [Aggregations](Aggregations.html)|yes| +|postAggregations|See [Post Aggregations](Post Aggregations.html)|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/docs/Tutorial:-A-First-Look-at-Druid.md b/docs/Tutorial:-A-First-Look-at-Druid.md index 4722dd173c0..c3de0df1d91 100644 --- a/docs/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/Tutorial:-A-First-Look-at-Druid.md @@ -41,7 +41,7 @@ These metrics track the number of characters added, deleted, and changed. Setting Up ---------- -There are two ways to setup Druid: download a tarball, or [[Build From Source]]. You only need to do one of these. +There are two ways to setup Druid: download a tarball, or [Build From Source](Build From Source.html). You only need to do one of these. ### Download a Tarball @@ -64,7 +64,7 @@ You should see a bunch of files: Running Example Scripts ----------------------- -Letโ€™s start doing stuff. You can start a Druid [[Realtime]] node by issuing: +Letโ€™s start doing stuff. You can start a Druid [Realtime](Realtime.html) node by issuing: ./run_example_server.sh @@ -176,7 +176,7 @@ As you can probably tell, the result is indicating the maximum and minimum times Return to your favorite editor and create the file:
timeseries_query.body
-We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file:

 {
     "queryType": "timeseries", 
@@ -200,7 +200,7 @@ We are going to make a slightly more complicated query, the [[TimeseriesQuery]].
 }
 
-You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +You are probably wondering, what are these [Granularities](Granularities.html) and [Aggregations](Aggregations.html) things? What the query is doing is aggregating some metrics over some span of time. To issue the query and get some results, run the following in your command line:
curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
 
@@ -275,7 +275,7 @@ This gives us something like the following:
 Solving a Problem
 -----------------
 
-One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top pages in the US are, ordered by the number of edits over the last few minutes youโ€™ve been going through this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
+One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top pages in the US are, ordered by the number of edits over the last few minutes youโ€™ve been going through this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [GroupByQuery](GroupByQuery.html). It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
 
 Letโ€™s create the file:
 
@@ -317,7 +317,7 @@ Letโ€™s create the file:
     }
     
 
-Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [OrderBy](OrderBy.html) thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
 
 If you issue the query:
 
@@ -357,9 +357,9 @@ Feel free to tweak other query parameters to answer other questions you may have
 Next Steps
 ----------
 
-What to know even more information about the Druid Cluster? Check out [[Tutorial: The Druid Cluster]]
+What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
 
-Druid is even more fun if you load your own data into it! To learn how to load your data, see [[Loading Your Data]].
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading Your Data.html).
 
 Additional Information
 ----------------------
diff --git a/docs/Tutorial:-The-Druid-Cluster.md b/docs/Tutorial:-The-Druid-Cluster.md
index e2eff84f505..286447cc3cd 100644
--- a/docs/Tutorial:-The-Druid-Cluster.md
+++ b/docs/Tutorial:-The-Druid-Cluster.md
@@ -19,7 +19,7 @@ tar -zxvf druid-services-*-bin.tar.gz
 cd druid-services-*
 ```
 
-You can also [[Build From Source]].
+You can also [Build From Source](Build From Source.html).
 
 ## External Dependencies ##
 
diff --git a/docs/Tutorial:-Webstream.md b/docs/Tutorial:-Webstream.md
index 973204f31d4..bbfb42450fd 100644
--- a/docs/Tutorial:-Webstream.md
+++ b/docs/Tutorial:-Webstream.md
@@ -145,7 +145,7 @@ As you can probably tell, the result is indicating the maximum and minimum times
 Return to your favorite editor and create the file:
 
timeseries_query.body
-We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file:

 {
     "queryType": "timeseries", 
@@ -168,7 +168,7 @@ We are going to make a slightly more complicated query, the [[TimeseriesQuery]].
 }
 
-You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +You are probably wondering, what are these [Granularities](Granularities.html) and [Aggregations](Aggregations.html) things? What the query is doing is aggregating some metrics over some span of time. To issue the query and get some results, run the following in your command line:
curl -X POST 'http://localhost:8083/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
 
@@ -246,7 +246,7 @@ This gives us something like the following:
 Solving a Problem
 -----------------
 
-One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top states in the US are, ordered by the number of visits by known users over the last few minutes? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
+One of Druidโ€™s main powers is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top states in the US are, ordered by the number of visits by known users over the last few minutes? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [GroupByQuery](GroupByQuery.html). It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
 
 Letโ€™s create the file:
 
@@ -292,7 +292,7 @@ Letโ€™s create the file:
     }
     
 
-Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [OrderBy](OrderBy.html) thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
 
 If you issue the query:
 
@@ -346,8 +346,8 @@ Feel free to tweak other query parameters to answer other questions you may have
 Next Steps
 ----------
 
-What to know even more information about the Druid Cluster? Check out [[Tutorial: The Druid Cluster]]
-Druid is even more fun if you load your own data into it! To learn how to load your data, see [[Loading Your Data]].
+What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading Your Data.html).
 
 Additional Information
 ----------------------
diff --git a/docs/Twitter-Tutorial.md b/docs/Twitter-Tutorial.md
index cedd26b9250..dc2151ec3e4 100644
--- a/docs/Twitter-Tutorial.md
+++ b/docs/Twitter-Tutorial.md
@@ -1,7 +1,7 @@
 ---
 layout: default
 ---
-Greetings! We see youโ€™ve taken an interest in Druid. Thatโ€™s awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [[Examples]], and issue some basic Druid queries. The data source weโ€™ll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on!
+Greetings! We see youโ€™ve taken an interest in Druid. Thatโ€™s awesome! Hopefully this tutorial will help clarify some core Druid concepts. We will go through one of the Real-time [Examples](Examples.html), and issue some basic Druid queries. The data source weโ€™ll be working with is the [Twitter spritzer stream](https://dev.twitter.com/docs/streaming-apis/streams/public). If you are ready to explore Druid, brave its challenges, and maybe learn a thing or two, read on!
 
 Setting Up
 ----------
@@ -52,7 +52,7 @@ You can find the example executables in the examples/bin directory:
 Running Example Scripts
 -----------------------
 
-Letโ€™s start doing stuff. You can start a Druid [[Realtime]] node by issuing:
+Letโ€™s start doing stuff. You can start a Druid [Realtime](Realtime.html) node by issuing:
 
     ./run_example_server.sh
 
@@ -175,7 +175,7 @@ If you said the result is indicating the maximum and minimum timestamps we've se
 Return to your favorite editor and create the file:
 
timeseries_query.body
-We are going to make a slightly more complicated query, the [[TimeseriesQuery]]. Copy and paste the following into the file: +We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file:
{
   "queryType":"timeseries",
   "dataSource":"twitterstream",
@@ -188,7 +188,7 @@ We are going to make a slightly more complicated query, the [[TimeseriesQuery]].
 }
 
-You are probably wondering, what are these [[Granularities]] and [[Aggregations]] things? What the query is doing is aggregating some metrics over some span of time. +You are probably wondering, what are these [Granularities](Granularities.html) and [Aggregations](Aggregations.html) things? What the query is doing is aggregating some metrics over some span of time. To issue the query and get some results, run the following in your command line:
curl -X POST 'http://localhost:8080/druid/v2/?pretty' -H 'content-type: application/json'  -d  ````timeseries\_query.body
 
@@ -252,7 +252,7 @@ This gives us something like the following:
 Solving a Problem
 -----------------
 
-One of Druidโ€™s main powers (see what we did there?) is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes youโ€™ve been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [[GroupByQuery]]. It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
+One of Druidโ€™s main powers (see what we did there?) is to provide answers to problems, so letโ€™s pose a problem. What if we wanted to know what the top hash tags are, ordered by the number tweets, where the language is english, over the last few minutes youโ€™ve been reading this tutorial? To solve this problem, we have to return to the query we introduced at the very beginning of this tutorial, the [GroupByQuery](GroupByQuery.html). It would be nice if we could group by results by dimension value and somehow sort those resultsโ€ฆ and it turns out we can!
 
 Letโ€™s create the file:
 
@@ -272,7 +272,7 @@ Letโ€™s create the file:
     }
     
 
-Woah! Our query just got a way more complicated. Now we have these [[Filters]] things and this [[OrderBy]] thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
+Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [OrderBy](OrderBy.html) thing. Fear not, it turns out the new objects weโ€™ve introduced to our query can help define the format of our results and provide an answer to our question.
 
 If you issue the query:
 
@@ -324,6 +324,6 @@ Feel free to tweak other query parameters to answer other questions you may have
 Additional Information
 ----------------------
 
-This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to [[Loading Your Data]].
+This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to [Loading Your Data](Loading Your Data.html).
 
 And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
diff --git a/docs/Versioning.md b/docs/Versioning.md
index 7b9fa24045c..6b9e79fe9d3 100644
--- a/docs/Versioning.md
+++ b/docs/Versioning.md
@@ -21,4 +21,4 @@ For external deployments, we recommend running the stable release tag. Releases
 Tagging strategy
 ----------------
 
-Tags of the codebase are equivalent to release candidates. We tag the code every time we want to take it through our release process, which includes some QA cycles and deployments. So, it is not safe to assume that a tag is a stable release, it is a solidification of the code as it goes through our production QA cycle and deployment. Tags will never change, but we often go through a number of iterations of tags before actually getting a stable release onto production. So, it is recommended that if you are not aware of what is on a tag, to stick to the stable releases listed on the [[Download]] page.
+Tags of the codebase are equivalent to release candidates. We tag the code every time we want to take it through our release process, which includes some QA cycles and deployments. So, it is not safe to assume that a tag is a stable release, it is a solidification of the code as it goes through our production QA cycle and deployment. Tags will never change, but we often go through a number of iterations of tags before actually getting a stable release onto production. So, it is recommended that if you are not aware of what is on a tag, to stick to the stable releases listed on the [Download](Download.html) page.
diff --git a/docs/ZooKeeper.md b/docs/ZooKeeper.md
index 03f2b1b8e0c..d3e24e29ceb 100644
--- a/docs/ZooKeeper.md
+++ b/docs/ZooKeeper.md
@@ -3,9 +3,9 @@ layout: default
 ---
 Druid uses ZooKeeper (ZK) for management of current cluster state. The operations that happen over ZK are
 
-1.  [[Master]] leader election
-2.  Segment โ€œpublishingโ€ protocol from [[Compute]] and [[Realtime]]
-3.  Segment load/drop protocol between [[Master]] and [[Compute]]
+1.  [Master](Master.html) leader election
+2.  Segment โ€œpublishingโ€ protocol from [Compute](Compute.html) and [Realtime](Realtime.html)
+3.  Segment load/drop protocol between [Master](Master.html) and [Compute](Compute.html)
 
 ### Property Configuration
 
@@ -41,7 +41,7 @@ We use the Curator LeadershipLatch recipe to do leader election at path
 
 The `announcementsPath` and `servedSegmentsPath` are used for this.
 
-All [[Compute]] and [[Realtime]] nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
+All [Compute](Compute.html) and [Realtime](Realtime.html) nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
 
     ${druid.zk.paths.announcementsPath}/${druid.host}
 
@@ -53,13 +53,13 @@ And as they load up segments, they will attach ephemeral znodes that look like
 
     ${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_
 
-Nodes like the [[Master]] and [[Broker]] can then watch these paths to see which nodes are currently serving which segments.
+Nodes like the [Master](Master.html) and [Broker](Broker.html) can then watch these paths to see which nodes are currently serving which segments.
 
 ### Segment load/drop protocol between Master and Compute
 
 The `loadQueuePath` is used for this.
 
-When the [[Master]] decides that a [[Compute]] node should load or drop a segment, it writes an ephemeral znode to
+When the [Master](Master.html) decides that a [Compute](Compute.html) node should load or drop a segment, it writes an ephemeral znode to
 
     ${druid.zk.paths.loadQueuePath}/_host_of_compute_node/_segment_identifier
 
diff --git a/docs/contents.md b/docs/contents.md
index 23b56bc33a5..0d3f7f9cb62 100644
--- a/docs/contents.md
+++ b/docs/contents.md
@@ -2,70 +2,70 @@
 layout: default
 ---
 Contents
-\* [[Introduction|Home]]
-\* [[Download]]
-\* [[Support]]
-\* [[Contribute]]
+\* [Introduction|Home](Introduction|Home.html)
+\* [Download](Download.html)
+\* [Support](Support.html)
+\* [Contribute](Contribute.html)
 ========================
 
 Getting Started
-\* [[Tutorial: A First Look at Druid]]
-\* [[Tutorial: The Druid Cluster]]
-\* [[Loading Your Data]]
-\* [[Querying Your Data]]
-\* [[Booting a Production Cluster]]
-\* [[Examples]]
-\* [[Cluster Setup]]
-\* [[Configuration]]
+\* [Tutorial: A First Look at Druid](Tutorial: A First Look at Druid.html)
+\* [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
+\* [Loading Your Data](Loading Your Data.html)
+\* [Querying Your Data](Querying Your Data.html)
+\* [Booting a Production Cluster](Booting a Production Cluster.html)
+\* [Examples](Examples.html)
+\* [Cluster Setup](Cluster Setup.html)
+\* [Configuration](Configuration.html)
 --------------------------------------
 
 Data Ingestion
-\* [[Realtime]]
-\* [[Batch|Batch Ingestion]]
-\* [[Indexing Service]]
+\* [Realtime](Realtime.html)
+\* [Batch|Batch Ingestion](Batch|Batch Ingestion.html)
+\* [Indexing Service](Indexing Service.html)
 ----------------------------
 
 Querying
-\* [[Querying]]
+\* [Querying](Querying.html)
 **\* ]
-**\* [[Aggregations]]
+**\* [Aggregations](Aggregations.html)
 **\* ]
-**\* [[Granularities]]
+**\* [Granularities](Granularities.html)
 \* Query Types
 **\* ]
 ****\* ]
 ****\* ]
-**\* [[SearchQuery]]
+**\* [SearchQuery](SearchQuery.html)
 **\* ]
-** [[SegmentMetadataQuery]]
+** [SegmentMetadataQuery](SegmentMetadataQuery.html)
 **\* ]
-**\* [[TimeseriesQuery]]
+**\* [TimeseriesQuery](TimeseriesQuery.html)
 ---------------------------
 
 Architecture
-\* [[Design]]
-\* [[Segments]]
+\* [Design](Design.html)
+\* [Segments](Segments.html)
 \* Node Types
 **\* ]
-**\* [[Broker]]
+**\* [Broker](Broker.html)
 **\* ]
 ****\* ]
-**\* [[Realtime]]
+**\* [Realtime](Realtime.html)
 **\* ]
-**\* [[Plumber]]
+**\* [Plumber](Plumber.html)
 \* External Dependencies
 **\* ]
-**\* [[MySQL]]
+**\* [MySQL](MySQL.html)
 **\* ]
-** [[Concepts and Terminology]]
+** [Concepts and Terminology](Concepts and Terminology.html)
 -------------------------------
 
 Development
-\* [[Versioning]]
-\* [[Build From Source]]
-\* [[Libraries]]
+\* [Versioning](Versioning.html)
+\* [Build From Source](Build From Source.html)
+\* [Libraries](Libraries.html)
 ------------------------
 
 Misc
-\* [[Thanks]]
+\* [Thanks](Thanks.html)
 -------------

From cabae7993da168ef6baf31eb8335f063c642520e Mon Sep 17 00:00:00 2001
From: fjy 
Date: Mon, 16 Sep 2013 16:03:47 -0700
Subject: [PATCH 80/92] port over multi threaded realtime and also fix broken
 realtime nodes that can't start up

---
 .../examples/guice/RealtimeExampleModule.java |  4 +-
 .../io/druid/indexing/common/TaskToolbox.java |  9 ++++
 .../indexing/common/TaskToolboxFactory.java   |  7 ++++
 .../common/task/RealtimeIndexTask.java        |  2 +
 .../coordinator/TaskLifecycleTest.java        |  1 +
 .../indexing/coordinator/TaskQueueTest.java   |  2 +
 .../worker/WorkerTaskMonitorTest.java         |  2 +-
 .../java/io/druid/guice/RealtimeModule.java   | 20 ++++++++-
 .../plumber/RealtimePlumberSchool.java        | 41 ++++++++++++++-----
 .../plumber/RealtimePlumberSchoolTest.java    |  2 +
 .../java/io/druid/cli/CliRealtimeExample.java |  2 +-
 11 files changed, 77 insertions(+), 15 deletions(-)

diff --git a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java
index a6c08b1e6fd..816a8d367da 100644
--- a/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java
+++ b/examples/src/main/java/druid/examples/guice/RealtimeExampleModule.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
 import com.google.inject.TypeLiteral;
+import com.metamx.common.guava.LazySequence;
 import com.metamx.common.logger.Logger;
 import druid.examples.flights.FlightsFirehoseFactory;
 import druid.examples.rand.RandomFirehoseFactory;
@@ -34,6 +35,7 @@ import io.druid.client.InventoryView;
 import io.druid.client.ServerView;
 import io.druid.guice.FireDepartmentsProvider;
 import io.druid.guice.JsonConfigProvider;
+import io.druid.guice.LazySingleton;
 import io.druid.guice.ManageLifecycle;
 import io.druid.guice.NoopSegmentPublisherProvider;
 import io.druid.guice.RealtimeManagerConfig;
@@ -71,7 +73,7 @@ public class RealtimeExampleModule implements DruidModule
         new TypeLiteral>()
         {
         }
-    ).toProvider(FireDepartmentsProvider.class);
+    ).toProvider(FireDepartmentsProvider.class).in(LazySingleton.class);
     binder.bind(RealtimeManager.class).in(ManageLifecycle.class);
   }
 
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
index dcef1147d17..51e0d227d21 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java
@@ -39,6 +39,7 @@ import io.druid.timeline.DataSegment;
 import java.io.File;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutorService;
 
 /**
  * Stuff that may be needed by a Task in order to conduct its business.
@@ -55,6 +56,7 @@ public class TaskToolbox
   private final ServerView newSegmentServerView;
   private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
   private final MonitorScheduler monitorScheduler;
+  private final ExecutorService queryExecutorService;
   private final SegmentLoader segmentLoader;
   private final ObjectMapper objectMapper;
 
@@ -68,6 +70,7 @@ public class TaskToolbox
       DataSegmentAnnouncer segmentAnnouncer,
       ServerView newSegmentServerView,
       QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
+      ExecutorService queryExecutorService,
       MonitorScheduler monitorScheduler,
       SegmentLoader segmentLoader,
       ObjectMapper objectMapper
@@ -82,6 +85,7 @@ public class TaskToolbox
     this.segmentAnnouncer = segmentAnnouncer;
     this.newSegmentServerView = newSegmentServerView;
     this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
+    this.queryExecutorService = queryExecutorService;
     this.monitorScheduler = monitorScheduler;
     this.segmentLoader = segmentLoader;
     this.objectMapper = objectMapper;
@@ -127,6 +131,11 @@ public class TaskToolbox
     return queryRunnerFactoryConglomerate;
   }
 
+  public ExecutorService getQueryExecutorService()
+  {
+    return queryExecutorService;
+  }
+
   public MonitorScheduler getMonitorScheduler()
   {
     return monitorScheduler;
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
index fc1e8db390b..0cac5acbce7 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java
@@ -24,6 +24,7 @@ import com.google.inject.Inject;
 import com.metamx.emitter.service.ServiceEmitter;
 import com.metamx.metrics.MonitorScheduler;
 import io.druid.client.ServerView;
+import io.druid.guice.annotations.Processing;
 import io.druid.indexing.common.actions.TaskActionClientFactory;
 import io.druid.indexing.common.config.TaskConfig;
 import io.druid.indexing.common.task.Task;
@@ -33,6 +34,8 @@ import io.druid.segment.loading.DataSegmentPusher;
 import io.druid.segment.loading.SegmentLoader;
 import io.druid.server.coordination.DataSegmentAnnouncer;
 
+import java.util.concurrent.ExecutorService;
+
 /**
  * Stuff that may be needed by a Task in order to conduct its business.
  */
@@ -46,6 +49,7 @@ public class TaskToolboxFactory
   private final DataSegmentAnnouncer segmentAnnouncer;
   private final ServerView newSegmentServerView;
   private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
+  private final ExecutorService queryExecutorService;
   private final MonitorScheduler monitorScheduler;
   private final SegmentLoader segmentLoader;
   private final ObjectMapper objectMapper;
@@ -60,6 +64,7 @@ public class TaskToolboxFactory
       DataSegmentAnnouncer segmentAnnouncer,
       ServerView newSegmentServerView,
       QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
+      @Processing ExecutorService queryExecutorService,
       MonitorScheduler monitorScheduler,
       SegmentLoader segmentLoader,
       ObjectMapper objectMapper
@@ -73,6 +78,7 @@ public class TaskToolboxFactory
     this.segmentAnnouncer = segmentAnnouncer;
     this.newSegmentServerView = newSegmentServerView;
     this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
+    this.queryExecutorService = queryExecutorService;
     this.monitorScheduler = monitorScheduler;
     this.segmentLoader = segmentLoader;
     this.objectMapper = objectMapper;
@@ -90,6 +96,7 @@ public class TaskToolboxFactory
         segmentAnnouncer,
         newSegmentServerView,
         queryRunnerFactoryConglomerate,
+        queryExecutorService,
         monitorScheduler,
         segmentLoader,
         objectMapper
diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
index 63181410f8b..691b7a89a27 100644
--- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
+++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java
@@ -116,6 +116,7 @@ public class RealtimeIndexTask extends AbstractTask
         id == null
         ? makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString())
         :id,
+
         String.format(
             "index_realtime_%s",
             schema.getDataSource()
@@ -274,6 +275,7 @@ public class RealtimeIndexTask extends AbstractTask
     // NOTE: "same" segment.
     realtimePlumberSchool.setDataSegmentPusher(toolbox.getSegmentPusher());
     realtimePlumberSchool.setConglomerate(toolbox.getQueryRunnerFactoryConglomerate());
+    realtimePlumberSchool.setQueryExecutorService(toolbox.getQueryExecutorService());
     realtimePlumberSchool.setVersioningPolicy(versioningPolicy);
     realtimePlumberSchool.setSegmentAnnouncer(lockingSegmentAnnouncer);
     realtimePlumberSchool.setSegmentPublisher(segmentPublisher);
diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java
index ceb30ffdea6..10f5beb853c 100644
--- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskLifecycleTest.java
@@ -137,6 +137,7 @@ public class TaskLifecycleTest
         null, // segment announcer
         null, // new segment server view
         null, // query runner factory conglomerate corporation unionized collective
+        null, // query executor service
         null, // monitor scheduler
         null, // segment loader
         new DefaultObjectMapper()
diff --git a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java
index 7cb5076d21d..af98107a406 100644
--- a/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/coordinator/TaskQueueTest.java
@@ -169,6 +169,7 @@ public class TaskQueueTest
         null,
         null,
         null,
+        null,
         null
     );
 
@@ -230,6 +231,7 @@ public class TaskQueueTest
         null,
         null,
         null,
+        null,
         null
     );
 
diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java
index 399ff4ab05a..39139f29efe 100644
--- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java
+++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java
@@ -115,7 +115,7 @@ public class WorkerTaskMonitorTest
         new ThreadPoolTaskRunner(
             new TaskToolboxFactory(
                 new TaskConfig(tmp.toString(), null, null, 0),
-                null, null, null, null, null, null, null, null, null, jsonMapper
+                null, null, null, null, null, null, null, null, null, null, jsonMapper
             )
         ),
         new WorkerConfig().setCapacity(1)
diff --git a/realtime/src/main/java/io/druid/guice/RealtimeModule.java b/realtime/src/main/java/io/druid/guice/RealtimeModule.java
index 9bdac37a89f..bba4a7d6240 100644
--- a/realtime/src/main/java/io/druid/guice/RealtimeModule.java
+++ b/realtime/src/main/java/io/druid/guice/RealtimeModule.java
@@ -19,19 +19,24 @@
 
 package io.druid.guice;
 
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.inject.Binder;
 import com.google.inject.Module;
 import com.google.inject.TypeLiteral;
 import com.metamx.common.logger.Logger;
+import io.druid.initialization.DruidModule;
 import io.druid.segment.realtime.FireDepartment;
 import io.druid.segment.realtime.RealtimeManager;
 import io.druid.segment.realtime.SegmentPublisher;
+import io.druid.segment.realtime.firehose.KafkaFirehoseFactory;
 
+import java.util.Arrays;
 import java.util.List;
 
 /**
  */
-public class RealtimeModule implements Module
+public class RealtimeModule implements DruidModule
 {
   private static final Logger log = new Logger(RealtimeModule.class);
 
@@ -46,7 +51,18 @@ public class RealtimeModule implements Module
         new TypeLiteral>()
         {
         }
-    ).toProvider(FireDepartmentsProvider.class);
+    ).toProvider(FireDepartmentsProvider.class).in(LazySingleton.class);
     binder.bind(RealtimeManager.class).in(ManageLifecycle.class);
   }
+
+  @Override
+  public List getJacksonModules()
+  {
+    return Arrays.asList(
+        new SimpleModule("RealtimeModule")
+            .registerSubtypes(
+                new NamedType(KafkaFirehoseFactory.class, "kafka-0.7.2")
+            )
+    );
+  }
 }
diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java
index ed6ed355ece..64daf729484 100644
--- a/realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java
+++ b/realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java
@@ -43,6 +43,7 @@ import io.druid.client.DruidServer;
 import io.druid.client.ServerView;
 import io.druid.common.guava.ThreadRenamingCallable;
 import io.druid.common.guava.ThreadRenamingRunnable;
+import io.druid.guice.annotations.Processing;
 import io.druid.query.MetricsEmittingQueryRunner;
 import io.druid.query.Query;
 import io.druid.query.QueryRunner;
@@ -75,6 +76,7 @@ import org.joda.time.Interval;
 import org.joda.time.Period;
 
 import javax.annotation.Nullable;
+import javax.validation.constraints.NotNull;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
@@ -91,24 +93,44 @@ import java.util.concurrent.ScheduledExecutorService;
 public class RealtimePlumberSchool implements PlumberSchool
 {
   private static final EmittingLogger log = new EmittingLogger(RealtimePlumberSchool.class);
-  private static final ListeningExecutorService EXEC = MoreExecutors.sameThreadExecutor();
 
   private final Period windowPeriod;
   private final File basePersistDirectory;
   private final IndexGranularity segmentGranularity;
   private final Object handoffCondition = new Object();
 
+  @JacksonInject
+  @NotNull
   private volatile ServiceEmitter emitter;
+
+  @JacksonInject
+  @NotNull
   private volatile QueryRunnerFactoryConglomerate conglomerate = null;
+
+  @JacksonInject
+  @NotNull
   private volatile DataSegmentPusher dataSegmentPusher = null;
+
+  @JacksonInject
+  @NotNull
   private volatile DataSegmentAnnouncer segmentAnnouncer = null;
+
+  @JacksonInject
+  @NotNull
   private volatile SegmentPublisher segmentPublisher = null;
+
+  @JacksonInject
+  @NotNull
   private volatile ServerView serverView = null;
 
+  @JacksonInject
+  @NotNull
+  @Processing
+  private volatile ExecutorService queryExecutorService = null;
+
   private volatile VersioningPolicy versioningPolicy = null;
   private volatile RejectionPolicyFactory rejectionPolicyFactory = null;
 
-
   @JsonCreator
   public RealtimePlumberSchool(
       @JsonProperty("windowPeriod") Period windowPeriod,
@@ -139,42 +161,41 @@ public class RealtimePlumberSchool implements PlumberSchool
     this.rejectionPolicyFactory = factory;
   }
 
-  @JacksonInject
   public void setEmitter(ServiceEmitter emitter)
   {
     this.emitter = emitter;
   }
 
-  @JacksonInject
   public void setConglomerate(QueryRunnerFactoryConglomerate conglomerate)
   {
     this.conglomerate = conglomerate;
   }
 
-  @JacksonInject
   public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher)
   {
     this.dataSegmentPusher = dataSegmentPusher;
   }
 
-  @JacksonInject
   public void setSegmentAnnouncer(DataSegmentAnnouncer segmentAnnouncer)
   {
     this.segmentAnnouncer = segmentAnnouncer;
   }
 
-  @JacksonInject
   public void setSegmentPublisher(SegmentPublisher segmentPublisher)
   {
     this.segmentPublisher = segmentPublisher;
   }
 
-  @JacksonInject
   public void setServerView(ServerView serverView)
   {
     this.serverView = serverView;
   }
 
+  public void setQueryExecutorService(ExecutorService executorService)
+  {
+    this.queryExecutorService = queryExecutorService;
+  }
+
   @Override
   public Plumber findPlumber(final Schema schema, final FireDepartmentMetrics metrics)
   {
@@ -262,7 +283,7 @@ public class RealtimePlumberSchool implements PlumberSchool
 
         return toolchest.mergeResults(
             factory.mergeRunners(
-                EXEC,
+                queryExecutorService,
                 FunctionalIterable
                     .create(querySinks)
                     .transform(
@@ -277,7 +298,7 @@ public class RealtimePlumberSchool implements PlumberSchool
                                     emitter,
                                     builderFn,
                                     factory.mergeRunners(
-                                        EXEC,
+                                        MoreExecutors.sameThreadExecutor(),
                                         Iterables.transform(
                                             theSink,
                                             new Function>()
diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
index 8b92bff01c2..71eb4659909 100644
--- a/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
+++ b/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java
@@ -23,6 +23,7 @@ import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.io.Files;
+import com.google.common.util.concurrent.MoreExecutors;
 import com.metamx.common.ISE;
 import com.metamx.emitter.service.ServiceEmitter;
 import io.druid.client.ServerView;
@@ -114,6 +115,7 @@ public class RealtimePlumberSchoolTest
     realtimePlumberSchool.setDataSegmentPusher(dataSegmentPusher);
     realtimePlumberSchool.setServerView(serverView);
     realtimePlumberSchool.setEmitter(emitter);
+    realtimePlumberSchool.setQueryExecutorService(MoreExecutors.sameThreadExecutor());
 
     plumber = realtimePlumberSchool.findPlumber(schema, new FireDepartmentMetrics());
   }
diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java
index adc6f78ccba..3af6b116e6a 100644
--- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java
+++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java
@@ -39,7 +39,7 @@ import java.util.List;
 /**
  */
 @Command(
-    name = "example realtime",
+    name = "realtime",
     description = "Runs a standalone realtime node for examples, see https://github.com/metamx/druid/wiki/Realtime for a description"
 )
 public class CliRealtimeExample extends ServerRunnable

From 946a9e502fe544ea8e08218a4fd0d0a683cab44b Mon Sep 17 00:00:00 2001
From: Russell Jurney 
Date: Mon, 16 Sep 2013 16:19:49 -0700
Subject: [PATCH 81/92] Replaced spaces with dashes

---
 docs/Batch-ingestion.md                 |  2 +-
 docs/Booting-a-production-cluster.md    |  2 +-
 docs/Configuration.md                   |  2 +-
 docs/Druid-Personal-Demo-Cluster.md     |  2 +-
 docs/Examples.md                        |  2 +-
 docs/GroupByQuery.md                    |  2 +-
 docs/Libraries.md                       |  3 +++
 docs/Loading-Your-Data.md               |  4 ++--
 docs/Master.md                          |  2 +-
 docs/MySQL.md                           |  2 +-
 docs/Querying-your-data.md              |  6 +++---
 docs/TimeseriesQuery.md                 |  2 +-
 docs/Tutorial:-A-First-Look-at-Druid.md |  4 ++--
 docs/Tutorial:-The-Druid-Cluster.md     |  2 +-
 docs/Tutorial:-Webstream.md             |  4 ++--
 docs/contents.md                        | 20 ++++++++++----------
 16 files changed, 32 insertions(+), 29 deletions(-)

diff --git a/docs/Batch-ingestion.md b/docs/Batch-ingestion.md
index 6511b85b452..42a42ac7b29 100644
--- a/docs/Batch-ingestion.md
+++ b/docs/Batch-ingestion.md
@@ -9,7 +9,7 @@ There are two choices for batch data ingestion to your Druid cluster, you can us
 Which should I use?
 -------------------
 
-The [Indexing service](Indexing service.html) is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data.
+The [Indexing service](Indexing-service.html) is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data.
 
 The `HadoopDruidIndexerMain` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and donโ€™t want to spend the time configuring and deploying the [Indexing service](Indexing service.html) just yet.
 
diff --git a/docs/Booting-a-production-cluster.md b/docs/Booting-a-production-cluster.md
index d5fc38c8ce5..f7e5444ab8e 100644
--- a/docs/Booting-a-production-cluster.md
+++ b/docs/Booting-a-production-cluster.md
@@ -3,7 +3,7 @@ layout: default
 ---
 # Booting a Single Node Cluster #
 
-[Loading Your Data](Loading Your Data.html) and [Querying Your Data](Querying Your Data.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz).
+[Loading Your Data](Loading-Your-Data.html) and [Querying Your Data](Querying-Your-Data.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.5.51-SNAPSHOT-bin.tar.gz).
 
 The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables:
 
diff --git a/docs/Configuration.md b/docs/Configuration.md
index 544b9ea4f55..4042d02d825 100644
--- a/docs/Configuration.md
+++ b/docs/Configuration.md
@@ -91,7 +91,7 @@ These properties are for connecting with S3 and using it to pull down segments.
 
 ### JDBC connection
 
-These properties specify the jdbc connection and other configuration around the โ€œsegments tableโ€ database. The only processes that connect to the DB with these properties are the [Master](Master.html) and [Indexing service](Indexing service.html). This is tested on MySQL.
+These properties specify the jdbc connection and other configuration around the โ€œsegments tableโ€ database. The only processes that connect to the DB with these properties are the [Master](Master.html) and [Indexing service](Indexing-service.html). This is tested on MySQL.
 
 |Property|Description|Default|
 |--------|-----------|-------|
diff --git a/docs/Druid-Personal-Demo-Cluster.md b/docs/Druid-Personal-Demo-Cluster.md
index 498f8ff8e14..0ef9834f198 100644
--- a/docs/Druid-Personal-Demo-Cluster.md
+++ b/docs/Druid-Personal-Demo-Cluster.md
@@ -3,7 +3,7 @@ layout: default
 ---
 # Druid Personal Demo Cluster (DPDC)
 
-Note, there are currently some issues with the CloudFormation.  We are working through them and will update the documentation here when things work properly.  In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version.  If you just want to get a feel for the types of data and queries that you can issue, check out [Realtime Examples](Realtime Examples.html)
+Note, there are currently some issues with the CloudFormation.  We are working through them and will update the documentation here when things work properly.  In the meantime, the simplest way to get your feet wet with a cluster setup is to run through the instructions at [housejester/druid-test-harness](https://github.com/housejester/druid-test-harness), though it is based on an older version.  If you just want to get a feel for the types of data and queries that you can issue, check out [Realtime Examples](Realtime-Examples.html)
 
 ## Introduction
 To make it easy for you to get started with Druid, we created an AWS (Amazon Web Services) [CloudFormation](http://aws.amazon.com/cloudformation/) Template that allows you to create a small pre-configured Druid cluster using your own AWS account. The cluster contains a pre-loaded sample workload, the Wikipedia edit stream, and a basic query interface that gets you familiar with Druid capabilities like drill-downs and filters. 
diff --git a/docs/Examples.md b/docs/Examples.md
index 2f48f60b1b5..4207911464b 100644
--- a/docs/Examples.md
+++ b/docs/Examples.md
@@ -34,7 +34,7 @@ Clone Druid and build it:
 Twitter Example
 ---------------
 
-For a full tutorial based on the twitter example, check out this [Twitter Tutorial](Twitter Tutorial.html).
+For a full tutorial based on the twitter example, check out this [Twitter Tutorial](Twitter-Tutorial.html).
 
 This Example uses a feature of Twitter that allows for sampling of itโ€™s stream. We sample the Twitter stream via our [TwitterSpritzerFirehoseFactory](https://github.com/metamx/druid/blob/master/examples/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java) class and use it to simulate the kinds of data you might ingest into Druid. Then, with the client part, the sample shows what kinds of analytics explorations you can do during and after the data is loaded.
 
diff --git a/docs/GroupByQuery.md b/docs/GroupByQuery.md
index 7e95ebcbdee..01edc6bdc7e 100644
--- a/docs/GroupByQuery.md
+++ b/docs/GroupByQuery.md
@@ -98,7 +98,7 @@ There are 9 main parts to a groupBy query:
 |granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes|
 |filter|See [Filters](Filters.html)|no|
 |aggregations|See [Aggregations](Aggregations.html)|yes|
-|postAggregations|See [Post Aggregations](Post Aggregations.html)|no|
+|postAggregations|See [Post Aggregations](Post-Aggregations.html)|no|
 |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
 |context|An additional JSON Object which can be used to specify certain flags.|no|
 
diff --git a/docs/Libraries.md b/docs/Libraries.md
index 75bc17c633c..0c57ffab3e8 100644
--- a/docs/Libraries.md
+++ b/docs/Libraries.md
@@ -13,6 +13,9 @@ Some great folks have written their own libraries to interact with Druid
 #### Ruby
 \* [madvertise/ruby-druid](https://github.com/madvertise/ruby-druid) - A ruby client for Druid
 
+#### Python
+\* [metamx/pydruid](https://github.com/metamx/pydruid) - A python client for Druid
+
 #### Helper Libraries
 
 -   [madvertise/druid-dumbo](https://github.com/madvertise/druid-dumbo) - Scripts to help generate batch configs for the ingestion of data into Druid
diff --git a/docs/Loading-Your-Data.md b/docs/Loading-Your-Data.md
index a5edd9d65ea..2e27fad8303 100644
--- a/docs/Loading-Your-Data.md
+++ b/docs/Loading-Your-Data.md
@@ -165,7 +165,7 @@ curl -X POST "http://localhost:8080/druid/v2/?pretty" \
   }
 } ]
 ```
-Now you're ready for [Querying Your Data](Querying Your Data.html)!
+Now you're ready for [Querying Your Data](Querying-Your-Data.html)!
 
 ## Loading Data with the HadoopDruidIndexer ##
 
@@ -367,4 +367,4 @@ Now its time to run the Hadoop [Batch-ingestion](Batch-ingestion.html) job, Hado
 java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=realtime.spec -classpath lib/* com.metamx.druid.indexer.HadoopDruidIndexerMain batchConfig.json
 ```
 
-You can now move on to [Querying Your Data](Querying Your Data.html)!
\ No newline at end of file
+You can now move on to [Querying Your Data](Querying-Your-Data.html)!
\ No newline at end of file
diff --git a/docs/Master.md b/docs/Master.md
index c96af56dea9..eb86a3e81fd 100644
--- a/docs/Master.md
+++ b/docs/Master.md
@@ -15,7 +15,7 @@ Rules
 
 Segments are loaded and dropped from the cluster based on a set of rules. Rules indicate how segments should be assigned to different compute node tiers and how many replicants of a segment should exist in each tier. Rules may also indicate when segments should be dropped entirely from the cluster. The master loads a set of rules from the database. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The master will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule
 
-For more information on rules, see [Rule Configuration](Rule Configuration.html).
+For more information on rules, see [Rule Configuration](Rule-Configuration.html).
 
 Cleaning Up Segments
 --------------------
diff --git a/docs/MySQL.md b/docs/MySQL.md
index 88ef75006cf..713ad0ab18d 100644
--- a/docs/MySQL.md
+++ b/docs/MySQL.md
@@ -44,4 +44,4 @@ The config table is used to store runtime configuration objects. We do not have
 Task-related Tables
 -------------------
 
-There are also a number of tables created and used by the [Indexing Service](Indexing Service.html) in the course of its work.
+There are also a number of tables created and used by the [Indexing Service](Indexing-Service.html) in the course of its work.
diff --git a/docs/Querying-your-data.md b/docs/Querying-your-data.md
index 5bf72a6fa54..dc3e04d645c 100644
--- a/docs/Querying-your-data.md
+++ b/docs/Querying-your-data.md
@@ -3,7 +3,7 @@ layout: default
 ---
 # Setup #
 
-Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading Your Data.html) we setup a [Realtime](Realtime.html), [Compute](Compute.html) and [Master](Master.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'.
+Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading-Your-Data.html) we setup a [Realtime](Realtime.html), [Compute](Compute.html) and [Master](Master.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'.
 
 ## Booting a Broker Node ##
 
@@ -98,7 +98,7 @@ com.metamx.druid.http.ComputeMain
 
 # Querying Your Data #
 
-Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [Loading Your Data](Loading Your Data.html). Having done that, its time to query our data! For a complete specification of queries, see [Querying](Querying.html).
+Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [Loading Your Data](Loading-Your-Data.html). Having done that, its time to query our data! For a complete specification of queries, see [Querying](Querying.html).
 
 ## Querying Different Nodes ##
 
@@ -363,4 +363,4 @@ Check out [Filters](Filters.html) for more.
 
 ## Learn More ##
 
-You can learn more about querying at [Querying](Querying.html)! Now check out [Booting a production cluster](Booting a production cluster.html)!
\ No newline at end of file
+You can learn more about querying at [Querying](Querying.html)! Now check out [Booting a production cluster](Booting-a-production-cluster.html)!
\ No newline at end of file
diff --git a/docs/TimeseriesQuery.md b/docs/TimeseriesQuery.md
index 9ea79fcfa75..62ebcee59f1 100644
--- a/docs/TimeseriesQuery.md
+++ b/docs/TimeseriesQuery.md
@@ -87,7 +87,7 @@ There are 7 main parts to a timeseries query:
 |granularity|Defines the granularity of the query. See [Granularities](Granularities.html)|yes|
 |filter|See [Filters](Filters.html)|no|
 |aggregations|See [Aggregations](Aggregations.html)|yes|
-|postAggregations|See [Post Aggregations](Post Aggregations.html)|no|
+|postAggregations|See [Post Aggregations](Post-Aggregations.html)|no|
 |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
 |context|An additional JSON Object which can be used to specify certain flags.|no|
 
diff --git a/docs/Tutorial:-A-First-Look-at-Druid.md b/docs/Tutorial:-A-First-Look-at-Druid.md
index c3de0df1d91..987cf89fa28 100644
--- a/docs/Tutorial:-A-First-Look-at-Druid.md
+++ b/docs/Tutorial:-A-First-Look-at-Druid.md
@@ -357,9 +357,9 @@ Feel free to tweak other query parameters to answer other questions you may have
 Next Steps
 ----------
 
-What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
+What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
 
-Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading Your Data.html).
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading-Your-Data.html).
 
 Additional Information
 ----------------------
diff --git a/docs/Tutorial:-The-Druid-Cluster.md b/docs/Tutorial:-The-Druid-Cluster.md
index 286447cc3cd..282ec9fa7f8 100644
--- a/docs/Tutorial:-The-Druid-Cluster.md
+++ b/docs/Tutorial:-The-Druid-Cluster.md
@@ -19,7 +19,7 @@ tar -zxvf druid-services-*-bin.tar.gz
 cd druid-services-*
 ```
 
-You can also [Build From Source](Build From Source.html).
+You can also [Build From Source](Build-From-Source.html).
 
 ## External Dependencies ##
 
diff --git a/docs/Tutorial:-Webstream.md b/docs/Tutorial:-Webstream.md
index bbfb42450fd..bfb7ed73bed 100644
--- a/docs/Tutorial:-Webstream.md
+++ b/docs/Tutorial:-Webstream.md
@@ -346,8 +346,8 @@ Feel free to tweak other query parameters to answer other questions you may have
 Next Steps
 ----------
 
-What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
-Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading Your Data.html).
+What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
+Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading-Your-Data.html).
 
 Additional Information
 ----------------------
diff --git a/docs/contents.md b/docs/contents.md
index 0d3f7f9cb62..963f88926e1 100644
--- a/docs/contents.md
+++ b/docs/contents.md
@@ -9,20 +9,20 @@ Contents
 ========================
 
 Getting Started
-\* [Tutorial: A First Look at Druid](Tutorial: A First Look at Druid.html)
-\* [Tutorial: The Druid Cluster](Tutorial: The Druid Cluster.html)
-\* [Loading Your Data](Loading Your Data.html)
-\* [Querying Your Data](Querying Your Data.html)
-\* [Booting a Production Cluster](Booting a Production Cluster.html)
+\* [Tutorial: A First Look at Druid](Tutorial:-A-First-Look-at-Druid.html)
+\* [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
+\* [Loading Your Data](Loading-Your-Data.html)
+\* [Querying Your Data](Querying-Your-Data.html)
+\* [Booting a Production Cluster](Booting-a-Production-Cluster.html)
 \* [Examples](Examples.html)
-\* [Cluster Setup](Cluster Setup.html)
+\* [Cluster Setup](Cluster-Setup.html)
 \* [Configuration](Configuration.html)
 --------------------------------------
 
 Data Ingestion
 \* [Realtime](Realtime.html)
-\* [Batch|Batch Ingestion](Batch|Batch Ingestion.html)
-\* [Indexing Service](Indexing Service.html)
+\* [Batch|Batch Ingestion](Batch|Batch-Ingestion.html)
+\* [Indexing Service](Indexing-Service.html)
 ----------------------------
 
 Querying
@@ -57,12 +57,12 @@ Architecture
 **\* ]
 **\* [MySQL](MySQL.html)
 **\* ]
-** [Concepts and Terminology](Concepts and Terminology.html)
+** [Concepts and Terminology](Concepts-and-Terminology.html)
 -------------------------------
 
 Development
 \* [Versioning](Versioning.html)
-\* [Build From Source](Build From Source.html)
+\* [Build From Source](Build-From-Source.html)
 \* [Libraries](Libraries.html)
 ------------------------
 

From 6df31408d9fbdab0242113cbd3ebbb6485a2a812 Mon Sep 17 00:00:00 2001
From: Russell Jurney 
Date: Mon, 16 Sep 2013 17:09:59 -0700
Subject: [PATCH 82/92] made layouts work

---
 docs/_layouts/default.html            |  179 +-
 docs/_layouts/docs.html               |    8 +
 docs/_layouts/page.html               |   11 +
 docs/_layouts/post.html               |   43 +-
 docs/css/bootstrap-responsive.css     | 1058 +++++
 docs/css/bootstrap-responsive.min.css |    9 +
 docs/css/bootstrap.css                | 5774 +++++++++++++++++++++++++
 docs/css/bootstrap.min.css            |    9 +
 docs/css/custom.css                   |  592 +++
 docs/css/default.html                 |  147 +
 docs/css/docs.html                    |    8 +
 docs/css/page.html                    |   11 +
 docs/css/pie.htc                      |   96 +
 docs/css/post.html                    |   44 +
 docs/css/syntax.css                   |    2 +-
 15 files changed, 7948 insertions(+), 43 deletions(-)
 create mode 100644 docs/_layouts/docs.html
 create mode 100644 docs/_layouts/page.html
 create mode 100644 docs/css/bootstrap-responsive.css
 create mode 100644 docs/css/bootstrap-responsive.min.css
 create mode 100644 docs/css/bootstrap.css
 create mode 100644 docs/css/bootstrap.min.css
 create mode 100644 docs/css/custom.css
 create mode 100644 docs/css/default.html
 create mode 100644 docs/css/docs.html
 create mode 100644 docs/css/page.html
 create mode 100644 docs/css/pie.htc
 create mode 100644 docs/css/post.html

diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
index 22e7e3f2a31..12106274e0f 100644
--- a/docs/_layouts/default.html
+++ b/docs/_layouts/default.html
@@ -1,44 +1,147 @@
 
-
-    
-        
-        
-        {{ page.title }}
-        
-
-        
-        
-
-        
-        
-
-    
-    
-
-        
- - - {{ content }} - -