diff --git common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java index 5d475f4..50dc7df 100644 --- common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java +++ common/src/java/org/apache/hadoop/hive/common/JvmPauseMonitor.java @@ -37,6 +37,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; /** * Based on the JvmPauseMonitor from Hadoop. @@ -172,7 +173,7 @@ public String toString() { private class Monitor implements Runnable { @Override public void run() { - Stopwatch sw = new Stopwatch(); + Stopwatch sw = Stopwatch.createStarted(); Map gcTimesBeforeSleep = getGcTimes(); while (shouldRun) { sw.reset().start(); @@ -181,7 +182,7 @@ public void run() { } catch (InterruptedException ie) { return; } - long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS; + long extraSleepTime = sw.elapsed(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS; Map gcTimesAfterSleep = getGcTimes(); if (extraSleepTime > warnThresholdMs) { diff --git common/src/java/org/apache/hadoop/hive/conf/Constants.java common/src/java/org/apache/hadoop/hive/conf/Constants.java index 6c42163..9d1a48f 100644 --- common/src/java/org/apache/hadoop/hive/conf/Constants.java +++ common/src/java/org/apache/hadoop/hive/conf/Constants.java @@ -26,7 +26,11 @@ /* Constants for Druid storage handler */ public static final String DRUID_HIVE_STORAGE_HANDLER_ID = "org.apache.hadoop.hive.druid.DruidStorageHandler"; + public static final String DRUID_HIVE_OUTPUT_FORMAT = + "org.apache.hadoop.hive.druid.io.DruidOutputFormat"; public static final String DRUID_DATA_SOURCE = "druid.datasource"; + public static final String DRUID_SEGMENT_GRANULARITY = "druid.segment.granularity"; + public static final String DRUID_TIMESTAMP_GRANULARITY_COL_NAME = "__time_granularity"; public static final String DRUID_QUERY_JSON = "druid.query.json"; public static final String DRUID_QUERY_TYPE = "druid.query.type"; public static final String DRUID_QUERY_FETCH = "druid.query.fetch"; @@ -34,4 +38,6 @@ public static final String HIVE_SERVER2_JOB_CREDSTORE_PASSWORD_ENVVAR = "HIVE_JOB_CREDSTORE_PASSWORD"; public static final String HADOOP_CREDENTIAL_PASSWORD_ENVVAR = "HADOOP_CREDSTORE_PASSWORD"; public static final String HADOOP_CREDENTIAL_PROVIDER_PATH_CONFIG = "hadoop.security.credential.provider.path"; + + public static final String DRUID_SEGMENT_DIRECTORY = "druid.storage.storageDirectory"; } diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 9a5d604..2345c2e 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -1929,6 +1929,11 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal new TimeValidator(TimeUnit.MILLISECONDS), "Frequency of WriteSet reaper runs"), // For Druid storage handler + HIVE_DRUID_INDEXING_GRANULARITY("hive.druid.indexer.segments.granularity", "DAY", new PatternSet("YEAR", "MONTH", "WEEK", "DAY", "HOUR", "MINUTE", "SECOND"), "Granularity for the segments created by the Druid storage handler"), + HIVE_DRUID_MAX_PARTITION_SIZE("hive.druid.indexer.partition.size.max", 5000000, + "Maximum number of records per segment partition"), + HIVE_DRUID_MAX_ROW_IN_MEMORY("hive.druid.indexer.memory.rownum.max", 75000, + "Maximum number of records in memory while storing data in Druid"), HIVE_DRUID_BROKER_DEFAULT_ADDRESS("hive.druid.broker.address.default", "localhost:8082", "Address of the Druid broker. If we are querying Druid from Hive, this address needs to be\n" + "declared"), @@ -1938,7 +1943,9 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal "number of records of the query results is larger than this threshold, we split the query in\n" + "total number of rows/threshold parts across the time dimension. Note that we assume the\n" + "records to be split uniformly across the time dimension"), - + HIVE_DRUID_BASE_PERSIST_DIRECTORY("hive.druid.basePersisitDirectory", "/tmp", "local base persist directory used while indexing data"), + DRUID_SEGMENT_DIRECTORY("hive.druid.storage.storageDirectory", "/druid/segments" + , "druid deep storage location"), // For HBase storage handler HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true, "Whether writes to HBase should be forced to the write-ahead log. \n" + diff --git druid-handler/README.md druid-handler/README.md new file mode 100644 index 0000000..57c381c --- /dev/null +++ druid-handler/README.md @@ -0,0 +1,3 @@ +# Druid Storage Handler + +The handler will try to create the druid segments table if it does not exist, this will be done as part of the pre-create process. diff --git druid-handler/pom.xml druid-handler/pom.xml index e4fa8fd..cd35abd 100644 --- druid-handler/pom.xml +++ druid-handler/pom.xml @@ -38,6 +38,7 @@ org.apache.hive hive-exec ${project.version} + provided io.netty @@ -47,6 +48,10 @@ io.netty netty + + com.google.guava + guava + @@ -58,29 +63,96 @@ org.apache.hadoop hadoop-common + provided ${hadoop.version} true - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - + + + org.slf4j + slf4j-log4j12 + + + com.google.guava + guava + + + commmons-logging + commons-logging + org.apache.hadoop + provided hadoop-mapreduce-client-core ${hadoop.version} true + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + io.netty netty + + com.google.guava + guava + + + + + + joda-time + joda-time + 2.8.2 + + + org.apache.calcite + calcite-druid + ${calcite.version} + + + io.druid + druid-server + ${druid.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + + + io.druid + java-util + ${druid.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + @@ -107,6 +179,21 @@ + + io.druid.extensions + druid-hdfs-storage + ${druid.version} + + + io.druid.extensions + mysql-metadata-storage + ${druid.version} + + + io.druid.extensions + postgresql-metadata-storage + ${druid.version} + @@ -115,6 +202,12 @@ ${junit.version} test + + io.druid + druid-indexing-hadoop + ${druid.version} + test + @@ -174,11 +267,16 @@ io.druid:* + io.druid.extensions:* com.metamx:* io.netty:* com.fasterxml.jackson.core:* com.fasterxml.jackson.datatype:* com.fasterxml.jackson.dataformat:* + it.unimi.dsi:* + org.jdbi:* + net.jpountz.lz4:* + org.apache.commons:* diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index 8242385..6bdb17b 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -17,36 +17,118 @@ */ package org.apache.hadoop.hive.druid; -import org.apache.commons.lang3.StringUtils; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.MetadataStorageTablesConfig; +import io.druid.metadata.SQLMetadataConnector; +import io.druid.metadata.storage.mysql.MySQLConnector; +import io.druid.metadata.storage.postgresql.PostgreSQLConnector; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.druid.io.DruidOutputFormat; +import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat; import org.apache.hadoop.hive.druid.serde.DruidSerDe; import org.apache.hadoop.hive.metastore.HiveMetaHook; -import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler; +import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde2.AbstractSerDe; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.OutputFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.Collection; +import java.util.List; + /** * DruidStorageHandler provides a HiveStorageHandler implementation for Druid. */ -@SuppressWarnings({"deprecation","rawtypes"}) +@SuppressWarnings({ "deprecation", "rawtypes" }) public class DruidStorageHandler extends DefaultStorageHandler implements HiveMetaHook { protected static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandler.class); + public static final String SEGMENTS_DESCRIPTOR_DIR_NAME = "segmentsDescriptorDir"; + + private final SQLMetadataConnector connector; + + private final SQLMetadataStorageUpdaterJobHandler druidSqlMetadataStorageUpdaterJobHandler; + + private final MetadataStorageTablesConfig druidMetadataStorageTablesConfig; + + public DruidStorageHandler() { + //this is the default value in druid + final String base = SessionState.getSessionConf().get("hive.druid.metadata.base", "druid"); + // default to mysql to avoid issue with creating of connector. + final String dbType = SessionState.getSessionConf().get("hive.druid.metadata.db.type", "mysql"); + final String username = SessionState.getSessionConf().get("hive.druid.metadata.username", ""); + final String password = SessionState.getSessionConf().get("hive.druid.metadata.password", ""); + final String uri = SessionState.getSessionConf() + .get("hive.druid.metadata.uri", "jdbc:mysql://localhost/druid"); + + druidMetadataStorageTablesConfig = MetadataStorageTablesConfig.fromBase(base); + + final Supplier storageConnectorConfigSupplier = Suppliers.ofInstance( + new MetadataStorageConnectorConfig() { + @Override + public String getConnectURI() { + return uri; + } + + @Override + public String getUser() { + return username; + } + + @Override + public String getPassword() { + return password; + } + }); + + if (dbType.equals("mysql")) { + connector = new MySQLConnector(storageConnectorConfigSupplier, + Suppliers.ofInstance(druidMetadataStorageTablesConfig) + ); + } else if (dbType.equals("postgres")) { + connector = new PostgreSQLConnector(storageConnectorConfigSupplier, + Suppliers.ofInstance(druidMetadataStorageTablesConfig) + ); + } else { + throw new IllegalStateException(String.format("Unknown metadata storage type [%s]", dbType)); + } + druidSqlMetadataStorageUpdaterJobHandler = new SQLMetadataStorageUpdaterJobHandler(connector); + } + + @VisibleForTesting + public DruidStorageHandler(SQLMetadataConnector connector, + SQLMetadataStorageUpdaterJobHandler druidSqlMetadataStorageUpdaterJobHandler, + MetadataStorageTablesConfig druidMetadataStorageTablesConfig + ) { + this.connector = connector; + this.druidSqlMetadataStorageUpdaterJobHandler = druidSqlMetadataStorageUpdaterJobHandler; + this.druidMetadataStorageTablesConfig = druidMetadataStorageTablesConfig; + } + @Override public Class getInputFormatClass() { - return HiveDruidQueryBasedInputFormat.class; + return DruidQueryBasedInputFormat.class; } @Override public Class getOutputFormatClass() { - return HiveDruidOutputFormat.class; + return DruidOutputFormat.class; } @Override @@ -62,28 +144,128 @@ public HiveMetaHook getMetaHook() { @Override public void preCreateTable(Table table) throws MetaException { // Do safety checks - if (!MetaStoreUtils.isExternalTable(table)) { - throw new MetaException("Table in Druid needs to be declared as EXTERNAL"); - } - if (!StringUtils.isEmpty(table.getSd().getLocation())) { - throw new MetaException("LOCATION may not be specified for Druid"); - } + /* + @TODO ASK Jesus if this is needed, i think we don't needed anymore + if (MetaStoreUtils.isExternalTable(table) && !StringUtils.isEmpty(table.getSd().getLocation())) { + throw new MetaException("LOCATION may not be specified for Druid existing sources"); + }*/ if (table.getPartitionKeysSize() != 0) { throw new MetaException("PARTITIONED BY may not be specified for Druid"); } if (table.getSd().getBucketColsSize() != 0) { throw new MetaException("CLUSTERED BY may not be specified for Druid"); } + String dataSourceName = Preconditions + .checkNotNull(table.getParameters().get(Constants.DRUID_DATA_SOURCE), + "WTF dataSource name is null !" + ); + try { + connector.createSegmentTable(); + } catch (Exception e) { + LOG.error("Exception while trying to create druid segments table", e); + throw new MetaException(e.getMessage()); + } + Collection existingDataSources = DruidStorageHandlerUtils + .getAllDataSourceNames(connector, druidMetadataStorageTablesConfig); + LOG.debug(String.format("pre-create data source with name [%s]", dataSourceName)); + if (existingDataSources.contains(dataSourceName)) { + throw new MetaException(String.format("Data source [%s] already existing", dataSourceName)); + } } @Override public void rollbackCreateTable(Table table) throws MetaException { - // Nothing to do + final Path segmentDescriptorDir = new Path(table.getSd().getLocation()); + try { + List dataSegmentList = DruidStorageHandlerUtils + .getPublishedSegments(segmentDescriptorDir, getConf()); + for (DataSegment dataSegment : + dataSegmentList) { + try { + deleteSegment(dataSegment); + } catch (SegmentLoadingException e) { + LOG.error(String.format("Error while trying to clean the segment [%s]", dataSegment), e); + } + } + } catch (IOException e) { + LOG.error("Exception while rollback", e); + Throwables.propagate(e); + } } @Override public void commitCreateTable(Table table) throws MetaException { - // Nothing to do + LOG.info(String.format("Committing table [%s] to the druid metastore", table.getDbName())); + final Path tableDir = new Path(table.getSd().getLocation()); + try { + druidSqlMetadataStorageUpdaterJobHandler.publishSegments( + druidMetadataStorageTablesConfig.getSegmentsTable(), + DruidStorageHandlerUtils.getPublishedSegments(tableDir, getConf()), + DruidStorageHandlerUtils.JSON_MAPPER + ); + } catch (IOException e) { + LOG.error("Exception while commit", e); + Throwables.propagate(e); + } + } + + @VisibleForTesting + protected void deleteSegment(DataSegment segment) throws SegmentLoadingException { + + final Path path = getPath(segment); + LOG.info(String.format("removing segment[%s], located at path[%s]", segment.getIdentifier(), + path + )); + + try { + if (path.getName().endsWith(".zip")) { + + final FileSystem fs = path.getFileSystem(getConf()); + + if (!fs.exists(path)) { + LOG.warn(String.format( + "Segment Path [%s] does not exist. It appears to have been deleted already.", + path + )); + return; + } + + // path format -- > .../dataSource/interval/version/partitionNum/xxx.zip + Path partitionNumDir = path.getParent(); + if (!fs.delete(partitionNumDir, true)) { + throw new SegmentLoadingException( + "Unable to kill segment, failed to delete dir [%s]", + partitionNumDir.toString() + ); + } + + //try to delete other directories if possible + Path versionDir = partitionNumDir.getParent(); + if (safeNonRecursiveDelete(fs, versionDir)) { + Path intervalDir = versionDir.getParent(); + if (safeNonRecursiveDelete(fs, intervalDir)) { + Path dataSourceDir = intervalDir.getParent(); + safeNonRecursiveDelete(fs, dataSourceDir); + } + } + } else { + throw new SegmentLoadingException("Unknown file type[%s]", path); + } + } catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to kill segment"); + } + } + + private static Path getPath(DataSegment dataSegment) { + return new Path(String.valueOf(dataSegment.getLoadSpec().get("path"))); + } + + private static boolean safeNonRecursiveDelete(FileSystem fs, Path path) { + try { + return fs.delete(path, false); + } catch (Exception ex) { + return false; + } } @Override @@ -98,7 +280,36 @@ public void rollbackDropTable(Table table) throws MetaException { @Override public void commitDropTable(Table table, boolean deleteData) throws MetaException { - // Nothing to do + String dataSourceName = Preconditions + .checkNotNull(table.getParameters().get(Constants.DRUID_DATA_SOURCE), + "WTF dataSource name is null !" + ); + + if (deleteData == true) { + LOG.info(String.format("Dropping with purge all the data for data source [%s]", + dataSourceName + )); + List dataSegmentList = DruidStorageHandlerUtils + .getDataSegment(connector, druidMetadataStorageTablesConfig, dataSourceName); + if (dataSegmentList.isEmpty()) { + LOG.info(String.format("Nothing to delete for data source [%s]", dataSourceName)); + return; + } + for (DataSegment dataSegment : + dataSegmentList) { + try { + deleteSegment(dataSegment); + } catch (SegmentLoadingException e) { + LOG.error(String.format("Error while deleting segment [%s]", dataSegment.getIdentifier()), + e + ); + } + } + } + if (DruidStorageHandlerUtils + .disableDataSource(connector, druidMetadataStorageTablesConfig, dataSourceName)) { + LOG.info(String.format("Successfully dropped druid data source [%s]", dataSourceName)); + } } @Override diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java index c6b8024..bd026f0 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java @@ -17,22 +17,62 @@ */ package org.apache.hadoop.hive.druid; -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.util.concurrent.ExecutionException; - -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpMethod; - import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; +import com.google.common.collect.Lists; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.NoopEmitter; +import com.metamx.emitter.service.ServiceEmitter; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.response.InputStreamResponseHandler; - import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.MapUtils; +import io.druid.metadata.MetadataStorageTablesConfig; +import io.druid.metadata.SQLMetadataConnector; +import io.druid.metadata.storage.mysql.MySQLConnector; import io.druid.query.BaseQuery; +import io.druid.segment.IndexIO; +import io.druid.segment.IndexMergerV9; +import io.druid.segment.column.ColumnConfig; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.retry.RetryPolicies; +import org.apache.hadoop.io.retry.RetryProxy; +import org.jboss.netty.handler.codec.http.HttpHeaders; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.skife.jdbi.v2.FoldController; +import org.skife.jdbi.v2.Folder3; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.StatementContext; +import org.skife.jdbi.v2.TransactionCallback; +import org.skife.jdbi.v2.TransactionStatus; +import org.skife.jdbi.v2.tweak.HandleCallback; +import org.skife.jdbi.v2.util.ByteArrayMapper; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.URL; +import java.net.UnknownHostException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; /** * Utils class for Druid storage handler. @@ -51,9 +91,42 @@ */ public static final ObjectMapper SMILE_MAPPER = new DefaultObjectMapper(new SmileFactory()); + private static final int NUM_RETRIES = 8; + + private static final int SECONDS_BETWEEN_RETRIES = 2; + + private static final int DEFAULT_FS_BUFFER_SIZE = 1 << 18; // 256KB + + private static final int DEFAULT_STREAMING_RESULT_SIZE = 100; + + public static final IndexIO INDEX_IO = new IndexIO(JSON_MAPPER, new ColumnConfig() { + @Override + public int columnCacheSizeBytes() { + return 0; + } + }); + + public static final IndexMergerV9 INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, + DruidStorageHandlerUtils.INDEX_IO + ); + + public static final Interner DATA_SEGMENT_INTERNER = Interners.newWeakInterner(); + + static { + JSON_MAPPER.registerSubtypes(new NamedType(LinearShardSpec.class, "linear")); + JSON_MAPPER.setTimeZone(TimeZone.getTimeZone("UTC")); + try { + EmittingLogger.registerEmitter( + new ServiceEmitter("druid-hive-indexer", InetAddress.getLocalHost().getHostName(), + new NoopEmitter() + )); + } catch (UnknownHostException e) { + Throwables.propagate(e); + } + } + /** * Method that creates a request for Druid JSON query (using SMILE). - * @param mapper * @param address * @param query * @return @@ -87,4 +160,208 @@ public static InputStream submitRequest(HttpClient client, Request request) return response; } + /** + * @param tableDir path to the table directory containing the segments descriptor info + * the descriptor path will be .../tableDir/task_id/{@link DruidStorageHandler#SEGMENTS_DESCRIPTOR_DIR_NAME}/*.json + * @param conf hadoop conf to get the file system + * + * @return List of DataSegments + * + * @throws IOException can be for the case we did not produce data. + */ + + public static List getPublishedSegments(Path tableDir, Configuration conf) + throws IOException { + ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); + FileSystem fs = tableDir.getFileSystem(conf); + for (FileStatus status : fs.listStatus(tableDir)) { + Path taskDir = new Path(status.getPath(), DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME); + if (!fs.isDirectory(taskDir)) { + continue; + } + for (FileStatus fileStatus : fs.listStatus(taskDir)) { + final DataSegment segment = JSON_MAPPER + .readValue(fs.open(fileStatus.getPath()), DataSegment.class); + publishedSegmentsBuilder.add(segment); + } + } + List publishedSegments = publishedSegmentsBuilder.build(); + return publishedSegments; + } + + public static void writeSegmentDescriptor( + final FileSystem outputFS, + final DataSegment segment, + final Path descriptorPath + ) + throws IOException { + final DataPusher descriptorPusher = (DataPusher) RetryProxy.create( + DataPusher.class, new DataPusher() { + @Override + public long push() throws IOException { + try { + if (outputFS.exists(descriptorPath)) { + if (!outputFS.delete(descriptorPath, false)) { + throw new IOException( + String.format("Failed to delete descriptor at [%s]", descriptorPath)); + } + } + try (final OutputStream descriptorOut = outputFS.create( + descriptorPath, + true, + DEFAULT_FS_BUFFER_SIZE + )) { + JSON_MAPPER.writeValue(descriptorOut, segment); + descriptorOut.flush(); + } + } catch (RuntimeException | IOException ex) { + throw ex; + } + return -1; + } + }, + RetryPolicies + .exponentialBackoffRetry(NUM_RETRIES, SECONDS_BETWEEN_RETRIES, TimeUnit.SECONDS) + ); + descriptorPusher.push(); + } + + public static Collection getAllDataSourceNames(SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig + ) { + return connector.getDBI().withHandle( + new HandleCallback>() { + @Override + public List withHandle(Handle handle) throws Exception { + return handle.createQuery( + String.format("SELECT DISTINCT(datasource) FROM %s WHERE used = true", + metadataStorageTablesConfig.getSegmentsTable() + ) + ) + .fold( + Lists.newArrayList(), + new Folder3, Map>() { + @Override + public ArrayList fold( + ArrayList druidDataSources, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException { + druidDataSources.add( + MapUtils.getString(stringObjectMap, "datasource") + ); + return druidDataSources; + } + } + ); + + } + } + ); + + } + + public static boolean disableDataSource(SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig, final String dataSource + ) { + try { + if (!getAllDataSourceNames(connector, metadataStorageTablesConfig).contains(dataSource)) { + DruidStorageHandler.LOG + .warn(String.format("Cannot delete data source [%s], does not exist", dataSource)); + return false; + } + + connector.getDBI().withHandle( + new HandleCallback() { + @Override + public Void withHandle(Handle handle) throws Exception { + handle.createStatement( + String.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", + metadataStorageTablesConfig.getSegmentsTable() + ) + ) + .bind("dataSource", dataSource) + .execute(); + + return null; + } + } + ); + + } catch (Exception e) { + DruidStorageHandler.LOG.error(String.format("Error removing dataSource %s", dataSource), e); + return false; + } + return true; + } + + public static List getDataSegment(final SQLMetadataConnector connector, + final MetadataStorageTablesConfig metadataStorageTablesConfig, final String dataSource + ) { + List segmentList = connector.retryTransaction( + new TransactionCallback>() { + @Override + public List inTransaction( + Handle handle, TransactionStatus status + ) throws Exception { + return handle + .createQuery(String.format( + "SELECT payload FROM %s WHERE dataSource = :dataSource", + metadataStorageTablesConfig.getSegmentsTable() + )) + .setFetchSize(getStreamingFetchSize(connector)) + .bind("dataSource", dataSource) + .map(ByteArrayMapper.FIRST) + .fold( + new ArrayList(), + new Folder3, byte[]>() { + @Override + public List fold(List accumulator, + byte[] payload, FoldController control, + StatementContext ctx + ) throws SQLException { + try { + final DataSegment segment = DATA_SEGMENT_INTERNER.intern( + JSON_MAPPER.readValue( + payload, + DataSegment.class + )); + + accumulator.add(segment); + return accumulator; + } catch (Exception e) { + throw new SQLException(e.toString()); + } + } + } + ); + } + } + , 3, SQLMetadataConnector.DEFAULT_MAX_TRIES); + return segmentList; + } + + private static int getStreamingFetchSize(SQLMetadataConnector connector) { + if (connector instanceof MySQLConnector) { + return Integer.MIN_VALUE; + } + return DEFAULT_STREAMING_RESULT_SIZE; + } + + public static Path makeSegmentDescriptorOutputPath(DataSegment pushedSegment, + Path segmentsDescriptorDir + ) { + return new Path( + segmentsDescriptorDir, + String.format("%s.json", pushedSegment.getIdentifier().replace(":", "")) + ); + } + + /** + * Simple interface for retry operations + */ + public interface DataPusher { + long push() throws IOException; + } } diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java deleted file mode 100644 index 45e31d6..0000000 --- druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.druid; - -import java.io.IOException; -import java.util.Properties; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.io.HiveOutputFormat; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordWriter; -import org.apache.hadoop.util.Progressable; - -/** - * Place holder for Druid output format. Currently not implemented. - */ -@SuppressWarnings("rawtypes") -public class HiveDruidOutputFormat implements HiveOutputFormat { - - @Override - public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name, - Progressable progress) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jc, Path finalOutPath, - Class valueClass, boolean isCompressed, Properties tableProperties, Progressable progress) - throws IOException { - throw new UnsupportedOperationException(); - } - -} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java deleted file mode 100644 index 787cd52..0000000 --- druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java +++ /dev/null @@ -1,370 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.druid; - -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.calcite.adapter.druid.DruidDateTimeUtils; -import org.apache.calcite.adapter.druid.DruidTable; -import org.apache.commons.lang3.StringEscapeUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.Constants; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader; -import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader; -import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader; -import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader; -import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader; -import org.apache.hadoop.hive.druid.serde.DruidWritable; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.joda.time.Interval; -import org.joda.time.chrono.ISOChronology; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; - -import io.druid.query.Druids; -import io.druid.query.Druids.SegmentMetadataQueryBuilder; -import io.druid.query.Druids.SelectQueryBuilder; -import io.druid.query.Druids.TimeBoundaryQueryBuilder; -import io.druid.query.Query; -import io.druid.query.Result; -import io.druid.query.metadata.metadata.SegmentAnalysis; -import io.druid.query.metadata.metadata.SegmentMetadataQuery; -import io.druid.query.select.PagingSpec; -import io.druid.query.select.SelectQuery; -import io.druid.query.spec.MultipleIntervalSegmentSpec; -import io.druid.query.timeboundary.TimeBoundaryQuery; -import io.druid.query.timeboundary.TimeBoundaryResultValue; - -/** - * Druid query based input format. - * - * Given a query and the Druid broker address, it will send it, and retrieve - * and parse the results. - */ -public class HiveDruidQueryBasedInputFormat extends InputFormat - implements org.apache.hadoop.mapred.InputFormat { - - protected static final Logger LOG = LoggerFactory.getLogger(HiveDruidQueryBasedInputFormat.class); - - @Override - public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits) - throws IOException { - return getInputSplits(job); - } - - @Override - public List getSplits(JobContext context) throws IOException, InterruptedException { - return Arrays. asList(getInputSplits(context.getConfiguration())); - } - - @SuppressWarnings("deprecation") - private HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException { - String address = HiveConf.getVar(conf, - HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS); - if (StringUtils.isEmpty(address)) { - throw new IOException("Druid broker address not specified in configuration"); - } - String druidQuery = StringEscapeUtils.unescapeJava(conf.get(Constants.DRUID_QUERY_JSON)); - String druidQueryType; - if (StringUtils.isEmpty(druidQuery)) { - // Empty, maybe because CBO did not run; we fall back to - // full Select query - if (LOG.isWarnEnabled()) { - LOG.warn("Druid query is empty; creating Select query"); - } - String dataSource = conf.get(Constants.DRUID_DATA_SOURCE); - if (dataSource == null) { - throw new IOException("Druid data source cannot be empty"); - } - druidQuery = createSelectStarQuery(address, dataSource); - druidQueryType = Query.SELECT; - } else { - druidQueryType = conf.get(Constants.DRUID_QUERY_TYPE); - if (druidQueryType == null) { - throw new IOException("Druid query type not recognized"); - } - } - - // hive depends on FileSplits - Job job = new Job(conf); - JobContext jobContext = ShimLoader.getHadoopShims().newJobContext(job); - Path [] paths = FileInputFormat.getInputPaths(jobContext); - - switch (druidQueryType) { - case Query.TIMESERIES: - case Query.TOPN: - case Query.GROUP_BY: - return new HiveDruidSplit[] { new HiveDruidSplit(address, druidQuery, paths[0]) }; - case Query.SELECT: - return splitSelectQuery(conf, address, druidQuery, paths[0]); - default: - throw new IOException("Druid query type not recognized"); - } - } - - private static String createSelectStarQuery(String address, String dataSource) throws IOException { - // Create Select query - SelectQueryBuilder builder = new Druids.SelectQueryBuilder(); - builder.dataSource(dataSource); - builder.intervals(Arrays.asList(DruidTable.DEFAULT_INTERVAL)); - builder.pagingSpec(PagingSpec.newSpec(1)); - Map context = new HashMap<>(); - context.put(Constants.DRUID_QUERY_FETCH, false); - builder.context(context); - return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(builder.build()); - } - - /* Method that splits Select query depending on the threshold so read can be - * parallelized */ - private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address, - String druidQuery, Path dummyPath) throws IOException { - final int selectThreshold = (int) HiveConf.getIntVar( - conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD); - - SelectQuery query; - try { - query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, SelectQuery.class); - } catch (Exception e) { - throw new IOException(e); - } - - final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false); - if (isFetch) { - // If it has a limit, we use it and we do not split the query - return new HiveDruidSplit[] { new HiveDruidSplit( - address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) }; - } - - // We do not have the number of rows, thus we need to execute a - // Segment Metadata query to obtain number of rows - SegmentMetadataQueryBuilder metadataBuilder = new Druids.SegmentMetadataQueryBuilder(); - metadataBuilder.dataSource(query.getDataSource()); - metadataBuilder.intervals(query.getIntervals()); - metadataBuilder.merge(true); - metadataBuilder.analysisTypes(); - SegmentMetadataQuery metadataQuery = metadataBuilder.build(); - - HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle()); - InputStream response; - try { - response = DruidStorageHandlerUtils.submitRequest(client, - DruidStorageHandlerUtils.createRequest(address, metadataQuery)); - } catch (Exception e) { - throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - - // Retrieve results - List metadataList; - try { - metadataList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response, - new TypeReference>() {}); - } catch (Exception e) { - response.close(); - throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - if (metadataList == null || metadataList.isEmpty()) { - throw new IOException("Connected to Druid but could not retrieve datasource information"); - } - if (metadataList.size() != 1) { - throw new IOException("Information about segments should have been merged"); - } - - final long numRows = metadataList.get(0).getNumRows(); - - query = query.withPagingSpec(PagingSpec.newSpec(Integer.MAX_VALUE)); - if (numRows <= selectThreshold) { - // We are not going to split it - return new HiveDruidSplit[] { new HiveDruidSplit(address, - DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) }; - } - - // If the query does not specify a timestamp, we obtain the total time using - // a Time Boundary query. Then, we use the information to split the query - // following the Select threshold configuration property - final List intervals = new ArrayList<>(); - if (query.getIntervals().size() == 1 && query.getIntervals().get(0).withChronology( - ISOChronology.getInstanceUTC()).equals(DruidTable.DEFAULT_INTERVAL)) { - // Default max and min, we should execute a time boundary query to get a - // more precise range - TimeBoundaryQueryBuilder timeBuilder = new Druids.TimeBoundaryQueryBuilder(); - timeBuilder.dataSource(query.getDataSource()); - TimeBoundaryQuery timeQuery = timeBuilder.build(); - - try { - response = DruidStorageHandlerUtils.submitRequest(client, - DruidStorageHandlerUtils.createRequest(address, timeQuery)); - } catch (Exception e) { - throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - - // Retrieve results - List> timeList; - try { - timeList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response, - new TypeReference>>() {}); - } catch (Exception e) { - response.close(); - throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - if (timeList == null || timeList.isEmpty()) { - throw new IOException("Connected to Druid but could not retrieve time boundary information"); - } - if (timeList.size() != 1) { - throw new IOException("We should obtain a single time boundary"); - } - - intervals.add(new Interval(timeList.get(0).getValue().getMinTime().getMillis(), - timeList.get(0).getValue().getMaxTime().getMillis(), ISOChronology.getInstanceUTC())); - } else { - intervals.addAll(query.getIntervals()); - } - - // Create (numRows/default threshold) input splits - int numSplits = (int) Math.ceil((double) numRows / selectThreshold); - List> newIntervals = createSplitsIntervals(intervals, numSplits); - HiveDruidSplit[] splits = new HiveDruidSplit[numSplits]; - for (int i = 0; i < numSplits; i++) { - // Create partial Select query - final SelectQuery partialQuery = query.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(newIntervals.get(i))); - splits[i] = new HiveDruidSplit(address, - DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath); - } - return splits; - } - - private static List> createSplitsIntervals(List intervals, int numSplits) { - final long totalTime = DruidDateTimeUtils.extractTotalTime(intervals); - long startTime = intervals.get(0).getStartMillis(); - long endTime = startTime; - long currTime = 0; - List> newIntervals = new ArrayList<>(); - for (int i = 0, posIntervals = 0; i < numSplits; i++) { - final long rangeSize = Math.round( (double) (totalTime * (i + 1)) / numSplits) - - Math.round( (double) (totalTime * i) / numSplits); - // Create the new interval(s) - List currentIntervals = new ArrayList<>(); - while (posIntervals < intervals.size()) { - final Interval interval = intervals.get(posIntervals); - final long expectedRange = rangeSize - currTime; - if (interval.getEndMillis() - startTime >= expectedRange) { - endTime = startTime + expectedRange; - currentIntervals.add(new Interval(startTime, endTime, ISOChronology.getInstanceUTC())); - startTime = endTime; - currTime = 0; - break; - } - endTime = interval.getEndMillis(); - currentIntervals.add(new Interval(startTime, endTime, ISOChronology.getInstanceUTC())); - currTime += (endTime - startTime); - startTime = intervals.get(++posIntervals).getStartMillis(); - } - newIntervals.add(currentIntervals); - } - assert endTime == intervals.get(intervals.size()-1).getEndMillis(); - return newIntervals; - } - - @Override - public org.apache.hadoop.mapred.RecordReader getRecordReader( - org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter) - throws IOException { - // We need to provide a different record reader for every type of Druid query. - // The reason is that Druid results format is different for each type. - final DruidQueryRecordReader reader; - final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE); - if (druidQueryType == null) { - reader = new DruidSelectQueryRecordReader(); // By default - reader.initialize((HiveDruidSplit)split, job); - return reader; - } - switch (druidQueryType) { - case Query.TIMESERIES: - reader = new DruidTimeseriesQueryRecordReader(); - break; - case Query.TOPN: - reader = new DruidTopNQueryRecordReader(); - break; - case Query.GROUP_BY: - reader = new DruidGroupByQueryRecordReader(); - break; - case Query.SELECT: - reader = new DruidSelectQueryRecordReader(); - break; - default: - throw new IOException("Druid query type not recognized"); - } - reader.initialize((HiveDruidSplit)split, job); - return reader; - } - - @Override - public RecordReader createRecordReader(InputSplit split, - TaskAttemptContext context) throws IOException, InterruptedException { - // We need to provide a different record reader for every type of Druid query. - // The reason is that Druid results format is different for each type. - final String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE); - if (druidQueryType == null) { - return new DruidSelectQueryRecordReader(); // By default - } - final DruidQueryRecordReader reader; - switch (druidQueryType) { - case Query.TIMESERIES: - reader = new DruidTimeseriesQueryRecordReader(); - break; - case Query.TOPN: - reader = new DruidTopNQueryRecordReader(); - break; - case Query.GROUP_BY: - reader = new DruidGroupByQueryRecordReader(); - break; - case Query.SELECT: - reader = new DruidSelectQueryRecordReader(); - break; - default: - throw new IOException("Druid query type not recognized"); - } - return reader; - } - -} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java deleted file mode 100644 index 3fba5d0..0000000 --- druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.druid; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.FileSplit; - -/** - * Druid split. Its purpose is to trigger query execution in Druid. - */ -public class HiveDruidSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit { - - private String address; - private String druidQuery; - - // required for deserialization - public HiveDruidSplit() { - super((Path) null, 0, 0, (String[]) null); - } - - public HiveDruidSplit(String address, String druidQuery, Path dummyPath) { - super(dummyPath, 0, 0, (String[]) null); - this.address = address; - this.druidQuery = druidQuery; - } - - @Override - public void write(DataOutput out) throws IOException { - super.write(out); - out.writeUTF(address); - out.writeUTF(druidQuery); - } - - @Override - public void readFields(DataInput in) throws IOException { - super.readFields(in); - address = in.readUTF(); - druidQuery = in.readUTF(); - } - - @Override - public long getLength() { - return 0L; - } - - @Override - public String[] getLocations() { - return new String[] {""} ; - } - - public String getAddress() { - return address; - } - - public String getDruidQuery() { - return druidQuery; - } - - @Override - public String toString() { - return "HiveDruidSplit{" + address + ", " + druidQuery + "}"; - } - -} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java new file mode 100644 index 0000000..2d9c25f --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidOutputFormat.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.druid.io; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.Granularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.granularity.GranularitySpec; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.plumber.CustomVersioningPolicy; +import io.druid.storage.hdfs.HdfsDataSegmentPusher; +import io.druid.storage.hdfs.HdfsDataSegmentPusherConfig; +import org.apache.calcite.adapter.druid.DruidTable; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; +import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.util.Progressable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hadoop.hive.druid.DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME; + +public class DruidOutputFormat implements HiveOutputFormat { + + protected static final Logger LOG = LoggerFactory.getLogger(DruidOutputFormat.class); + + @Override + public FileSinkOperator.RecordWriter getHiveRecordWriter( + JobConf jc, + Path finalOutPath, + Class valueClass, + boolean isCompressed, + Properties tableProperties, + Progressable progress + ) throws IOException { + + final String segmentGranularity = + tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY) != null ? + tableProperties.getProperty(Constants.DRUID_SEGMENT_GRANULARITY) : + HiveConf.getVar(jc, HiveConf.ConfVars.HIVE_DRUID_INDEXING_GRANULARITY); + final String dataSource = tableProperties.getProperty(Constants.DRUID_DATA_SOURCE); + final String segmentDirectory = + tableProperties.getProperty(Constants.DRUID_SEGMENT_DIRECTORY) != null + ? tableProperties.getProperty(Constants.DRUID_SEGMENT_DIRECTORY) + : HiveConf.getVar(jc, HiveConf.ConfVars.DRUID_SEGMENT_DIRECTORY); + + final HdfsDataSegmentPusherConfig hdfsDataSegmentPusherConfig = new HdfsDataSegmentPusherConfig(); + hdfsDataSegmentPusherConfig.setStorageDirectory(segmentDirectory); + final DataSegmentPusher hdfsDataSegmentPusher = new HdfsDataSegmentPusher( + hdfsDataSegmentPusherConfig, jc, DruidStorageHandlerUtils.JSON_MAPPER); + + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularity.valueOf(segmentGranularity), + null, + null + ); + + final String columnNameProperty = tableProperties.getProperty(serdeConstants.LIST_COLUMNS); + final String columnTypeProperty = tableProperties.getProperty(serdeConstants.LIST_COLUMN_TYPES); + + if (StringUtils.isEmpty(columnNameProperty) || StringUtils.isEmpty(columnTypeProperty)) { + throw new IllegalStateException( + String.format("List of columns names [%s] or columns type [%s] is/are not present", + columnNameProperty, columnTypeProperty + )); + } + ArrayList columnNames = new ArrayList(); + for (String name : columnNameProperty.split(",")) { + columnNames.add(name); + } + if (!columnNames.contains(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) { + throw new IllegalStateException("Timestamp column (' " + DruidTable.DEFAULT_TIMESTAMP_COLUMN + + "') not specified in create table; list of columns is : " + + tableProperties.getProperty(serdeConstants.LIST_COLUMNS)); + } + ArrayList columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty); + + // Default, all columns that are not metrics or timestamp, are treated as dimensions + final List dimensions = new ArrayList<>(); + ImmutableList.Builder aggregatorFactoryBuilder = ImmutableList.builder(); + for (int i = 0; i < columnTypes.size(); i++) { + TypeInfo f = columnTypes.get(i); + assert f.getCategory() == ObjectInspector.Category.PRIMITIVE; + AggregatorFactory af; + switch (f.getTypeName()) { + case serdeConstants.TINYINT_TYPE_NAME: + case serdeConstants.SMALLINT_TYPE_NAME: + case serdeConstants.INT_TYPE_NAME: + case serdeConstants.BIGINT_TYPE_NAME: + af = new LongSumAggregatorFactory(columnNames.get(i), columnNames.get(i)); + break; + case serdeConstants.FLOAT_TYPE_NAME: + case serdeConstants.DOUBLE_TYPE_NAME: + af = new DoubleSumAggregatorFactory(columnNames.get(i), columnNames.get(i)); + break; + default: + // Dimension or timestamp + String columnName = columnNames.get(i); + if (!columnName.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN) && !columnName + .equals(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME)) { + dimensions.add(new StringDimensionSchema(columnName)); + } + continue; + } + aggregatorFactoryBuilder.add(af); + } + List aggregatorFactories = aggregatorFactoryBuilder.build(); + final InputRowParser inputRowParser = new MapInputRowParser(new TimeAndDimsParseSpec( + new TimestampSpec(DruidTable.DEFAULT_TIMESTAMP_COLUMN, "auto", null), + new DimensionsSpec(dimensions, + Lists.newArrayList(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME), null + ) + )); + + Map inputParser = DruidStorageHandlerUtils.JSON_MAPPER + .convertValue(inputRowParser, Map.class); + + final DataSchema dataSchema = new DataSchema( + Preconditions.checkNotNull(dataSource, "Data source name is null"), + inputParser, + aggregatorFactories.toArray(new AggregatorFactory[aggregatorFactories.size()]), + granularitySpec, + DruidStorageHandlerUtils.JSON_MAPPER + ); + + Integer maxPartitionSize = HiveConf + .getIntVar(jc, HiveConf.ConfVars.HIVE_DRUID_MAX_PARTITION_SIZE); + String basePersistDirectory = HiveConf + .getVar(jc, HiveConf.ConfVars.HIVE_DRUID_BASE_PERSIST_DIRECTORY); + final RealtimeTuningConfig realtimeTuningConfig = RealtimeTuningConfig + .makeDefaultTuningConfig(new File( + basePersistDirectory)).withVersioningPolicy(new CustomVersioningPolicy(null)); + + LOG.debug(String.format("running with Data schema [%s] ", dataSchema)); + return new DruidRecordWriter( + dataSchema, + realtimeTuningConfig, + hdfsDataSegmentPusher, maxPartitionSize, + makeSegmentDescriptorOutputDir(finalOutPath), + finalOutPath.getFileSystem(jc) + ); + } + + private Path makeSegmentDescriptorOutputDir(Path finalOutPath) { + return new Path(finalOutPath, SEGMENTS_DESCRIPTOR_DIR_NAME); + } + + @Override + public RecordWriter getRecordWriter( + FileSystem ignored, JobConf job, String name, Progressable progress + ) throws IOException { + throw new UnsupportedOperationException("please implement me !"); + } + + @Override + public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException { + throw new UnsupportedOperationException("not implemented yet"); + } +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java new file mode 100644 index 0000000..bbe8656 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidQueryBasedInputFormat.java @@ -0,0 +1,391 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.druid.io; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; +import io.druid.query.Druids; +import io.druid.query.Druids.SegmentMetadataQueryBuilder; +import io.druid.query.Druids.SelectQueryBuilder; +import io.druid.query.Druids.TimeBoundaryQueryBuilder; +import io.druid.query.Query; +import io.druid.query.Result; +import io.druid.query.metadata.metadata.SegmentAnalysis; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.select.PagingSpec; +import io.druid.query.select.SelectQuery; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import org.apache.calcite.adapter.druid.DruidDateTimeUtils; +import org.apache.calcite.adapter.druid.DruidTable; +import org.apache.commons.lang3.StringEscapeUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; +import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader; +import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Druid query based input format. + * + * Given a query and the Druid broker address, it will send it, and retrieve + * and parse the results. + */ +public class DruidQueryBasedInputFormat extends InputFormat + implements org.apache.hadoop.mapred.InputFormat { + + protected static final Logger LOG = LoggerFactory.getLogger(DruidQueryBasedInputFormat.class); + + @Override + public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits) + throws IOException { + return getInputSplits(job); + } + + @Override + public List getSplits(JobContext context) throws IOException, InterruptedException { + return Arrays.asList(getInputSplits(context.getConfiguration())); + } + + @SuppressWarnings("deprecation") + private HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException { + String address = HiveConf.getVar(conf, + HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS + ); + if (StringUtils.isEmpty(address)) { + throw new IOException("Druid broker address not specified in configuration"); + } + String druidQuery = StringEscapeUtils.unescapeJava(conf.get(Constants.DRUID_QUERY_JSON)); + String druidQueryType; + if (StringUtils.isEmpty(druidQuery)) { + // Empty, maybe because CBO did not run; we fall back to + // full Select query + if (LOG.isWarnEnabled()) { + LOG.warn("Druid query is empty; creating Select query"); + } + String dataSource = conf.get(Constants.DRUID_DATA_SOURCE); + if (dataSource == null) { + throw new IOException("Druid data source cannot be empty"); + } + druidQuery = createSelectStarQuery(dataSource); + druidQueryType = Query.SELECT; + } else { + druidQueryType = conf.get(Constants.DRUID_QUERY_TYPE); + if (druidQueryType == null) { + throw new IOException("Druid query type not recognized"); + } + } + + // hive depends on FileSplits + Job job = new Job(conf); + JobContext jobContext = ShimLoader.getHadoopShims().newJobContext(job); + Path[] paths = FileInputFormat.getInputPaths(jobContext); + + switch (druidQueryType) { + case Query.TIMESERIES: + case Query.TOPN: + case Query.GROUP_BY: + return new HiveDruidSplit[] { new HiveDruidSplit(address, druidQuery, paths[0]) }; + case Query.SELECT: + return splitSelectQuery(conf, address, druidQuery, paths[0]); + default: + throw new IOException("Druid query type not recognized"); + } + } + + private static String createSelectStarQuery(String dataSource) throws IOException { + // Create Select query + SelectQueryBuilder builder = new Druids.SelectQueryBuilder(); + builder.dataSource(dataSource); + builder.intervals(Arrays.asList(DruidTable.DEFAULT_INTERVAL)); + builder.pagingSpec(PagingSpec.newSpec(1)); + Map context = new HashMap<>(); + context.put(Constants.DRUID_QUERY_FETCH, false); + builder.context(context); + return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(builder.build()); + } + + /* Method that splits Select query depending on the threshold so read can be + * parallelized */ + private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address, + String druidQuery, Path dummyPath + ) throws IOException { + final int selectThreshold = (int) HiveConf.getIntVar( + conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD); + + SelectQuery query; + try { + query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, SelectQuery.class); + } catch (Exception e) { + throw new IOException(e); + } + + final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false); + if (isFetch) { + // If it has a limit, we use it and we do not split the query + return new HiveDruidSplit[] { new HiveDruidSplit( + address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) }; + } + + // We do not have the number of rows, thus we need to execute a + // Segment Metadata query to obtain number of rows + SegmentMetadataQueryBuilder metadataBuilder = new Druids.SegmentMetadataQueryBuilder(); + metadataBuilder.dataSource(query.getDataSource()); + metadataBuilder.intervals(query.getIntervals()); + metadataBuilder.merge(true); + metadataBuilder.analysisTypes(); + SegmentMetadataQuery metadataQuery = metadataBuilder.build(); + final Lifecycle lifecycle = new Lifecycle(); + HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), lifecycle); + try { + lifecycle.start(); + } catch (Exception e) { + LOG.error("Lifecycle start issue", e); + } + InputStream response; + try { + response = DruidStorageHandlerUtils.submitRequest(client, + DruidStorageHandlerUtils.createRequest(address, metadataQuery) + ); + } catch (Exception e) { + throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); + } finally { + lifecycle.stop(); + } + + // Retrieve results + List metadataList; + try { + metadataList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response, + new TypeReference>() { + } + ); + } catch (Exception e) { + response.close(); + throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); + } + if (metadataList == null || metadataList.isEmpty()) { + throw new IOException("Connected to Druid but could not retrieve datasource information"); + } + if (metadataList.size() != 1) { + throw new IOException("Information about segments should have been merged"); + } + + final long numRows = metadataList.get(0).getNumRows(); + + query = query.withPagingSpec(PagingSpec.newSpec(Integer.MAX_VALUE)); + if (numRows <= selectThreshold) { + // We are not going to split it + return new HiveDruidSplit[] { new HiveDruidSplit(address, + DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath + ) }; + } + + // If the query does not specify a timestamp, we obtain the total time using + // a Time Boundary query. Then, we use the information to split the query + // following the Select threshold configuration property + final List intervals = new ArrayList<>(); + if (query.getIntervals().size() == 1 && query.getIntervals().get(0).withChronology( + ISOChronology.getInstanceUTC()).equals(DruidTable.DEFAULT_INTERVAL)) { + // Default max and min, we should execute a time boundary query to get a + // more precise range + TimeBoundaryQueryBuilder timeBuilder = new Druids.TimeBoundaryQueryBuilder(); + timeBuilder.dataSource(query.getDataSource()); + TimeBoundaryQuery timeQuery = timeBuilder.build(); + + try { + response = DruidStorageHandlerUtils.submitRequest(client, + DruidStorageHandlerUtils.createRequest(address, timeQuery) + ); + } catch (Exception e) { + throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); + } + + // Retrieve results + List> timeList; + try { + timeList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response, + new TypeReference>>() { + } + ); + } catch (Exception e) { + response.close(); + throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e)); + } + if (timeList == null || timeList.isEmpty()) { + throw new IOException( + "Connected to Druid but could not retrieve time boundary information"); + } + if (timeList.size() != 1) { + throw new IOException("We should obtain a single time boundary"); + } + + intervals.add(new Interval(timeList.get(0).getValue().getMinTime().getMillis(), + timeList.get(0).getValue().getMaxTime().getMillis(), ISOChronology.getInstanceUTC() + )); + } else { + intervals.addAll(query.getIntervals()); + } + + // Create (numRows/default threshold) input splits + int numSplits = (int) Math.ceil((double) numRows / selectThreshold); + List> newIntervals = createSplitsIntervals(intervals, numSplits); + HiveDruidSplit[] splits = new HiveDruidSplit[numSplits]; + for (int i = 0; i < numSplits; i++) { + // Create partial Select query + final SelectQuery partialQuery = query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(newIntervals.get(i))); + splits[i] = new HiveDruidSplit(address, + DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath + ); + } + return splits; + } + + private static List> createSplitsIntervals(List intervals, int numSplits + ) { + final long totalTime = DruidDateTimeUtils.extractTotalTime(intervals); + long startTime = intervals.get(0).getStartMillis(); + long endTime = startTime; + long currTime = 0; + List> newIntervals = new ArrayList<>(); + for (int i = 0, posIntervals = 0; i < numSplits; i++) { + final long rangeSize = Math.round((double) (totalTime * (i + 1)) / numSplits) - + Math.round((double) (totalTime * i) / numSplits); + // Create the new interval(s) + List currentIntervals = new ArrayList<>(); + while (posIntervals < intervals.size()) { + final Interval interval = intervals.get(posIntervals); + final long expectedRange = rangeSize - currTime; + if (interval.getEndMillis() - startTime >= expectedRange) { + endTime = startTime + expectedRange; + currentIntervals.add(new Interval(startTime, endTime, ISOChronology.getInstanceUTC())); + startTime = endTime; + currTime = 0; + break; + } + endTime = interval.getEndMillis(); + currentIntervals.add(new Interval(startTime, endTime, ISOChronology.getInstanceUTC())); + currTime += (endTime - startTime); + startTime = intervals.get(++posIntervals).getStartMillis(); + } + newIntervals.add(currentIntervals); + } + assert endTime == intervals.get(intervals.size() - 1).getEndMillis(); + return newIntervals; + } + + @Override + public org.apache.hadoop.mapred.RecordReader getRecordReader( + org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter + ) + throws IOException { + // We need to provide a different record reader for every type of Druid query. + // The reason is that Druid results format is different for each type. + final DruidQueryRecordReader reader; + final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE); + if (druidQueryType == null) { + reader = new DruidSelectQueryRecordReader(); // By default + reader.initialize((HiveDruidSplit) split, job); + return reader; + } + switch (druidQueryType) { + case Query.TIMESERIES: + reader = new DruidTimeseriesQueryRecordReader(); + break; + case Query.TOPN: + reader = new DruidTopNQueryRecordReader(); + break; + case Query.GROUP_BY: + reader = new DruidGroupByQueryRecordReader(); + break; + case Query.SELECT: + reader = new DruidSelectQueryRecordReader(); + break; + default: + throw new IOException("Druid query type not recognized"); + } + reader.initialize((HiveDruidSplit) split, job); + return reader; + } + + @Override + public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext context + ) throws IOException, InterruptedException { + // We need to provide a different record reader for every type of Druid query. + // The reason is that Druid results format is different for each type. + final String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE); + if (druidQueryType == null) { + return new DruidSelectQueryRecordReader(); // By default + } + final DruidQueryRecordReader reader; + switch (druidQueryType) { + case Query.TIMESERIES: + reader = new DruidTimeseriesQueryRecordReader(); + break; + case Query.TOPN: + reader = new DruidTopNQueryRecordReader(); + break; + case Query.GROUP_BY: + reader = new DruidGroupByQueryRecordReader(); + break; + case Query.SELECT: + reader = new DruidSelectQueryRecordReader(); + break; + default: + throw new IOException("Druid query type not recognized"); + } + return reader; + } + +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java new file mode 100644 index 0000000..af6c611 --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/DruidRecordWriter.java @@ -0,0 +1,266 @@ +package org.apache.hadoop.hive.druid.io; + +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Throwables; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; +import io.druid.data.input.Committer; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.java.util.common.Granularity; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.realtime.FireDepartmentMetrics; +import io.druid.segment.realtime.appenderator.Appenderator; +import io.druid.segment.realtime.appenderator.DefaultOfflineAppenderatorFactory; +import io.druid.segment.realtime.appenderator.SegmentIdentifier; +import io.druid.segment.realtime.appenderator.SegmentNotWritableException; +import io.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import io.druid.segment.realtime.plumber.Committers; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.LinearShardSpec; +import org.apache.calcite.adapter.druid.DruidTable; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; +import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.concurrent.ExecutionException; + +public class DruidRecordWriter implements RecordWriter, + org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter { + protected static final Logger LOG = LoggerFactory.getLogger(DruidRecordWriter.class); + + private final DataSchema dataSchema; + + private final Appenderator appenderator; + + private final RealtimeTuningConfig tuningConfig; + + private final Path segmentsDescriptorDir; + + private SegmentIdentifier currentOpenSegment = null; + + private final Integer maxPartitionSize; + + private final FileSystem fileSystem; + + private final Supplier committerSupplier; + + public DruidRecordWriter( + DataSchema dataSchema, + RealtimeTuningConfig realtimeTuningConfig, + DataSegmentPusher dataSegmentPusher, + int maxPartitionSize, + final Path segmentsDescriptorsDir, + final FileSystem fileSystem + ) { + DefaultOfflineAppenderatorFactory defaultOfflineAppenderatorFactory = new DefaultOfflineAppenderatorFactory( + Preconditions.checkNotNull(dataSegmentPusher, "dataSegmentPusher is null"), + DruidStorageHandlerUtils.JSON_MAPPER, + DruidStorageHandlerUtils.INDEX_IO, + DruidStorageHandlerUtils.INDEX_MERGER_V9 + ); + this.tuningConfig = Preconditions + .checkNotNull(realtimeTuningConfig, "realtimeTuningConfig is null"); + this.dataSchema = Preconditions.checkNotNull(dataSchema, "data schema is null"); + appenderator = defaultOfflineAppenderatorFactory.build( + this.dataSchema, + tuningConfig, + new FireDepartmentMetrics() + ); + Preconditions.checkArgument(maxPartitionSize > 0, "maxPartitionSize need to be greater than 0"); + this.maxPartitionSize = maxPartitionSize; + appenderator.startJob(); // maybe we need to move this out of the constructor + this.segmentsDescriptorDir = Preconditions + .checkNotNull(segmentsDescriptorsDir, "segmentsDescriptorsDir is null"); + this.fileSystem = Preconditions.checkNotNull(fileSystem, "file system is null"); + committerSupplier = Suppliers.ofInstance(Committers.nil()); + LOG.debug(String.format("Data schema is [%s]", this.dataSchema)); + } + + /** + * This function computes the segment identifier and push the current open segment if max size is reached or the event belongs to the next interval. + * Note that this function assumes that timestamps are pseudo sorted. + * This function will close and move to the next segment granularity as soon as it we get an event from the next interval. + * The sorting is done by the previous stage. + * + * @return segmentIdentifier with respect to the timestamp and maybe push the current open segment. + */ + private SegmentIdentifier getSegmentIdentifierAndMaybePush(long truncatedTime) { + + final Granularity segmentGranularity = dataSchema.getGranularitySpec() + .getSegmentGranularity(); + + final Interval interval = new Interval( + new DateTime(truncatedTime), + segmentGranularity.increment(new DateTime(truncatedTime)) + ); + + SegmentIdentifier retVal; + if (currentOpenSegment == null) { + retVal = new SegmentIdentifier( + dataSchema.getDataSource(), + interval, + tuningConfig.getVersioningPolicy().getVersion(interval), + new LinearShardSpec(0) + ); + currentOpenSegment = retVal; + return retVal; + } else if (currentOpenSegment.getInterval().equals(interval)) { + retVal = currentOpenSegment; + int rowCount = appenderator.getRowCount(retVal); + if (rowCount < maxPartitionSize) { + return retVal; + } else { + retVal = new SegmentIdentifier( + dataSchema.getDataSource(), + interval, + tuningConfig.getVersioningPolicy().getVersion(interval), + new LinearShardSpec(currentOpenSegment.getShardSpec().getPartitionNum() + 1) + ); + pushSegments(Lists.newArrayList(currentOpenSegment)); + currentOpenSegment = retVal; + return retVal; + } + } else { + retVal = new SegmentIdentifier( + dataSchema.getDataSource(), + interval, + tuningConfig.getVersioningPolicy().getVersion(interval), + new LinearShardSpec(0) + ); + pushSegments(Lists.newArrayList(currentOpenSegment)); + currentOpenSegment = retVal; + return retVal; + } + } + + private void pushSegments(List segmentsToPush) { + try { + SegmentsAndMetadata segmentsAndMetadata = appenderator + .push(segmentsToPush, committerSupplier.get()).get(); + final HashSet pushedSegmentIdentifierHashSet = new HashSet<>(); + + for (DataSegment pushedSegment : segmentsAndMetadata.getSegments()) { + pushedSegmentIdentifierHashSet + .add(SegmentIdentifier.fromDataSegment(pushedSegment).getIdentifierAsString()); + final Path segmentDescriptorOutputPath = DruidStorageHandlerUtils + .makeSegmentDescriptorOutputPath(pushedSegment, segmentsDescriptorDir); + DruidStorageHandlerUtils + .writeSegmentDescriptor(fileSystem, pushedSegment, segmentDescriptorOutputPath); + + LOG.info( + String.format( + "Pushed the segment [%s] and persisted the descriptor located at [%s]", + pushedSegment, + segmentDescriptorOutputPath + ) + ); + } + + final HashSet toPushSegmentsHashSet = new HashSet( + FluentIterable.from(segmentsToPush) + .transform(new Function() { + @Nullable + @Override + public String apply( + @Nullable SegmentIdentifier input + ) { + return input.getIdentifierAsString(); + } + }) + .toList()); + + if (!pushedSegmentIdentifierHashSet.equals(toPushSegmentsHashSet)) { + throw new IllegalStateException(String.format( + "was asked to publish [%s] but was able to publish only [%s]", + Joiner.on(", ").join(toPushSegmentsHashSet), + Joiner.on(", ").join(pushedSegmentIdentifierHashSet) + )); + } + + LOG.info(String.format("Published [%,d] segments.", segmentsToPush.size())); + } catch (InterruptedException e) { + LOG.error(String.format("got interrupted, failed to push [%,d] segments.", + segmentsToPush.size() + ), e); + Thread.currentThread().interrupt(); + } catch (IOException | ExecutionException e) { + LOG.error(String.format("Failed to push [%,d] segments.", segmentsToPush.size()), e); + Throwables.propagate(e); + } + } + + @Override + public void write(Writable w) throws IOException { + if (w == null) { + return; + } + DruidWritable record = (DruidWritable) w; + final long timestamp = (long) record.getValue().get(DruidTable.DEFAULT_TIMESTAMP_COLUMN); + final long truncatedTime = (long) record.getValue() + .get(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME); + + InputRow inputRow = new MapBasedInputRow( + timestamp, + dataSchema.getParser() + .getParseSpec() + .getDimensionsSpec() + .getDimensionNames(), + record.getValue() + ); + + try { + appenderator + .add(getSegmentIdentifierAndMaybePush(truncatedTime), inputRow, committerSupplier); + } catch (SegmentNotWritableException e) { + throw new IOException(e); + } + } + + @Override + public void close(boolean abort) throws IOException { + try { + if (abort == false) { + final List segmentsToPush = Lists.newArrayList(); + segmentsToPush.addAll(appenderator.getSegments()); + pushSegments(segmentsToPush); + } + appenderator.clear(); + } catch (InterruptedException e) { + Throwables.propagate(e); + } finally { + appenderator.close(); + } + } + + @Override + public void write(NullWritable key, DruidWritable value) throws IOException { + this.write(value); + } + + @Override + public void close(Reporter reporter) throws IOException { + this.close(true); + } + +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java new file mode 100644 index 0000000..861075d --- /dev/null +++ druid-handler/src/java/org/apache/hadoop/hive/druid/io/HiveDruidSplit.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.druid.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; + +/** + * Druid split. Its purpose is to trigger query execution in Druid. + */ +public class HiveDruidSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit { + + private String address; + + private String druidQuery; + + // required for deserialization + public HiveDruidSplit() { + super((Path) null, 0, 0, (String[]) null); + } + + public HiveDruidSplit(String address, String druidQuery, Path dummyPath) { + super(dummyPath, 0, 0, (String[]) null); + this.address = address; + this.druidQuery = druidQuery; + } + + @Override + public void write(DataOutput out) throws IOException { + super.write(out); + out.writeUTF(address); + out.writeUTF(druidQuery); + } + + @Override + public void readFields(DataInput in) throws IOException { + super.readFields(in); + address = in.readUTF(); + druidQuery = in.readUTF(); + } + + @Override + public long getLength() { + return 0L; + } + + @Override + public String[] getLocations() { + return new String[] { "" }; + } + + public String getAddress() { + return address; + } + + public String getDruidQuery() { + return druidQuery; + } + + @Override + public String toString() { + return "HiveDruidSplit{" + address + ", " + druidQuery + "}"; + } + +} diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java index f97f820..9e8b439 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java @@ -42,7 +42,9 @@ extends DruidQueryRecordReader { private Row current; + private int[] indexes = new int[0]; + // Row objects returned by GroupByQuery have different access paths depending on // whether the result for the metric is a Float or a Long, thus we keep track // using these converters @@ -62,11 +64,14 @@ protected GroupByQuery createQuery(String content) throws IOException { @Override protected List createResultsList(InputStream content) throws IOException { return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content, - new TypeReference>(){}); + new TypeReference>() { + } + ); } private void initExtractors() throws IOException { - extractors = new Extract[query.getAggregatorSpecs().size() + query.getPostAggregatorSpecs().size()]; + extractors = new Extract[query.getAggregatorSpecs().size() + query.getPostAggregatorSpecs() + .size()]; int counter = 0; for (int i = 0; i < query.getAggregatorSpecs().size(); i++, counter++) { AggregatorFactory af = query.getAggregatorSpecs().get(i); @@ -103,7 +108,7 @@ public boolean nextKeyValue() { if (results.hasNext()) { current = results.next(); indexes = new int[query.getDimensions().size()]; - for (int i=0; i < query.getDimensions().size(); i++) { + for (int i = 0; i < query.getDimensions().size(); i++) { DimensionSpec ds = query.getDimensions().get(i); indexes[i] = current.getDimension(ds.getDimension()).size() - 1; } @@ -124,7 +129,7 @@ public DruidWritable getCurrentValue() throws IOException, InterruptedException // 1) The timestamp column value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis()); // 2) The dimension columns - for (int i=0; i < query.getDimensions().size(); i++) { + for (int i = 0; i < query.getDimensions().size(); i++) { DimensionSpec ds = query.getDimensions().get(i); List dims = current.getDimension(ds.getDimension()); if (dims.size() == 0) { @@ -163,7 +168,7 @@ public boolean next(NullWritable key, DruidWritable value) { // 1) The timestamp column value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis()); // 2) The dimension columns - for (int i=0; i < query.getDimensions().size(); i++) { + for (int i = 0; i < query.getDimensions().size(); i++) { DimensionSpec ds = query.getDimensions().get(i); List dims = current.getDimension(ds.getDimension()); if (dims.size() == 0) { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java index 96bcee87..9e0db0c 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java @@ -17,14 +17,15 @@ */ package org.apache.hadoop.hive.druid.serde; -import java.io.IOException; -import java.io.InputStream; -import java.util.Iterator; -import java.util.List; - +import com.google.common.collect.Iterators; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; +import io.druid.query.BaseQuery; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; -import org.apache.hadoop.hive.druid.HiveDruidSplit; +import org.apache.hadoop.hive.druid.io.HiveDruidSplit; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; @@ -32,24 +33,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Iterators; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; - -import io.druid.query.BaseQuery; +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.List; /** * Base record reader for given a Druid query. This class contains the logic to * send the query to the broker and retrieve the results. The transformation to * emit records needs to be done by the classes that extend the reader. - * + * * The key for each record will be a NullWritable, while the value will be a * DruidWritable containing the timestamp as well as all values resulting from * the query. */ -public abstract class DruidQueryRecordReader,R extends Comparable> +public abstract class DruidQueryRecordReader, R extends Comparable> extends RecordReader implements org.apache.hadoop.mapred.RecordReader { @@ -81,10 +79,17 @@ public void initialize(InputSplit split, Configuration conf) throws IOException LOG.info("Retrieving from druid using query:\n " + query); } - HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle()); + final Lifecycle lifecycle = new Lifecycle(); + HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), lifecycle); + try { + lifecycle.start(); + } catch (Exception e) { + LOG.error("Issues with lifecycle start", e); + } InputStream response = DruidStorageHandlerUtils.submitRequest(client, - DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getAddress(), query)); - + DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getAddress(), query) + ); + lifecycle.stop(); // Retrieve results List resultsList; try { diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java index c30ac56..8a41e91 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java @@ -41,6 +41,7 @@ extends DruidQueryRecordReader> { private Result current; + private Iterator values = Iterators.emptyIterator(); @Override @@ -49,9 +50,12 @@ protected SelectQuery createQuery(String content) throws IOException { } @Override - protected List> createResultsList(InputStream content) throws IOException { + protected List> createResultsList(InputStream content) + throws IOException { return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content, - new TypeReference>>(){}); + new TypeReference>>() { + } + ); } @Override diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java index 238f7a3..8811294 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java @@ -17,19 +17,33 @@ */ package org.apache.hadoop.hive.druid.serde; -import java.io.IOException; -import java.io.InputStream; -import java.sql.Timestamp; -import java.util.ArrayList; -import java.util.List; -import java.util.Map.Entry; -import java.util.Properties; - +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.metamx.common.lifecycle.Lifecycle; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.HttpClientConfig; +import com.metamx.http.client.HttpClientInit; +import io.druid.query.Druids; +import io.druid.query.Druids.SegmentMetadataQueryBuilder; +import io.druid.query.Query; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.metadata.metadata.ColumnAnalysis; +import io.druid.query.metadata.metadata.SegmentAnalysis; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.select.SelectQuery; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.topn.TopNQuery; import org.apache.calcite.adapter.druid.DruidTable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.AbstractSerDe; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.SerDeSpec; @@ -37,51 +51,48 @@ import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.DoubleWritable; import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.collect.Lists; -import com.metamx.common.lifecycle.Lifecycle; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.HttpClientConfig; -import com.metamx.http.client.HttpClientInit; - -import io.druid.query.Druids; -import io.druid.query.Druids.SegmentMetadataQueryBuilder; -import io.druid.query.Query; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.PostAggregator; -import io.druid.query.dimension.DimensionSpec; -import io.druid.query.groupby.GroupByQuery; -import io.druid.query.metadata.metadata.ColumnAnalysis; -import io.druid.query.metadata.metadata.SegmentAnalysis; -import io.druid.query.metadata.metadata.SegmentMetadataQuery; -import io.druid.query.select.SelectQuery; -import io.druid.query.timeseries.TimeseriesQuery; -import io.druid.query.topn.TopNQuery; +import java.io.IOException; +import java.io.InputStream; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; /** * DruidSerDe that is used to deserialize objects from a Druid data source. */ -@SerDeSpec(schemaProps = {Constants.DRUID_DATA_SOURCE}) +@SerDeSpec(schemaProps = { Constants.DRUID_DATA_SOURCE }) public class DruidSerDe extends AbstractSerDe { protected static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class); private String[] columns; + private PrimitiveTypeInfo[] types; - private ObjectInspector inspector; + private ObjectInspector inspector; @Override public void initialize(Configuration configuration, Properties properties) throws SerDeException { @@ -92,52 +103,88 @@ public void initialize(Configuration configuration, Properties properties) throw // Druid query String druidQuery = properties.getProperty(Constants.DRUID_QUERY_JSON); if (druidQuery == null) { - // No query. We need to create a Druid Segment Metadata query that retrieves all - // columns present in the data source (dimensions and metrics). - // Create Segment Metadata Query - String dataSource = properties.getProperty(Constants.DRUID_DATA_SOURCE); - if (dataSource == null) { - throw new SerDeException("Druid data source not specified; use " + - Constants.DRUID_DATA_SOURCE + " in table properties"); - } - SegmentMetadataQueryBuilder builder = new Druids.SegmentMetadataQueryBuilder(); - builder.dataSource(dataSource); - builder.merge(true); - builder.analysisTypes(); - SegmentMetadataQuery query = builder.build(); + // No query. Either it is a CTAS, or we need to create a Druid + // Segment Metadata query that retrieves all columns present in + // the data source (dimensions and metrics). + if (!org.apache.commons.lang3.StringUtils + .isEmpty(properties.getProperty(serdeConstants.LIST_COLUMNS)) + && !org.apache.commons.lang3.StringUtils + .isEmpty(properties.getProperty(serdeConstants.LIST_COLUMN_TYPES))) { + columnNames.addAll(Utilities.getColumnNames(properties)); + if (!columnNames.contains(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) { + throw new SerDeException("Timestamp column (' " + DruidTable.DEFAULT_TIMESTAMP_COLUMN + + "') not specified in create table; list of columns is : " + + properties.getProperty(serdeConstants.LIST_COLUMNS)); + } + columnTypes.addAll(Lists.transform(Utilities.getColumnTypes(properties), + new Function() { + @Override + public PrimitiveTypeInfo apply(String type) { + return TypeInfoFactory.getPrimitiveTypeInfo(type); + } + } + )); + inspectors.addAll(Lists.transform(columnTypes, + new Function() { + @Override + public ObjectInspector apply(PrimitiveTypeInfo type) { + return PrimitiveObjectInspectorFactory + .getPrimitiveWritableObjectInspector(type); + } + } + )); + columns = columnNames.toArray(new String[columnNames.size()]); + types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); + inspector = ObjectInspectorFactory + .getStandardStructObjectInspector(columnNames, inspectors); + } else { + String dataSource = properties.getProperty(Constants.DRUID_DATA_SOURCE); + if (dataSource == null) { + throw new SerDeException("Druid data source not specified; use " + + Constants.DRUID_DATA_SOURCE + " in table properties"); + } + SegmentMetadataQueryBuilder builder = new Druids.SegmentMetadataQueryBuilder(); + builder.dataSource(dataSource); + builder.merge(true); + builder.analysisTypes(); + SegmentMetadataQuery query = builder.build(); - // Execute query in Druid - String address = HiveConf.getVar(configuration, - HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS); - if (org.apache.commons.lang3.StringUtils.isEmpty(address)) { - throw new SerDeException("Druid broker address not specified in configuration"); - } + // Execute query in Druid + String address = HiveConf.getVar(configuration, + HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS + ); + if (org.apache.commons.lang3.StringUtils.isEmpty(address)) { + throw new SerDeException("Druid broker address not specified in configuration"); + } - // Infer schema - SegmentAnalysis schemaInfo; - try { - schemaInfo = submitMetadataRequest(address, query); - } catch (IOException e) { - throw new SerDeException(e); - } - for (Entry columnInfo : schemaInfo.getColumns().entrySet()) { - if (columnInfo.getKey().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) { - // Special handling for timestamp column + // Infer schema + SegmentAnalysis schemaInfo; + try { + schemaInfo = submitMetadataRequest(address, query); + } catch (IOException e) { + throw new SerDeException(e); + } + for (Entry columnInfo : schemaInfo.getColumns().entrySet()) { + if (columnInfo.getKey().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) { + // Special handling for timestamp column + columnNames.add(columnInfo.getKey()); // field name + PrimitiveTypeInfo type = TypeInfoFactory.timestampTypeInfo; // field type + columnTypes.add(type); + inspectors + .add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type)); + continue; + } columnNames.add(columnInfo.getKey()); // field name - PrimitiveTypeInfo type = TypeInfoFactory.timestampTypeInfo; // field type + PrimitiveTypeInfo type = DruidSerDeUtils.convertDruidToHiveType( + columnInfo.getValue().getType()); // field type columnTypes.add(type); inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type)); - continue; } - columnNames.add(columnInfo.getKey()); // field name - PrimitiveTypeInfo type = DruidSerDeUtils.convertDruidToHiveType( - columnInfo.getValue().getType()); // field type - columnTypes.add(type); - inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type)); + columns = columnNames.toArray(new String[columnNames.size()]); + types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); + inspector = ObjectInspectorFactory + .getStandardStructObjectInspector(columnNames, inspectors); } - columns = columnNames.toArray(new String[columnNames.size()]); - types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]); - inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } else { // Query is specified, we can extract the results schema from the query Query query; @@ -163,13 +210,14 @@ public void initialize(Configuration configuration, Properties properties) throw default: throw new SerDeException("Not supported Druid query"); } - + columns = new String[columnNames.size()]; types = new PrimitiveTypeInfo[columnNames.size()]; for (int i = 0; i < columnTypes.size(); ++i) { columns[i] = columnNames.get(i); types[i] = columnTypes.get(i); - inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(types[i])); + inspectors + .add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(types[i])); } inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } @@ -184,20 +232,28 @@ public void initialize(Configuration configuration, Properties properties) throw /* Submits the request and returns */ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query) throws SerDeException, IOException { - HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle()); + + final Lifecycle lifecycle = new Lifecycle(); + HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), lifecycle); InputStream response; try { + lifecycle.start(); response = DruidStorageHandlerUtils.submitRequest(client, - DruidStorageHandlerUtils.createRequest(address, query)); + DruidStorageHandlerUtils.createRequest(address, query) + ); } catch (Exception e) { throw new SerDeException(StringUtils.stringifyException(e)); + } finally { + lifecycle.stop(); } // Retrieve results List resultsList; try { resultsList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response, - new TypeReference>() {}); + new TypeReference>() { + } + ); } catch (Exception e) { response.close(); throw new SerDeException(StringUtils.stringifyException(e)); @@ -214,7 +270,8 @@ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQ /* Timeseries query */ private void inferSchema(TimeseriesQuery query, List columnNames, - List columnTypes) { + List columnTypes + ) { // Timestamp column columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN); columnTypes.add(TypeInfoFactory.timestampTypeInfo); @@ -231,7 +288,9 @@ private void inferSchema(TimeseriesQuery query, List columnNames, } /* TopN query */ - private void inferSchema(TopNQuery query, List columnNames, List columnTypes) { + private void inferSchema(TopNQuery query, List columnNames, + List columnTypes + ) { // Timestamp column columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN); columnTypes.add(TypeInfoFactory.timestampTypeInfo); @@ -252,7 +311,8 @@ private void inferSchema(TopNQuery query, List columnNames, List columnNames, - List columnTypes) { + List columnTypes + ) { // Timestamp column columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN); columnTypes.add(TypeInfoFactory.timestampTypeInfo); @@ -269,7 +329,9 @@ private void inferSchema(SelectQuery query, List columnNames, } /* GroupBy query */ - private void inferSchema(GroupByQuery query, List columnNames, List columnTypes) { + private void inferSchema(GroupByQuery query, List columnNames, + List columnTypes + ) { // Timestamp column columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN); columnTypes.add(TypeInfoFactory.timestampTypeInfo); @@ -292,17 +354,67 @@ private void inferSchema(GroupByQuery query, List columnNames, List getSerializedClass() { - return NullWritable.class; + return DruidWritable.class; } @Override public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException { - return NullWritable.get(); + if (objectInspector.getCategory() != ObjectInspector.Category.STRUCT) { + throw new SerDeException(getClass().toString() + + " can only serialize struct types, but we got: " + + objectInspector.getTypeName()); + } + + // Prepare the field ObjectInspectors + StructObjectInspector soi = (StructObjectInspector) objectInspector; + List fields = soi.getAllStructFieldRefs(); + List values = soi.getStructFieldsDataAsList(o); + // We deserialize the result + Map value = new HashMap<>(); + for (int i = 0; i < columns.length; i++) { + if (values.get(i) == null) { + // null, we just add it + value.put(columns[i], null); + continue; + } + final Object res; + switch (types[i].getPrimitiveCategory()) { + case TIMESTAMP: + res = ((TimestampObjectInspector) fields.get(i).getFieldObjectInspector()) + .getPrimitiveJavaObject( + values.get(i)).getTime(); + break; + case LONG: + res = ((LongObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); + break; + case FLOAT: + res = ((FloatObjectInspector) fields.get(i).getFieldObjectInspector()).get(values.get(i)); + break; + case DOUBLE: + res = ((DoubleObjectInspector) fields.get(i).getFieldObjectInspector()) + .get(values.get(i)); + break; + case STRING: + res = ((StringObjectInspector) fields.get(i).getFieldObjectInspector()) + .getPrimitiveJavaObject( + values.get(i)); + break; + default: + throw new SerDeException("Unknown type: " + types[i].getPrimitiveCategory()); + } + value.put(columns[i], res); + } + value.put(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME, + ((TimestampObjectInspector) fields.get(columns.length).getFieldObjectInspector()) + .getPrimitiveJavaObject(values.get(columns.length)).getTime() + ); + return new DruidWritable(value); } @Override public SerDeStats getSerDeStats() { - throw new UnsupportedOperationException("SerdeStats not supported."); + // no support for statistics + return null; } @Override @@ -317,13 +429,16 @@ public Object deserialize(Writable writable) throws SerDeException { } switch (types[i].getPrimitiveCategory()) { case TIMESTAMP: - output.add(new TimestampWritable(new Timestamp((Long)value))); + output.add(new TimestampWritable(new Timestamp((Long) value))); break; case LONG: - output.add(new LongWritable(((Number)value).longValue())); + output.add(new LongWritable(((Number) value).longValue())); break; case FLOAT: - output.add(new FloatWritable(((Number)value).floatValue())); + output.add(new FloatWritable(((Number) value).floatValue())); + break; + case DOUBLE: + output.add(new DoubleWritable(((Number) value).floatValue())); break; case STRING: output.add(new Text(value.toString())); diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java index 29b8845..64a19f6 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java @@ -31,14 +31,16 @@ private static final Logger LOG = LoggerFactory.getLogger(DruidSerDeUtils.class); protected static final String FLOAT_TYPE = "FLOAT"; + protected static final String LONG_TYPE = "LONG"; + protected static final String STRING_TYPE = "STRING"; /* This method converts from the String representation of Druid type * to the corresponding Hive type */ public static PrimitiveTypeInfo convertDruidToHiveType(String typeName) { typeName = typeName.toUpperCase(); - switch(typeName) { + switch (typeName) { case FLOAT_TYPE: return TypeInfoFactory.floatTypeInfo; case LONG_TYPE: @@ -61,7 +63,7 @@ public static PrimitiveTypeInfo convertDruidToHiveType(String typeName) { * to the String representation of the corresponding Hive type */ public static String convertDruidToHiveTypeString(String typeName) { typeName = typeName.toUpperCase(); - switch(typeName) { + switch (typeName) { case FLOAT_TYPE: return serdeConstants.FLOAT_TYPE_NAME; case LONG_TYPE: diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java index b91178c..8c2fb10 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java @@ -45,9 +45,12 @@ protected TimeseriesQuery createQuery(String content) throws IOException { } @Override - protected List> createResultsList(InputStream content) throws IOException { + protected List> createResultsList(InputStream content) + throws IOException { return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content, - new TypeReference>>(){}); + new TypeReference>>() { + } + ); } @Override diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java index 22599c3..d431925 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java @@ -41,6 +41,7 @@ extends DruidQueryRecordReader> { private Result current; + private Iterator values = Iterators.emptyIterator(); @Override @@ -49,9 +50,12 @@ protected TopNQuery createQuery(String content) throws IOException { } @Override - protected List> createResultsList(InputStream content) throws IOException { + protected List> createResultsList(InputStream content) + throws IOException { return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content, - new TypeReference>>(){}); + new TypeReference>>() { + } + ); } @Override diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/DruidStorageHandlerTest.java druid-handler/src/test/org/apache/hadoop/hive/druid/DruidStorageHandlerTest.java new file mode 100644 index 0000000..211dc50 --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/druid/DruidStorageHandlerTest.java @@ -0,0 +1,179 @@ +package org.apache.hadoop.hive.druid; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.indexer.JobHelper; +import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.NoneShardSpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; +import org.skife.jdbi.v2.Handle; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Map; +import java.util.UUID; + +public class DruidStorageHandlerTest { + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final String DATA_SOURCE_NAME = "testName"; + + private String segmentsTable; + + private String tablePath; + + private DataSegment dataSegment = DataSegment.builder().dataSource(DATA_SOURCE_NAME).version("v1") + .interval(new Interval(100, 170)).shardSpec(NoneShardSpec.instance()).build(); + + @Before + public void before() throws Throwable { + tablePath = temporaryFolder.newFolder().getAbsolutePath(); + segmentsTable = derbyConnectorRule.metadataTablesConfigSupplier().get().getSegmentsTable(); + Map mockMap = ImmutableMap.of(Constants.DRUID_DATA_SOURCE, DATA_SOURCE_NAME); + Mockito.when(tableMock.getParameters()).thenReturn(mockMap); + Mockito.when(tableMock.getPartitionKeysSize()).thenReturn(0); + StorageDescriptor storageDes = Mockito.mock(StorageDescriptor.class); + Mockito.when(storageDes.getBucketColsSize()).thenReturn(0); + Mockito.when(storageDes.getLocation()).thenReturn(tablePath); + Mockito.when(tableMock.getSd()).thenReturn(storageDes); + Mockito.when(tableMock.getDbName()).thenReturn(DATA_SOURCE_NAME); + } + + Table tableMock = Mockito.mock(Table.class); + + @Test + public void testPreCreateTableWillCreateSegmentsTable() throws MetaException { + DruidStorageHandler druidStorageHandler = new DruidStorageHandler( + derbyConnectorRule.getConnector(), + new SQLMetadataStorageUpdaterJobHandler(derbyConnectorRule.getConnector()), + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); + + try (Handle handle = derbyConnectorRule.getConnector().getDBI().open()) { + Assert.assertFalse(derbyConnectorRule.getConnector() + .tableExists(handle, + segmentsTable + )); + druidStorageHandler.preCreateTable(tableMock); + Assert.assertTrue(derbyConnectorRule.getConnector() + .tableExists(handle, + segmentsTable + )); + } + + } + + @Test(expected = MetaException.class) + public void testPreCreateTableWhenDataSourceExists() throws MetaException { + derbyConnectorRule.getConnector().createSegmentTable(); + SQLMetadataStorageUpdaterJobHandler sqlMetadataStorageUpdaterJobHandler = new SQLMetadataStorageUpdaterJobHandler( + derbyConnectorRule.getConnector()); + sqlMetadataStorageUpdaterJobHandler.publishSegments(segmentsTable, Arrays.asList(dataSegment), + DruidStorageHandlerUtils.JSON_MAPPER + ); + DruidStorageHandler druidStorageHandler = new DruidStorageHandler( + derbyConnectorRule.getConnector(), + new SQLMetadataStorageUpdaterJobHandler(derbyConnectorRule.getConnector()), + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); + druidStorageHandler.preCreateTable(tableMock); + } + + @Test + public void testCommitCreateTablePlusCommitDropTableWithoutPurge() + throws MetaException, IOException { + DruidStorageHandler druidStorageHandler = new DruidStorageHandler( + derbyConnectorRule.getConnector(), + new SQLMetadataStorageUpdaterJobHandler(derbyConnectorRule.getConnector()), + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); + druidStorageHandler.preCreateTable(tableMock); + Configuration config = new Configuration(); + druidStorageHandler.setConf(config); + LocalFileSystem localFileSystem = FileSystem.getLocal(config); + /* + final descriptor path is in the form tablePath/taskId_Attempt_ID/segmentDescriptorDir/segmentIdentifier.json + UUID.randomUUID() will fake the taskId_attemptID + */ + Path taskDirPath = new Path(tablePath, UUID.randomUUID().toString()); + Path descriptorPath = DruidStorageHandlerUtils.makeSegmentDescriptorOutputPath(dataSegment, + new Path(taskDirPath, DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME) + ); + DruidStorageHandlerUtils.writeSegmentDescriptor(localFileSystem, dataSegment, descriptorPath); + druidStorageHandler.commitCreateTable(tableMock); + Assert.assertArrayEquals(Lists.newArrayList(DATA_SOURCE_NAME).toArray(), Lists.newArrayList( + DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get() + )).toArray()); + druidStorageHandler.commitDropTable(tableMock, false); + Assert.assertArrayEquals(Lists.newArrayList().toArray(), Lists.newArrayList( + DruidStorageHandlerUtils.getAllDataSourceNames(derbyConnectorRule.getConnector(), + derbyConnectorRule.metadataTablesConfigSupplier().get() + )).toArray()); + + } + + @Test + public void testDeleteSegment() throws IOException, SegmentLoadingException { + DruidStorageHandler druidStorageHandler = new DruidStorageHandler( + derbyConnectorRule.getConnector(), + new SQLMetadataStorageUpdaterJobHandler(derbyConnectorRule.getConnector()), + derbyConnectorRule.metadataTablesConfigSupplier().get() + ); + + String segmentRootPath = temporaryFolder.newFolder().getAbsolutePath(); + Configuration config = new Configuration(); + druidStorageHandler.setConf(config); + LocalFileSystem localFileSystem = FileSystem.getLocal(config); + + Path segmentOutputPath = JobHelper + .makeSegmentOutputPath(new Path(segmentRootPath), localFileSystem, dataSegment); + Path indexPath = new Path(segmentOutputPath, "index.zip"); + DataSegment dataSegmentWithLoadspect = DataSegment.builder(dataSegment).loadSpec( + ImmutableMap.of("path", indexPath)).build(); + OutputStream outputStream = localFileSystem.create(indexPath, true); + outputStream.close(); + Assert.assertTrue("index file is not created ??", localFileSystem.exists(indexPath)); + Assert.assertTrue(localFileSystem.exists(segmentOutputPath)); + + druidStorageHandler.deleteSegment(dataSegmentWithLoadspect); + // path format -- > .../dataSource/interval/version/partitionNum/xxx.zip + Assert.assertFalse("Index file still there ??", localFileSystem.exists(indexPath)); + // path format of segmentOutputPath -- > .../dataSource/interval/version/partitionNum/ + Assert.assertFalse("PartitionNum directory still there ??", + localFileSystem.exists(segmentOutputPath) + ); + Assert.assertFalse("Version directory still there ??", + localFileSystem.exists(segmentOutputPath.getParent()) + ); + Assert.assertFalse("Interval directory still there ??", + localFileSystem.exists(segmentOutputPath.getParent().getParent()) + ); + Assert.assertFalse("Data source directory still there ??", + localFileSystem.exists(segmentOutputPath.getParent().getParent().getParent()) + ); + } + +} \ No newline at end of file diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java index 2b4df78..8dc8091 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java @@ -41,34 +41,34 @@ // + "\"usingDefaultInterval\":true,\"lenientAggregatorMerge\":false,\"descending\":false}"; private static final String RESPONSE = "[ {\r\n " - + " \"id\" : \"merged\",\r\n " - + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n " - + " \"columns\" : {\r\n " - + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n " - + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " - + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " - + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n " - + " },\r\n " - + " \"aggregators\" : {\r\n " - + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n " - + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n " - + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n " - + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n " - + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n " - + " },\r\n " - + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n " - + " \"size\" : 300000,\r\n " - + " \"numRows\" : 5000000\r\n} ]"; + + " \"id\" : \"merged\",\r\n " + + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n " + + " \"columns\" : {\r\n " + + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n " + + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n " + + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n " + + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n " + + " },\r\n " + + " \"aggregators\" : {\r\n " + + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n " + + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n " + + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n " + + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n " + + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n " + + " },\r\n " + + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n " + + " \"size\" : 300000,\r\n " + + " \"numRows\" : 5000000\r\n} ]"; /* Submits the request and returns */ @Override @@ -78,7 +78,9 @@ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQ List resultsList; try { resultsList = DruidStorageHandlerUtils.JSON_MAPPER.readValue(RESPONSE, - new TypeReference>() {}); + new TypeReference>() { + } + ); } catch (Exception e) { throw new SerDeException(StringUtils.stringifyException(e)); } diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/TestDerbyConnector.java druid-handler/src/test/org/apache/hadoop/hive/druid/TestDerbyConnector.java new file mode 100644 index 0000000..03f3d6f --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestDerbyConnector.java @@ -0,0 +1,108 @@ +package org.apache.hadoop.hive.druid; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import io.druid.metadata.MetadataStorageConnectorConfig; +import io.druid.metadata.MetadataStorageTablesConfig; +import io.druid.metadata.storage.derby.DerbyConnector; +import org.junit.Assert; +import org.junit.rules.ExternalResource; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException; + +import java.sql.SQLException; +import java.util.UUID; + +public class TestDerbyConnector extends DerbyConnector { + private final String jdbcUri; + + public TestDerbyConnector( + Supplier config, + Supplier dbTables + ) { + this(config, dbTables, "jdbc:derby:memory:druidTest" + dbSafeUUID()); + } + + protected TestDerbyConnector( + Supplier config, + Supplier dbTables, + String jdbcUri + ) { + super(config, dbTables, new DBI(jdbcUri + ";create=true")); + this.jdbcUri = jdbcUri; + } + + public void tearDown() { + try { + new DBI(jdbcUri + ";drop=true").open().close(); + } catch (UnableToObtainConnectionException e) { + SQLException cause = (SQLException) e.getCause(); + // error code "08006" indicates proper shutdown + Assert.assertEquals(String.format("Derby not shutdown: [%s]", cause.toString()), "08006", + cause.getSQLState() + ); + } + } + + public static String dbSafeUUID() { + return UUID.randomUUID().toString().replace("-", ""); + } + + public String getJdbcUri() { + return jdbcUri; + } + + public static class DerbyConnectorRule extends ExternalResource { + private TestDerbyConnector connector; + + private final Supplier dbTables; + + private final MetadataStorageConnectorConfig connectorConfig; + + public DerbyConnectorRule() { + this("druidTest" + dbSafeUUID()); + } + + private DerbyConnectorRule( + final String defaultBase + ) { + this(Suppliers.ofInstance(MetadataStorageTablesConfig.fromBase(defaultBase))); + } + + public DerbyConnectorRule( + Supplier dbTables + ) { + this.dbTables = dbTables; + this.connectorConfig = new MetadataStorageConnectorConfig() { + @Override + public String getConnectURI() { + return connector.getJdbcUri(); + } + }; + } + + @Override + protected void before() throws Throwable { + connector = new TestDerbyConnector(Suppliers.ofInstance(connectorConfig), dbTables); + connector.getDBI().open().close(); // create db + } + + @Override + protected void after() { + connector.tearDown(); + } + + public TestDerbyConnector getConnector() { + return connector; + } + + public MetadataStorageConnectorConfig getMetadataConnectorConfig() { + return connectorConfig; + } + + public Supplier metadataTablesConfigSupplier() { + return dbTables; + } + } + +} \ No newline at end of file diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java index 1343939..a495165 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java @@ -74,374 +74,408 @@ // Timeseries query private static final String TIMESERIES_QUERY = "{ \"queryType\": \"timeseries\", " - + " \"dataSource\": \"sample_datasource\", " - + " \"granularity\": \"day\", " - + " \"descending\": \"true\", " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" }, " - + " { \"type\": \"or\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" }, " - + " { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" } " - + " ] " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" }, " - + " { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } " - + " ], " - + " \"postAggregations\": [ " - + " { \"type\": \"arithmetic\", " - + " \"name\": \"sample_divide\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, " - + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}"; + + " \"dataSource\": \"sample_datasource\", " + + " \"granularity\": \"day\", " + + " \"descending\": \"true\", " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" }, " + + " { \"type\": \"or\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" }, " + + " { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" } " + + " ] " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" }, " + + " { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } " + + " ], " + + " \"postAggregations\": [ " + + " { \"type\": \"arithmetic\", " + + " \"name\": \"sample_divide\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, " + + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}"; + // Timeseries query results private static final String TIMESERIES_QUERY_RESULTS = "[ " - + "{ " - + " \"timestamp\": \"2012-01-01T00:00:00.000Z\", " - + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 } " - + "}, " - + "{ " - + " \"timestamp\": \"2012-01-02T00:00:00.000Z\", " - + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 } " - + "}]"; + + "{ " + + " \"timestamp\": \"2012-01-01T00:00:00.000Z\", " + + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 } " + + "}, " + + "{ " + + " \"timestamp\": \"2012-01-02T00:00:00.000Z\", " + + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 } " + + "}]"; + // Timeseries query results as records private static final Object[][] TIMESERIES_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new LongWritable(0), new FloatWritable(1.0F), new FloatWritable(2.2222F) } , - new Object[] { new TimestampWritable(new Timestamp(1325462400000L)), new LongWritable(2), new FloatWritable(3.32F), new FloatWritable(4F)} + new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new LongWritable(0), + new FloatWritable(1.0F), new FloatWritable(2.2222F) }, + new Object[] { new TimestampWritable(new Timestamp(1325462400000L)), new LongWritable(2), + new FloatWritable(3.32F), new FloatWritable(4F) } }; // TopN query private static final String TOPN_QUERY = "{ \"queryType\": \"topN\", " - + " \"dataSource\": \"sample_data\", " - + " \"dimension\": \"sample_dim\", " - + " \"threshold\": 5, " - + " \"metric\": \"count\", " - + " \"granularity\": \"all\", " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { " - + " \"type\": \"selector\", " - + " \"dimension\": \"dim1\", " - + " \"value\": \"some_value\" " - + " }, " - + " { " - + " \"type\": \"selector\", " - + " \"dimension\": \"dim2\", " - + " \"value\": \"some_other_val\" " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { " - + " \"type\": \"longSum\", " - + " \"name\": \"count\", " - + " \"fieldName\": \"count\" " - + " }, " - + " { " - + " \"type\": \"doubleSum\", " - + " \"name\": \"some_metric\", " - + " \"fieldName\": \"some_metric\" " - + " } " - + " ], " - + " \"postAggregations\": [ " - + " { " - + " \"type\": \"arithmetic\", " - + " \"name\": \"sample_divide\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { " - + " \"type\": \"fieldAccess\", " - + " \"name\": \"some_metric\", " - + " \"fieldName\": \"some_metric\" " - + " }, " - + " { " - + " \"type\": \"fieldAccess\", " - + " \"name\": \"count\", " - + " \"fieldName\": \"count\" " - + " } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ " - + " \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" " - + " ]}"; + + " \"dataSource\": \"sample_data\", " + + " \"dimension\": \"sample_dim\", " + + " \"threshold\": 5, " + + " \"metric\": \"count\", " + + " \"granularity\": \"all\", " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { " + + " \"type\": \"selector\", " + + " \"dimension\": \"dim1\", " + + " \"value\": \"some_value\" " + + " }, " + + " { " + + " \"type\": \"selector\", " + + " \"dimension\": \"dim2\", " + + " \"value\": \"some_other_val\" " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { " + + " \"type\": \"longSum\", " + + " \"name\": \"count\", " + + " \"fieldName\": \"count\" " + + " }, " + + " { " + + " \"type\": \"doubleSum\", " + + " \"name\": \"some_metric\", " + + " \"fieldName\": \"some_metric\" " + + " } " + + " ], " + + " \"postAggregations\": [ " + + " { " + + " \"type\": \"arithmetic\", " + + " \"name\": \"sample_divide\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { " + + " \"type\": \"fieldAccess\", " + + " \"name\": \"some_metric\", " + + " \"fieldName\": \"some_metric\" " + + " }, " + + " { " + + " \"type\": \"fieldAccess\", " + + " \"name\": \"count\", " + + " \"fieldName\": \"count\" " + + " } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ " + + " \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" " + + " ]}"; + // TopN query results private static final String TOPN_QUERY_RESULTS = "[ " - + " { " - + " \"timestamp\": \"2013-08-31T00:00:00.000Z\", " - + " \"result\": [ " - + " { " - + " \"sample_dim\": \"dim1_val\", " - + " \"count\": 111, " - + " \"some_metric\": 10669, " - + " \"sample_divide\": 96.11711711711712 " - + " }, " - + " { " - + " \"sample_dim\": \"another_dim1_val\", " - + " \"count\": 88, " - + " \"some_metric\": 28344, " - + " \"sample_divide\": 322.09090909090907 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val3\", " - + " \"count\": 70, " - + " \"some_metric\": 871, " - + " \"sample_divide\": 12.442857142857143 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val4\", " - + " \"count\": 62, " - + " \"some_metric\": 815, " - + " \"sample_divide\": 13.14516129032258 " - + " }, " - + " { " - + " \"sample_dim\": \"dim1_val5\", " - + " \"count\": 60, " - + " \"some_metric\": 2787, " - + " \"sample_divide\": 46.45 " - + " } " - + " ] " - + " }]"; + + " { " + + " \"timestamp\": \"2013-08-31T00:00:00.000Z\", " + + " \"result\": [ " + + " { " + + " \"sample_dim\": \"dim1_val\", " + + " \"count\": 111, " + + " \"some_metric\": 10669, " + + " \"sample_divide\": 96.11711711711712 " + + " }, " + + " { " + + " \"sample_dim\": \"another_dim1_val\", " + + " \"count\": 88, " + + " \"some_metric\": 28344, " + + " \"sample_divide\": 322.09090909090907 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val3\", " + + " \"count\": 70, " + + " \"some_metric\": 871, " + + " \"sample_divide\": 12.442857142857143 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val4\", " + + " \"count\": 62, " + + " \"some_metric\": 815, " + + " \"sample_divide\": 13.14516129032258 " + + " }, " + + " { " + + " \"sample_dim\": \"dim1_val5\", " + + " \"count\": 60, " + + " \"some_metric\": 2787, " + + " \"sample_divide\": 46.45 " + + " } " + + " ] " + + " }]"; + // TopN query results as records private static final Object[][] TOPN_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val"), new LongWritable(111), new FloatWritable(10669F), new FloatWritable(96.11711711711712F) } , - new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), new FloatWritable(322.09090909090907F) } , - new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), new FloatWritable(12.442857142857143F) } , - new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), new FloatWritable(13.14516129032258F) } , - new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), new FloatWritable(46.45F) } + new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val"), + new LongWritable(111), new FloatWritable(10669F), + new FloatWritable(96.11711711711712F) }, + new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), + new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), + new FloatWritable(322.09090909090907F) }, + new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), + new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), + new FloatWritable(12.442857142857143F) }, + new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), + new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), + new FloatWritable(13.14516129032258F) }, + new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), + new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), + new FloatWritable(46.45F) } }; // GroupBy query private static final String GROUP_BY_QUERY = "{ " - + " \"queryType\": \"groupBy\", " - + " \"dataSource\": \"sample_datasource\", " - + " \"granularity\": \"day\", " - + " \"dimensions\": [\"country\", \"device\"], " - + " \"limitSpec\": {" - + " \"type\": \"default\"," - + " \"limit\": 5000," - + " \"columns\": [\"country\", \"data_transfer\"] }, " - + " \"filter\": { " - + " \"type\": \"and\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" }, " - + " { \"type\": \"or\", " - + " \"fields\": [ " - + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" }, " - + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" } " - + " ] " - + " } " - + " ] " - + " }, " - + " \"aggregations\": [ " - + " { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" }, " - + " { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } " - + " ], " - + " \"postAggregations\": [ " - + " { \"type\": \"arithmetic\", " - + " \"name\": \"avg_usage\", " - + " \"fn\": \"/\", " - + " \"fields\": [ " - + " { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" }, " - + " { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" } " - + " ] " - + " } " - + " ], " - + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], " - + " \"having\": { " - + " \"type\": \"greaterThan\", " - + " \"aggregation\": \"total_usage\", " - + " \"value\": 100 " - + " }}"; + + " \"queryType\": \"groupBy\", " + + " \"dataSource\": \"sample_datasource\", " + + " \"granularity\": \"day\", " + + " \"dimensions\": [\"country\", \"device\"], " + + " \"limitSpec\": {" + + " \"type\": \"default\"," + + " \"limit\": 5000," + + " \"columns\": [\"country\", \"data_transfer\"] }, " + + " \"filter\": { " + + " \"type\": \"and\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" }, " + + " { \"type\": \"or\", " + + " \"fields\": [ " + + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" }, " + + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" } " + + " ] " + + " } " + + " ] " + + " }, " + + " \"aggregations\": [ " + + " { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" }, " + + " { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } " + + " ], " + + " \"postAggregations\": [ " + + " { \"type\": \"arithmetic\", " + + " \"name\": \"avg_usage\", " + + " \"fn\": \"/\", " + + " \"fields\": [ " + + " { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" }, " + + " { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" } " + + " ] " + + " } " + + " ], " + + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], " + + " \"having\": { " + + " \"type\": \"greaterThan\", " + + " \"aggregation\": \"total_usage\", " + + " \"value\": 100 " + + " }}"; + // GroupBy query results private static final String GROUP_BY_QUERY_RESULTS = "[ " - + " { " - + " \"version\" : \"v1\", " - + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " - + " \"event\" : { " - + " \"country\" : \"India\", " - + " \"device\" : \"phone\", " - + " \"total_usage\" : 88, " - + " \"data_transfer\" : 29.91233453, " - + " \"avg_usage\" : 60.32 " - + " } " - + " }, " - + " { " - + " \"version\" : \"v1\", " - + " \"timestamp\" : \"2012-01-01T00:00:12.000Z\", " - + " \"event\" : { " - + " \"country\" : \"Spain\", " - + " \"device\" : \"pc\", " - + " \"total_usage\" : 16, " - + " \"data_transfer\" : 172.93494959, " - + " \"avg_usage\" : 6.333333 " - + " } " - + " }]"; + + " { " + + " \"version\" : \"v1\", " + + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", " + + " \"event\" : { " + + " \"country\" : \"India\", " + + " \"device\" : \"phone\", " + + " \"total_usage\" : 88, " + + " \"data_transfer\" : 29.91233453, " + + " \"avg_usage\" : 60.32 " + + " } " + + " }, " + + " { " + + " \"version\" : \"v1\", " + + " \"timestamp\" : \"2012-01-01T00:00:12.000Z\", " + + " \"event\" : { " + + " \"country\" : \"Spain\", " + + " \"device\" : \"pc\", " + + " \"total_usage\" : 16, " + + " \"data_transfer\" : 172.93494959, " + + " \"avg_usage\" : 6.333333 " + + " } " + + " }]"; + // GroupBy query results as records private static final Object[][] GROUP_BY_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new Text("India"), new Text("phone"), new LongWritable(88), new FloatWritable(29.91233453F), new FloatWritable(60.32F) } , - new Object[] { new TimestampWritable(new Timestamp(1325376012000L)), new Text("Spain"), new Text("pc"), new LongWritable(16), new FloatWritable(172.93494959F), new FloatWritable(6.333333F) } + new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new Text("India"), + new Text("phone"), new LongWritable(88), new FloatWritable(29.91233453F), + new FloatWritable(60.32F) }, + new Object[] { new TimestampWritable(new Timestamp(1325376012000L)), new Text("Spain"), + new Text("pc"), new LongWritable(16), new FloatWritable(172.93494959F), + new FloatWritable(6.333333F) } }; // Select query private static final String SELECT_QUERY = "{ \"queryType\": \"select\", " - + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " - + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " - + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " - + " \"granularity\": \"all\", " - + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], " - + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }"; + + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", " + + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], " + + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], " + + " \"granularity\": \"all\", " + + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], " + + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }"; + // Select query results private static final String SELECT_QUERY_RESULTS = "[{ " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"result\" : { " - + " \"pagingIdentifiers\" : { " - + " \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4 }, " - + " \"events\" : [ { " - + " \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 0, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"robot\" : \"1\", " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"11._korpus_(NOVJ)\", " - + " \"language\" : \"sl\", " - + " \"newpage\" : \"0\", " - + " \"user\" : \"EmausBot\", " - + " \"count\" : 1.0, " - + " \"added\" : 39.0, " - + " \"delta\" : 39.0, " - + " \"variation\" : 39.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 1, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " - + " \"robot\" : \"0\", " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"112_U.S._580\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 70.0, " - + " \"delta\" : 70.0, " - + " \"variation\" : 70.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 2, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : \"0\", " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._243\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 77.0, " - + " \"delta\" : 77.0, " - + " \"variation\" : 77.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 3, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : \"0\", " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._73\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 70.0, " - + " \"delta\" : 70.0, " - + " \"variation\" : 70.0, " - + " \"deleted\" : 0.0 " - + " } " - + " }, { " - + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " - + " \"offset\" : 4, " - + " \"event\" : { " - + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " - + " \"robot\" : \"0\", " - + " \"namespace\" : \"article\", " - + " \"anonymous\" : \"0\", " - + " \"unpatrolled\" : \"0\", " - + " \"page\" : \"113_U.S._756\", " - + " \"language\" : \"en\", " - + " \"newpage\" : \"1\", " - + " \"user\" : \"MZMcBride\", " - + " \"count\" : 1.0, " - + " \"added\" : 68.0, " - + " \"delta\" : 68.0, " - + " \"variation\" : 68.0, " - + " \"deleted\" : 0.0 " - + " } " - + " } ] }} ]"; + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"result\" : { " + + " \"pagingIdentifiers\" : { " + + " \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4 }, " + + " \"events\" : [ { " + + " \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 0, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"robot\" : \"1\", " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"11._korpus_(NOVJ)\", " + + " \"language\" : \"sl\", " + + " \"newpage\" : \"0\", " + + " \"user\" : \"EmausBot\", " + + " \"count\" : 1.0, " + + " \"added\" : 39.0, " + + " \"delta\" : 39.0, " + + " \"variation\" : 39.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 1, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", " + + " \"robot\" : \"0\", " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"112_U.S._580\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 70.0, " + + " \"delta\" : 70.0, " + + " \"variation\" : 70.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 2, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : \"0\", " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._243\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 77.0, " + + " \"delta\" : 77.0, " + + " \"variation\" : 77.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 3, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : \"0\", " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._73\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 70.0, " + + " \"delta\" : 70.0, " + + " \"variation\" : 70.0, " + + " \"deleted\" : 0.0 " + + " } " + + " }, { " + + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", " + + " \"offset\" : 4, " + + " \"event\" : { " + + " \"timestamp\" : \"2013-01-01T00:00:12.000Z\", " + + " \"robot\" : \"0\", " + + " \"namespace\" : \"article\", " + + " \"anonymous\" : \"0\", " + + " \"unpatrolled\" : \"0\", " + + " \"page\" : \"113_U.S._756\", " + + " \"language\" : \"en\", " + + " \"newpage\" : \"1\", " + + " \"user\" : \"MZMcBride\", " + + " \"count\" : 1.0, " + + " \"added\" : 68.0, " + + " \"delta\" : 68.0, " + + " \"variation\" : 68.0, " + + " \"deleted\" : 0.0 " + + " } " + + " } ] }} ]"; + // Select query results as records private static final Object[][] SELECT_QUERY_RESULTS_RECORDS = new Object[][] { - new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("1"), new Text("article"), new Text("0"), new Text("0"), - new Text("11._korpus_(NOVJ)"), new Text("sl"), new Text("0"), new Text("EmausBot"), - new FloatWritable(1.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(0.0F) } , - new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"), - new Text("112_U.S._580"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } , - new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._243"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new FloatWritable(1.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(0.0F) } , - new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._73"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } , - new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"), - new Text("113_U.S._756"), new Text("en"), new Text("1"), new Text("MZMcBride"), - new FloatWritable(1.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(0.0F) } + new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("1"), + new Text("article"), new Text("0"), new Text("0"), + new Text("11._korpus_(NOVJ)"), new Text("sl"), new Text("0"), + new Text("EmausBot"), + new FloatWritable(1.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), + new FloatWritable(39.0F), new FloatWritable(0.0F) }, + new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), + new Text("article"), new Text("0"), new Text("0"), + new Text("112_U.S._580"), new Text("en"), new Text("1"), new Text("MZMcBride"), + new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), + new FloatWritable(70.0F), new FloatWritable(0.0F) }, + new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), + new Text("article"), new Text("0"), new Text("0"), + new Text("113_U.S._243"), new Text("en"), new Text("1"), new Text("MZMcBride"), + new FloatWritable(1.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), + new FloatWritable(77.0F), new FloatWritable(0.0F) }, + new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), + new Text("article"), new Text("0"), new Text("0"), + new Text("113_U.S._73"), new Text("en"), new Text("1"), new Text("MZMcBride"), + new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), + new FloatWritable(70.0F), new FloatWritable(0.0F) }, + new Object[] { new TimestampWritable(new Timestamp(1356998412000L)), new Text("0"), + new Text("article"), new Text("0"), new Text("0"), + new Text("113_U.S._756"), new Text("en"), new Text("1"), new Text("MZMcBride"), + new FloatWritable(1.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), + new FloatWritable(68.0F), new FloatWritable(0.0F) } }; - /** * Test the default behavior of the objects and object inspectors. - * @throws IOException - * @throws IllegalAccessException - * @throws IllegalArgumentException - * @throws SecurityException - * @throws NoSuchFieldException - * @throws JsonMappingException - * @throws JsonParseException - * @throws InvocationTargetException - * @throws NoSuchMethodException + * @throws IOException + * @throws IllegalAccessException + * @throws IllegalArgumentException + * @throws SecurityException + * @throws NoSuchFieldException + * @throws JsonMappingException + * @throws JsonParseException + * @throws InvocationTargetException + * @throws NoSuchMethodException */ @Test public void testDruidSerDe() @@ -457,25 +491,31 @@ public void testDruidSerDe() tbl = createPropertiesQuery("sample_datasource", Query.TIMESERIES, TIMESERIES_QUERY); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); deserializeQueryResults(serDe, Query.TIMESERIES, TIMESERIES_QUERY, - TIMESERIES_QUERY_RESULTS, TIMESERIES_QUERY_RESULTS_RECORDS); + TIMESERIES_QUERY_RESULTS, TIMESERIES_QUERY_RESULTS_RECORDS + ); // TopN query tbl = createPropertiesQuery("sample_data", Query.TOPN, TOPN_QUERY); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); deserializeQueryResults(serDe, Query.TOPN, TOPN_QUERY, - TOPN_QUERY_RESULTS, TOPN_QUERY_RESULTS_RECORDS); + TOPN_QUERY_RESULTS, TOPN_QUERY_RESULTS_RECORDS + ); // GroupBy query tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GROUP_BY_QUERY); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); deserializeQueryResults(serDe, Query.GROUP_BY, GROUP_BY_QUERY, - GROUP_BY_QUERY_RESULTS, GROUP_BY_QUERY_RESULTS_RECORDS); + GROUP_BY_QUERY_RESULTS, GROUP_BY_QUERY_RESULTS_RECORDS + ); // Select query tbl = createPropertiesQuery("wikipedia", Query.SELECT, SELECT_QUERY); SerDeUtils.initializeSerDe(serDe, conf, tbl, null); deserializeQueryResults(serDe, Query.SELECT, SELECT_QUERY, - SELECT_QUERY_RESULTS, SELECT_QUERY_RESULTS_RECORDS); + SELECT_QUERY_RESULTS, SELECT_QUERY_RESULTS_RECORDS + ); } - private static Properties createPropertiesQuery(String dataSource, String queryType, String jsonQuery) { + private static Properties createPropertiesQuery(String dataSource, String queryType, + String jsonQuery + ) { Properties tbl = new Properties(); // Set the configuration parameters @@ -486,14 +526,15 @@ private static Properties createPropertiesQuery(String dataSource, String queryT } private static void deserializeQueryResults(DruidSerDe serDe, String queryType, String jsonQuery, - String resultString, Object[][] records) throws SerDeException, JsonParseException, + String resultString, Object[][] records + ) throws SerDeException, JsonParseException, JsonMappingException, IOException, NoSuchFieldException, SecurityException, IllegalArgumentException, IllegalAccessException, InterruptedException, NoSuchMethodException, InvocationTargetException { // Initialize Query query = null; - DruidQueryRecordReader reader = null; + DruidQueryRecordReader reader = null; List resultsList = null; ObjectMapper mapper = new DefaultObjectMapper(); switch (queryType) { @@ -501,25 +542,33 @@ private static void deserializeQueryResults(DruidSerDe serDe, String queryType, query = mapper.readValue(jsonQuery, TimeseriesQuery.class); reader = new DruidTimeseriesQueryRecordReader(); resultsList = mapper.readValue(resultString, - new TypeReference>>() {}); + new TypeReference>>() { + } + ); break; case Query.TOPN: query = mapper.readValue(jsonQuery, TopNQuery.class); reader = new DruidTopNQueryRecordReader(); resultsList = mapper.readValue(resultString, - new TypeReference>>() {}); + new TypeReference>>() { + } + ); break; case Query.GROUP_BY: query = mapper.readValue(jsonQuery, GroupByQuery.class); reader = new DruidGroupByQueryRecordReader(); resultsList = mapper.readValue(resultString, - new TypeReference>() {}); + new TypeReference>() { + } + ); break; case Query.SELECT: query = mapper.readValue(jsonQuery, SelectQuery.class); reader = new DruidSelectQueryRecordReader(); resultsList = mapper.readValue(resultString, - new TypeReference>>() {}); + new TypeReference>>() { + } + ); break; } @@ -534,7 +583,7 @@ private static void deserializeQueryResults(DruidSerDe serDe, String queryType, } Field field2 = DruidQueryRecordReader.class.getDeclaredField("results"); field2.setAccessible(true); - + // Get the row structure StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector(); List fieldRefs = oi.getAllStructFieldRefs(); diff --git druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java index 9ccd48e..4fde3eb 100644 --- druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java +++ druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.hadoop.hive.druid.io.DruidQueryBasedInputFormat; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; import org.junit.Test; @@ -29,14 +30,15 @@ import junit.framework.TestCase; public class TestHiveDruidQueryBasedInputFormat extends TestCase { - + @SuppressWarnings("unchecked") @Test public void testCreateSplitsIntervals() throws Exception { - HiveDruidQueryBasedInputFormat input = new HiveDruidQueryBasedInputFormat(); + DruidQueryBasedInputFormat input = new DruidQueryBasedInputFormat(); - Method method1 = HiveDruidQueryBasedInputFormat.class.getDeclaredMethod("createSplitsIntervals", - List.class, int.class); + Method method1 = DruidQueryBasedInputFormat.class.getDeclaredMethod("createSplitsIntervals", + List.class, int.class + ); method1.setAccessible(true); List intervals; @@ -48,10 +50,14 @@ public void testCreateSplitsIntervals() throws Exception { intervals.add(new Interval(1262304000000L, 1293840000000L, ISOChronology.getInstanceUTC())); resultList = (List>) method1.invoke(input, intervals, 4); expectedResultList = new ArrayList<>(); - expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1270188000000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1270188000000L, 1278072000000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1278072000000L, 1285956000000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1285956000000L, 1293840000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1262304000000L, 1270188000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1270188000000L, 1278072000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1278072000000L, 1285956000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1285956000000L, 1293840000000L, ISOChronology.getInstanceUTC()))); assertEquals(expectedResultList, resultList); // Test 2 : two splits, create 4 @@ -60,11 +66,16 @@ public void testCreateSplitsIntervals() throws Exception { intervals.add(new Interval(1325376000000L, 1356998400000L, ISOChronology.getInstanceUTC())); resultList = (List>) method1.invoke(input, intervals, 4); expectedResultList = new ArrayList<>(); - expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1278093600000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1278093600000L, 1293840000000L, ISOChronology.getInstanceUTC()), - new Interval(1325376000000L, 1325419200000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1325419200000L, 1341208800000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1341208800000L, 1356998400000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1262304000000L, 1278093600000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1278093600000L, 1293840000000L, ISOChronology.getInstanceUTC()), + new Interval(1325376000000L, 1325419200000L, ISOChronology.getInstanceUTC()) + )); + expectedResultList.add(Arrays + .asList(new Interval(1325419200000L, 1341208800000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1341208800000L, 1356998400000L, ISOChronology.getInstanceUTC()))); assertEquals(expectedResultList, resultList); // Test 3 : two splits, create 5 @@ -73,29 +84,49 @@ public void testCreateSplitsIntervals() throws Exception { intervals.add(new Interval(1325376000000L, 1356998400000L, ISOChronology.getInstanceUTC())); resultList = (List>) method1.invoke(input, intervals, 5); expectedResultList = new ArrayList<>(); - expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1274935680000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1274935680000L, 1287567360000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1287567360000L, 1293840000000L, ISOChronology.getInstanceUTC()), - new Interval(1325376000000L, 1331735040000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1331735040000L, 1344366720000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1344366720000L, 1356998400000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1262304000000L, 1274935680000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1274935680000L, 1287567360000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1287567360000L, 1293840000000L, ISOChronology.getInstanceUTC()), + new Interval(1325376000000L, 1331735040000L, ISOChronology.getInstanceUTC()) + )); + expectedResultList.add(Arrays + .asList(new Interval(1331735040000L, 1344366720000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1344366720000L, 1356998400000L, ISOChronology.getInstanceUTC()))); assertEquals(expectedResultList, resultList); // Test 4 : three splits, different ranges, create 6 intervals = new ArrayList<>(); - intervals.add(new Interval(1199145600000L, 1201824000000L, ISOChronology.getInstanceUTC())); // one month - intervals.add(new Interval(1325376000000L, 1356998400000L, ISOChronology.getInstanceUTC())); // one year - intervals.add(new Interval(1407283200000L, 1407888000000L, ISOChronology.getInstanceUTC())); // 7 days + intervals.add(new Interval(1199145600000L, 1201824000000L, + ISOChronology.getInstanceUTC() + )); // one month + intervals.add(new Interval(1325376000000L, 1356998400000L, + ISOChronology.getInstanceUTC() + )); // one year + intervals.add(new Interval(1407283200000L, 1407888000000L, + ISOChronology.getInstanceUTC() + )); // 7 days resultList = (List>) method1.invoke(input, intervals, 6); expectedResultList = new ArrayList<>(); - expectedResultList.add(Arrays.asList(new Interval(1199145600000L, 1201824000000L, ISOChronology.getInstanceUTC()), - new Interval(1325376000000L, 1328515200000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1328515200000L, 1334332800000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1334332800000L, 1340150400000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1340150400000L, 1345968000000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1345968000000L, 1351785600000L, ISOChronology.getInstanceUTC()))); - expectedResultList.add(Arrays.asList(new Interval(1351785600000L, 1356998400000L, ISOChronology.getInstanceUTC()), - new Interval(1407283200000L, 1407888000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1199145600000L, 1201824000000L, ISOChronology.getInstanceUTC()), + new Interval(1325376000000L, 1328515200000L, ISOChronology.getInstanceUTC()) + )); + expectedResultList.add(Arrays + .asList(new Interval(1328515200000L, 1334332800000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1334332800000L, 1340150400000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1340150400000L, 1345968000000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1345968000000L, 1351785600000L, ISOChronology.getInstanceUTC()))); + expectedResultList.add(Arrays + .asList(new Interval(1351785600000L, 1356998400000L, ISOChronology.getInstanceUTC()), + new Interval(1407283200000L, 1407888000000L, ISOChronology.getInstanceUTC()) + )); assertEquals(expectedResultList, resultList); } diff --git druid-handler/src/test/org/apache/hadoop/hive/ql/io/DruidRecordWriterTest.java druid-handler/src/test/org/apache/hadoop/hive/ql/io/DruidRecordWriterTest.java new file mode 100644 index 0000000..6cea93a --- /dev/null +++ druid-handler/src/test/org/apache/hadoop/hive/ql/io/DruidRecordWriterTest.java @@ -0,0 +1,220 @@ +package org.apache.hadoop.hive.ql.io; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.data.input.Firehose; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.granularity.QueryGranularities; +import io.druid.java.util.common.Granularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexStorageAdapter; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import io.druid.segment.loading.DataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPuller; +import io.druid.segment.loading.LocalDataSegmentPusher; +import io.druid.segment.loading.LocalDataSegmentPusherConfig; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.segment.realtime.firehose.IngestSegmentFirehose; +import io.druid.segment.realtime.firehose.WindowedStorageAdapter; +import io.druid.timeline.DataSegment; +import org.apache.calcite.adapter.druid.DruidTable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.druid.DruidStorageHandler; +import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils; +import org.apache.hadoop.hive.druid.io.DruidRecordWriter; +import org.apache.hadoop.hive.druid.serde.DruidWritable; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class DruidRecordWriterTest { + private ObjectMapper objectMapper = DruidStorageHandlerUtils.JSON_MAPPER; + + private static final Interval INTERVAL_FULL = new Interval("2014-10-22T00:00:00Z/P1D"); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private DruidRecordWriter druidRecordWriter; + + final List> expectedRows = ImmutableList.of( + ImmutableMap.of( + DruidTable.DEFAULT_TIMESTAMP_COLUMN, + DateTime.parse("2014-10-22T00:00:00.000Z").getMillis(), + "host", ImmutableList.of("a.example.com"), + "visited_sum", 190L, + "unique_hosts", 1.0d + ), + ImmutableMap.of( + DruidTable.DEFAULT_TIMESTAMP_COLUMN, + DateTime.parse("2014-10-22T01:00:00.000Z").getMillis(), + "host", ImmutableList.of("b.example.com"), + "visited_sum", 175L, + "unique_hosts", 1.0d + ), + ImmutableMap.of( + DruidTable.DEFAULT_TIMESTAMP_COLUMN, + DateTime.parse("2014-10-22T02:00:00.000Z").getMillis(), + "host", ImmutableList.of("c.example.com"), + "visited_sum", 270L, + "unique_hosts", 1.0d + ) + ); + + // This test need this patch https://github.com/druid-io/druid/pull/3483 + @Ignore + @Test + public void testWrite() throws IOException, SegmentLoadingException { + + final String datasourceName = "testDataSource"; + final File segmentOutputDir = temporaryFolder.newFolder(); + final File segmentDescriptorOutputDir = temporaryFolder.newFolder(); + Configuration config = new Configuration(); + + final InputRowParser inputRowParser = new MapInputRowParser(new TimeAndDimsParseSpec( + new TimestampSpec(DruidTable.DEFAULT_TIMESTAMP_COLUMN, "auto", null), + new DimensionsSpec(ImmutableList.of(new StringDimensionSchema("host")), + null, null + ) + )); + final Map parserMap = objectMapper.convertValue(inputRowParser, Map.class); + + DataSchema dataSchema = new DataSchema( + datasourceName, + parserMap, + new AggregatorFactory[] { + new LongSumAggregatorFactory("visited_sum", "visited_sum"), + new HyperUniquesAggregatorFactory("unique_hosts", "unique_hosts") + }, + new UniformGranularitySpec( + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(INTERVAL_FULL) + ), + objectMapper + ); + + RealtimeTuningConfig tuningConfig = RealtimeTuningConfig + .makeDefaultTuningConfig(temporaryFolder.newFolder()); + LocalFileSystem localFileSystem = FileSystem.getLocal(config); + DataSegmentPusher dataSegmentPusher = new LocalDataSegmentPusher( + new LocalDataSegmentPusherConfig() { + @Override + public File getStorageDirectory() {return segmentOutputDir;} + }, objectMapper); + + druidRecordWriter = new DruidRecordWriter(dataSchema, tuningConfig, dataSegmentPusher, 20, + new Path(segmentDescriptorOutputDir.getAbsolutePath(), + DruidStorageHandler.SEGMENTS_DESCRIPTOR_DIR_NAME + ), localFileSystem + ); + + List druidWritables = Lists.transform(expectedRows, + new Function, DruidWritable>() { + @Nullable + @Override + public DruidWritable apply(@Nullable ImmutableMap input + ) { + return new DruidWritable(ImmutableMap.builder().putAll(input) + .put(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME, + Granularity.DAY.truncate( + new DateTime((long) input + .get(DruidTable.DEFAULT_TIMESTAMP_COLUMN))) + .getMillis() + ).build()); + } + } + ); + for (DruidWritable druidWritable : druidWritables) { + druidRecordWriter.write(druidWritable); + } + druidRecordWriter.close(false); + List dataSegmentList = DruidStorageHandlerUtils + .getPublishedSegments(new Path(segmentDescriptorOutputDir.getParent()), config); + Assert.assertEquals(1, dataSegmentList.size()); + File tmpUnzippedSegmentDir = temporaryFolder.newFolder(); + new LocalDataSegmentPuller().getSegmentFiles(dataSegmentList.get(0), tmpUnzippedSegmentDir); + final QueryableIndex queryableIndex = DruidStorageHandlerUtils.INDEX_IO + .loadIndex(tmpUnzippedSegmentDir); + + QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(queryableIndex); + + Firehose firehose = new IngestSegmentFirehose( + ImmutableList.of(new WindowedStorageAdapter(adapter, adapter.getInterval())), + ImmutableList.of("host"), + ImmutableList.of("visited_sum", "unique_hosts"), + null, + QueryGranularities.NONE + ); + + List rows = Lists.newArrayList(); + while (firehose.hasMore()) { + rows.add(firehose.nextRow()); + } + + verifyRows(expectedRows, rows); + + } + + private void verifyRows(List> expectedRows, + List actualRows + ) { + System.out.println("actualRows = " + actualRows); + Assert.assertEquals(expectedRows.size(), actualRows.size()); + + for (int i = 0; i < expectedRows.size(); i++) { + Map expected = expectedRows.get(i); + InputRow actual = actualRows.get(i); + + Assert.assertEquals(ImmutableList.of("host"), actual.getDimensions()); + + Assert.assertEquals(expected.get(DruidTable.DEFAULT_TIMESTAMP_COLUMN), + actual.getTimestamp().getMillis() + ); + Assert.assertEquals(expected.get("host"), actual.getDimension("host")); + Assert.assertEquals(expected.get("visited_sum"), actual.getLongMetric("visited_sum")); + Assert.assertEquals( + (Double) expected.get("unique_hosts"), + (Double) HyperUniquesAggregatorFactory + .estimateCardinality(actual.getRaw("unique_hosts")), + 0.001 + ); + } + } + + @Test + public void testSerDesr() throws IOException { + String segment = "{\"dataSource\":\"datasource2015\",\"interval\":\"2015-06-01T00:00:00.000-04:00/2015-06-02T00:00:00.000-04:00\",\"version\":\"2016-11-04T19:24:01.732-04:00\",\"loadSpec\":{\"type\":\"hdfs\",\"path\":\"hdfs://cn105-10.l42scl.hortonworks.com:8020/apps/hive/warehouse/druid.db/.hive-staging_hive_2016-11-04_19-23-50_168_1550339856804207572-1/_task_tmp.-ext-10002/_tmp.000000_0/datasource2015/20150601T000000.000-0400_20150602T000000.000-0400/2016-11-04T19_24_01.732-04_00/0/index.zip\"},\"dimensions\":\"dimension1\",\"metrics\":\"bigint\",\"shardSpec\":{\"type\":\"linear\",\"partitionNum\":0},\"binaryVersion\":9,\"size\":1765,\"identifier\":\"datasource2015_2015-06-01T00:00:00.000-04:00_2015-06-02T00:00:00.000-04:00_2016-11-04T19:24:01.732-04:00\"}"; + DataSegment dataSegment = objectMapper.readerFor(DataSegment.class) + .readValue(segment); + Assert.assertTrue(dataSegment.getDataSource().equals("datasource2015")); + } + +} \ No newline at end of file diff --git llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java index 87bd5c8..5e71d4d 100644 --- llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java +++ llap-server/src/java/org/apache/hadoop/hive/llap/daemon/impl/TaskRunnerCallable.java @@ -17,19 +17,12 @@ */ package org.apache.hadoop.hive.llap.daemon.impl; -import java.net.InetSocketAddress; -import java.nio.ByteBuffer; -import java.security.PrivilegedExceptionAction; -import java.util.HashMap; -import java.util.Map; -import java.util.Stack; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; - +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.llap.daemon.FragmentCompletionHandler; @@ -50,7 +43,6 @@ import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.log4j.MDC; import org.apache.log4j.NDC; import org.apache.tez.common.CallableWithNdc; @@ -58,10 +50,7 @@ import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.TokenCache; import org.apache.tez.dag.api.TezConstants; -import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskAttemptID; -import org.apache.tez.dag.records.TezTaskID; -import org.apache.tez.dag.records.TezVertexID; import org.apache.tez.hadoop.shim.HadoopShim; import org.apache.tez.runtime.api.ExecutionContext; import org.apache.tez.runtime.api.impl.TaskSpec; @@ -75,15 +64,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Joiner; -import com.google.common.base.Stopwatch; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.security.PrivilegedExceptionAction; +import java.util.HashMap; +import java.util.Map; +import java.util.Stack; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; /** * @@ -116,8 +107,8 @@ private final String queryId; private final HadoopShim tezHadoopShim; private boolean shouldRunTask = true; - final Stopwatch runtimeWatch = new Stopwatch(); - final Stopwatch killtimerWatch = new Stopwatch(); + final Stopwatch runtimeWatch = Stopwatch.createStarted(); + final Stopwatch killtimerWatch = Stopwatch.createStarted(); private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isCompleted = new AtomicBoolean(false); private final AtomicBoolean killInvoked = new AtomicBoolean(false); @@ -265,7 +256,7 @@ public LlapTaskUmbilicalProtocol run() throws Exception { } finally { FileSystem.closeAllForUGI(taskUgi); LOG.info("ExecutionTime for Container: " + request.getContainerIdString() + "=" + - runtimeWatch.stop().elapsedMillis()); + runtimeWatch.stop().elapsed(TimeUnit.MILLISECONDS)); if (LOG.isDebugEnabled()) { LOG.debug( "canFinish post completion: " + taskSpec.getTaskAttemptID() + ": " + canFinish()); @@ -458,14 +449,14 @@ public void onSuccess(TaskRunner2Result result) { LOG.info("Killed task {}", requestId); if (killtimerWatch.isRunning()) { killtimerWatch.stop(); - long elapsed = killtimerWatch.elapsedMillis(); + long elapsed = killtimerWatch.elapsed(TimeUnit.MILLISECONDS); LOG.info("Time to die for task {}", elapsed); if (metrics != null) { metrics.addMetricsPreemptionTimeToKill(elapsed); } } if (metrics != null) { - metrics.addMetricsPreemptionTimeLost(runtimeWatch.elapsedMillis()); + metrics.addMetricsPreemptionTimeLost(runtimeWatch.elapsed(TimeUnit.MILLISECONDS)); metrics.incrExecutorTotalKilled(); } break; diff --git pom.xml pom.xml index 8c58f4c..f01e568 100644 --- pom.xml +++ pom.xml @@ -136,8 +136,8 @@ 10.10.2.0 3.1.0 0.1.2 - 0.9.1.1 - 14.0.1 + 0.9.2 + 16.0.1 2.4.4 2.7.2 ${basedir}/${hive.path.to.root}/testutils/hadoop diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 54d619c..3fea2b9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -265,7 +265,7 @@ public Stat getStat() { protected transient FileSystem fs; protected transient Serializer serializer; protected final transient LongWritable row_count = new LongWritable(); - private transient boolean isNativeTable = true; + private transient boolean needsNativeStorage = true; /** * The evaluators for the multiFile sprayer. If the table under consideration has 1000 buckets, @@ -334,7 +334,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { try { this.hconf = hconf; filesCreated = false; - isNativeTable = !conf.getTableInfo().isNonNative(); + needsNativeStorage = conf.getTableInfo().needsNativeStorage(); isTemporary = conf.isTemporary(); multiFileSpray = conf.isMultiFileSpray(); totalFiles = conf.getTotalFiles(); @@ -556,7 +556,7 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { assert filesIdx == numFiles; // in recent hadoop versions, use deleteOnExit to clean tmp files. - if (isNativeTable && fs != null && fsp != null) { + if (needsNativeStorage && fs != null && fsp != null) { autoDelete = fs.deleteOnExit(fsp.outPaths[0]); } } catch (Exception e) { @@ -570,7 +570,7 @@ protected void createBucketFiles(FSPaths fsp) throws HiveException { protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) throws HiveException { try { - if (isNativeTable) { + if (needsNativeStorage) { fsp.finalPaths[filesIdx] = fsp.getFinalPath(taskId, fsp.tmpPath, null); if (isInfoEnabled) { LOG.info("Final Path: FS " + fsp.finalPaths[filesIdx]); @@ -597,7 +597,7 @@ protected void createBucketForFileIdx(FSPaths fsp, int filesIdx) LOG.info("New Final Path: FS " + fsp.finalPaths[filesIdx]); } - if (isNativeTable) { + if (needsNativeStorage) { // in recent hadoop versions, use deleteOnExit to clean tmp files. autoDelete = fs.deleteOnExit(fsp.outPaths[filesIdx]); } @@ -1067,7 +1067,7 @@ public void closeOp(boolean abort) throws HiveException { } } - if (isNativeTable) { + if (needsNativeStorage) { fsp.commit(fs); } } @@ -1080,7 +1080,7 @@ public void closeOp(boolean abort) throws HiveException { // Hadoop always call close() even if an Exception was thrown in map() or // reduce(). for (FSPaths fsp : valToPaths.values()) { - fsp.abortWriters(fs, abort, !autoDelete && isNativeTable); + fsp.abortWriters(fs, abort, !autoDelete && needsNativeStorage); } } fsp = prevFsp = null; @@ -1103,7 +1103,7 @@ static public String getOperatorName() { public void jobCloseOp(Configuration hconf, boolean success) throws HiveException { try { - if ((conf != null) && isNativeTable) { + if ((conf != null) && needsNativeStorage) { Path specPath = conf.getDirName(); DynamicPartitionCtx dpCtx = conf.getDynPartCtx(); if (conf.isLinkedFileSink() && (dpCtx != null)) { diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 2b1d1ce..c902254 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -3054,8 +3054,8 @@ private static Path createDummyFileForEmptyPartition(Path path, JobConf job, Map // The input file does not exist, replace it by a empty file PartitionDesc partDesc = work.getPathToPartitionInfo().get(path); - if (partDesc.getTableDesc().isNonNative()) { - // if this isn't a hive table we can't create an empty file for it. + if (!partDesc.getTableDesc().needsNativeStorage()) { + // if it does not need native storage, we can't create an empty file for it. return path; } @@ -3090,8 +3090,8 @@ private static Path createDummyFileForEmptyTable(JobConf job, MapWork work, throws Exception { TableDesc tableDesc = work.getAliasToPartnInfo().get(alias).getTableDesc(); - if (tableDesc.isNonNative()) { - // if this isn't a hive table we can't create an empty file for it. + if (!tableDesc.needsNativeStorage()) { + // if it does not need native storage, we can't create an empty file for it. return null; } diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java index 178a2de..27cb36d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java @@ -18,21 +18,14 @@ package org.apache.hadoop.hive.ql.hooks; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import com.google.gson.stream.JsonWriter; import org.apache.commons.collections.SetUtils; import org.apache.commons.io.output.StringBuilderWriter; import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.common.ObjectPair; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -50,11 +43,18 @@ import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; -import com.google.gson.stream.JsonWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * Implementation of a post execute hook that logs lineage info to a log file. @@ -462,7 +462,7 @@ private void writeVertices(JsonWriter writer, Set vertices) throws IOExc */ private String getQueryHash(String queryStr) { Hasher hasher = Hashing.md5().newHasher(); - hasher.putString(queryStr); + hasher.putString(queryStr, Charsets.UTF_8); return hasher.hash().toString(); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index eaf0abc..d9c2ff4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -122,6 +122,8 @@ public void initialize(HiveConf hiveConf) { transformations.add(new SortedDynPartitionOptimizer()); } + transformations.add(new SortedDynPartitionTimeGranularityOptimizer()); + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) { transformations.add(new PartitionPruner()); transformations.add(new PartitionConditionRemover()); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java new file mode 100644 index 0000000..bd5025b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java @@ -0,0 +1,353 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.optimizer; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.Constants; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.RowSchema; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.Utilities.ReduceField; +import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PlanUtils; +import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorDay; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorHour; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorMinute; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorMonth; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorSecond; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorWeek; +import org.apache.hadoop.hive.ql.udf.UDFDateFloorYear; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +/** + * Introduces a RS before FS to partition data by configuration specified + * time granularity. + */ +public class SortedDynPartitionTimeGranularityOptimizer extends Transform { + + @Override + public ParseContext transform(ParseContext pCtx) throws SemanticException { + // create a walker which walks the tree in a DFS manner while maintaining the + // operator stack. The dispatcher generates the plan from the operator tree + Map opRules = new LinkedHashMap(); + + String FS = FileSinkOperator.getOperatorName() + "%"; + + opRules.put(new RuleRegExp("Sorted Dynamic Partition Time Granularity", FS), getSortDynPartProc(pCtx)); + + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null); + GraphWalker ogw = new DefaultGraphWalker(disp); + + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pCtx.getTopOps().values()); + ogw.startWalking(topNodes, null); + + return pCtx; + } + + private NodeProcessor getSortDynPartProc(ParseContext pCtx) { + return new SortedDynamicPartitionProc(pCtx); + } + + class SortedDynamicPartitionProc implements NodeProcessor { + + private final Logger LOG = LoggerFactory.getLogger(SortedDynPartitionTimeGranularityOptimizer.class); + protected ParseContext parseCtx; + + public SortedDynamicPartitionProc(ParseContext pCtx) { + this.parseCtx = pCtx; + } + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + + // introduce RS and EX before FS + FileSinkOperator fsOp = (FileSinkOperator) nd; + + final String sh = fsOp.getConf().getTableInfo().getOutputFileFormatClassName(); + if (sh == null || !sh.equals(Constants.DRUID_HIVE_OUTPUT_FORMAT)) { + // Bail out, nothing to do + return null; + } + + LOG.info("Sorted dynamic partitioning on time granularity optimization kicked in..."); + + // unlink connection between FS and its parent + Operator fsParent = fsOp.getParentOperators().get(0); + fsParent = fsOp.getParentOperators().get(0); + fsParent.getChildOperators().clear(); + + // Create SelectOp with granularity column + Operator granularitySelOp = getGranularitySelOp(fsParent); + + // Create ReduceSinkOp operator + ArrayList parentCols = Lists.newArrayList(granularitySelOp.getSchema().getSignature()); + ArrayList allRSCols = Lists.newArrayList(); + for (ColumnInfo ci : parentCols) { + allRSCols.add(new ExprNodeColumnDesc(ci)); + } + // Get the key positions + List keyPositions = new ArrayList<>(); + keyPositions.add(allRSCols.size() - 1); + List sortOrder = new ArrayList(1); + sortOrder.add(1); // asc + List sortNullOrder = new ArrayList(1); + sortNullOrder.add(0); // nulls first + ReduceSinkOperator rsOp = getReduceSinkOp(keyPositions, sortOrder, + sortNullOrder, allRSCols, granularitySelOp, fsOp.getConf().getWriteType()); + + // Create backtrack SelectOp + List descs = new ArrayList(allRSCols.size()); + List colNames = new ArrayList(); + String colName; + for (int i = 0; i < allRSCols.size(); i++) { + ExprNodeDesc col = allRSCols.get(i); + colName = col.getExprString(); + colNames.add(colName); + if (keyPositions.contains(i)) { + descs.add(new ExprNodeColumnDesc(col.getTypeInfo(), ReduceField.KEY.toString()+"."+colName, null, false)); + } else { + descs.add(new ExprNodeColumnDesc(col.getTypeInfo(), ReduceField.VALUE.toString()+"."+colName, null, false)); + } + } + RowSchema selRS = new RowSchema(granularitySelOp.getSchema()); + SelectDesc selConf = new SelectDesc(descs, colNames); + SelectOperator backtrackSelOp = (SelectOperator) OperatorFactory.getAndMakeChild( + selConf, selRS, rsOp); + + // Link backtrack SelectOp to FileSinkOp + fsOp.getParentOperators().clear(); + fsOp.getParentOperators().add(backtrackSelOp); + backtrackSelOp.getChildOperators().add(fsOp); + + // Update file sink descriptor + fsOp.getConf().setDpSortState(FileSinkDesc.DPSortState.PARTITION_SORTED); + fsOp.getConf().setPartitionCols(rsOp.getConf().getPartitionCols()); + ColumnInfo ci = new ColumnInfo(granularitySelOp.getSchema().getSignature().get( + granularitySelOp.getSchema().getSignature().size() - 1)); // granularity column + fsOp.getSchema().getSignature().add(ci); + + LOG.info("Inserted " + granularitySelOp.getOperatorId() + ", " + rsOp.getOperatorId() + " and " + + backtrackSelOp.getOperatorId() + " as parent of " + fsOp.getOperatorId() + + " and child of " + fsParent.getOperatorId()); + + parseCtx.setReduceSinkAddedBySortedDynPartition(true); + return null; + } + + private Operator getGranularitySelOp( + Operator fsParent) throws SemanticException { + ArrayList parentCols = Lists.newArrayList(fsParent.getSchema().getSignature()); + ArrayList descs = Lists.newArrayList(); + List colNames = Lists.newArrayList(); + int timestampPos = -1; + for (int i = 0; i < parentCols.size(); i++) { + ColumnInfo ci = parentCols.get(i); + ExprNodeColumnDesc columnDesc = new ExprNodeColumnDesc(ci); + descs.add(columnDesc); + colNames.add(columnDesc.getExprString()); + if (columnDesc.getTypeInfo().getCategory() == ObjectInspector.Category.PRIMITIVE + && ((PrimitiveTypeInfo) columnDesc.getTypeInfo()).getPrimitiveCategory() == PrimitiveCategory.TIMESTAMP) { + if (timestampPos != -1) { + throw new SemanticException("Multiple columns with timestamp type on query result; " + + "could not resolve which one is the timestamp column"); + } + timestampPos = i; + } + } + if (timestampPos == -1) { + throw new SemanticException("No column with timestamp type on query result; " + + "one column should be of timestamp type"); + } + RowSchema selRS = new RowSchema(fsParent.getSchema()); + // Granularity (partition) column + String udfName; + Class udfClass; + switch (parseCtx.getConf().getVar(HiveConf.ConfVars.HIVE_DRUID_INDEXING_GRANULARITY)) { + case "YEAR": + udfName = "floor_year"; + udfClass = UDFDateFloorYear.class; + break; + case "MONTH": + udfName = "floor_month"; + udfClass = UDFDateFloorMonth.class; + break; + case "WEEK": + udfName = "floor_week"; + udfClass = UDFDateFloorWeek.class; + break; + case "DAY": + udfName = "floor_day"; + udfClass = UDFDateFloorDay.class; + break; + case "HOUR": + udfName = "floor_hour"; + udfClass = UDFDateFloorHour.class; + break; + case "MINUTE": + udfName = "floor_minute"; + udfClass = UDFDateFloorMinute.class; + break; + case "SECOND": + udfName = "floor_second"; + udfClass = UDFDateFloorSecond.class; + break; + default: + throw new SemanticException("Granularity for Druid segment not recognized"); + } + ExprNodeDesc expr = new ExprNodeColumnDesc(parentCols.get(timestampPos)); + descs.add(new ExprNodeGenericFuncDesc( + TypeInfoFactory.timestampTypeInfo, + new GenericUDFBridge(udfName, false, udfClass.getName()), + Lists.newArrayList(expr))); + colNames.add(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME); + // Add granularity to the row schema + ColumnInfo ci = new ColumnInfo(Constants.DRUID_TIMESTAMP_GRANULARITY_COL_NAME, TypeInfoFactory.timestampTypeInfo, + selRS.getSignature().get(0).getTabAlias(), false, false); + selRS.getSignature().add(ci); + + // Create SelectDesc + SelectDesc selConf = new SelectDesc(descs, colNames); + + // Create Select Operator + SelectOperator selOp = (SelectOperator) OperatorFactory.getAndMakeChild( + selConf, selRS, fsParent); + + return selOp; + } + + private ReduceSinkOperator getReduceSinkOp(List keyPositions, List sortOrder, + List sortNullOrder, ArrayList allCols, Operator parent, + AcidUtils.Operation writeType) throws SemanticException { + + ArrayList keyCols = Lists.newArrayList(); + // we will clone here as RS will update bucket column key with its + // corresponding with bucket number and hence their OIs + for (Integer idx : keyPositions) { + keyCols.add(allCols.get(idx).clone()); + } + + ArrayList valCols = Lists.newArrayList(); + for (int i = 0; i < allCols.size(); i++) { + if (!keyPositions.contains(i)) { + valCols.add(allCols.get(i).clone()); + } + } + + ArrayList partCols = Lists.newArrayList(); + for (Integer idx : keyPositions) { + partCols.add(allCols.get(idx).clone()); + } + + // map _col0 to KEY._col0, etc + Map colExprMap = Maps.newHashMap(); + Map nameMapping = new HashMap<>(); + ArrayList keyColNames = Lists.newArrayList(); + for (ExprNodeDesc keyCol : keyCols) { + String keyColName = keyCol.getExprString(); + keyColNames.add(keyColName); + colExprMap.put(Utilities.ReduceField.KEY + "." +keyColName, keyCol); + nameMapping.put(keyColName, Utilities.ReduceField.KEY + "." + keyColName); + } + ArrayList valColNames = Lists.newArrayList(); + for (ExprNodeDesc valCol : valCols) { + String colName = valCol.getExprString(); + valColNames.add(colName); + colExprMap.put(Utilities.ReduceField.VALUE + "." + colName, valCol); + nameMapping.put(colName, Utilities.ReduceField.VALUE + "." + colName); + } + + // order and null order + String orderStr = StringUtils.repeat("+", sortOrder.size()); + String nullOrderStr = StringUtils.repeat("a", sortNullOrder.size()); + + // Create Key/Value TableDesc. When the operator plan is split into MR tasks, + // the reduce operator will initialize Extract operator with information + // from Key and Value TableDesc + List fields = PlanUtils.getFieldSchemasFromColumnList(keyCols, + keyColNames, 0, ""); + TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, orderStr, nullOrderStr); + List valFields = PlanUtils.getFieldSchemasFromColumnList(valCols, + valColNames, 0, ""); + TableDesc valueTable = PlanUtils.getReduceValueTableDesc(valFields); + List> distinctColumnIndices = Lists.newArrayList(); + + // Number of reducers is set to default (-1) + ReduceSinkDesc rsConf = new ReduceSinkDesc(keyCols, keyCols.size(), valCols, + keyColNames, distinctColumnIndices, valColNames, -1, partCols, -1, keyTable, + valueTable, writeType); + + ArrayList signature = new ArrayList<>(); + for (int index = 0; index < parent.getSchema().getSignature().size(); index++) { + ColumnInfo colInfo = new ColumnInfo(parent.getSchema().getSignature().get(index)); + colInfo.setInternalName(nameMapping.get(colInfo.getInternalName())); + signature.add(colInfo); + } + ReduceSinkOperator op = (ReduceSinkOperator) OperatorFactory.getAndMakeChild( + rsConf, new RowSchema(signature), parent); + op.setColumnExprMap(colExprMap); + return op; + } + + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index d55db0a..13a5251 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -11712,10 +11712,6 @@ ASTNode analyzeCreateTable( storageFormat.fillDefaultStorageFormat(isExt, false); - if ((command_type == CTAS) && (storageFormat.getStorageHandler() != null)) { - throw new SemanticException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg()); - } - // check for existence of table if (ifNotExists) { try { diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java index b15ad34..74769e4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.llap.LlapOutputFormat; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; @@ -46,7 +47,6 @@ import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; -import org.apache.hadoop.hive.llap.LlapOutputFormat; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; @@ -71,6 +71,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputFormat; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapred.TextInputFormat; @@ -306,17 +307,26 @@ public static TableDesc getDefaultQueryOutputTableDesc(String cols, String colTy public static TableDesc getTableDesc(CreateTableDesc crtTblDesc, String cols, String colTypes) { - Class serdeClass = LazySimpleSerDe.class; - String separatorCode = Integer.toString(Utilities.ctrlaCode); - String columns = cols; - String columnTypes = colTypes; - boolean lastColumnTakesRestOfTheLine = false; TableDesc ret; + // Resolve storage handler (if any) try { - if (crtTblDesc.getSerName() != null) { - Class c = JavaUtils.loadClass(crtTblDesc.getSerName()); - serdeClass = c; + HiveStorageHandler storageHandler = null; + if (crtTblDesc.getStorageHandler() != null) { + storageHandler = HiveUtils.getStorageHandler( + SessionState.getSessionConf(), crtTblDesc.getStorageHandler()); + } + + Class serdeClass = LazySimpleSerDe.class; + String separatorCode = Integer.toString(Utilities.ctrlaCode); + String columns = cols; + String columnTypes = colTypes; + boolean lastColumnTakesRestOfTheLine = false; + + if (storageHandler != null) { + serdeClass = storageHandler.getSerDeClass(); + } else if (crtTblDesc.getSerName() != null) { + serdeClass = JavaUtils.loadClass(crtTblDesc.getSerName()); } if (crtTblDesc.getFieldDelim() != null) { @@ -329,6 +339,12 @@ public static TableDesc getTableDesc(CreateTableDesc crtTblDesc, String cols, // set other table properties Properties properties = ret.getProperties(); + if (crtTblDesc.getStorageHandler() != null) { + properties.setProperty( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, + crtTblDesc.getStorageHandler()); + } + if (crtTblDesc.getCollItemDelim() != null) { properties.setProperty(serdeConstants.COLLECTION_DELIM, crtTblDesc .getCollItemDelim()); @@ -367,15 +383,24 @@ public static TableDesc getTableDesc(CreateTableDesc crtTblDesc, String cols, // replace the default input & output file format with those found in // crtTblDesc - Class c1 = JavaUtils.loadClass(crtTblDesc.getInputFormat()); - Class c2 = JavaUtils.loadClass(crtTblDesc.getOutputFormat()); - Class in_class = c1; - Class out_class = c2; - + Class in_class; + if (storageHandler != null) { + in_class = storageHandler.getInputFormatClass(); + } else { + in_class = JavaUtils.loadClass(crtTblDesc.getInputFormat()); + } + Class out_class; + if (storageHandler != null) { + out_class = storageHandler.getOutputFormatClass(); + } else { + out_class = JavaUtils.loadClass(crtTblDesc.getOutputFormat()); + } ret.setInputFileFormatClass(in_class); ret.setOutputFileFormatClass(out_class); } catch (ClassNotFoundException e) { throw new RuntimeException("Unable to find class in getTableDesc: " + e.getMessage(), e); + } catch (HiveException e) { + throw new RuntimeException("Error loading storage handler in getTableDesc: " + e.getMessage(), e); } return ret; } @@ -1101,4 +1126,4 @@ public static ReadEntity getParentViewInfo(String alias_id, return currentInput; } -} +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java index 1da8e91..fee11cc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java @@ -25,24 +25,33 @@ import java.util.Properties; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.Constants; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.ql.plan.Explain.Level; +import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.OutputFormat; - import org.apache.hive.common.util.HiveStringUtils; import org.apache.hive.common.util.ReflectionUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * TableDesc. * */ public class TableDesc implements Serializable, Cloneable { + + private static final Logger LOG = LoggerFactory.getLogger(TableDesc.class); + private static final long serialVersionUID = 1L; private Class inputFileFormatClass; private Class outputFileFormatClass; @@ -165,6 +174,28 @@ public boolean isNonNative() { return (properties.getProperty(hive_metastoreConstants.META_TABLE_STORAGE) != null); } + public boolean needsNativeStorage() { + // TODO: We add this exception for the moment. We should allow storage handlers to + // specify their desired behavior. + String handlerClass = properties.getProperty( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE); + if (handlerClass == null) { + // Native + return true; + } + try { + HiveStorageHandler storageHandler = HiveUtils.getStorageHandler( + SessionState.getSessionConf(), handlerClass); + if (storageHandler.toString().equals(Constants.DRUID_HIVE_STORAGE_HANDLER_ID)) { + return true; + } + } catch (HiveException e) { + LOG.warn("Could not resolve storage handler"); + } + // Safe bail out + return false; + } + @Override public Object clone() { TableDesc ret = new TableDesc();