diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6970489ebeb..76af3436ca2 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -33,15 +33,18 @@ - com.metamx.druid - druid-server - ${project.parent.version} + io.druid + druid-api com.netflix.astyanax astyanax 1.0.1 + + commons-io + commons-io + diff --git a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentConfig.java b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentConfig.java similarity index 96% rename from cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentConfig.java rename to cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentConfig.java index 2a8fdafab84..8069dcd4baf 100644 --- a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentConfig.java +++ b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading.cassandra; +package io.druid.storage.cassandra; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPuller.java b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java similarity index 97% rename from cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPuller.java rename to cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java index 01f9264d3cc..0296d1045a9 100644 --- a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPuller.java +++ b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPuller.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading.cassandra; +package io.druid.storage.cassandra; import com.google.common.io.Files; import com.google.inject.Inject; @@ -28,10 +28,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 io.druid.common.utils.CompressionUtils; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPusher.java b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java similarity index 90% rename from cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPusher.java rename to cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java index 76b4ac52ddd..1d436796450 100644 --- a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDataSegmentPusher.java +++ b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDataSegmentPusher.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading.cassandra; +package io.druid.storage.cassandra; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; @@ -26,11 +26,11 @@ import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.netflix.astyanax.MutationBatch; import com.netflix.astyanax.recipes.storage.ChunkedStorage; -import io.druid.common.utils.CompressionUtils; -import io.druid.segment.IndexIO; +import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import java.io.File; import java.io.FileInputStream; @@ -57,7 +57,13 @@ public class CassandraDataSegmentPusher extends CassandraStorage implements Data this.jsonMapper=jsonMapper; } - @Override + @Override + public String getPathForHadoop(String dataSource) + { + throw new UnsupportedOperationException("Cassandra storage does not support indexing via Hadoop"); + } + + @Override public DataSegment push(final File indexFilesDir, DataSegment segment) throws IOException { log.info("Writing [%s] to C*", indexFilesDir); @@ -71,7 +77,7 @@ public class CassandraDataSegmentPusher extends CassandraStorage implements Data long indexSize = CompressionUtils.zip(indexFilesDir, compressedIndexFile); log.info("Wrote compressed file [%s] to [%s]", compressedIndexFile.getAbsolutePath(), key); - int version = IndexIO.getVersionFromDir(indexFilesDir); + int version = SegmentUtils.getVersionFromDir(indexFilesDir); try { diff --git a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDruidModule.java b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java similarity index 93% rename from cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDruidModule.java rename to cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java index 029f295e5b9..8215f3fe42a 100644 --- a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraDruidModule.java +++ b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraDruidModule.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading.cassandra; +package io.druid.storage.cassandra; import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Key; -import io.druid.guice.DruidBinders; +import io.druid.guice.Binders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.PolyBind; @@ -45,7 +45,7 @@ public class CassandraDruidModule implements DruidModule @Override public void configure(Binder binder) { - DruidBinders.dataSegmentPullerBinder(binder) + Binders.dataSegmentPullerBinder(binder) .addBinding("c*") .to(CassandraDataSegmentPuller.class) .in(LazySingleton.class); diff --git a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraStorage.java b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraStorage.java similarity index 98% rename from cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraStorage.java rename to cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraStorage.java index 3a693575c18..67081354830 100644 --- a/cassandra-storage/src/main/java/io/druid/segment/loading/cassandra/CassandraStorage.java +++ b/cassandra-storage/src/main/java/io/druid/storage/cassandra/CassandraStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading.cassandra; +package io.druid.storage.cassandra; import com.netflix.astyanax.AstyanaxContext; import com.netflix.astyanax.Keyspace; diff --git a/cassandra-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/cassandra-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index ade20b5ba28..04144072177 100644 --- a/cassandra-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/cassandra-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -1 +1 @@ -io.druid.segment.loading.cassandra.CassandraDruidModule \ No newline at end of file +io.druid.storage.cassandra.CassandraDruidModule \ No newline at end of file diff --git a/common/src/main/java/io/druid/common/guava/Runnables.java b/common/src/main/java/io/druid/common/guava/Runnables.java deleted file mode 100644 index e855f4081a3..00000000000 --- a/common/src/main/java/io/druid/common/guava/Runnables.java +++ /dev/null @@ -1,43 +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.common.guava; - -/** - */ -public class Runnables -{ - public static Runnable threadNaming(final String threadName, final Runnable runnable) - { - return new ThreadRenamingRunnable(threadName) - { - @Override - public void doRun() - { - runnable.run(); - } - }; - } - - public static Runnable getNoopRunnable(){ - return new Runnable(){ - public void run(){} - }; - } -} diff --git a/common/src/main/java/io/druid/common/utils/CompressionUtils.java b/common/src/main/java/io/druid/common/utils/CompressionUtils.java deleted file mode 100644 index dcca115a982..00000000000 --- a/common/src/main/java/io/druid/common/utils/CompressionUtils.java +++ /dev/null @@ -1,142 +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.common.utils; - -import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; -import com.google.common.io.Files; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.zip.GZIPInputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; -import java.util.zip.ZipOutputStream; - -/** - */ -public class CompressionUtils -{ - private static final Logger log = new Logger(CompressionUtils.class); - - public static long zip(File directory, File outputZipFile) throws IOException - { - if (!outputZipFile.getName().endsWith(".zip")) { - log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory); - } - - final FileOutputStream out = new FileOutputStream(outputZipFile); - try { - final long retVal = zip(directory, out); - - out.close(); - - return retVal; - } - finally { - Closeables.closeQuietly(out); - } - } - - public static long zip(File directory, OutputStream out) throws IOException - { - if (!directory.isDirectory()) { - throw new IOException(String.format("directory[%s] is not a directory", directory)); - } - - long totalSize = 0; - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(out); - File[] files = directory.listFiles(); - for (File file : files) { - log.info("Adding file[%s] with size[%,d]. Total size so far[%,d]", file, file.length(), totalSize); - if (file.length() >= Integer.MAX_VALUE) { - zipOut.finish(); - throw new IOException(String.format("file[%s] too large [%,d]", file, file.length())); - } - zipOut.putNextEntry(new ZipEntry(file.getName())); - totalSize += ByteStreams.copy(Files.newInputStreamSupplier(file), zipOut); - } - zipOut.closeEntry(); - } - finally { - if (zipOut != null) { - zipOut.finish(); - } - } - - return totalSize; - } - - public static void unzip(File pulledFile, File outDir) throws IOException - { - if (!(outDir.exists() && outDir.isDirectory())) { - throw new ISE("outDir[%s] must exist and be a directory", outDir); - } - - log.info("Unzipping file[%s] to [%s]", pulledFile, outDir); - InputStream in = null; - try { - in = new BufferedInputStream(new FileInputStream(pulledFile)); - unzip(in, outDir); - } - finally { - Closeables.closeQuietly(in); - } - } - - public static void unzip(InputStream in, File outDir) throws IOException - { - ZipInputStream zipIn = new ZipInputStream(in); - - ZipEntry entry; - while ((entry = zipIn.getNextEntry()) != null) { - FileOutputStream out = null; - try { - out = new FileOutputStream(new File(outDir, entry.getName())); - ByteStreams.copy(zipIn, out); - zipIn.closeEntry(); - out.close(); - } - finally { - Closeables.closeQuietly(out); - } - } - } - - public static void gunzip(File pulledFile, File outDir) throws IOException - { - log.info("Gunzipping file[%s] to [%s]", pulledFile, outDir); - StreamUtils.copyToFileAndClose(new GZIPInputStream(new FileInputStream(pulledFile)), outDir); - if (!pulledFile.delete()) { - log.error("Could not delete tmpFile[%s].", pulledFile); - } - } - -} diff --git a/common/src/main/java/io/druid/guice/PolyBind.java b/common/src/main/java/io/druid/guice/PolyBind.java deleted file mode 100644 index 62b56efd9a9..00000000000 --- a/common/src/main/java/io/druid/guice/PolyBind.java +++ /dev/null @@ -1,157 +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.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/io/druid/guice/PolyBindTest.java b/common/src/test/java/io/druid/guice/PolyBindTest.java deleted file mode 100644 index 0c90878e169..00000000000 --- a/common/src/test/java/io/druid/guice/PolyBindTest.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.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", "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", "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", "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", "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/examples/pom.xml b/examples/pom.xml index 2a8310e92ba..4891972dbf4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -63,6 +63,13 @@ twitter4j-stream 3.0.3 + + + + junit + junit + test + diff --git a/examples/src/main/java/druid/examples/ExamplesDruidModule.java b/examples/src/main/java/druid/examples/ExamplesDruidModule.java new file mode 100644 index 00000000000..5a3a024fd33 --- /dev/null +++ b/examples/src/main/java/druid/examples/ExamplesDruidModule.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 druid.examples; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +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.util.Arrays; +import java.util.List; + +/** + */ +public class ExamplesDruidModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("ExamplesModule") + .registerSubtypes( + new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"), + new NamedType(FlightsFirehoseFactory.class, "flights"), + new NamedType(RandomFirehoseFactory.class, "rand"), + new NamedType(WebFirehoseFactory.class, "webstream") + ) + ); + } + + @Override + public void configure(Binder binder) + { + + } +} diff --git a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java index 42f62440736..fce667f5df3 100644 --- a/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/flights/FlightsFirehoseFactory.java @@ -28,7 +28,7 @@ import com.google.common.io.Closeables; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.impl.StringInputRowParser; import java.io.BufferedReader; import java.io.File; diff --git a/examples/src/main/java/druid/examples/web/WebFirehoseFactory.java b/examples/src/main/java/druid/examples/web/WebFirehoseFactory.java index 2c7ccfa09a5..e4ae9ac4c1a 100644 --- a/examples/src/main/java/druid/examples/web/WebFirehoseFactory.java +++ b/examples/src/main/java/druid/examples/web/WebFirehoseFactory.java @@ -25,11 +25,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Throwables; import com.metamx.common.parsers.TimestampParser; import com.metamx.emitter.EmittingLogger; -import io.druid.common.guava.Runnables; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; +import io.druid.utils.Runnables; import org.joda.time.DateTime; import java.io.IOException; diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml new file mode 100644 index 00000000000..fb555601dc9 --- /dev/null +++ b/hdfs-storage/pom.xml @@ -0,0 +1,61 @@ + + + + + 4.0.0 + com.metamx.druid + druid-hdfs-storage + druid-hdfs-storage + druid-hdfs-storage + + + com.metamx + druid + 0.6.0-SNAPSHOT + + + + + io.druid + druid-api + + + org.apache.hadoop + hadoop-core + 1.0.3 + compile + + + com.metamx + emitter + + + commons-io + commons-io + + + + + junit + junit + test + + + diff --git a/server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPuller.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java similarity index 94% rename from server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPuller.java rename to hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java index d19c4cef793..6af99afcbcd 100644 --- a/server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPuller.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java @@ -17,12 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.hdfs; import com.google.common.io.Closeables; import com.google.inject.Inject; -import io.druid.common.utils.CompressionUtils; +import io.druid.segment.loading.DataSegmentPuller; +import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; 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/io/druid/segment/loading/HdfsDataSegmentPusher.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java similarity index 86% rename from server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPusher.java rename to hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java index eb620698689..2fd4f071b32 100644 --- a/server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPusher.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -17,18 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.hdfs; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; 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 io.druid.common.utils.CompressionUtils; -import io.druid.segment.IndexIO; +import io.druid.segment.SegmentUtils; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -60,6 +63,17 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher this.jsonMapper = jsonMapper; } + @Override + public String getPathForHadoop(String dataSource) + { + try { + return new Path(config.getStorageDirectory(), dataSource).makeQualified(FileSystem.get(hadoopConfig)).toString(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + @Override public DataSegment push(File inDir, DataSegment segment) throws IOException { @@ -85,7 +99,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher return createDescriptorFile( segment.withLoadSpec(makeLoadSpec(outFile)) .withSize(size) - .withBinaryVersion(IndexIO.CURRENT_VERSION_ID), + .withBinaryVersion(SegmentUtils.getVersionFromDir(inDir)), outFile.getParent(), fs ); diff --git a/server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPusherConfig.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusherConfig.java similarity index 96% rename from server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPusherConfig.java rename to hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusherConfig.java index e329fdeb4c4..e262bd665d3 100644 --- a/server/src/main/java/io/druid/segment/loading/HdfsDataSegmentPusherConfig.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusherConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.hdfs; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java new file mode 100644 index 00000000000..e2fb1475742 --- /dev/null +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java @@ -0,0 +1,71 @@ +/* + * 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.storage.hdfs; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Inject; +import io.druid.guice.Binders; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.initialization.DruidModule; +import org.apache.hadoop.conf.Configuration; + +import java.util.List; +import java.util.Properties; + +/** + */ +public class HdfsStorageDruidModule implements DruidModule +{ + private Properties props = null; + + @Inject + public void setProperties(Properties props) + { + this.props = props; + } + + @Override + public List getJacksonModules() + { + return ImmutableList.of(); + } + + @Override + public void configure(Binder binder) + { + Binders.dataSegmentPullerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); + Binders.dataSegmentPusherBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPusher.class).in(LazySingleton.class); + + final Configuration conf = new Configuration(); + if (props != null) { + for (String propName : System.getProperties().stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), System.getProperty(propName)); + } + } + } + + binder.bind(Configuration.class).toInstance(conf); + JsonConfigProvider.bind(binder, "druid.storage", HdfsDataSegmentPusherConfig.class); + } +} diff --git a/hdfs-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/hdfs-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 00000000000..f3ccd6b05ca --- /dev/null +++ b/hdfs-storage/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.storage.hdfs.HdfsStorageDruidModule \ No newline at end of file diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 569a61db92b..d09b34b13b0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -75,11 +75,6 @@ com.google.code.findbugs jsr305 - - net.java.dev.jets3t - jets3t - 0.7.1 - diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java index 9f83a2374f9..cd1d3bb753c 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DeterminePartitionsJob.java @@ -141,7 +141,7 @@ public class DeterminePartitionsJob implements Jobby groupByJob.setOutputKeyClass(BytesWritable.class); groupByJob.setOutputValueClass(NullWritable.class); groupByJob.setOutputFormatClass(SequenceFileOutputFormat.class); - groupByJob.setJarByClass(DeterminePartitionsJob.class); + JobHelper.setupClasspath(config, groupByJob); config.addInputPaths(groupByJob); config.intoConfiguration(groupByJob); @@ -189,9 +189,9 @@ public class DeterminePartitionsJob implements Jobby dimSelectionJob.setOutputKeyClass(BytesWritable.class); dimSelectionJob.setOutputValueClass(Text.class); dimSelectionJob.setOutputFormatClass(DeterminePartitionsDimSelectionOutputFormat.class); - dimSelectionJob.setJarByClass(DeterminePartitionsJob.class); dimSelectionJob.setPartitionerClass(DeterminePartitionsDimSelectionPartitioner.class); dimSelectionJob.setNumReduceTasks(config.getGranularitySpec().bucketIntervals().size()); + JobHelper.setupClasspath(config, dimSelectionJob); config.intoConfiguration(dimSelectionJob); FileOutputFormat.setOutputPath(dimSelectionJob, config.makeIntermediatePath()); @@ -486,7 +486,7 @@ public class DeterminePartitionsJob implements Jobby private Iterable combineRows(Iterable input) { - return new CombiningIterable( + return new CombiningIterable<>( Iterables.transform( input, new Function() @@ -758,14 +758,19 @@ public class DeterminePartitionsJob implements Jobby log.info("Chosen partitions:"); for (ShardSpec shardSpec : chosenShardSpecs) { - log.info(" %s", shardSpec); + log.info(" %s", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(shardSpec)); } + System.out.println(HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(chosenShardSpecs)); + try { - HadoopDruidIndexerConfig.jsonMapper.writerWithType(new TypeReference>() {}).writeValue( - out, - chosenShardSpecs - ); + HadoopDruidIndexerConfig.jsonMapper + .writerWithType( + new TypeReference>() + { + } + ) + .writeValue(out, chosenShardSpecs); } finally { Closeables.close(out, false); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 1156885bbd1..ec51d872f33 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -21,7 +21,6 @@ package io.druid.indexer; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -33,24 +32,31 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; 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.metamx.common.Granularity; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; -import io.druid.data.input.DataSpec; import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; -import io.druid.data.input.TimestampSpec; -import io.druid.data.input.ToLowercaseDataSpec; +import io.druid.data.input.impl.DataSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.data.input.impl.ToLowercaseDataSpec; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; import io.druid.indexer.granularity.GranularitySpec; import io.druid.indexer.granularity.UniformGranularitySpec; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; import io.druid.indexer.rollup.DataRollupSpec; import io.druid.indexer.updater.DbUpdaterJobSpec; -import io.druid.jackson.DefaultObjectMapper; +import io.druid.initialization.Initialization; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.ShardSpec; import org.apache.hadoop.conf.Configuration; @@ -78,11 +84,28 @@ public class HadoopDruidIndexerConfig public static final Splitter tabSplitter = Splitter.on("\t"); public static final Joiner tabJoiner = Joiner.on("\t"); + + private static final Injector injector; + public static final ObjectMapper jsonMapper; static { - jsonMapper = new DefaultObjectMapper(); - jsonMapper.configure(JsonGenerator.Feature.ESCAPE_NON_ASCII, true); + injector = Initialization.makeInjectorWithModules( + Initialization.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", "localhost", -1) + ); + } + } + ) + ); + jsonMapper = injector.getInstance(ObjectMapper.class); } public static enum IndexJobCounters diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java index cbd579731a9..11d37df65ad 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java @@ -21,7 +21,7 @@ package io.druid.indexer; import com.metamx.common.RE; import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.impl.StringInputRowParser; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Mapper; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 4d62a83ee22..c4b9738e191 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -33,12 +33,13 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.impl.StringInputRowParser; import io.druid.indexer.rollup.DataRollupSpec; 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.SegmentUtils; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.timeline.DataSegment; @@ -136,7 +137,7 @@ public class IndexGeneratorJob implements Jobby config.addInputPaths(job); config.intoConfiguration(job); - job.setJarByClass(IndexGeneratorJob.class); + JobHelper.setupClasspath(config, job); job.submit(); log.info("Job %s submitted, status available at %s", job.getJobName(), job.getTrackingURL()); @@ -446,7 +447,7 @@ public class IndexGeneratorJob implements Jobby dimensionNames, metricNames, config.getShardSpec(bucket).getActualSpec(), - IndexIO.getVersionFromDir(mergedBase), + SegmentUtils.getVersionFromDir(mergedBase), size ); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java new file mode 100644 index 00000000000..ded0801f41a --- /dev/null +++ b/indexing-hadoop/src/main/java/io/druid/indexer/JobHelper.java @@ -0,0 +1,97 @@ +/* + * 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.indexer; + +import com.google.api.client.util.Sets; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.google.common.io.OutputSupplier; +import com.metamx.common.logger.Logger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.Job; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Set; + +/** + */ +public class JobHelper +{ + private static final Logger log = new Logger(JobHelper.class); + + private static final Set existing = Sets.newHashSet(); + + + public static void setupClasspath( + HadoopDruidIndexerConfig config, + Job groupByJob + ) + throws IOException + { + String classpathProperty = System.getProperty("druid.hadoop.internal.classpath"); + if (classpathProperty == null) { + classpathProperty = System.getProperty("java.class.path"); + } + + String[] jarFiles = classpathProperty.split(File.pathSeparator); + + final Configuration conf = groupByJob.getConfiguration(); + final FileSystem fs = FileSystem.get(conf); + Path distributedClassPath = new Path(config.getJobOutputDir(), "classpath"); + + if (fs instanceof LocalFileSystem) { + return; + } + + for (String jarFilePath : jarFiles) { + File jarFile = new File(jarFilePath); + if (jarFile.getName().endsWith(".jar")) { + final Path hdfsPath = new Path(distributedClassPath, jarFile.getName()); + + if (! existing.contains(hdfsPath)) { + if (jarFile.getName().endsWith("SNAPSHOT.jar") || !fs.exists(hdfsPath)) { + log.info("Uploading jar to path[%s]", hdfsPath); + ByteStreams.copy( + Files.newInputStreamSupplier(jarFile), + new OutputSupplier() + { + @Override + public OutputStream getOutput() throws IOException + { + return fs.create(hdfsPath); + } + } + ); + } + + existing.add(hdfsPath); + } + + DistributedCache.addFileToClassPath(hdfsPath, conf, fs); + } + } + } +} diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 30c847821de..45c2fb0aa7d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -130,10 +130,6 @@ com.sun.jersey.contribs jersey-guice - - net.java.dev.jets3t - jets3t - org.eclipse.jetty jetty-server @@ -163,5 +159,10 @@ curator-test test + + org.apache.hadoop + hadoop-core + test + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java index 168d3e9a7b3..f4033d8f475 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/EventReceiverFirehoseFactory.java @@ -32,7 +32,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.MapInputRowParser; +import io.druid.data.input.impl.MapInputRowParser; import javax.ws.rs.POST; import javax.ws.rs.Path; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index fdbc676ac40..b66d5a1d63e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -35,6 +35,7 @@ import io.druid.query.QueryRunner; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.FireHydrant; @@ -150,7 +151,7 @@ public class YeOldePlumberSchool implements PlumberSchool final DataSegment segmentToUpload = theSink.getSegment() .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) - .withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload)); + .withBinaryVersion(SegmentUtils.getVersionFromDir(fileToUpload)); dataSegmentPusher.push(fileToUpload, segmentToUpload); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 65d8274b112..09f42e56017 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -32,9 +32,7 @@ import io.druid.indexer.HadoopDruidIndexerJob; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SegmentInsertAction; -import io.druid.segment.loading.S3DataSegmentPusher; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; @@ -102,21 +100,7 @@ public class HadoopIndexTask extends AbstractTask // Set workingPath to some reasonable default configCopy.setJobOutputDir(toolbox.getConfig().getHadoopWorkingPath()); - if (toolbox.getSegmentPusher() instanceof S3DataSegmentPusher) { - // Hack alert! Bypassing DataSegmentPusher... - S3DataSegmentPusher segmentPusher = (S3DataSegmentPusher) toolbox.getSegmentPusher(); - String s3Path = String.format( - "s3n://%s/%s/%s", - segmentPusher.getConfig().getBucket(), - segmentPusher.getConfig().getBaseKey(), - getDataSource() - ); - - log.info("Setting segment output path to: %s", s3Path); - configCopy.setSegmentOutputDir(s3Path); - } else { - throw new IllegalStateException("Sorry, we only work with S3DataSegmentPushers! Bummer!"); - } + configCopy.setSegmentOutputDir(toolbox.getSegmentPusher().getPathForHadoop(getDataSource())); HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(configCopy); configCopy.verify(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java index 659ad4cfd3a..84f6211ad52 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexGeneratorTask.java @@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -33,7 +32,6 @@ import io.druid.data.input.InputRow; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; -import io.druid.indexing.common.actions.LockListAction; import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.index.YeOldePlumberSchool; import io.druid.segment.loading.DataSegmentPusher; @@ -48,7 +46,6 @@ import org.joda.time.Interval; import java.io.File; import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; public class IndexGeneratorTask extends AbstractTask @@ -126,6 +123,12 @@ public class IndexGeneratorTask extends AbstractTask final List pushedSegments = new CopyOnWriteArrayList(); final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher() { + @Override + public String getPathForHadoop(String dataSource) + { + return toolbox.getSegmentPusher().getPathForHadoop(dataSource); + } + @Override public DataSegment push(File file, DataSegment segment) throws IOException { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 8056e71dc11..6e7d9f61f17 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -26,6 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.indexer.granularity.GranularitySpec; import io.druid.indexing.common.TaskStatus; @@ -33,7 +34,6 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SpawnTasksAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.incremental.SpatialDimensionSchema; import io.druid.segment.realtime.Schema; import io.druid.timeline.partition.NoneShardSpec; import org.joda.time.DateTime; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/NoopTaskLogs.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/NoopTaskLogs.java deleted file mode 100644 index 7adddf18f64..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/NoopTaskLogs.java +++ /dev/null @@ -1,45 +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.indexing.common.tasklogs; - -import com.google.common.base.Optional; -import com.google.common.io.InputSupplier; -import com.metamx.common.logger.Logger; - -import java.io.File; -import java.io.IOException; -import java.io.InputStream; - -public class NoopTaskLogs implements TaskLogs -{ - private final Logger log = new Logger(TaskLogs.class); - - @Override - public Optional> streamTaskLog(String taskid, long offset) throws IOException - { - return Optional.absent(); - } - - @Override - public void pushTaskLog(String taskid, File logFile) throws IOException - { - log.info("Not pushing logs for task: %s", taskid); - } -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java index 606c370ba9a..e5de451e622 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/SwitchingTaskLogStreamer.java @@ -23,6 +23,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.google.common.io.InputSupplier; import com.google.inject.Inject; +import io.druid.tasklogs.TaskLogStreamer; import java.io.IOException; import java.io.InputStream; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogPusher.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogPusher.java deleted file mode 100644 index c40f7497fff..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogPusher.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.indexing.common.tasklogs; - -import java.io.File; -import java.io.IOException; - -/** - * Something that knows how to persist local task logs to some form of long-term storage. - */ -public interface TaskLogPusher -{ - public void pushTaskLog(String taskid, File logFile) throws IOException; -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogStreamer.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogStreamer.java deleted file mode 100644 index e260ffc29b0..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogStreamer.java +++ /dev/null @@ -1,42 +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.indexing.common.tasklogs; - -import com.google.common.base.Optional; -import com.google.common.io.InputSupplier; - -import java.io.IOException; -import java.io.InputStream; - -/** - * Something that knows how to stream logs for tasks. - */ -public interface TaskLogStreamer -{ - /** - * Stream log for a task. - * - * @param offset If zero, stream the entire log. If positive, attempt to read from this position onwards. If - * negative, attempt to read this many bytes from the end of the file (like tail -n). - * - * @return input supplier for this log, if available from this provider - */ - public Optional> streamTaskLog(String taskid, long offset) throws IOException; -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogs.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogs.java deleted file mode 100644 index ba89db1bd38..00000000000 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskLogs.java +++ /dev/null @@ -1,24 +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.indexing.common.tasklogs; - -public interface TaskLogs extends TaskLogStreamer, TaskLogPusher -{ -} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java index 03ad653abc3..c77c1497d76 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/TaskRunnerTaskLogStreamer.java @@ -24,6 +24,7 @@ import com.google.common.io.InputSupplier; import com.google.inject.Inject; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskRunner; +import io.druid.tasklogs.TaskLogStreamer; import java.io.IOException; import java.io.InputStream; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 1e69516db95..330ba961d62 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -43,10 +43,10 @@ import com.metamx.emitter.EmittingLogger; import io.druid.guice.annotations.Self; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; -import io.druid.indexing.common.tasklogs.TaskLogPusher; -import io.druid.indexing.common.tasklogs.TaskLogStreamer; import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; +import io.druid.tasklogs.TaskLogStreamer; import org.apache.commons.io.FileUtils; import java.io.File; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java index 4a090f4fa47..1716d62602d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java @@ -22,9 +22,9 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import io.druid.guice.annotations.Self; -import io.druid.indexing.common.tasklogs.TaskLogPusher; import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.server.DruidNode; +import io.druid.tasklogs.TaskLogPusher; import java.util.Properties; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index fb212a164d9..303a8365796 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -45,12 +45,12 @@ import com.metamx.http.client.response.StatusResponseHolder; import io.druid.curator.cache.PathChildrenCacheFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.Task; -import io.druid.indexing.common.tasklogs.TaskLogStreamer; import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.server.initialization.ZkPathsConfig; +import io.druid.tasklogs.TaskLogStreamer; 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/overlord/http/OldOverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java index 96c66ba8dde..6897490624a 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OldOverlordResource.java @@ -21,9 +21,9 @@ package io.druid.indexing.overlord.http; import com.google.inject.Inject; import io.druid.common.config.JacksonConfigManager; -import io.druid.indexing.common.tasklogs.TaskLogStreamer; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskStorageQueryAdapter; +import io.druid.tasklogs.TaskLogStreamer; import javax.ws.rs.Path; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 2ffc124a5bb..ef195b6f6cc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -31,7 +31,6 @@ import io.druid.common.config.JacksonConfigManager; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.actions.TaskActionHolder; import io.druid.indexing.common.task.Task; -import io.druid.indexing.common.tasklogs.TaskLogStreamer; import io.druid.indexing.overlord.TaskMaster; import io.druid.indexing.overlord.TaskQueue; import io.druid.indexing.overlord.TaskRunner; @@ -39,6 +38,7 @@ import io.druid.indexing.overlord.TaskRunnerWorkItem; import io.druid.indexing.overlord.TaskStorageQueryAdapter; import io.druid.indexing.overlord.scaling.ResourceManagementScheduler; import io.druid.indexing.overlord.setup.WorkerSetupData; +import io.druid.tasklogs.TaskLogStreamer; import io.druid.timeline.DataSegment; import javax.ws.rs.Consumes; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 9c3ace35806..6594b860a55 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; import com.metamx.common.Granularity; -import io.druid.data.input.JSONDataSpec; +import io.druid.data.input.impl.JSONDataSpec; import io.druid.granularity.QueryGranularity; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.granularity.UniformGranularitySpec; diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 5575b87eb2e..b155b31dd0f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -127,6 +127,12 @@ public class TaskLifecycleTest newMockEmitter(), new DataSegmentPusher() { + @Override + public String getPathForHadoop(String dataSource) + { + throw new UnsupportedOperationException(); + } + @Override public DataSegment push(File file, DataSegment segment) throws IOException { diff --git a/pom.xml b/pom.xml index 8d1a3d034cc..04a8a4d45a8 100644 --- a/pom.xml +++ b/pom.xml @@ -47,11 +47,12 @@ examples indexing-hadoop indexing-service - realtime server services processing cassandra-storage + hdfs-storage + s3-extensions @@ -59,7 +60,7 @@ io.druid druid-api - 1.0.0-SNAPSHOT + 0.1.0-SNAPSHOT @@ -122,7 +123,7 @@ com.amazonaws aws-java-sdk - 1.3.27 + 1.6.0.1 javax.mail @@ -146,7 +147,7 @@ io.airlift airline - 0.5 + 0.6 org.skife.config @@ -173,17 +174,6 @@ curator-x-discovery ${apache.curator.version} - - org.apache.hadoop - hadoop-core - 0.20.2 - - - org.mortbay.jetty - servlet-api-2.5 - - - it.uniroma3.mat extendedset @@ -299,11 +289,6 @@ jersey-server 1.17.1 - - net.java.dev.jets3t - jets3t - 0.8.1 - org.eclipse.jetty jetty-server @@ -394,8 +379,28 @@ aether-api 0.9.0.M2 - - + + kafka + core-kafka + 0.7.2-mmx1 + + + log4j + log4j + + + + + com.rabbitmq + amqp-client + 3.1.1 + + + org.apache.hadoop + hadoop-core + 1.0.3 + provided + diff --git a/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java b/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java deleted file mode 100644 index d033013bc6d..00000000000 --- a/processing/src/main/java/io/druid/data/input/ByteBufferInputRowParser.java +++ /dev/null @@ -1,33 +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.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/processing/src/main/java/io/druid/data/input/CSVDataSpec.java b/processing/src/main/java/io/druid/data/input/CSVDataSpec.java deleted file mode 100644 index c0cedfe2095..00000000000 --- a/processing/src/main/java/io/druid/data/input/CSVDataSpec.java +++ /dev/null @@ -1,98 +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.data.input; - -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.parsers.CSVParser; -import com.metamx.common.parsers.Parser; -import io.druid.segment.incremental.SpatialDimensionSchema; - -import java.util.List; - -/** - */ -public class CSVDataSpec implements DataSpec -{ - private final List columns; - private final List dimensions; - private final List spatialDimensions; - - @JsonCreator - public CSVDataSpec( - @JsonProperty("columns") List columns, - @JsonProperty("dimensions") List dimensions, - @JsonProperty("spatialDimensions") List spatialDimensions - ) - { - Preconditions.checkNotNull(columns, "columns"); - for (String column : columns) { - Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); - } - - this.columns = columns; - this.dimensions = dimensions; - this.spatialDimensions = (spatialDimensions == null) - ? Lists.newArrayList() - : spatialDimensions; - } - - @JsonProperty("columns") - public List getColumns() - { - return columns; - } - - @JsonProperty("dimensions") - @Override - public List getDimensions() - { - return dimensions; - } - - @JsonProperty("spatialDimensions") - @Override - public List getSpatialDimensions() - { - return spatialDimensions; - } - - @Override - public void verify(List usedCols) - { - for (String columnName : usedCols) { - Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); - } - } - - @Override - public boolean hasCustomDimensions() - { - return !(dimensions == null || dimensions.isEmpty()); - } - - @Override - public Parser getParser() - { - return new CSVParser(columns); - } -} diff --git a/processing/src/main/java/io/druid/data/input/DataSpec.java b/processing/src/main/java/io/druid/data/input/DataSpec.java deleted file mode 100644 index b45faec2a9a..00000000000 --- a/processing/src/main/java/io/druid/data/input/DataSpec.java +++ /dev/null @@ -1,48 +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.data.input; - -import com.fasterxml.jackson.annotation.JsonSubTypes; -import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.common.parsers.Parser; -import io.druid.segment.incremental.SpatialDimensionSchema; - -import java.util.List; - -/** - */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format", defaultImpl = DelimitedDataSpec.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "json", value = JSONDataSpec.class), - @JsonSubTypes.Type(name = "csv", value = CSVDataSpec.class), - @JsonSubTypes.Type(name = "tsv", value = DelimitedDataSpec.class) -}) -public interface DataSpec -{ - public void verify(List usedCols); - - public boolean hasCustomDimensions(); - - public List getDimensions(); - - public List getSpatialDimensions(); - - public Parser getParser(); -} diff --git a/processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java b/processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java deleted file mode 100644 index f9e819f07ec..00000000000 --- a/processing/src/main/java/io/druid/data/input/DelimitedDataSpec.java +++ /dev/null @@ -1,109 +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.data.input; - -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.parsers.DelimitedParser; -import com.metamx.common.parsers.Parser; -import io.druid.segment.incremental.SpatialDimensionSchema; - -import java.util.List; - -/** - */ -public class DelimitedDataSpec implements DataSpec -{ - private final String delimiter; - private final List columns; - private final List dimensions; - private final List spatialDimensions; - - @JsonCreator - public DelimitedDataSpec( - @JsonProperty("delimiter") String delimiter, - @JsonProperty("columns") List columns, - @JsonProperty("dimensions") List dimensions, - @JsonProperty("spatialDimensions") List spatialDimensions - ) - { - Preconditions.checkNotNull(columns); - for (String column : columns) { - Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column); - } - - this.delimiter = (delimiter == null) ? DelimitedParser.DEFAULT_DELIMITER : delimiter; - this.columns = columns; - this.dimensions = dimensions; - this.spatialDimensions = (spatialDimensions == null) - ? Lists.newArrayList() - : spatialDimensions; - } - - @JsonProperty("delimiter") - public String getDelimiter() - { - return delimiter; - } - - @JsonProperty("columns") - public List getColumns() - { - return columns; - } - - @JsonProperty("dimensions") - @Override - public List getDimensions() - { - return dimensions; - } - - @JsonProperty("spatialDimensions") - @Override - public List getSpatialDimensions() - { - return spatialDimensions; - } - - @Override - public void verify(List usedCols) - { - for (String columnName : usedCols) { - Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName); - } - } - - @Override - public boolean hasCustomDimensions() - { - return !(dimensions == null || dimensions.isEmpty()); - } - - @Override - public Parser getParser() - { - Parser retVal = new DelimitedParser(delimiter); - retVal.setFieldNames(columns); - return retVal; - } -} diff --git a/processing/src/main/java/io/druid/data/input/InputRowParser.java b/processing/src/main/java/io/druid/data/input/InputRowParser.java deleted file mode 100644 index ea1a0dfa8a0..00000000000 --- a/processing/src/main/java/io/druid/data/input/InputRowParser.java +++ /dev/null @@ -1,28 +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.data.input; - -import com.metamx.common.exception.FormattedException; - -public interface InputRowParser -{ - public InputRow parse(T input) throws FormattedException; - public void addDimensionExclusion(String dimension); -} diff --git a/processing/src/main/java/io/druid/data/input/JSONDataSpec.java b/processing/src/main/java/io/druid/data/input/JSONDataSpec.java deleted file mode 100644 index 9badb3e2527..00000000000 --- a/processing/src/main/java/io/druid/data/input/JSONDataSpec.java +++ /dev/null @@ -1,80 +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.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 io.druid.segment.incremental.SpatialDimensionSchema; - -import java.util.List; - -/** - */ -public class JSONDataSpec implements DataSpec -{ - private final List dimensions; - private final List spatialDimensions; - - @JsonCreator - public JSONDataSpec( - @JsonProperty("dimensions") List dimensions, - @JsonProperty("spatialDimensions") List spatialDimensions - ) - { - this.dimensions = dimensions; - this.spatialDimensions = (spatialDimensions == null) - ? Lists.newArrayList() - : spatialDimensions; - } - - @JsonProperty("dimensions") - @Override - public List getDimensions() - { - return dimensions; - } - - @JsonProperty("spatialDimensions") - @Override - public List getSpatialDimensions() - { - return spatialDimensions; - } - - @Override - public void verify(List usedCols) - { - } - - @Override - public boolean hasCustomDimensions() - { - return !(dimensions == null || dimensions.isEmpty()); - } - - @Override - public Parser getParser() - { - return new JSONParser(); - } -} diff --git a/processing/src/main/java/io/druid/data/input/MapInputRowParser.java b/processing/src/main/java/io/druid/data/input/MapInputRowParser.java deleted file mode 100644 index adb4cc534e5..00000000000 --- a/processing/src/main/java/io/druid/data/input/MapInputRowParser.java +++ /dev/null @@ -1,117 +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.data.input; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.metamx.common.exception.FormattedException; -import org.joda.time.DateTime; - -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class MapInputRowParser implements InputRowParser> -{ - private final TimestampSpec timestampSpec; - private List dimensions; - private final Set dimensionExclusions; - - @JsonCreator - public MapInputRowParser( - @JsonProperty("timestampSpec") TimestampSpec timestampSpec, - @JsonProperty("dimensions") List dimensions, - @JsonProperty("dimensionExclusions") List dimensionExclusions - ) - { - this.timestampSpec = timestampSpec; - if (dimensions != null) { - this.dimensions = ImmutableList.copyOf(dimensions); - } - this.dimensionExclusions = Sets.newHashSet(); - if (dimensionExclusions != null) { - for (String dimensionExclusion : dimensionExclusions) { - this.dimensionExclusions.add(dimensionExclusion.toLowerCase()); - } - } - this.dimensionExclusions.add(timestampSpec.getTimestampColumn().toLowerCase()); - } - - @Override - public InputRow parse(Map theMap) throws FormattedException - { - final List dimensions = hasCustomDimensions() - ? this.dimensions - : Lists.newArrayList(Sets.difference(theMap.keySet(), dimensionExclusions)); - - final DateTime timestamp; - try { - timestamp = timestampSpec.extractTimestamp(theMap); - if (timestamp == null) { - final String input = theMap.toString(); - throw new NullPointerException( - String.format( - "Null timestamp in input: %s", - input.length() < 100 ? input : input.substring(0, 100) + "..." - ) - ); - } - } - catch (Exception e) { - throw new FormattedException.Builder() - .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_TIMESTAMP) - .withMessage(e.toString()) - .build(); - } - - return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap); - } - - private boolean hasCustomDimensions() { - return dimensions != null; - } - - @Override - public void addDimensionExclusion(String dimension) - { - dimensionExclusions.add(dimension); - } - - @JsonProperty - public TimestampSpec getTimestampSpec() - { - return timestampSpec; - } - - @JsonProperty - public List getDimensions() - { - return dimensions; - } - - @JsonProperty - public Set getDimensionExclusions() - { - return dimensionExclusions; - } -} diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index f1f8ea875f5..87c08fa9ba9 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -29,6 +29,8 @@ import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimestampSpec; import java.io.InputStream; import java.nio.ByteBuffer; diff --git a/processing/src/main/java/io/druid/data/input/StringInputRowParser.java b/processing/src/main/java/io/druid/data/input/StringInputRowParser.java deleted file mode 100644 index f2d9b10df07..00000000000 --- a/processing/src/main/java/io/druid/data/input/StringInputRowParser.java +++ /dev/null @@ -1,137 +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.data.input; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Charsets; -import com.google.common.collect.ImmutableList; -import com.metamx.common.exception.FormattedException; -import com.metamx.common.parsers.Parser; -import com.metamx.common.parsers.ToLowerCaseParser; - -import java.nio.ByteBuffer; -import java.nio.CharBuffer; -import java.nio.charset.CoderResult; -import java.nio.charset.CodingErrorAction; -import java.util.List; -import java.util.Map; - -/** - */ -public class StringInputRowParser implements ByteBufferInputRowParser -{ - private final MapInputRowParser inputRowCreator; - private final Parser parser; - private final DataSpec dataSpec; - - private CharBuffer chars = null; - - @JsonCreator - public StringInputRowParser( - @JsonProperty("timestampSpec") TimestampSpec timestampSpec, - @JsonProperty("data") DataSpec dataSpec, - @JsonProperty("dimensionExclusions") List dimensionExclusions) - { - this.dataSpec = dataSpec; - this.inputRowCreator = new MapInputRowParser(timestampSpec, dataSpec.getDimensions(), dimensionExclusions); - this.parser = new ToLowerCaseParser(dataSpec.getParser()); - } - - public void addDimensionExclusion(String dimension) - { - inputRowCreator.addDimensionExclusion(dimension); - } - - @Override - public InputRow parse(ByteBuffer input) throws FormattedException - { - return parseMap(buildStringKeyMap(input)); - } - - private Map buildStringKeyMap(ByteBuffer input) - { - int payloadSize = input.remaining(); - - if (chars == null || chars.remaining() < payloadSize) - { - chars = CharBuffer.allocate(payloadSize); - } - - final CoderResult coderResult = Charsets.UTF_8.newDecoder() - .onMalformedInput(CodingErrorAction.REPLACE) - .onUnmappableCharacter(CodingErrorAction.REPLACE) - .decode(input, chars, true); - - Map theMap; - if (coderResult.isUnderflow()) - { - chars.flip(); - try - { - theMap = parseString(chars.toString()); - } finally - { - chars.clear(); - } - } - else - { - throw new FormattedException.Builder() - .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_ROW) - .withMessage(String.format("Failed with CoderResult[%s]", coderResult)) - .build(); - } - return theMap; - } - - private Map parseString(String inputString) - { - return parser.parse(inputString); - } - - public InputRow parse(String input) throws FormattedException - { - return parseMap(parseString(input)); - } - - private InputRow parseMap(Map theMap) - { - return inputRowCreator.parse(theMap); - } - - @JsonProperty - public TimestampSpec getTimestampSpec() - { - return inputRowCreator.getTimestampSpec(); - } - - @JsonProperty("data") - public DataSpec getDataSpec() - { - return dataSpec; - } - - @JsonProperty - public List getDimensionExclusions() - { - return ImmutableList.copyOf(inputRowCreator.getDimensionExclusions()); - } -} diff --git a/processing/src/main/java/io/druid/data/input/TimestampSpec.java b/processing/src/main/java/io/druid/data/input/TimestampSpec.java deleted file mode 100644 index ef10d726c4e..00000000000 --- a/processing/src/main/java/io/druid/data/input/TimestampSpec.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.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; - -/** - */ -public class TimestampSpec -{ - private static final String defaultFormat = "auto"; - - private final String timestampColumn; - private final String timestampFormat; - private final Function timestampConverter; - - @JsonCreator - public TimestampSpec( - @JsonProperty("column") String timestampColumn, - @JsonProperty("format") String format - ) - { - this.timestampColumn = timestampColumn; - this.timestampFormat = format == null ? defaultFormat : format; - this.timestampConverter = ParserUtils.createTimestampParser(timestampFormat); - } - - @JsonProperty("column") - public String getTimestampColumn() - { - return timestampColumn; - } - - @JsonProperty("format") - public String getTimestampFormat() - { - return timestampFormat; - } - - public DateTime extractTimestamp(Map input) - { - final Object o = input.get(timestampColumn); - - return o == null ? null : timestampConverter.apply(o.toString()); - } -} diff --git a/processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java b/processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java deleted file mode 100644 index cba57080af6..00000000000 --- a/processing/src/main/java/io/druid/data/input/ToLowercaseDataSpec.java +++ /dev/null @@ -1,77 +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.data.input; - -import com.fasterxml.jackson.annotation.JsonValue; -import com.metamx.common.parsers.Parser; -import com.metamx.common.parsers.ToLowerCaseParser; -import io.druid.segment.incremental.SpatialDimensionSchema; - -import java.util.List; - -/** - */ -public class ToLowercaseDataSpec implements DataSpec -{ - private final DataSpec delegate; - - public ToLowercaseDataSpec( - DataSpec delegate - ) - { - this.delegate = delegate; - } - - @Override - public void verify(List usedCols) - { - delegate.verify(usedCols); - } - - @Override - public boolean hasCustomDimensions() - { - return delegate.hasCustomDimensions(); - } - - @Override - public List getDimensions() - { - return delegate.getDimensions(); - } - - @Override - public List getSpatialDimensions() - { - return delegate.getSpatialDimensions(); - } - - @Override - public Parser getParser() - { - return new ToLowerCaseParser(delegate.getParser()); - } - - @JsonValue - public DataSpec getDelegate() - { - return delegate; - } -} diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 990893c4b13..403789190ea 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -125,7 +125,7 @@ public class IndexIO public static QueryableIndex loadIndex(File inDir) throws IOException { init(); - final int version = getVersionFromDir(inDir); + final int version = SegmentUtils.getVersionFromDir(inDir); final IndexLoader loader = indexLoaders.get(version); @@ -187,7 +187,7 @@ public class IndexIO public static boolean convertSegment(File toConvert, File converted) throws IOException { - final int version = getVersionFromDir(toConvert); + final int version = SegmentUtils.getVersionFromDir(toConvert); switch (version) { case 1: diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 77d99e44cbc..dfdd83dab76 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -36,6 +36,7 @@ import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index 4ebea50e24d..b72530ba47a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -23,6 +23,7 @@ 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 io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.query.aggregation.Aggregator; import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java index 5bd0250c90f..369e504f31e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java @@ -22,6 +22,7 @@ package io.druid.segment.incremental; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java index d5d8f4662de..01c1ba505fc 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java @@ -30,6 +30,7 @@ import com.google.common.collect.Sets; import com.google.common.primitives.Floats; import com.metamx.common.ISE; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.SpatialDimensionSchema; import java.util.Arrays; import java.util.List; diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java deleted file mode 100644 index 6ef1e788c6c..00000000000 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionSchema.java +++ /dev/null @@ -1,67 +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.incremental; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Function; -import com.google.common.collect.Lists; - -import java.util.List; - -/** - */ -public class SpatialDimensionSchema -{ - private final String dimName; - private final List dims; - - @JsonCreator - public SpatialDimensionSchema( - @JsonProperty("dimName") String dimName, - @JsonProperty("dims") List dims - ) - { - this.dimName = dimName.toLowerCase(); - this.dims = Lists.transform( - dims, - new Function() - { - @Override - public String apply(String input) - { - return input.toLowerCase(); - } - } - ); - } - - @JsonProperty - public String getDimName() - { - return dimName; - } - - @JsonProperty - public List getDims() - { - return dims; - } -} diff --git a/processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java b/processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java deleted file mode 100644 index a57962a6f37..00000000000 --- a/processing/src/test/java/io/druid/data/input/InputRowParserSerdeTest.java +++ /dev/null @@ -1,88 +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.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 io.druid.jackson.DefaultObjectMapper; -import io.druid.segment.incremental.SpatialDimensionSchema; -import junit.framework.Assert; -import org.joda.time.DateTime; -import org.junit.Test; - -import java.nio.ByteBuffer; - -public class InputRowParserSerdeTest -{ - private final ObjectMapper jsonMapper = new DefaultObjectMapper(); - - @Test - public void testStringInputRowParserSerde() throws Exception - { - final StringInputRowParser parser = new StringInputRowParser( - new TimestampSpec("timestamp", "iso"), - new JSONDataSpec( - ImmutableList.of("foo", "bar"), ImmutableList.of() - ), - ImmutableList.of("baz") - ); - final ByteBufferInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsBytes(parser), - ByteBufferInputRowParser.class - ); - final InputRow parsed = parser2.parse( - ByteBuffer.wrap( - "{\"foo\":\"x\",\"bar\":\"y\",\"qux\":\"z\",\"timestamp\":\"2000\"}".getBytes(Charsets.UTF_8) - ) - ); - Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions()); - Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo")); - Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar")); - Assert.assertEquals(new DateTime("2000").getMillis(), parsed.getTimestampFromEpoch()); - } - - @Test - public void testMapInputRowParserSerde() throws Exception - { - final MapInputRowParser parser = new MapInputRowParser( - new TimestampSpec("timestamp", "iso"), - ImmutableList.of("foo", "bar"), - ImmutableList.of("baz") - ); - final MapInputRowParser parser2 = jsonMapper.readValue( - jsonMapper.writeValueAsBytes(parser), - MapInputRowParser.class - ); - final InputRow parsed = parser2.parse( - ImmutableMap.of( - "foo", "x", - "bar", "y", - "qux", "z", - "timestamp", "2000" - ) - ); - Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions()); - Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo")); - Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar")); - Assert.assertEquals(new DateTime("2000").getMillis(), parsed.getTimestampFromEpoch()); - } -} diff --git a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java index aa98a3bccb9..c7013f85f3a 100644 --- a/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java +++ b/processing/src/test/java/io/druid/data/input/ProtoBufInputRowParserTest.java @@ -19,6 +19,7 @@ package io.druid.data.input; +import io.druid.data.input.impl.TimestampSpec; import org.joda.time.DateTime; import org.junit.Test; diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 8b13c6ce9e8..0fd53185727 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -25,9 +25,9 @@ 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 io.druid.data.input.DelimitedDataSpec; -import io.druid.data.input.StringInputRowParser; -import io.druid.data.input.TimestampSpec; +import io.druid.data.input.impl.DelimitedDataSpec; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index 8ab77be16a6..8e423dfcb47 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; @@ -46,7 +47,6 @@ import io.druid.segment.Segment; import io.druid.segment.TestHelper; 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/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 24ed5b9e149..b59d4a3d59f 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.metamx.collections.spatial.search.RadiusBound; import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; @@ -46,7 +47,6 @@ import io.druid.segment.Segment; import io.druid.segment.TestHelper; 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/realtime/pom.xml b/realtime/pom.xml deleted file mode 100644 index 9d8e461c631..00000000000 --- a/realtime/pom.xml +++ /dev/null @@ -1,175 +0,0 @@ - - - - - 4.0.0 - com.metamx.druid - druid-realtime - druid-realtime - druid-realtime - - - com.metamx - druid - 0.6.0-SNAPSHOT - - - - - com.metamx.druid - druid-server - ${project.parent.version} - - - - com.metamx - emitter - - - com.metamx - http-client - - - com.metamx - java-util - - - com.metamx - server-metrics - - - - org.apache.directory.studio - org.apache.commons.collections - 3.2.1 - - - commons-io - commons-io - - - org.skife.config - config-magic - - - com.google.guava - guava - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.dataformat - jackson-dataformat-smile - - - org.jdbi - jdbi - - - net.java.dev.jets3t - jets3t - - - org.eclipse.jetty - jetty-server - - - joda-time - joda-time - - - com.google.code.findbugs - jsr305 - - - kafka - core-kafka - 0.7.2-mmx1 - - - log4j - log4j - - - - - com.rabbitmq - amqp-client - 3.1.1 - - - - - commons-codec - commons-codec - true - - - commons-logging - commons-logging - true - - - commons-httpclient - commons-httpclient - true - - - com.jamesmurty.utils - java-xmlbuilder - true - - - com.ircclouds.irc - irc-api - - - com.maxmind.geoip2 - geoip2 - - - - - - junit - junit - test - - - org.easymock - easymock - test - - - com.metamx.druid - druid-server - ${project.parent.version} - test-jar - test - - - - diff --git a/realtime/src/main/java/io/druid/segment/realtime/S3SegmentPusher.java b/realtime/src/main/java/io/druid/segment/realtime/S3SegmentPusher.java deleted file mode 100644 index ce11848c462..00000000000 --- a/realtime/src/main/java/io/druid/segment/realtime/S3SegmentPusher.java +++ /dev/null @@ -1,44 +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.realtime; - -import com.fasterxml.jackson.databind.ObjectMapper; -import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.S3DataSegmentPusher; -import io.druid.segment.loading.S3DataSegmentPusherConfig; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; - -/** - * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible - * - * Exists in 0.2, can be removed from 0.3 on - */ -@Deprecated -public class S3SegmentPusher extends S3DataSegmentPusher implements DataSegmentPusher -{ - public S3SegmentPusher( - RestS3Service s3Client, - S3DataSegmentPusherConfig config, - ObjectMapper jsonMapper - ) - { - super(s3Client, config, jsonMapper); - } -} diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/FileIteratingFirehose.java b/realtime/src/main/java/io/druid/segment/realtime/firehose/FileIteratingFirehose.java deleted file mode 100644 index a8a6be14002..00000000000 --- a/realtime/src/main/java/io/druid/segment/realtime/firehose/FileIteratingFirehose.java +++ /dev/null @@ -1,96 +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.realtime.firehose; - -import com.google.common.base.Throwables; -import io.druid.common.guava.Runnables; -import io.druid.data.input.Firehose; -import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; -import org.apache.commons.io.LineIterator; - -import java.io.IOException; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.Queue; - -/** - */ -public class FileIteratingFirehose implements Firehose -{ - private final Iterator lineIterators; - private final StringInputRowParser parser; - - private LineIterator lineIterator = null; - - public FileIteratingFirehose( - Iterator lineIterators, - StringInputRowParser parser - ) - { - this.lineIterators = lineIterators; - this.parser = parser; - } - - @Override - public boolean hasMore() - { - try { - return lineIterators.hasNext() || (lineIterator != null && lineIterator.hasNext()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - @Override - public InputRow nextRow() - { - try { - if (lineIterator == null || !lineIterator.hasNext()) { - // Close old streams, maybe. - if (lineIterator != null) { - lineIterator.close(); - } - - lineIterator = lineIterators.next(); - } - - return parser.parse(lineIterator.next()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - - @Override - public Runnable commit() - { - return Runnables.getNoopRunnable(); - } - - @Override - public void close() throws IOException - { - if (lineIterator != null) { - lineIterator.close(); - } - } -} diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml new file mode 100644 index 00000000000..4450d2a33bb --- /dev/null +++ b/s3-extensions/pom.xml @@ -0,0 +1,60 @@ + + + + + 4.0.0 + com.metamx.druid + druid-s3-extensions + druid-s3-extensions + druid-s3-extensions + + + com.metamx + druid + 0.6.0-SNAPSHOT + + + + + io.druid + druid-api + + + net.java.dev.jets3t + jets3t + 0.8.1 + + + com.metamx + emitter + + + commons-io + commons-io + + + + + junit + junit + test + + + diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/StaticS3FirehoseFactory.java b/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java similarity index 91% rename from indexing-service/src/main/java/io/druid/indexing/common/index/StaticS3FirehoseFactory.java rename to s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java index 69bdfa8d5ab..e4d6741f414 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/StaticS3FirehoseFactory.java +++ b/s3-extensions/src/main/java/io/druid/firehose/s3/StaticS3FirehoseFactory.java @@ -17,25 +17,25 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.common.index; +package io.druid.firehose.s3; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.metamx.common.logger.Logger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.StringInputRowParser; -import io.druid.segment.realtime.firehose.FileIteratingFirehose; +import io.druid.data.input.impl.FileIteratingFirehose; +import io.druid.data.input.impl.StringInputRowParser; import org.apache.commons.io.IOUtils; import org.apache.commons.io.LineIterator; import org.jets3t.service.S3Service; +import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import java.io.BufferedReader; @@ -51,7 +51,6 @@ import java.util.zip.GZIPInputStream; /** * Builds firehoses that read from a predefined list of S3 objects and then dry up. */ -@JsonTypeName("s3") public class StaticS3FirehoseFactory implements FirehoseFactory { private static final Logger log = new Logger(StaticS3FirehoseFactory.class); @@ -119,8 +118,9 @@ public class StaticS3FirehoseFactory implements FirehoseFactory log.info("Reading from bucket[%s] object[%s] (%s)", s3Bucket, s3Object.getKey(), nextURI); try { - final InputStream innerInputStream = s3Client.getObject(s3Bucket, s3Object.getKey()) - .getDataInputStream(); + final InputStream innerInputStream = s3Client.getObject( + new S3Bucket(s3Bucket), s3Object.getKey()) + .getDataInputStream(); final InputStream outerInputStream = s3Object.getKey().endsWith(".gz") ? new GZIPInputStream(innerInputStream) diff --git a/server/src/main/java/io/druid/segment/loading/AWSCredentialsConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java similarity index 97% rename from server/src/main/java/io/druid/segment/loading/AWSCredentialsConfig.java rename to s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java index b9a032fd4ea..e19289ec8ee 100644 --- a/server/src/main/java/io/druid/segment/loading/AWSCredentialsConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/segment/loading/S3DataSegmentKiller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java similarity index 94% rename from server/src/main/java/io/druid/segment/loading/S3DataSegmentKiller.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java index 314822679ae..b9a44f631c6 100644 --- a/server/src/main/java/io/druid/segment/loading/S3DataSegmentKiller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentKiller.java @@ -17,11 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.s3; import com.google.inject.Inject; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; +import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPuller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java similarity index 88% rename from server/src/main/java/io/druid/segment/loading/S3DataSegmentPuller.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index abf4b3502cf..12312cb0e52 100644 --- a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPuller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.s3; import com.google.common.base.Throwables; import com.google.common.io.ByteStreams; @@ -27,14 +27,15 @@ import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import io.druid.common.utils.CompressionUtils; -import io.druid.storage.s3.S3Utils; +import io.druid.segment.loading.DataSegmentPuller; +import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import org.apache.commons.io.FileUtils; -import org.jets3t.service.ServiceException; +import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; -import org.jets3t.service.model.StorageObject; import java.io.File; import java.io.IOException; @@ -92,7 +93,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller S3Object s3Obj = null; try { - s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path); + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); InputStream in = null; try { @@ -154,7 +155,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller @Override public Boolean call() throws Exception { - return s3Client.isObjectInBucket(coords.bucket, coords.path); + return S3Utils.isObjectInBucket(s3Client, coords.bucket, coords.path); } } ); @@ -165,7 +166,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller catch (IOException e) { throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); } - catch (ServiceException e) { + catch (S3ServiceException e) { throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); } } @@ -175,13 +176,13 @@ public class S3DataSegmentPuller implements DataSegmentPuller { final S3Coords coords = new S3Coords(segment); try { - final StorageObject objDetails = S3Utils.retryS3Operation( - new Callable() + final S3Object objDetails = S3Utils.retryS3Operation( + new Callable() { @Override - public StorageObject call() throws Exception + public S3Object call() throws Exception { - return s3Client.getObjectDetails(coords.bucket, coords.path); + return s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); } } ); @@ -190,10 +191,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller catch (InterruptedException e) { throw Throwables.propagate(e); } - catch (IOException e) { - throw new SegmentLoadingException(e, e.getMessage()); - } - catch (ServiceException e) { + catch (S3ServiceException | IOException e) { throw new SegmentLoadingException(e, e.getMessage()); } } diff --git a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPusher.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java similarity index 90% rename from server/src/main/java/io/druid/segment/loading/S3DataSegmentPusher.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java index a1469446f31..735a9365628 100644 --- a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPusher.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusher.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.s3; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; @@ -27,11 +27,13 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; -import io.druid.common.utils.CompressionUtils; -import io.druid.segment.IndexIO; -import io.druid.storage.s3.S3Utils; +import io.druid.segment.SegmentUtils; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.loading.DataSegmentPusherUtil; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import org.jets3t.service.ServiceException; +import org.jets3t.service.acl.AccessControlList; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; @@ -61,9 +63,10 @@ public class S3DataSegmentPusher implements DataSegmentPusher this.jsonMapper = jsonMapper; } - public S3DataSegmentPusherConfig getConfig() + @Override + public String getPathForHadoop(String dataSource) { - return config; + return String.format("s3n://%s/%s/%s", config.getBucket(), config.getBaseKey(), dataSource); } @Override @@ -90,7 +93,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher toPush.setBucketName(outputBucket); toPush.setKey(outputKey + "/index.zip"); if (!config.getDisableAcl()) { - toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + toPush.setAcl(AccessControlList.REST_CANNED_AUTHENTICATED_READ); } log.info("Pushing %s.", toPush); @@ -107,7 +110,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher toPush.getKey() ) ) - .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); + .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(inSegment)), descriptorFile); diff --git a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPusherConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java similarity index 97% rename from server/src/main/java/io/druid/segment/loading/S3DataSegmentPusherConfig.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java index b74d22597d7..d57972f7ae3 100644 --- a/server/src/main/java/io/druid/segment/loading/S3DataSegmentPusherConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPusherConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.segment.loading; +package io.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java new file mode 100644 index 00000000000..34c23f850c9 --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.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 io.druid.storage.s3; + +import com.fasterxml.jackson.databind.Module; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.ProvisionException; +import io.druid.guice.Binders; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.initialization.DruidModule; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.util.List; + +/** + */ +public class S3StorageDruidModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of(); + } + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.s3", AWSCredentialsConfig.class); + + Binders.dataSegmentPullerBinder(binder).addBinding("s3_zip").to(S3DataSegmentPuller.class).in(LazySingleton.class); + Binders.dataSegmentKillerBinder(binder).addBinding("s3_zip").to(S3DataSegmentKiller.class).in(LazySingleton.class); + Binders.dataSegmentPusherBinder(binder).addBinding("s3").to(S3DataSegmentPusher.class).in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentPusherConfig.class); + + Binders.taskLogsBinder(binder).addBinding("s3").to(S3TaskLogs.class); + JsonConfigProvider.bind(binder, "druid.indexer.logs", S3TaskLogsConfig.class); + binder.bind(S3TaskLogs.class).in(LazySingleton.class); + } + + @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(credentials); + } + catch (S3ServiceException e) { + throw new ProvisionException("Unable to create a RestS3Service", e); + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogs.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java similarity index 98% rename from indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogs.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java index b6bf910d485..572959a05ed 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogs.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogs.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.common.tasklogs; +package io.druid.storage.s3; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.io.InputSupplier; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import io.druid.storage.s3.S3Utils; +import io.druid.tasklogs.TaskLogs; import org.jets3t.service.ServiceException; import org.jets3t.service.StorageService; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogsConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogsConfig.java similarity index 96% rename from indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogsConfig.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogsConfig.java index 6d4cb1fdf60..fb830d3b38d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/tasklogs/S3TaskLogsConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3TaskLogsConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.indexing.common.tasklogs; +package io.druid.storage.s3; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/server/src/main/java/io/druid/storage/s3/S3Utils.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java similarity index 70% rename from server/src/main/java/io/druid/storage/s3/S3Utils.java rename to s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java index 044b3fa76c9..0dfa685d3bd 100644 --- a/server/src/main/java/io/druid/storage/s3/S3Utils.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3Utils.java @@ -21,7 +21,9 @@ package io.druid.storage.s3; import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; -import org.jets3t.service.ServiceException; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import java.io.IOException; @@ -53,7 +55,7 @@ public class S3Utils * Retries S3 operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not * found, etc) are not retried. */ - public static T retryS3Operation(Callable f) throws IOException, ServiceException, InterruptedException + public static T retryS3Operation(Callable f) throws IOException, S3ServiceException, InterruptedException { int nTry = 0; final int maxTries = 3; @@ -69,10 +71,10 @@ public class S3Utils throw e; } } - catch (ServiceException e) { + catch (S3ServiceException e) { if (nTry <= maxTries && (e.getCause() instanceof IOException || - (e.getErrorCode() != null && e.getErrorCode().equals("RequestTimeout")))) { + (e.getS3ErrorCode() != null && e.getS3ErrorCode().equals("RequestTimeout")))) { awaitNextRetry(e, nTry); } else { throw e; @@ -96,4 +98,29 @@ public class S3Utils log.info(e, "S3 fail on try %d, retrying in %,dms.", nTry, sleepMillis); Thread.sleep(sleepMillis); } + + public static boolean isObjectInBucket(RestS3Service s3Client, String bucketName, String objectKey) + throws S3ServiceException + { + try { + s3Client.getObjectDetails(new S3Bucket(bucketName), objectKey); + } + catch (S3ServiceException e) { + if (404 == e.getResponseCode() + || "NoSuchKey".equals(e.getS3ErrorCode()) + || "NoSuchBucket".equals(e.getS3ErrorCode())) + { + return false; + } + if ("AccessDenied".equals(e.getS3ErrorCode())) + { + // Object is inaccessible to current user, but does exist. + return true; + } + // Something else has gone wrong + throw e; + } + return true; + } + } diff --git a/s3-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/s3-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 00000000000..47bd0408e62 --- /dev/null +++ b/s3-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.storage.cassandra.S3StorageDruidModule \ No newline at end of file diff --git a/server/pom.xml b/server/pom.xml index 1259afa949a..8a9f0d3e40d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -136,10 +136,6 @@ com.sun.jersey.contribs jersey-guice - - net.java.dev.jets3t - jets3t - org.eclipse.jetty jetty-server @@ -164,10 +160,6 @@ org.slf4j slf4j-log4j12 - - org.apache.hadoop - hadoop-core - io.tesla.aether tesla-aether @@ -176,29 +168,6 @@ org.eclipse.aether aether-api - - - - - commons-codec - commons-codec - true - - - commons-logging - commons-logging - true - - - commons-httpclient - commons-httpclient - true - - - com.jamesmurty.utils - java-xmlbuilder - true - org.antlr antlr4-runtime @@ -208,8 +177,8 @@ spymemcached - net.jpountz.lz4 - lz4 + net.jpountz.lz4 + lz4 org.eclipse.jetty @@ -223,6 +192,22 @@ org.eclipse.jetty jetty-servlets + + kafka + core-kafka + + + com.rabbitmq + amqp-client + + + com.ircclouds.irc + irc-api + + + com.maxmind.geoip2 + geoip2 + diff --git a/server/src/main/java/io/druid/guice/AWSModule.java b/server/src/main/java/io/druid/guice/AWSModule.java index 9a9b6d3b0f0..76ea8bbe142 100644 --- a/server/src/main/java/io/druid/guice/AWSModule.java +++ b/server/src/main/java/io/druid/guice/AWSModule.java @@ -23,13 +23,10 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.services.ec2.AmazonEC2; import com.amazonaws.services.ec2.AmazonEC2Client; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; -import com.google.inject.ProvisionException; -import io.druid.segment.loading.AWSCredentialsConfig; -import org.jets3t.service.S3ServiceException; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** */ @@ -48,29 +45,30 @@ public class AWSModule implements Module 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(credentials); - } - catch (S3ServiceException e) { - throw new ProvisionException("Unable to create a RestS3Service", e); - } - } - @Provides @LazySingleton public AmazonEC2 getEc2Client(AWSCredentials credentials) { return new AmazonEC2Client(credentials); } + + public static class AWSCredentialsConfig + { + @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/io/druid/guice/DataSegmentPusherPullerModule.java b/server/src/main/java/io/druid/guice/DataSegmentPusherPullerModule.java index b68b63ad431..af45ea18f17 100644 --- a/server/src/main/java/io/druid/guice/DataSegmentPusherPullerModule.java +++ b/server/src/main/java/io/druid/guice/DataSegmentPusherPullerModule.java @@ -23,18 +23,11 @@ import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Module; import io.druid.segment.loading.DataSegmentPusher; -import io.druid.segment.loading.HdfsDataSegmentPuller; -import io.druid.segment.loading.HdfsDataSegmentPusher; -import io.druid.segment.loading.HdfsDataSegmentPusherConfig; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.LocalDataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusherConfig; import io.druid.segment.loading.OmniSegmentLoader; -import io.druid.segment.loading.S3DataSegmentPuller; -import io.druid.segment.loading.S3DataSegmentPusher; -import io.druid.segment.loading.S3DataSegmentPusherConfig; import io.druid.segment.loading.SegmentLoader; -import org.apache.hadoop.conf.Configuration; /** */ @@ -46,8 +39,6 @@ public class DataSegmentPusherPullerModule implements Module binder.bind(SegmentLoader.class).to(OmniSegmentLoader.class).in(LazySingleton.class); bindDeepStorageLocal(binder); - bindDeepStorageS3(binder); - bindDeepStorageHdfs(binder); PolyBind.createChoice( binder, "druid.storage.type", Key.get(DataSegmentPusher.class), Key.get(LocalDataSegmentPusher.class) @@ -56,7 +47,7 @@ public class DataSegmentPusherPullerModule implements Module private static void bindDeepStorageLocal(Binder binder) { - DruidBinders.dataSegmentPullerBinder(binder) + Binders.dataSegmentPullerBinder(binder) .addBinding("local") .to(LocalDataSegmentPuller.class) .in(LazySingleton.class); @@ -67,34 +58,4 @@ public class DataSegmentPusherPullerModule implements Module .in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.storage", LocalDataSegmentPusherConfig.class); } - - private static void bindDeepStorageS3(Binder binder) - { - DruidBinders.dataSegmentPullerBinder(binder) - .addBinding("s3_zip") - .to(S3DataSegmentPuller.class) - .in(LazySingleton.class); - - PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class)) - .addBinding("s3") - .to(S3DataSegmentPusher.class) - .in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.storage", S3DataSegmentPusherConfig.class); - } - - private static void bindDeepStorageHdfs(Binder binder) - { - DruidBinders.dataSegmentPullerBinder(binder) - .addBinding("hdfs") - .to(HdfsDataSegmentPuller.class) - .in(LazySingleton.class); - - binder.bind(Configuration.class).toInstance(new Configuration()); - - PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class)) - .addBinding("hdfs") - .to(HdfsDataSegmentPusher.class) - .in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.storage", HdfsDataSegmentPusherConfig.class); - } } diff --git a/realtime/src/main/java/io/druid/guice/DbSegmentPublisherProvider.java b/server/src/main/java/io/druid/guice/DbSegmentPublisherProvider.java similarity index 100% rename from realtime/src/main/java/io/druid/guice/DbSegmentPublisherProvider.java rename to server/src/main/java/io/druid/guice/DbSegmentPublisherProvider.java diff --git a/server/src/main/java/io/druid/guice/DruidBinders.java b/server/src/main/java/io/druid/guice/DruidBinders.java index 1ebf72488ee..6ab21acaf65 100644 --- a/server/src/main/java/io/druid/guice/DruidBinders.java +++ b/server/src/main/java/io/druid/guice/DruidBinders.java @@ -27,7 +27,6 @@ import com.metamx.metrics.Monitor; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; -import io.druid.segment.loading.DataSegmentPuller; import io.druid.server.DruidNode; /** @@ -48,11 +47,6 @@ public class DruidBinders ); } - public static MapBinder dataSegmentPullerBinder(Binder binder) - { - return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class); - } - public static Multibinder> discoveryAnnouncementBinder(Binder binder) { return Multibinder.newSetBinder(binder, new TypeLiteral>(){}); diff --git a/realtime/src/main/java/io/druid/guice/FireDepartmentsProvider.java b/server/src/main/java/io/druid/guice/FireDepartmentsProvider.java similarity index 100% rename from realtime/src/main/java/io/druid/guice/FireDepartmentsProvider.java rename to server/src/main/java/io/druid/guice/FireDepartmentsProvider.java diff --git a/services/src/main/java/io/druid/guice/FirehoseModule.java b/server/src/main/java/io/druid/guice/FirehoseModule.java similarity index 74% rename from services/src/main/java/io/druid/guice/FirehoseModule.java rename to server/src/main/java/io/druid/guice/FirehoseModule.java index 8312149f1c1..acaa370a18d 100644 --- a/services/src/main/java/io/druid/guice/FirehoseModule.java +++ b/server/src/main/java/io/druid/guice/FirehoseModule.java @@ -23,17 +23,12 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; -import druid.examples.flights.FlightsFirehoseFactory; -import druid.examples.rand.RandomFirehoseFactory; -import druid.examples.twitter.TwitterSpritzerFirehoseFactory; -import druid.examples.web.WebFirehoseFactory; -import io.druid.indexing.common.index.EventReceiverFirehoseFactory; -import io.druid.segment.realtime.firehose.LocalFirehoseFactory; -import io.druid.indexing.common.index.StaticS3FirehoseFactory; +import io.druid.data.input.ProtoBufInputRowParser; import io.druid.initialization.DruidModule; import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; import io.druid.segment.realtime.firehose.IrcFirehoseFactory; import io.druid.segment.realtime.firehose.KafkaFirehoseFactory; +import io.druid.segment.realtime.firehose.LocalFirehoseFactory; import io.druid.segment.realtime.firehose.RabbitMQFirehoseFactory; import io.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; @@ -55,19 +50,15 @@ public class FirehoseModule implements DruidModule return Arrays.asList( new SimpleModule("FirehoseModule") .registerSubtypes( - new NamedType(TwitterSpritzerFirehoseFactory.class, "twitzer"), - new NamedType(FlightsFirehoseFactory.class, "flights"), - new NamedType(RandomFirehoseFactory.class, "rand"), - new NamedType(WebFirehoseFactory.class, "webstream"), new NamedType(KafkaFirehoseFactory.class, "kafka-0.7.2"), new NamedType(RabbitMQFirehoseFactory.class, "rabbitmq"), new NamedType(ClippedFirehoseFactory.class, "clipped"), new NamedType(TimedShutoffFirehoseFactory.class, "timed"), new NamedType(IrcFirehoseFactory.class, "irc"), - new NamedType(StaticS3FirehoseFactory.class, "s3"), - new NamedType(EventReceiverFirehoseFactory.class, "receiver"), - new NamedType(LocalFirehoseFactory.class, "local") + new NamedType(LocalFirehoseFactory.class, "local"), + new NamedType(ProtoBufInputRowParser.class, "protobuf") ) ); } + } diff --git a/realtime/src/main/java/io/druid/guice/NoopSegmentPublisherProvider.java b/server/src/main/java/io/druid/guice/NoopSegmentPublisherProvider.java similarity index 100% rename from realtime/src/main/java/io/druid/guice/NoopSegmentPublisherProvider.java rename to server/src/main/java/io/druid/guice/NoopSegmentPublisherProvider.java diff --git a/realtime/src/main/java/io/druid/guice/RealtimeManagerConfig.java b/server/src/main/java/io/druid/guice/RealtimeManagerConfig.java similarity index 100% rename from realtime/src/main/java/io/druid/guice/RealtimeManagerConfig.java rename to server/src/main/java/io/druid/guice/RealtimeManagerConfig.java diff --git a/realtime/src/main/java/io/druid/guice/SegmentPublisherProvider.java b/server/src/main/java/io/druid/guice/SegmentPublisherProvider.java similarity index 100% rename from realtime/src/main/java/io/druid/guice/SegmentPublisherProvider.java rename to server/src/main/java/io/druid/guice/SegmentPublisherProvider.java diff --git a/indexing-service/src/main/java/io/druid/guice/TaskLogsModule.java b/server/src/main/java/io/druid/guice/TaskLogsModule.java similarity index 71% rename from indexing-service/src/main/java/io/druid/guice/TaskLogsModule.java rename to server/src/main/java/io/druid/guice/TaskLogsModule.java index b7e69830dcd..eedd12caabc 100644 --- a/indexing-service/src/main/java/io/druid/guice/TaskLogsModule.java +++ b/server/src/main/java/io/druid/guice/TaskLogsModule.java @@ -23,11 +23,9 @@ import com.google.inject.Binder; import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.multibindings.MapBinder; -import io.druid.indexing.common.tasklogs.NoopTaskLogs; -import io.druid.indexing.common.tasklogs.S3TaskLogs; -import io.druid.indexing.common.tasklogs.S3TaskLogsConfig; -import io.druid.indexing.common.tasklogs.TaskLogPusher; -import io.druid.indexing.common.tasklogs.TaskLogs; +import io.druid.tasklogs.NoopTaskLogs; +import io.druid.tasklogs.TaskLogPusher; +import io.druid.tasklogs.TaskLogs; /** */ @@ -37,12 +35,8 @@ public class TaskLogsModule implements Module 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); + final MapBinder taskLogBinder = Binders.taskLogsBinder(binder); taskLogBinder.addBinding("noop").to(NoopTaskLogs.class).in(LazySingleton.class); binder.bind(NoopTaskLogs.class).in(LazySingleton.class); diff --git a/services/src/main/java/io/druid/cli/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java similarity index 71% rename from services/src/main/java/io/druid/cli/Initialization.java rename to server/src/main/java/io/druid/initialization/Initialization.java index a3d687bb70c..5a052b589fe 100644 --- a/services/src/main/java/io/druid/cli/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.cli; +package io.druid.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; @@ -38,12 +38,14 @@ import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; import io.druid.guice.DataSegmentPusherPullerModule; import io.druid.guice.DbConnectorModule; +import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidProcessingModule; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.FirehoseModule; import io.druid.guice.HttpClientModule; import io.druid.guice.IndexingServiceDiscoveryModule; import io.druid.guice.JacksonConfigManagerModule; +import io.druid.guice.JsonConfigProvider; import io.druid.guice.LifecycleModule; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; @@ -54,10 +56,12 @@ import io.druid.guice.TaskLogsModule; import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; -import io.druid.initialization.DruidModule; +import io.druid.jackson.JacksonModule; +import io.druid.server.initialization.ConfigModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.ExtensionsConfig; import io.druid.server.initialization.JettyServerModule; +import io.druid.server.initialization.PropertiesModule; import io.druid.server.metrics.MetricsModule; import io.tesla.aether.TeslaAether; import io.tesla.aether.internal.DefaultTeslaAether; @@ -68,10 +72,14 @@ 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.resolution.DependencyResolutionException; import org.eclipse.aether.util.artifact.JavaScopes; import org.eclipse.aether.util.filter.DependencyFilterUtils; +import java.io.IOException; +import java.io.OutputStream; import java.io.PrintStream; +import java.net.MalformedURLException; import java.net.URL; import java.net.URLClassLoader; import java.util.Collections; @@ -82,11 +90,10 @@ import java.util.Set; /** */ -public class - Initialization +public class Initialization { private static final Logger log = new Logger(Initialization.class); - private static final Map loadersMap = Maps.newHashMap(); + private static final Map loadersMap = Maps.newHashMap(); private static final Set exclusions = Sets.newHashSet( "io.druid", @@ -108,57 +115,7 @@ public class for (String coordinate : config.getCoordinates()) { log.info("Loading extension[%s]", coordinate); try { - 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); - } - } - - for (URL url : urls) { - log.info("Added URL[%s]", url); - } - - loader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader()); - loadersMap.put(coordinate, loader); - } + URLClassLoader loader = getClassLoaderForCoordinates(aether, coordinate); final ServiceLoader serviceLoader = ServiceLoader.load(clazz, loader); @@ -175,7 +132,64 @@ public class return retVal; } - private static DefaultTeslaAether getAetherClient(ExtensionsConfig config) + public static URLClassLoader getClassLoaderForCoordinates(TeslaAether aether, String coordinate) + throws DependencyResolutionException, MalformedURLException + { + URLClassLoader 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.debug("Skipped Artifact[%s]", artifact); + } + } + + for (URL url : urls) { + log.info("Added URL[%s]", url); + } + + loader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader()); + loadersMap.put(coordinate, loader); + } + return loader; + } + + public static DefaultTeslaAether getAetherClient(ExtensionsConfig config) { /* DefaultTeslaAether logs a bunch of stuff to System.out, which is annoying. We choose to disable that @@ -194,7 +208,28 @@ public class PrintStream oldOut = System.out; try { - System.setOut(new PrintStream(ByteStreams.nullOutputStream())); + System.setOut(new PrintStream( + new OutputStream() + { + @Override + public void write(int b) throws IOException + { + + } + + @Override + public void write(byte[] b) throws IOException + { + + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + + } + } + )); return new DefaultTeslaAether(config.getLocalRepository(), config.getRemoteRepositories()); } finally { @@ -244,6 +279,25 @@ public class return Guice.createInjector(Modules.override(defaultModules.getModules()).with(actualModules.getModules())); } + 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); + } + } + ); + } + private static class ModuleList { private final Injector baseInjector; diff --git a/server/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java b/server/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java deleted file mode 100644 index b64f26806ca..00000000000 --- a/server/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java +++ /dev/null @@ -1,63 +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.Joiner; -import io.druid.timeline.DataSegment; -import org.joda.time.format.ISODateTimeFormat; - -/** - */ -public class DataSegmentPusherUtil -{ - private static final Joiner JOINER = Joiner.on("/").skipNulls(); - - public static String getStorageDir(DataSegment segment) - { - return JOINER.join( - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() - ); - } - - /** - * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in - * path names. So we format paths differently for HDFS. - */ - public static String getHdfsStorageDir(DataSegment segment) - { - return JOINER.join( - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), - segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) - ), - segment.getVersion().replaceAll(":", "_"), - segment.getShardSpec().getPartitionNum() - ); - } -} diff --git a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPuller.java b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPuller.java index fd29d9192e9..831c77d79cd 100644 --- a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPuller.java +++ b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPuller.java @@ -22,8 +22,8 @@ package io.druid.segment.loading; import com.google.common.io.Files; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; -import io.druid.common.utils.CompressionUtils; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import java.io.File; import java.io.IOException; diff --git a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java index a252337499e..a28bfa61366 100644 --- a/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java +++ b/server/src/main/java/io/druid/segment/loading/LocalDataSegmentPusher.java @@ -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 io.druid.common.utils.CompressionUtils; -import io.druid.segment.IndexIO; +import io.druid.segment.SegmentUtils; import io.druid.timeline.DataSegment; +import io.druid.utils.CompressionUtils; import java.io.File; import java.io.IOException; @@ -51,6 +51,12 @@ public class LocalDataSegmentPusher implements DataSegmentPusher this.jsonMapper = jsonMapper; } + @Override + public String getPathForHadoop(String dataSource) + { + return String.format("file://%s/%s", config.getStorageDirectory(), dataSource); + } + @Override public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException { @@ -65,7 +71,7 @@ public class LocalDataSegmentPusher implements DataSegmentPusher return createDescriptorFile( segment.withLoadSpec(makeLoadSpec(outDir)) .withSize(size) - .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), outDir ); } @@ -78,7 +84,7 @@ public class LocalDataSegmentPusher implements DataSegmentPusher return createDescriptorFile( segment.withLoadSpec(makeLoadSpec(outFile)) .withSize(size) - .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + .withBinaryVersion(SegmentUtils.getVersionFromDir(dataSegmentFile)), outDir ); } diff --git a/server/src/main/java/io/druid/segment/loading/OmniDataSegmentKiller.java b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentKiller.java new file mode 100644 index 00000000000..1c87e6b2a15 --- /dev/null +++ b/server/src/main/java/io/druid/segment/loading/OmniDataSegmentKiller.java @@ -0,0 +1,60 @@ +/* + * 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.inject.Inject; +import com.metamx.common.MapUtils; +import io.druid.timeline.DataSegment; + +import java.util.Map; + +/** + */ +public class OmniDataSegmentKiller implements DataSegmentKiller +{ + private final Map killers; + + @Inject + public OmniDataSegmentKiller( + Map killers + ) + { + this.killers = killers; + } + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + getKiller(segment).kill(segment); + } + + private DataSegmentKiller getKiller(DataSegment segment) throws SegmentLoadingException + { + String type = MapUtils.getString(segment.getLoadSpec(), "type"); + DataSegmentKiller loader = killers.get(type); + + if (loader == null) { + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, killers.keySet()); + } + + return loader; + } + +} diff --git a/realtime/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java b/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java rename to server/src/main/java/io/druid/segment/realtime/DbSegmentPublisher.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/DbSegmentPublisherConfig.java b/server/src/main/java/io/druid/segment/realtime/DbSegmentPublisherConfig.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/DbSegmentPublisherConfig.java rename to server/src/main/java/io/druid/segment/realtime/DbSegmentPublisherConfig.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java b/server/src/main/java/io/druid/segment/realtime/FireDepartment.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/FireDepartment.java rename to server/src/main/java/io/druid/segment/realtime/FireDepartment.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/FireDepartmentConfig.java b/server/src/main/java/io/druid/segment/realtime/FireDepartmentConfig.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/FireDepartmentConfig.java rename to server/src/main/java/io/druid/segment/realtime/FireDepartmentConfig.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java b/server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java rename to server/src/main/java/io/druid/segment/realtime/FireDepartmentMetrics.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/FireHydrant.java b/server/src/main/java/io/druid/segment/realtime/FireHydrant.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/FireHydrant.java rename to server/src/main/java/io/druid/segment/realtime/FireHydrant.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/Indexer.java b/server/src/main/java/io/druid/segment/realtime/Indexer.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/Indexer.java rename to server/src/main/java/io/druid/segment/realtime/Indexer.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/NoopSegmentPublisher.java b/server/src/main/java/io/druid/segment/realtime/NoopSegmentPublisher.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/NoopSegmentPublisher.java rename to server/src/main/java/io/druid/segment/realtime/NoopSegmentPublisher.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/RealtimeCuratorDataSegmentAnnouncerConfig.java b/server/src/main/java/io/druid/segment/realtime/RealtimeCuratorDataSegmentAnnouncerConfig.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/RealtimeCuratorDataSegmentAnnouncerConfig.java rename to server/src/main/java/io/druid/segment/realtime/RealtimeCuratorDataSegmentAnnouncerConfig.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/RealtimeManager.java rename to server/src/main/java/io/druid/segment/realtime/RealtimeManager.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java b/server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java rename to server/src/main/java/io/druid/segment/realtime/RealtimeMetricsMonitor.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/Schema.java b/server/src/main/java/io/druid/segment/realtime/Schema.java similarity index 98% rename from realtime/src/main/java/io/druid/segment/realtime/Schema.java rename to server/src/main/java/io/druid/segment/realtime/Schema.java index 47cf034089c..948e8495088 100644 --- a/realtime/src/main/java/io/druid/segment/realtime/Schema.java +++ b/server/src/main/java/io/druid/segment/realtime/Schema.java @@ -23,9 +23,9 @@ 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 io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.incremental.SpatialDimensionSchema; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; diff --git a/realtime/src/main/java/io/druid/segment/realtime/SegmentPublisher.java b/server/src/main/java/io/druid/segment/realtime/SegmentPublisher.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/SegmentPublisher.java rename to server/src/main/java/io/druid/segment/realtime/SegmentPublisher.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/ClippedFirehoseFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/IrcDecoder.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcDecoder.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/IrcDecoder.java rename to server/src/main/java/io/druid/segment/realtime/firehose/IrcDecoder.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/JacksonifiedConnectionFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/JacksonifiedConnectionFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/JacksonifiedConnectionFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/JacksonifiedConnectionFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/KafkaFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/KafkaFirehoseFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/KafkaFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/KafkaFirehoseFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java similarity index 96% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java index eccf17ff206..f78bc0ac390 100644 --- a/realtime/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/LocalFirehoseFactory.java @@ -25,7 +25,8 @@ import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.collect.Lists; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; -import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.impl.FileIteratingFirehose; +import io.druid.data.input.impl.StringInputRowParser; import org.apache.commons.io.FileUtils; import org.apache.commons.io.LineIterator; diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java b/server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java rename to server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseConfig.java b/server/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseConfig.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseConfig.java rename to server/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseConfig.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java similarity index 99% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java index 2cfdddacd4a..aa0270df15d 100644 --- a/realtime/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/RabbitMQFirehoseFactory.java @@ -32,7 +32,7 @@ import com.rabbitmq.client.ShutdownSignalException; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.StringInputRowParser; +import io.druid.data.input.impl.StringInputRowParser; import java.io.IOException; diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java rename to server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java rename to server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java b/server/src/main/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java rename to server/src/main/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicy.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java b/server/src/main/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java rename to server/src/main/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java b/server/src/main/java/io/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java rename to server/src/main/java/io/druid/segment/realtime/plumber/NoopRejectionPolicyFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/Plumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/Plumber.java rename to server/src/main/java/io/druid/segment/realtime/plumber/Plumber.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java rename to server/src/main/java/io/druid/segment/realtime/plumber/PlumberSchool.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java rename to server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicy.java b/server/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicy.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicy.java rename to server/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicy.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicyFactory.java b/server/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicyFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicyFactory.java rename to server/src/main/java/io/druid/segment/realtime/plumber/RejectionPolicyFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java b/server/src/main/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java rename to server/src/main/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactory.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/io/druid/segment/realtime/plumber/Sink.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/Sink.java rename to server/src/main/java/io/druid/segment/realtime/plumber/Sink.java diff --git a/realtime/src/main/java/io/druid/segment/realtime/plumber/VersioningPolicy.java b/server/src/main/java/io/druid/segment/realtime/plumber/VersioningPolicy.java similarity index 100% rename from realtime/src/main/java/io/druid/segment/realtime/plumber/VersioningPolicy.java rename to server/src/main/java/io/druid/segment/realtime/plumber/VersioningPolicy.java diff --git a/server/src/main/java/io/druid/server/http/RedirectServlet.java b/server/src/main/java/io/druid/server/http/RedirectServlet.java index 2226fed3d83..d402d64cddf 100644 --- a/server/src/main/java/io/druid/server/http/RedirectServlet.java +++ b/server/src/main/java/io/druid/server/http/RedirectServlet.java @@ -21,7 +21,7 @@ package io.druid.server.http; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import org.mortbay.jetty.servlet.DefaultServlet; +import org.eclipse.jetty.servlet.DefaultServlet; import javax.servlet.ServletException; import javax.servlet.ServletRequest; diff --git a/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java b/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java index 46403358758..b4d2699a2a4 100644 --- a/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java @@ -67,4 +67,15 @@ public class ExtensionsConfig { return remoteRepositories; } + + @Override + public String toString() + { + return "ExtensionsConfig{" + + "searchCurrentClassloader=" + searchCurrentClassloader + + ", coordinates=" + coordinates + + ", localRepository='" + localRepository + '\'' + + ", remoteRepositories=" + remoteRepositories + + '}'; + } } diff --git a/server/src/main/java/io/druid/server/sql/SQLRunner.java b/server/src/main/java/io/druid/server/sql/SQLRunner.java index 561b10f2618..dc6cb8dfa12 100644 --- a/server/src/main/java/io/druid/server/sql/SQLRunner.java +++ b/server/src/main/java/io/druid/server/sql/SQLRunner.java @@ -22,6 +22,7 @@ package io.druid.server.sql; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.base.Charsets; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; @@ -47,13 +48,12 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.HelpFormatter; import org.apache.commons.cli.Options; -import org.apache.commons.httpclient.HttpClient; -import org.apache.commons.httpclient.methods.PostMethod; -import org.apache.commons.httpclient.methods.StringRequestEntity; import javax.annotation.Nullable; import java.io.BufferedReader; import java.io.InputStreamReader; +import java.net.URL; +import java.net.URLConnection; import java.util.ArrayList; import java.util.List; @@ -141,12 +141,11 @@ public class SQLRunner String queryStr = jsonWriter.writeValueAsString(query); if(cmd.hasOption("v")) System.err.println(queryStr); - PostMethod req = new PostMethod("http://" + hostname + "/druid/v2/?pretty"); - req.setRequestEntity(new StringRequestEntity(queryStr, "application/json", "utf-8")); - new HttpClient().executeMethod(req); - - BufferedReader stdInput = new BufferedReader(new - InputStreamReader(req.getResponseBodyAsStream())); + URL url = new URL(String.format("http://%s/druid/v2/?pretty", hostname)); + final URLConnection urlConnection = url.openConnection(); + urlConnection.addRequestProperty("content-type", "application/json"); + urlConnection.getOutputStream().write(queryStr.getBytes(Charsets.UTF_8)); + BufferedReader stdInput = new BufferedReader(new InputStreamReader(urlConnection.getInputStream())); Object res = objectMapper.readValue(stdInput, typeRef); diff --git a/server/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java b/server/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java deleted file mode 100644 index d58c210eb4b..00000000000 --- a/server/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java +++ /dev/null @@ -1,55 +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.collect.ImmutableMap; -import io.druid.segment.IndexIO; -import io.druid.timeline.DataSegment; -import io.druid.timeline.partition.NoneShardSpec; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; - -public class DataSegmentPusherUtilTest { - @Test - public void shouldNotHaveColonsInHdfsStorageDir() throws Exception { - - Interval interval = new Interval("2011-10-01/2011-10-02"); - ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); - - DataSegment segment = new DataSegment( - "something", - interval, - "brand:new:version", - loadSpec, - Arrays.asList("dim1", "dim2"), - Arrays.asList("met1", "met2"), - new NoneShardSpec(), - IndexIO.CURRENT_VERSION_ID, - 1 - ); - - String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment); - Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version/0", storageDir); - - } -} diff --git a/realtime/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java similarity index 98% rename from realtime/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java rename to server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index d955b228ebd..f3d87d4d122 100644 --- a/realtime/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -22,20 +22,20 @@ package io.druid.segment.realtime; import com.google.common.base.Stopwatch; import com.google.common.collect.Lists; import com.metamx.common.ISE; -import io.druid.common.guava.Runnables; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.SpatialDimensionSchema; 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.CountAggregatorFactory; -import io.druid.segment.incremental.SpatialDimensionSchema; import io.druid.segment.realtime.plumber.Plumber; import io.druid.segment.realtime.plumber.PlumberSchool; import io.druid.segment.realtime.plumber.Sink; import io.druid.timeline.partition.NoneShardSpec; +import io.druid.utils.Runnables; import junit.framework.Assert; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java similarity index 100% rename from realtime/src/test/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java rename to server/src/test/java/io/druid/segment/realtime/plumber/IntervalStartVersioningPolicyTest.java diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java similarity index 100% rename from realtime/src/test/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java rename to server/src/test/java/io/druid/segment/realtime/plumber/MessageTimeRejectionPolicyFactoryTest.java diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java similarity index 98% rename from realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java rename to server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 71eb4659909..39400f30cbf 100644 --- a/realtime/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -27,6 +27,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.ServerView; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; @@ -34,7 +35,6 @@ import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.IndexGranularity; -import io.druid.segment.incremental.SpatialDimensionSchema; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.Schema; diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java similarity index 100% rename from realtime/src/test/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java rename to server/src/test/java/io/druid/segment/realtime/plumber/ServerTimeRejectionPolicyFactoryTest.java diff --git a/realtime/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java similarity index 98% rename from realtime/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java rename to server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 9900c368ad1..baee6480789 100644 --- a/realtime/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -22,10 +22,10 @@ package io.druid.segment.realtime.plumber; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.segment.incremental.SpatialDimensionSchema; import io.druid.segment.realtime.FireHydrant; import io.druid.segment.realtime.Schema; import io.druid.timeline.partition.NoneShardSpec; diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index ec5db65ec97..98e88d46ade 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -19,29 +19,22 @@ package io.druid.cli; -import com.google.api.client.repackaged.com.google.common.base.Throwables; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.io.CharStreams; -import com.google.common.io.InputSupplier; -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.google.api.client.util.Lists; +import com.google.common.base.Joiner; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.airlift.command.Arguments; import io.airlift.command.Command; -import io.druid.guice.LazySingleton; -import io.druid.indexer.HadoopDruidIndexerConfig; -import io.druid.indexer.HadoopDruidIndexerJob; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; +import io.airlift.command.Option; +import io.druid.initialization.Initialization; +import io.druid.server.initialization.ExtensionsConfig; +import io.tesla.aether.internal.DefaultTeslaAether; import java.io.File; -import java.io.IOException; -import java.io.InputStreamReader; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.Arrays; import java.util.List; /** @@ -50,87 +43,55 @@ import java.util.List; name = "hadoop", description = "Runs the batch Hadoop Druid Indexer, see https://github.com/metamx/druid/wiki/Batch-ingestion for a description." ) -public class CliHadoopIndexer extends GuiceRunnable +public class CliHadoopIndexer implements Runnable { + private static final Logger log = new Logger(CliHadoopIndexer.class); + @Arguments(description = "A JSON object or the path to a file that contains a JSON object", required = true) private String argumentSpec; - private static final Logger log = new Logger(CliHadoopIndexer.class); + @Option(name = "hadoop", description = "The maven coordinates to the version of hadoop to run with. Defaults to org.apache.hadoop:hadoop-core:1.0.3") + private String hadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3"; - public CliHadoopIndexer() - { - super(log); - } - - @Override - protected List getModules() - { - return ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(HadoopDruidIndexerJob.class).in(LazySingleton.class); - } - - @Provides - @LazySingleton - public HadoopDruidIndexerConfig getHadoopDruidIndexerConfig() - { - try { - if (argumentSpec.startsWith("{")) { - return HadoopDruidIndexerConfig.fromString(argumentSpec); - } else if (argumentSpec.startsWith("s3://")) { - final Path s3nPath = new Path(String.format("s3n://%s", argumentSpec.substring("s3://".length()))); - final FileSystem fs = s3nPath.getFileSystem(new Configuration()); - - String configString = CharStreams.toString( - new InputSupplier() - { - @Override - public InputStreamReader getInput() throws IOException - { - return new InputStreamReader(fs.open(s3nPath)); - } - } - ); - - return HadoopDruidIndexerConfig.fromString(configString); - } else { - return HadoopDruidIndexerConfig.fromFile(new File(argumentSpec)); - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); - } + @Inject + private ExtensionsConfig extensionsConfig = null; @Override + @SuppressWarnings("unchecked") public void run() { try { - Injector injector = makeInjector(); - final HadoopDruidIndexerJob job = injector.getInstance(HadoopDruidIndexerJob.class); + final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); + final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates( + aetherClient, hadoopCoordinates + ); + final URL[] urLs = ((URLClassLoader) hadoopLoader).getURLs(); - Lifecycle lifecycle = initLifecycle(injector); + final URL[] nonHadoopUrls = ((URLClassLoader) CliHadoopIndexer.class.getClassLoader()).getURLs(); - job.run(); + List theURLS = Lists.newArrayList(); + theURLS.addAll(Arrays.asList(urLs)); + theURLS.addAll(Arrays.asList(nonHadoopUrls)); - try { - lifecycle.stop(); - } - catch (Throwable t) { - log.error(t, "Error when stopping. Failing."); - System.exit(1); - } + final URLClassLoader loader = new URLClassLoader(theURLS.toArray(new URL[theURLS.size()]), null); + Thread.currentThread().setContextClassLoader(loader); + System.setProperty("druid.hadoop.internal.classpath", Joiner.on(File.pathSeparator).join(nonHadoopUrls)); + + final Class mainClass = loader.loadClass(Main.class.getName()); + final Method mainMethod = mainClass.getMethod("main", String[].class); + + String[] args = new String[]{ + "internal", + "hadoop-indexer", + argumentSpec + }; + + mainMethod.invoke(null, new Object[]{args}); } catch (Exception e) { - throw Throwables.propagate(e); + log.error(e, "failure!!!!"); + System.exit(1); } } diff --git a/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java new file mode 100644 index 00000000000..217eed30905 --- /dev/null +++ b/services/src/main/java/io/druid/cli/CliInternalHadoopIndexer.java @@ -0,0 +1,76 @@ +/* + * 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.api.client.repackaged.com.google.common.base.Throwables; +import com.metamx.common.logger.Logger; +import io.airlift.command.Arguments; +import io.airlift.command.Command; +import io.druid.indexer.HadoopDruidIndexerConfig; +import io.druid.indexer.HadoopDruidIndexerJob; +import io.druid.timeline.partition.SingleDimensionShardSpec; + +import java.io.File; + +/** + */ +@Command( + name = "hadoop-indexer", + description = "Runs the batch Hadoop Druid Indexer, see https://github.com/metamx/druid/wiki/Batch-ingestion for a description." +) +public class CliInternalHadoopIndexer implements Runnable +{ + @Arguments(description = "A JSON object or the path to a file that contains a JSON object", required = true) + private String argumentSpec; + + private static final Logger log = new Logger(CliHadoopIndexer.class); + + @Override + public void run() + { + try { + System.out.println( + HadoopDruidIndexerConfig.jsonMapper.writeValueAsString( + new SingleDimensionShardSpec("billy", "a", "b", 1) + ) + ); + + final HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(getHadoopDruidIndexerConfig()); + job.run(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public HadoopDruidIndexerConfig getHadoopDruidIndexerConfig() + { + try { + if (argumentSpec.startsWith("{")) { + return HadoopDruidIndexerConfig.fromString(argumentSpec); + } else { + return HadoopDruidIndexerConfig.fromFile(new File(argumentSpec)); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} \ No newline at end of file diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 065e6c59039..fa0ed6d597e 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -44,8 +44,6 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.index.ChatHandlerProvider; import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; -import io.druid.indexing.common.tasklogs.TaskLogStreamer; -import io.druid.indexing.common.tasklogs.TaskLogs; import io.druid.indexing.common.tasklogs.TaskRunnerTaskLogStreamer; import io.druid.indexing.overlord.DbTaskStorage; import io.druid.indexing.overlord.ForkingTaskRunnerFactory; @@ -59,8 +57,8 @@ import io.druid.indexing.overlord.TaskRunnerFactory; import io.druid.indexing.overlord.TaskStorage; import io.druid.indexing.overlord.TaskStorageQueryAdapter; import io.druid.indexing.overlord.http.OldOverlordResource; -import io.druid.indexing.overlord.http.OverlordResource; import io.druid.indexing.overlord.http.OverlordRedirectInfo; +import io.druid.indexing.overlord.http.OverlordResource; import io.druid.indexing.overlord.scaling.AutoScalingStrategy; import io.druid.indexing.overlord.scaling.EC2AutoScalingStrategy; import io.druid.indexing.overlord.scaling.NoopAutoScalingStrategy; @@ -74,6 +72,8 @@ import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.server.http.RedirectFilter; import io.druid.server.http.RedirectInfo; import io.druid.server.initialization.JettyServerInitializer; +import io.druid.tasklogs.TaskLogStreamer; +import io.druid.tasklogs.TaskLogs; 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 b874ec69a7b..d91e03b111a 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -19,18 +19,20 @@ package io.druid.cli; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.multibindings.MapBinder; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.airlift.command.Arguments; import io.airlift.command.Command; import io.airlift.command.Option; +import io.druid.guice.Binders; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; @@ -47,6 +49,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.index.ChatHandlerProvider; +import io.druid.indexing.common.index.EventReceiverFirehoseFactory; import io.druid.indexing.common.index.NoopChatHandlerProvider; import io.druid.indexing.common.index.ServiceAnnouncingChatHandlerProvider; import io.druid.indexing.overlord.HeapMemoryTaskStorage; @@ -58,9 +61,10 @@ import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.executor.ChatHandlerResource; import io.druid.indexing.worker.executor.ExecutorLifecycle; import io.druid.indexing.worker.executor.ExecutorLifecycleConfig; +import io.druid.initialization.DruidModule; import io.druid.query.QuerySegmentWalker; import io.druid.segment.loading.DataSegmentKiller; -import io.druid.segment.loading.S3DataSegmentKiller; +import io.druid.segment.loading.OmniDataSegmentKiller; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.StorageLocationConfig; import io.druid.server.initialization.JettyServerInitializer; @@ -96,7 +100,7 @@ public class CliPeon extends GuiceRunnable protected List getModules() { return ImmutableList.of( - new Module() + new DruidModule() { @Override public void configure(Binder binder) @@ -124,7 +128,9 @@ public class CliPeon extends GuiceRunnable binder.bind(RetryPolicyFactory.class).in(LazySingleton.class); - binder.bind(DataSegmentKiller.class).to(S3DataSegmentKiller.class).in(LazySingleton.class); + // Build it to make it bind even if nothing binds to it. + Binders.dataSegmentKillerBinder(binder); + binder.bind(DataSegmentKiller.class).to(OmniDataSegmentKiller.class).in(LazySingleton.class); binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class); binder.bind(ExecutorLifecycleConfig.class).toInstance( @@ -173,6 +179,15 @@ public class CliPeon extends GuiceRunnable .to(RemoteTaskActionClientFactory.class).in(LazySingleton.class); } + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("PeonModule") + .registerSubtypes(new NamedType(EventReceiverFirehoseFactory.class, "receiver")) + ); + } } ); } diff --git a/services/src/main/java/io/druid/cli/CliRealtimeExample.java b/services/src/main/java/io/druid/cli/CliRealtimeExample.java index af3629f5058..59166401b37 100644 --- a/services/src/main/java/io/druid/cli/CliRealtimeExample.java +++ b/services/src/main/java/io/druid/cli/CliRealtimeExample.java @@ -108,6 +108,12 @@ public class CliRealtimeExample extends ServerRunnable private static class NoopDataSegmentPusher implements DataSegmentPusher { + @Override + public String getPathForHadoop(String dataSource) + { + return dataSource; + } + @Override public DataSegment push(File file, DataSegment segment) throws IOException { diff --git a/services/src/main/java/io/druid/cli/GuiceRunnable.java b/services/src/main/java/io/druid/cli/GuiceRunnable.java index 22d27cd6f7d..466104427ed 100644 --- a/services/src/main/java/io/druid/cli/GuiceRunnable.java +++ b/services/src/main/java/io/druid/cli/GuiceRunnable.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 io.druid.cli; import com.google.common.base.Throwables; @@ -5,6 +24,7 @@ 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.Initialization; import io.druid.initialization.LogLevelAdjuster; import java.util.List; diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 6f7755fd02c..750226ca871 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -19,21 +19,13 @@ 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.cli.convert.ConvertProperties; -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.initialization.Initialization; import io.druid.server.initialization.ExtensionsConfig; -import io.druid.server.initialization.PropertiesModule; import java.util.List; @@ -76,9 +68,9 @@ public class Main builder.withGroup("internal") .withDescription("Processes that Druid runs \"internally\", you should rarely use these directly") .withDefaultCommand(Help.class) - .withCommands(CliPeon.class); + .withCommands(CliPeon.class, CliInternalHadoopIndexer.class); - final Injector injector = makeStartupInjector(); + final Injector injector = Initialization.makeStartupInjector(); final ExtensionsConfig config = injector.getInstance(ExtensionsConfig.class); final List extensionCommands = Initialization.getFromExtensions(config, CliCommandCreator.class); @@ -89,7 +81,9 @@ public class Main final Cli cli = builder.build(); try { final Runnable command = cli.parse(args); - injector.injectMembers(command); + if (! (command instanceof Help)) { // Hack to work around Help not liking being injected + injector.injectMembers(command); + } command.run(); } catch (ParseException e) { @@ -99,23 +93,4 @@ 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/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index c4fa8aa6461..094efa93f75 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -57,11 +57,7 @@ public class RealtimeModule implements Module binder.bind(DbSegmentPublisher.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class); - binder.bind( - new TypeLiteral>() - { - } - ) + binder.bind(new TypeLiteral>(){}) .toProvider(FireDepartmentsProvider.class) .in(LazySingleton.class);