diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 7f4afd9..dbbddd0 100644
--- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -366,7 +366,6 @@
METASTORECONNECTURLKEY("javax.jdo.option.ConnectionURL",
"jdbc:derby:;databaseName=metastore_db;create=true",
"JDBC connect string for a JDBC metastore"),
-
HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 1,
"The number of times to retry a HMSHandler call if there were a connection error"),
HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", 1000,
@@ -1729,7 +1728,15 @@
"When auto reducer parallelism is enabled this factor will be used to over-partition data in shuffle edges."),
TEZ_MIN_PARTITION_FACTOR("hive.tez.min.partition.factor", 0.25f,
"When auto reducer parallelism is enabled this factor will be used to put a lower limit to the number\n" +
- "of reducers that tez specifies.")
+ "of reducers that tez specifies."),
+ TEZ_DYNAMIC_PARTITION_PRUNING(
+ "hive.tez.dynamic.partition.pruning", true,
+ "When dynamic pruning is enabled, joins on partition keys will be processed by sending events from the processing " +
+ "vertices to the tez application master. These events will be used to prune unnecessary partitions."),
+ TEZ_DYNAMIC_PARTITION_PRUNING_MAX_EVENT_SIZE("hive.tez.dynamic.partition.pruning.max.event.size", 1*1024*1024L,
+ "Maximum size of events sent by processors in dynamic pruning. If this size is crossed no pruning will take place."),
+ TEZ_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE("hive.tez.dynamic.parition.pruning.max.data.size", 100*1024*1024L,
+ "Maximum total data size of events in dynamic pruning.")
;
public final String varname;
diff --git conf/hive-default.xml.template conf/hive-default.xml.template
new file mode 100644
index 0000000..c6aa66e
--- /dev/null
+++ conf/hive-default.xml.template
@@ -0,0 +1,3054 @@
+
+
+
+
+
+
+
+
+
+
+ hive.exec.script.wrapper
+
+
+
+
+ hive.exec.plan
+
+
+
+
+ hive.plan.serialization.format
+ kryo
+
+ Query plan format serialization between client and task nodes.
+ Two supported values are : kryo and javaXML. Kryo is default.
+
+
+
+ hive.exec.scratchdir
+ /tmp/hive-${system:user.name}
+ Scratch space for Hive jobs
+
+
+ hive.exec.local.scratchdir
+ ${system:java.io.tmpdir}/${system:user.name}
+ Local scratch space for Hive jobs
+
+
+ hive.scratch.dir.permission
+ 700
+
+
+
+ hive.exec.submitviachild
+ false
+
+
+
+ hive.exec.submit.local.task.via.child
+ true
+
+ Determines whether local tasks (typically mapjoin hashtable generation phase) runs in
+ separate JVM (true recommended) or not.
+ Avoids the overhead of spawning new JVM, but can lead to out-of-memory issues.
+
+
+
+ hive.exec.script.maxerrsize
+ 100000
+
+ Maximum number of bytes a script is allowed to emit to standard error (per map-reduce task).
+ This prevents runaway scripts from filling logs partitions to capacity
+
+
+
+ hive.exec.script.allow.partial.consumption
+ false
+
+ When enabled, this option allows a user script to exit successfully without consuming
+ all the data from the standard input.
+
+
+
+ stream.stderr.reporter.prefix
+ reporter:
+ Streaming jobs that log to standard error with this prefix can log counter or status information.
+
+
+ stream.stderr.reporter.enabled
+ true
+ Enable consumption of status and counter messages for streaming jobs.
+
+
+ hive.exec.compress.output
+ false
+
+ This controls whether the final outputs of a query (to a local/HDFS file or a Hive table) is compressed.
+ The compression codec and other options are determined from Hadoop config variables mapred.output.compress*
+
+
+
+ hive.exec.compress.intermediate
+ false
+
+ This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed.
+ The compression codec and other options are determined from Hadoop config variables mapred.output.compress*
+
+
+
+ hive.intermediate.compression.codec
+
+
+
+
+ hive.intermediate.compression.type
+
+
+
+
+ hive.exec.reducers.bytes.per.reducer
+ 1000000000
+ size per reducer.The default is 1G, i.e if the input size is 10G, it will use 10 reducers.
+
+
+ hive.exec.reducers.max
+ 999
+
+ max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is
+ negative, Hive will use this one as the max number of reducers when automatically determine number of reducers.
+
+
+
+ hive.exec.pre.hooks
+
+
+ Comma-separated list of pre-execution hooks to be invoked for each statement.
+ A pre-execution hook is specified as the name of a Java class which implements the
+ org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+
+
+
+ hive.exec.post.hooks
+
+
+ Comma-separated list of post-execution hooks to be invoked for each statement.
+ A post-execution hook is specified as the name of a Java class which implements the
+ org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+
+
+
+ hive.exec.failure.hooks
+
+
+ Comma-separated list of on-failure hooks to be invoked for each statement.
+ An on-failure hook is specified as the name of Java class which implements the
+ org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
+
+
+
+ hive.client.stats.publishers
+
+
+ Comma-separated list of statistics publishers to be invoked on counters on each job.
+ A client stats publisher is specified as the name of a Java class which implements the
+ org.apache.hadoop.hive.ql.stats.ClientStatsPublisher interface.
+
+
+
+ hive.exec.parallel
+ false
+ Whether to execute jobs in parallel
+
+
+ hive.exec.parallel.thread.number
+ 8
+ How many jobs at most can be executed in parallel
+
+
+ hive.mapred.reduce.tasks.speculative.execution
+ true
+ Whether speculative execution for reducers should be turned on.
+
+
+ hive.exec.counters.pull.interval
+ 1000
+
+ The interval with which to poll the JobTracker for the counters the running job.
+ The smaller it is the more load there will be on the jobtracker, the higher it is the less granular the caught will be.
+
+
+
+ hive.exec.dynamic.partition
+ true
+ Whether or not to allow dynamic partitions in DML/DDL.
+
+
+ hive.exec.dynamic.partition.mode
+ strict
+
+ In strict mode, the user must specify at least one static partition
+ in case the user accidentally overwrites all partitions.
+
+
+
+ hive.exec.max.dynamic.partitions
+ 1000
+ Maximum number of dynamic partitions allowed to be created in total.
+
+
+ hive.exec.max.dynamic.partitions.pernode
+ 100
+ Maximum number of dynamic partitions allowed to be created in each mapper/reducer node.
+
+
+ hive.exec.max.created.files
+ 100000
+ Maximum number of HDFS files created by all mappers/reducers in a MapReduce job.
+
+
+ hive.downloaded.resources.dir
+ ${system:java.io.tmpdir}/${hive.session.id}_resources
+ Temporary local directory for added resources in the remote file system.
+
+
+ hive.exec.default.partition.name
+ __HIVE_DEFAULT_PARTITION__
+
+ The default partition name in case the dynamic partition column value is null/empty string or any other values that cannot be escaped.
+ This value must not contain any special character used in HDFS URI (e.g., ':', '%', '/' etc).
+ The user has to be aware that the dynamic partition value should not contain this value to avoid confusions.
+
+
+
+ hive.lockmgr.zookeeper.default.partition.name
+ __HIVE_DEFAULT_ZOOKEEPER_PARTITION__
+
+
+
+ hive.exec.show.job.failure.debug.info
+ true
+
+ If a job fails, whether to provide a link in the CLI to the task with the
+ most failures, along with debugging hints if applicable.
+
+
+
+ hive.exec.job.debug.capture.stacktraces
+ true
+
+ Whether or not stack traces parsed from the task logs of a sampled failed task
+ for each failed job should be stored in the SessionState
+
+
+
+ hive.exec.job.debug.timeout
+ 30000
+
+
+
+ hive.exec.tasklog.debug.timeout
+ 20000
+
+
+
+ hive.output.file.extension
+
+
+ String used as a file extension for output files.
+ If not set, defaults to the codec extension for text files (e.g. ".gz"), or no extension otherwise.
+
+
+
+ hive.exec.mode.local.auto
+ false
+ Let Hive determine whether to run in local mode automatically
+
+
+ hive.exec.mode.local.auto.inputbytes.max
+ 134217728
+ When hive.exec.mode.local.auto is true, input bytes should less than this for local mode.
+
+
+ hive.exec.mode.local.auto.input.files.max
+ 4
+ When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode.
+
+
+ hive.exec.drop.ignorenonexistent
+ true
+ Do not report an error if DROP TABLE/VIEW specifies a non-existent table/view
+
+
+ hive.ignore.mapjoin.hint
+ true
+ Ignore the mapjoin hint
+
+
+ hive.file.max.footer
+ 100
+ maximum number of lines for footer user can define for a table file
+
+
+ hive.resultset.use.unique.column.names
+ true
+
+ Make column names unique in the result set by qualifying column names with table alias if needed.
+ Table alias will be added to column names for queries of type "select *" or
+ if query explicitly uses table alias "select r1.x..".
+
+
+
+ fs.har.impl
+ org.apache.hadoop.hive.shims.HiveHarFileSystem
+ The implementation for accessing Hadoop Archives. Note that this won't be applicable to Hadoop versions less than 0.20
+
+
+ hive.metastore.metadb.dir
+
+
+
+
+ hive.metastore.warehouse.dir
+ /user/hive/warehouse
+ location of default database for the warehouse
+
+
+ hive.metastore.uris
+
+ Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore.
+
+
+ hive.metastore.connect.retries
+ 3
+ Number of retries while opening a connection to metastore
+
+
+ hive.metastore.failure.retries
+ 1
+ Number of retries upon failure of Thrift metastore calls
+
+
+ hive.metastore.client.connect.retry.delay
+ 1
+ Number of seconds for the client to wait between consecutive connection attempts
+
+
+ hive.metastore.client.socket.timeout
+ 600
+ MetaStore Client socket timeout in seconds
+
+
+ javax.jdo.option.ConnectionPassword
+ mine
+ password to use against metastore database
+
+
+ hive.metastore.ds.connection.url.hook
+
+ Name of the hook to use for retrieving the JDO connection URL. If empty, the value in javax.jdo.option.ConnectionURL is used
+
+
+ javax.jdo.option.Multithreaded
+ true
+ Set this to true if multiple threads access metastore through JDO concurrently.
+
+
+ javax.jdo.option.ConnectionURL
+ jdbc:derby:;databaseName=metastore_db;create=true
+ JDBC connect string for a JDBC metastore
+
+
+ hive.metastore.force.reload.conf
+ false
+
+ Whether to force reloading of the metastore configuration (including
+ the connection URL, before the next metastore query that accesses the
+ datastore. Once reloaded, this value is reset to false. Used for
+ testing only.
+
+
+
+ hive.hmshandler.retry.attempts
+ 1
+ The number of times to retry a HMSHandler call if there were a connection error
+
+
+ hive.hmshandler.retry.interval
+ 1000
+ The number of milliseconds between HMSHandler retry attempts
+
+
+ hive.hmshandler.force.reload.conf
+ false
+
+ Whether to force reloading of the HMSHandler configuration (including
+ the connection URL, before the next metastore query that accesses the
+ datastore. Once reloaded, this value is reset to false. Used for
+ testing only.
+
+
+
+ hive.metastore.server.min.threads
+ 200
+ Minimum number of worker threads in the Thrift server's pool.
+
+
+ hive.metastore.server.max.threads
+ 100000
+ Maximum number of worker threads in the Thrift server's pool.
+
+
+ hive.metastore.server.tcp.keepalive
+ true
+ Whether to enable TCP keepalive for the metastore server. Keepalive will prevent accumulation of half-open connections.
+
+
+ hive.metastore.archive.intermediate.original
+ _INTERMEDIATE_ORIGINAL
+
+ Intermediate dir suffixes used for archiving. Not important what they
+ are, as long as collisions are avoided
+
+
+
+ hive.metastore.archive.intermediate.archived
+ _INTERMEDIATE_ARCHIVED
+
+
+
+ hive.metastore.archive.intermediate.extracted
+ _INTERMEDIATE_EXTRACTED
+
+
+
+ hive.metastore.kerberos.keytab.file
+
+ The path to the Kerberos Keytab file containing the metastore Thrift server's service principal.
+
+
+ hive.metastore.kerberos.principal
+ hive-metastore/_HOST@EXAMPLE.COM
+
+ The service principal for the metastore Thrift server.
+ The special string _HOST will be replaced automatically with the correct host name.
+
+
+
+ hive.metastore.sasl.enabled
+ false
+ If true, the metastore Thrift interface will be secured with SASL. Clients must authenticate with Kerberos.
+
+
+ hive.metastore.thrift.framed.transport.enabled
+ false
+ If true, the metastore Thrift interface will use TFramedTransport. When false (default) a standard TTransport is used.
+
+
+ hive.cluster.delegation.token.store.class
+ org.apache.hadoop.hive.thrift.MemoryTokenStore
+ The delegation token store implementation. Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster.
+
+
+ hive.cluster.delegation.token.store.zookeeper.connectString
+
+ The ZooKeeper token store connect string.
+
+
+ hive.cluster.delegation.token.store.zookeeper.znode
+ /hive/cluster/delegation
+ The root path for token store data.
+
+
+ hive.cluster.delegation.token.store.zookeeper.acl
+
+ ACL for token store entries. List comma separated all server principals for the cluster.
+
+
+ hive.metastore.cache.pinobjtypes
+ Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order
+ List of comma separated metastore object types that should be pinned in the cache
+
+
+ datanucleus.connectionPoolingType
+ BONECP
+ Specify connection pool library for datanucleus
+
+
+ datanucleus.validateTables
+ false
+ validates existing schema against code. turn this on if you want to verify existing schema
+
+
+ datanucleus.validateColumns
+ false
+ validates existing schema against code. turn this on if you want to verify existing schema
+
+
+ datanucleus.validateConstraints
+ false
+ validates existing schema against code. turn this on if you want to verify existing schema
+
+
+ datanucleus.storeManagerType
+ rdbms
+ metadata store type
+
+
+ datanucleus.autoCreateSchema
+ true
+ creates necessary schema on a startup if one doesn't exist. set this to false, after creating it once
+
+
+ datanucleus.fixedDatastore
+ false
+
+
+
+ hive.metastore.schema.verification
+ false
+
+ Enforce metastore schema version consistency.
+ True: Verify that version information stored in metastore matches with one from Hive jars. Also disable automatic
+ schema migration attempt. Users are required to manually migrate schema after Hive upgrade which ensures
+ proper metastore schema migration. (Default)
+ False: Warn if the version information stored in metastore doesn't match with one from in Hive jars.
+
+
+
+ datanucleus.autoStartMechanismMode
+ checked
+ throw exception if metadata tables are incorrect
+
+
+ datanucleus.transactionIsolation
+ read-committed
+ Default transaction isolation level for identity generation.
+
+
+ datanucleus.cache.level2
+ false
+ Use a level 2 cache. Turn this off if metadata is changed independently of Hive metastore server
+
+
+ datanucleus.cache.level2.type
+ none
+
+
+
+ datanucleus.identifierFactory
+ datanucleus1
+
+ Name of the identifier factory to use when generating table/column names etc.
+ 'datanucleus1' is used for backward compatibility with DataNucleus v1
+
+
+
+ datanucleus.rdbms.useLegacyNativeValueStrategy
+ true
+
+
+
+ datanucleus.plugin.pluginRegistryBundleCheck
+ LOG
+ Defines what happens when plugin bundles are found and are duplicated [EXCEPTION|LOG|NONE]
+
+
+ hive.metastore.batch.retrieve.max
+ 300
+
+ Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch.
+ The higher the number, the less the number of round trips is needed to the Hive metastore server,
+ but it may also cause higher memory requirement at the client side.
+
+
+
+ hive.metastore.batch.retrieve.table.partition.max
+ 1000
+ Maximum number of table partitions that metastore internally retrieves in one batch.
+
+
+ hive.metastore.init.hooks
+
+
+ A comma separated list of hooks to be invoked at the beginning of HMSHandler initialization.
+ An init hook is specified as the name of Java class which extends org.apache.hadoop.hive.metastore.MetaStoreInitListener.
+
+
+
+ hive.metastore.pre.event.listeners
+
+ List of comma separated listeners for metastore events.
+
+
+ hive.metastore.event.listeners
+
+
+
+
+ hive.metastore.authorization.storage.checks
+ false
+
+ Should the metastore do authorization checks against the underlying storage (usually hdfs)
+ for operations like drop-partition (disallow the drop-partition if the user in
+ question doesn't have permissions to delete the corresponding directory
+ on the storage).
+
+
+
+ hive.metastore.event.clean.freq
+ 0
+ Frequency at which timer task runs to purge expired events in metastore(in seconds).
+
+
+ hive.metastore.event.expiry.duration
+ 0
+ Duration after which events expire from events table (in seconds)
+
+
+ hive.metastore.execute.setugi
+ true
+
+ In unsecure mode, setting this property to true will cause the metastore to execute DFS operations using
+ the client's reported user and group permissions. Note that this property must be set on
+ both the client and server sides. Further note that its best effort.
+ If client sets its to true and server sets it to false, client setting will be ignored.
+
+
+
+ hive.metastore.partition.name.whitelist.pattern
+
+ Partition names will be checked against this regex pattern and rejected if not matched.
+
+
+ hive.metastore.integral.jdo.pushdown
+ false
+
+ Allow JDO query pushdown for integral partition columns in metastore. Off by default. This
+ improves metastore perf for integral columns, especially if there's a large number of partitions.
+ However, it doesn't work correctly with integral values that are not normalized (e.g. have
+ leading zeroes, like 0012). If metastore direct SQL is enabled and works, this optimization
+ is also irrelevant.
+
+
+
+ hive.metastore.try.direct.sql
+ true
+
+
+
+ hive.metastore.try.direct.sql.ddl
+ true
+
+
+
+ hive.metastore.disallow.incompatible.col.type.changes
+ false
+
+ If true (default is false), ALTER TABLE operations which change the type of
+ a column (say STRING) to an incompatible type (say MAP<STRING, STRING>) are disallowed.
+ RCFile default SerDe (ColumnarSerDe) serializes the values in such a way that the
+ datatypes can be converted from string to any type. The map is also serialized as
+ a string, which can be read as a string as well. However, with any binary
+ serialization, this is not true. Blocking the ALTER TABLE prevents ClassCastExceptions
+ when subsequently trying to access old partitions.
+
+ Primitive types like INT, STRING, BIGINT, etc are compatible with each other and are
+ not blocked.
+
+ See HIVE-4409 for more details.
+
+
+
+ hive.table.parameters.default
+
+ Default property values for newly created tables
+
+
+ hive.ddl.createtablelike.properties.whitelist
+
+ Table Properties to copy over when executing a Create Table Like.
+
+
+ hive.metastore.rawstore.impl
+ org.apache.hadoop.hive.metastore.ObjectStore
+
+ Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface.
+ This class is used to store and retrieval of raw metadata objects such as table, database
+
+
+
+ javax.jdo.option.ConnectionDriverName
+ org.apache.derby.jdbc.EmbeddedDriver
+ Driver class name for a JDBC metastore
+
+
+ javax.jdo.PersistenceManagerFactoryClass
+ org.datanucleus.api.jdo.JDOPersistenceManagerFactory
+ class implementing the jdo persistence
+
+
+ hive.metastore.expression.proxy
+ org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore
+
+
+
+ javax.jdo.option.DetachAllOnCommit
+ true
+ Detaches all objects from session so that they can be used after transaction is committed
+
+
+ javax.jdo.option.NonTransactionalRead
+ true
+ Reads outside of transactions
+
+
+ javax.jdo.option.ConnectionUserName
+ APP
+ Username to use against metastore database
+
+
+ hive.metastore.end.function.listeners
+
+ List of comma separated listeners for the end of metastore functions.
+
+
+ hive.metastore.partition.inherit.table.properties
+
+
+ List of comma separated keys occurring in table properties which will get inherited to newly created partitions.
+ * implies all the keys will get inherited.
+
+
+
+ hive.metadata.export.location
+
+
+ When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener,
+ it is the location to which the metadata will be exported. The default is an empty string, which results in the
+ metadata being exported to the current user's home directory on HDFS.
+
+
+
+ hive.metadata.move.exported.metadata.to.trash
+ true
+
+ When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener,
+ this setting determines if the metadata that is exported will subsequently be moved to the user's trash directory
+ alongside the dropped table data. This ensures that the metadata will be cleaned up along with the dropped table data.
+
+
+
+ hive.cli.errors.ignore
+ false
+
+
+
+ hive.cli.print.current.db
+ false
+ Whether to include the current database in the Hive prompt.
+
+
+ hive.cli.prompt
+ hive
+
+ Command line prompt configuration value. Other hiveconf can be used in this configuration value.
+ Variable substitution will only be invoked at the Hive CLI startup.
+
+
+
+ hive.cli.pretty.output.num.cols
+ -1
+
+ The number of columns to use when formatting output generated by the DESCRIBE PRETTY table_name command.
+ If the value of this property is -1, then Hive will use the auto-detected terminal width.
+
+
+
+ hive.metastore.fs.handler.class
+ org.apache.hadoop.hive.metastore.HiveMetaStoreFsImpl
+
+
+
+ hive.session.id
+
+
+
+
+ hive.session.silent
+ false
+
+
+
+ hive.session.history.enabled
+ false
+ Whether to log Hive query, query plan, runtime statistics etc.
+
+
+ hive.query.string
+
+ Query being executed (might be multiple per a session)
+
+
+ hive.query.id
+
+ ID for query being executed (might be multiple per a session)
+
+
+ hive.jobname.length
+ 50
+ max jobname length
+
+
+ hive.jar.path
+
+
+
+
+ hive.aux.jars.path
+
+
+
+
+ hive.added.files.path
+
+
+
+
+ hive.added.jars.path
+
+
+
+
+ hive.added.archives.path
+
+
+
+
+ hive.auto.progress.timeout
+ 0
+
+ How long to run autoprogressor for the script/UDTF operators (in seconds).
+ Set to 0 for forever.
+
+
+
+ hive.table.name
+
+
+
+
+ hive.partition.name
+
+
+
+
+ hive.script.auto.progress
+ false
+
+ Whether Hive Transform/Map/Reduce Clause should automatically send progress information to TaskTracker
+ to avoid the task getting killed because of inactivity. Hive sends progress information when the script is
+ outputting to stderr. This option removes the need of periodically producing stderr messages,
+ but users should be cautious because this may prevent infinite loops in the scripts to be killed by TaskTracker.
+
+
+
+ hive.script.operator.id.env.var
+ HIVE_SCRIPT_OPERATOR_ID
+
+ Name of the environment variable that holds the unique script operator ID in the user's
+ transform function (the custom mapper/reducer that the user has specified in the query)
+
+
+
+ hive.script.operator.truncate.env
+ false
+ Truncate each environment variable for external script in scripts operator to 20KB (to fit system limits)
+
+
+ hive.mapred.mode
+ nonstrict
+
+ The mode in which the Hive operations are being performed.
+ In strict mode, some risky queries are not allowed to run. They include:
+ Cartesian Product.
+ No partition being picked up for a query.
+ Comparing bigints and strings.
+ Comparing bigints and doubles.
+ Orderby without limit.
+
+
+
+ hive.alias
+
+
+
+
+ hive.map.aggr
+ true
+ Whether to use map-side aggregation in Hive Group By queries
+
+
+ hive.groupby.skewindata
+ false
+ Whether there is skew in data to optimize group by queries
+
+
+ hive.optimize.multigroupby.common.distincts
+ true
+
+ Whether to optimize a multi-groupby query with the same distinct.
+ Consider a query like:
+
+ from src
+ insert overwrite table dest1 select col1, count(distinct colx) group by col1
+ insert overwrite table dest2 select col2, count(distinct colx) group by col2;
+
+ With this parameter set to true, first we spray by the distinct value (colx), and then
+ perform the 2 groups bys. This makes sense if map-side aggregation is turned off. However,
+ with maps-side aggregation, it might be useful in some cases to treat the 2 inserts independently,
+ thereby performing the query above in 2MR jobs instead of 3 (due to spraying by distinct key first).
+ If this parameter is turned off, we don't consider the fact that the distinct key is the same across
+ different MR jobs.
+
+
+
+ hive.join.emit.interval
+ 1000
+ How many rows in the right-most join operand Hive should buffer before emitting the join result.
+
+
+ hive.join.cache.size
+ 25000
+ How many rows in the joining tables (except the streaming table) should be cached in memory.
+
+
+ hive.mapjoin.bucket.cache.size
+ 100
+
+
+
+ hive.mapjoin.optimized.hashtable
+ true
+
+ Whether Hive should use memory-optimized hash table for MapJoin. Only works on Tez,
+ because memory-optimized hashtable cannot be serialized.
+
+
+
+ hive.mapjoin.optimized.keys
+ true
+
+ Whether MapJoin hashtable should use optimized (size-wise), keys, allowing the table to take less
+ memory. Depending on key, the memory savings for entire table can be 5-15% or so.
+
+
+
+ hive.mapjoin.lazy.hashtable
+ true
+
+ Whether MapJoin hashtable should deserialize values on demand. Depending on how many values in
+ the table the join will actually touch, it can save a lot of memory by not creating objects for
+ rows that are not needed. If all rows are needed obviously there's no gain.
+
+
+
+ hive.mapjoin.optimized.hashtable.wbsize
+ 10485760
+
+ Optimized hashtable (see hive.mapjoin.optimized.hashtable) uses a chain of buffers to
+ store data. This is one buffer size. HT may be slightly faster if this is larger, but for small
+ joins unnecessary memory will be allocated and then trimmed.
+
+
+
+ hive.smbjoin.cache.rows
+ 10000
+ How many rows with the same key value should be cached in memory per smb joined table.
+
+
+ hive.groupby.mapaggr.checkinterval
+ 100000
+ Number of rows after which size of the grouping keys/aggregation classes is performed
+
+
+ hive.map.aggr.hash.percentmemory
+ 0.5
+ Portion of total memory to be used by map-side group aggregation hash table
+
+
+ hive.mapjoin.followby.map.aggr.hash.percentmemory
+ 0.3
+ Portion of total memory to be used by map-side group aggregation hash table, when this group by is followed by map join
+
+
+ hive.map.aggr.hash.force.flush.memory.threshold
+ 0.9
+
+ The max memory to be used by map-side group aggregation hash table.
+ If the memory usage is higher than this number, force to flush data
+
+
+
+ hive.map.aggr.hash.min.reduction
+ 0.5
+
+ Hash aggregation will be turned off if the ratio between hash table size and input rows is bigger than this number.
+ Set to 1 to make sure hash aggregation is never turned off.
+
+
+
+ hive.multigroupby.singlereducer
+ true
+
+ Whether to optimize multi group by query to generate single M/R job plan. If the multi group by query has
+ common group by keys, it will be optimized to generate single M/R job.
+
+
+
+ hive.map.groupby.sorted
+ false
+
+ If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform
+ the group by in the mapper by using BucketizedHiveInputFormat. The only downside to this
+ is that it limits the number of mappers to the number of files.
+
+
+
+ hive.map.groupby.sorted.testmode
+ false
+
+ If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform
+ the group by in the mapper by using BucketizedHiveInputFormat. If the test mode is set, the plan
+ is not converted, but a query property is set to denote the same.
+
+
+
+ hive.groupby.orderby.position.alias
+ false
+ Whether to enable using Column Position Alias in Group By or Order By
+
+
+ hive.new.job.grouping.set.cardinality
+ 30
+
+ Whether a new map-reduce job should be launched for grouping sets/rollups/cubes.
+ For a query like: select a, b, c, count(1) from T group by a, b, c with rollup;
+ 4 rows are created per row: (a, b, c), (a, b, null), (a, null, null), (null, null, null).
+ This can lead to explosion across map-reduce boundary if the cardinality of T is very high,
+ and map-side aggregation does not do a very good job.
+
+ This parameter decides if Hive should add an additional map-reduce job. If the grouping set
+ cardinality (4 in the example above), is more than this value, a new MR job is added under the
+ assumption that the original group by will reduce the data size.
+
+
+
+ hive.udtf.auto.progress
+ false
+
+ Whether Hive should automatically send progress information to TaskTracker
+ when using UDTF's to prevent the task getting killed because of inactivity. Users should be cautious
+ because this may prevent TaskTracker from killing tasks with infinite loops.
+
+
+
+ hive.default.fileformat
+ TextFile
+
+ Default file format for CREATE TABLE statement.
+ Options are TextFile, SequenceFile, RCfile and ORC. Users can explicitly override it by CREATE TABLE ... STORED AS [FORMAT]
+
+
+
+ hive.query.result.fileformat
+ TextFile
+ Default file format for storing result of the query. Allows TextFile, SequenceFile and RCfile
+
+
+ hive.fileformat.check
+ true
+ Whether to check file format or not when loading data files
+
+
+ hive.default.rcfile.serde
+ org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe
+ The default SerDe Hive will use for the RCFile format
+
+
+ hive.default.serde
+ org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ The default SerDe Hive will use for storage formats that do not specify a SerDe.
+
+
+ hive.serdes.using.metastore.for.schema
+ org.apache.hadoop.hive.ql.io.orc.OrcSerde,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe,org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe,org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe,org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe,org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe,org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ SerDes retriving schema from metastore. This an internal parameter. Check with the hive dev. team
+
+
+ hive.querylog.location
+ ${system:java.io.tmpdir}/${system:user.name}
+ Location of Hive run time structured log file
+
+
+ hive.querylog.enable.plan.progress
+ true
+
+ Whether to log the plan's progress every time a job's progress is checked.
+ These logs are written to the location specified by hive.querylog.location
+
+
+
+ hive.querylog.plan.progress.interval
+ 60000
+
+ The interval to wait between logging the plan's progress in milliseconds.
+ If there is a whole number percentage change in the progress of the mappers or the reducers,
+ the progress is logged regardless of this value.
+ The actual interval will be the ceiling of (this value divided by the value of
+ hive.exec.counters.pull.interval) multiplied by the value of hive.exec.counters.pull.interval
+ I.e. if it is not divide evenly by the value of hive.exec.counters.pull.interval it will be
+ logged less frequently than specified.
+ This only has an effect if hive.querylog.enable.plan.progress is set to true.
+
+
+
+ hive.script.serde
+ org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ The default SerDe for transmitting input data to and reading output data from the user scripts.
+
+
+ hive.script.recordreader
+ org.apache.hadoop.hive.ql.exec.TextRecordReader
+ The default record reader for reading data from the user scripts.
+
+
+ hive.script.recordwriter
+ org.apache.hadoop.hive.ql.exec.TextRecordWriter
+ The default record writer for writing data to the user scripts.
+
+
+ hive.transform.escape.input
+ false
+
+ This adds an option to escape special chars (newlines, carriage returns and
+ tabs) when they are passed to the user script. This is useful if the Hive tables
+ can contain data that contains special characters.
+
+
+
+ hive.binary.record.max.length
+ 1000
+
+ Read from a binary stream and treat each hive.binary.record.max.length bytes as a record.
+ The last record before the end of stream can have less than hive.binary.record.max.length bytes
+
+
+
+ hive.hwi.listen.host
+ 0.0.0.0
+ This is the host address the Hive Web Interface will listen on
+
+
+ hive.hwi.listen.port
+ 9999
+ This is the port the Hive Web Interface will listen on
+
+
+ hive.hwi.war.file
+ ${system:HWI_WAR_FILE}
+ This sets the path to the HWI war file, relative to ${HIVE_HOME}.
+
+
+ hive.mapred.local.mem
+ 0
+ mapper/reducer memory in local mode
+
+
+ hive.mapjoin.smalltable.filesize
+ 25000000
+
+ The threshold for the input file size of the small tables; if the file size is smaller
+ than this threshold, it will try to convert the common join into map join
+
+
+
+ hive.sample.seednumber
+ 0
+ A number used to percentage sampling. By changing this number, user will change the subsets of data sampled.
+
+
+ hive.test.mode
+ false
+ Whether Hive is running in test mode. If yes, it turns on sampling and prefixes the output tablename.
+
+
+ hive.test.mode.prefix
+ test_
+ In test mode, specfies prefixes for the output table
+
+
+ hive.test.mode.samplefreq
+ 32
+
+ In test mode, specfies sampling frequency for table, which is not bucketed,
+ For example, the following query:
+ INSERT OVERWRITE TABLE dest SELECT col1 from src
+ would be converted to
+ INSERT OVERWRITE TABLE test_dest
+ SELECT col1 from src TABLESAMPLE (BUCKET 1 out of 32 on rand(1))
+
+
+
+ hive.test.mode.nosamplelist
+
+ In test mode, specifies comma separated table names which would not apply sampling
+
+
+ hive.test.dummystats.aggregator
+
+ internal variable for test
+
+
+ hive.test.dummystats.publisher
+
+ internal variable for test
+
+
+ hive.merge.mapfiles
+ true
+ Merge small files at the end of a map-only job
+
+
+ hive.merge.mapredfiles
+ false
+ Merge small files at the end of a map-reduce job
+
+
+ hive.merge.tezfiles
+ false
+ Merge small files at the end of a Tez DAG
+
+
+ hive.merge.size.per.task
+ 256000000
+ Size of merged files at the end of the job
+
+
+ hive.merge.smallfiles.avgsize
+ 16000000
+
+ When the average output file size of a job is less than this number, Hive will start an additional
+ map-reduce job to merge the output files into bigger files. This is only done for map-only jobs
+ if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true.
+
+
+
+ hive.merge.rcfile.block.level
+ true
+
+
+
+ hive.merge.input.format.block.level
+ org.apache.hadoop.hive.ql.io.rcfile.merge.RCFileBlockMergeInputFormat
+
+
+
+ hive.merge.current.job.has.dynamic.partitions
+ false
+
+
+
+ hive.exec.rcfile.use.explicit.header
+ true
+
+ If this is set the header for RCFiles will simply be RCF. If this is not
+ set the header will be that borrowed from sequence files, e.g. SEQ- followed
+ by the input and output RCFile formats.
+
+
+
+ hive.exec.rcfile.use.sync.cache
+ true
+
+
+
+ hive.io.rcfile.record.interval
+ 2147483647
+
+
+
+ hive.io.rcfile.column.number.conf
+ 0
+
+
+
+ hive.io.rcfile.tolerate.corruptions
+ false
+
+
+
+ hive.io.rcfile.record.buffer.size
+ 4194304
+
+
+
+ hive.exec.orc.memory.pool
+ 0.5
+ Maximum fraction of heap that can be used by ORC file writers
+
+
+ hive.exec.orc.write.format
+
+ Define the version of the file to write
+
+
+ hive.exec.orc.default.stripe.size
+ 268435456
+ Define the default ORC stripe size
+
+
+ hive.exec.orc.default.block.size
+ 268435456
+ Define the default file system block size for ORC files.
+
+
+ hive.exec.orc.dictionary.key.size.threshold
+ 0.8
+
+ If the number of keys in a dictionary is greater than this fraction of the total number of
+ non-null rows, turn off dictionary encoding. Use 1 to always use dictionary encoding.
+
+
+
+ hive.exec.orc.default.row.index.stride
+ 10000
+ Define the default ORC index stride
+
+
+ hive.exec.orc.default.buffer.size
+ 262144
+ Define the default ORC buffer size
+
+
+ hive.exec.orc.default.block.padding
+ true
+ Define the default block padding
+
+
+ hive.exec.orc.block.padding.tolerance
+ 0.05
+
+ Define the tolerance for block padding as a percentage of stripe size.
+ For the defaults of 64Mb ORC stripe and 256Mb HDFS blocks, a maximum of 3.2Mb will be reserved for padding within the 256Mb block.
+ In that case, if the available size within the block is more than 3.2Mb, a new smaller stripe will be inserted to fit within that space.
+ This will make sure that no stripe written will cross block boundaries and cause remote reads within a node local task.
+
+
+
+ hive.exec.orc.default.compress
+ ZLIB
+ Define the default compression codec for ORC file
+
+
+ hive.exec.orc.encoding.strategy
+ SPEED
+
+ Define the encoding strategy to use while writing data. Changing this will
+ only affect the light weight encoding for integers. This flag will not
+ change the compression level of higher level compression codec (like ZLIB).
+ Possible options are SPEED and COMPRESSION.
+
+
+
+ hive.orc.splits.include.file.footer
+ false
+
+ If turned on splits generated by orc will include metadata about the stripes in the file. This
+ data is read remotely (from the client or HS2 machine) and sent to all the tasks.
+
+
+
+ hive.orc.cache.stripe.details.size
+ 10000
+ Cache size for keeping meta info about orc splits cached in the client.
+
+
+ hive.orc.compute.splits.num.threads
+ 10
+ How many threads orc should use to create splits in parallel.
+
+
+ hive.exec.orc.skip.corrupt.data
+ false
+
+ If ORC reader encounters corrupt data, this value will be used to determine
+ whether to skip the corrupt data or throw exception. The default behavior is to throw exception.
+
+
+
+ hive.exec.orc.zerocopy
+ false
+ Use zerocopy reads with ORC.
+
+
+ hive.lazysimple.extended_boolean_literal
+ false
+
+ LazySimpleSerde uses this property to determine if it treats 'T', 't', 'F', 'f',
+ '1', and '0' as extened, legal boolean literal, in addition to 'TRUE' and 'FALSE'.
+ The default is false, which means only 'TRUE' and 'FALSE' are treated as legal
+ boolean literal.
+
+
+
+ hive.optimize.skewjoin
+ false
+
+ Whether to enable skew join optimization.
+ The algorithm is as follows: At runtime, detect the keys with a large skew. Instead of
+ processing those keys, store them temporarily in an HDFS directory. In a follow-up map-reduce
+ job, process those skewed keys. The same key need not be skewed for all the tables, and so,
+ the follow-up map-reduce job (for the skewed keys) would be much faster, since it would be a
+ map-join.
+
+
+
+ hive.auto.convert.join
+ true
+ Whether Hive enables the optimization about converting common join into mapjoin based on the input file size
+
+
+ hive.auto.convert.join.noconditionaltask
+ true
+
+ Whether Hive enables the optimization about converting common join into mapjoin based on the input file size.
+ If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the
+ specified size, the join is directly converted to a mapjoin (there is no conditional task).
+
+
+
+ hive.auto.convert.join.noconditionaltask.size
+ 10000000
+
+ If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect.
+ However, if it is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than this size,
+ the join is directly converted to a mapjoin(there is no conditional task). The default is 10MB
+
+
+
+ hive.auto.convert.join.use.nonstaged
+ false
+
+ For conditional joins, if input stream from a small alias can be directly applied to join operator without
+ filtering or projection, the alias need not to be pre-staged in distributed cache via mapred local task.
+ Currently, this is not working with vectorization or tez execution engine.
+
+
+
+ hive.skewjoin.key
+ 100000
+
+ Determine if we get a skew key in join. If we see more than the specified number of rows with the same key in join operator,
+ we think the key as a skew join key.
+
+
+
+ hive.skewjoin.mapjoin.map.tasks
+ 10000
+
+ Determine the number of map task used in the follow up map join job for a skew join.
+ It should be used together with hive.skewjoin.mapjoin.min.split to perform a fine grained control.
+
+
+
+ hive.skewjoin.mapjoin.min.split
+ 33554432
+
+ Determine the number of map task at most used in the follow up map join job for a skew join by specifying
+ the minimum split size. It should be used together with hive.skewjoin.mapjoin.map.tasks to perform a fine grained control.
+
+
+
+ hive.heartbeat.interval
+ 1000
+ Send a heartbeat after this interval - used by mapjoin and filter operators
+
+
+ hive.limit.row.max.size
+ 100000
+ When trying a smaller subset of data for simple LIMIT, how much size we need to guarantee each row to have at least.
+
+
+ hive.limit.optimize.limit.file
+ 10
+ When trying a smaller subset of data for simple LIMIT, maximum number of files we can sample.
+
+
+ hive.limit.optimize.enable
+ false
+ Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first.
+
+
+ hive.limit.optimize.fetch.max
+ 50000
+
+ Maximum number of rows allowed for a smaller subset of data for simple LIMIT, if it is a fetch query.
+ Insert queries are not restricted by this limit.
+
+
+
+ hive.limit.pushdown.memory.usage
+ -1.0
+ The max memory to be used for hash in RS operator for top K selection.
+
+
+ hive.limit.query.max.table.partition
+ -1
+
+ This controls how many partitions can be scanned for each partitioned table.
+ The default value "-1" means no limit.
+
+
+
+ hive.hashtable.initialCapacity
+ 100000
+
+
+
+ hive.hashtable.loadfactor
+ 0.75
+
+
+
+ hive.mapjoin.followby.gby.localtask.max.memory.usage
+ 0.55
+
+ This number means how much memory the local task can take to hold the key/value into an in-memory hash table
+ when this map join is followed by a group by. If the local task's memory usage is more than this number,
+ the local task will abort by itself. It means the data of the small table is too large to be held in memory.
+
+
+
+ hive.mapjoin.localtask.max.memory.usage
+ 0.9
+
+ This number means how much memory the local task can take to hold the key/value into an in-memory hash table.
+ If the local task's memory usage is more than this number, the local task will abort by itself.
+ It means the data of the small table is too large to be held in memory.
+
+
+
+ hive.mapjoin.check.memory.rows
+ 100000
+ The number means after how many rows processed it needs to check the memory usage
+
+
+ hive.debug.localtask
+ false
+
+
+
+ hive.input.format
+ org.apache.hadoop.hive.ql.io.CombineHiveInputFormat
+ The default input format. Set this to HiveInputFormat if you encounter problems with CombineHiveInputFormat.
+
+
+ hive.tez.input.format
+ org.apache.hadoop.hive.ql.io.HiveInputFormat
+ The default input format for tez. Tez groups splits in the AM.
+
+
+ hive.tez.container.size
+ -1
+ By default Tez will spawn containers of the size of a mapper. This can be used to overwrite.
+
+
+ hive.tez.java.opts
+
+ By default Tez will use the Java options from map tasks. This can be used to overwrite.
+
+
+ hive.tez.log.level
+ INFO
+
+ The log level to use for tasks executing as part of the DAG.
+ Used only if hive.tez.java.opts is used to configure Java options.
+
+
+
+ hive.enforce.bucketing
+ false
+ Whether bucketing is enforced. If true, while inserting into the table, bucketing is enforced.
+
+
+ hive.enforce.sorting
+ false
+ Whether sorting is enforced. If true, while inserting into the table, sorting is enforced.
+
+
+ hive.optimize.bucketingsorting
+ true
+
+ If hive.enforce.bucketing or hive.enforce.sorting is true, don't create a reducer for enforcing
+ bucketing/sorting for queries of the form:
+ insert overwrite table T2 select * from T1;
+ where T1 and T2 are bucketed/sorted by the same keys into the same number of buckets.
+
+
+
+ hive.mapred.partitioner
+ org.apache.hadoop.hive.ql.io.DefaultHivePartitioner
+
+
+
+ hive.enforce.sortmergebucketmapjoin
+ false
+ If the user asked for sort-merge bucketed map-side join, and it cannot be performed, should the query fail or not ?
+
+
+ hive.enforce.bucketmapjoin
+ false
+
+ If the user asked for bucketed map-side join, and it cannot be performed,
+ should the query fail or not ? For example, if the buckets in the tables being joined are
+ not a multiple of each other, bucketed map-side join cannot be performed, and the
+ query will fail if hive.enforce.bucketmapjoin is set to true.
+
+
+
+ hive.auto.convert.sortmerge.join
+ false
+ Will the join be automatically converted to a sort-merge join, if the joined tables pass the criteria for sort-merge join.
+
+
+ hive.auto.convert.sortmerge.join.bigtable.selection.policy
+ org.apache.hadoop.hive.ql.optimizer.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ
+
+ The policy to choose the big table for automatic conversion to sort-merge join.
+ By default, the table with the largest partitions is assigned the big table. All policies are:
+ . based on position of the table - the leftmost table is selected
+ org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSMJ.
+ . based on total size (all the partitions selected in the query) of the table
+ org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ.
+ . based on average size (all the partitions selected in the query) of the table
+ org.apache.hadoop.hive.ql.optimizer.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ.
+ New policies can be added in future.
+
+
+
+ hive.auto.convert.sortmerge.join.to.mapjoin
+ false
+
+ If hive.auto.convert.sortmerge.join is set to true, and a join was converted to a sort-merge join,
+ this parameter decides whether each table should be tried as a big table, and effectively a map-join should be
+ tried. That would create a conditional task with n+1 children for a n-way join (1 child for each table as the
+ big table), and the backup task will be the sort-merge join. In some cases, a map-join would be faster than a
+ sort-merge join, if there is no advantage of having the output bucketed and sorted. For example, if a very big sorted
+ and bucketed table with few files (say 10 files) are being joined with a very small sorter and bucketed table
+ with few files (10 files), the sort-merge join will only use 10 mappers, and a simple map-only join might be faster
+ if the complete small table can fit in memory, and a map-join can be performed.
+
+
+
+ hive.exec.script.trust
+ false
+
+
+
+ hive.exec.rowoffset
+ false
+ Whether to provide the row offset virtual column
+
+
+ hive.hadoop.supports.splittable.combineinputformat
+ false
+
+
+
+ hive.optimize.index.filter
+ false
+ Whether to enable automatic use of indexes
+
+
+ hive.optimize.index.autoupdate
+ false
+ Whether to update stale indexes automatically
+
+
+ hive.optimize.ppd
+ true
+ Whether to enable predicate pushdown
+
+
+ hive.ppd.recognizetransivity
+ true
+ Whether to transitively replicate predicate filters over equijoin conditions.
+
+
+ hive.ppd.remove.duplicatefilters
+ true
+ Whether to push predicates down into storage handlers. Ignored when hive.optimize.ppd is false.
+
+
+ hive.optimize.metadataonly
+ true
+
+
+
+ hive.optimize.null.scan
+ true
+ Dont scan relations which are guaranteed to not generate any rows
+
+
+ hive.optimize.ppd.storage
+ true
+ Whether to push predicates down to storage handlers
+
+
+ hive.optimize.groupby
+ true
+ Whether to enable the bucketed group by from bucketed partitions/tables.
+
+
+ hive.optimize.bucketmapjoin
+ false
+ Whether to try bucket mapjoin
+
+
+ hive.optimize.bucketmapjoin.sortedmerge
+ false
+ Whether to try sorted bucket merge map join
+
+
+ hive.optimize.reducededuplication
+ true
+
+ Remove extra map-reduce jobs if the data is already clustered by the same key which needs to be used again.
+ This should always be set to true. Since it is a new feature, it has been made configurable.
+
+
+
+ hive.optimize.reducededuplication.min.reducer
+ 4
+
+ Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS.
+ That means if reducer-num of the child RS is fixed (order by or forced bucketing) and small, it can make very slow, single MR.
+ The optimization will be automatically disabled if number of reducers would be less than specified value.
+
+
+
+ hive.optimize.sort.dynamic.partition
+ true
+
+ When enabled dynamic partitioning column will be globally sorted.
+ This way we can keep only one record writer open for each partition value
+ in the reducer thereby reducing the memory pressure on reducers.
+
+
+
+ hive.optimize.sampling.orderby
+ false
+
+
+
+ hive.optimize.sampling.orderby.number
+ 1000
+
+
+
+ hive.optimize.sampling.orderby.percent
+ 0.1
+
+
+
+ hive.optimize.union.remove
+ false
+
+ Whether to remove the union and push the operators between union and the filesink above union.
+ This avoids an extra scan of the output by union. This is independently useful for union
+ queries, and specially useful when hive.optimize.skewjoin.compiletime is set to true, since an
+ extra union is inserted.
+
+ The merge is triggered if either of hive.merge.mapfiles or hive.merge.mapredfiles is set to true.
+ If the user has set hive.merge.mapfiles to true and hive.merge.mapredfiles to false, the idea was the
+ number of reducers are few, so the number of files anyway are small. However, with this optimization,
+ we are increasing the number of files possibly by a big margin. So, we merge aggressively.
+
+
+
+ hive.optimize.correlation
+ false
+ exploit intra-query correlations.
+
+
+ hive.mapred.supports.subdirectories
+ false
+
+ Whether the version of Hadoop which is running supports sub-directories for tables/partitions.
+ Many Hive optimizations can be applied if the Hadoop version supports sub-directories for
+ tables/partitions. It was added by MAPREDUCE-1501
+
+
+
+ hive.optimize.skewjoin.compiletime
+ false
+
+ Whether to create a separate plan for skewed keys for the tables in the join.
+ This is based on the skewed keys stored in the metadata. At compile time, the plan is broken
+ into different joins: one for the skewed keys, and the other for the remaining keys. And then,
+ a union is performed for the 2 joins generated above. So unless the same skewed key is present
+ in both the joined tables, the join for the skewed key will be performed as a map-side join.
+
+ The main difference between this parameter and hive.optimize.skewjoin is that this parameter
+ uses the skew information stored in the metastore to optimize the plan at compile time itself.
+ If there is no skew information in the metadata, this parameter will not have any affect.
+ Both hive.optimize.skewjoin.compiletime and hive.optimize.skewjoin should be set to true.
+ Ideally, hive.optimize.skewjoin should be renamed as hive.optimize.skewjoin.runtime, but not doing
+ so for backward compatibility.
+
+ If the skew information is correctly stored in the metadata, hive.optimize.skewjoin.compiletime
+ would change the query plan to take care of it, and hive.optimize.skewjoin will be a no-op.
+
+
+
+ hive.optimize.index.filter.compact.minsize
+ 5368709120
+ Minimum size (in bytes) of the inputs on which a compact index is automatically used.
+
+
+ hive.optimize.index.filter.compact.maxsize
+ -1
+ Maximum size (in bytes) of the inputs on which a compact index is automatically used. A negative number is equivalent to infinity.
+
+
+ hive.index.compact.query.max.entries
+ 10000000
+ The maximum number of index entries to read during a query that uses the compact index. Negative value is equivalent to infinity.
+
+
+ hive.index.compact.query.max.size
+ 10737418240
+ The maximum number of bytes that a query using the compact index can read. Negative value is equivalent to infinity.
+
+
+ hive.index.compact.binary.search
+ true
+ Whether or not to use a binary search to find the entries in an index table that match the filter, where possible
+
+
+ hive.stats.autogather
+ true
+ A flag to gather statistics automatically during the INSERT OVERWRITE command.
+
+
+ hive.stats.dbclass
+ fs
+ The storage that stores temporary Hive statistics. Currently, jdbc, hbase, counter and custom type are supported.
+
+
+ hive.stats.jdbcdriver
+ org.apache.derby.jdbc.EmbeddedDriver
+ The JDBC driver for the database that stores temporary Hive statistics.
+
+
+ hive.stats.dbconnectionstring
+ jdbc:derby:;databaseName=TempStatsStore;create=true
+ The default connection string for the database that stores temporary Hive statistics.
+
+
+ hive.stats.default.publisher
+
+ The Java class (implementing the StatsPublisher interface) that is used by default if hive.stats.dbclass is custom type.
+
+
+ hive.stats.default.aggregator
+
+ The Java class (implementing the StatsAggregator interface) that is used by default if hive.stats.dbclass is custom type.
+
+
+ hive.stats.jdbc.timeout
+ 30
+ Timeout value (number of seconds) used by JDBC connection and statements.
+
+
+ hive.stats.atomic
+ false
+ whether to update metastore stats only if all stats are available
+
+
+ hive.stats.retries.max
+ 0
+
+ Maximum number of retries when stats publisher/aggregator got an exception updating intermediate database.
+ Default is no tries on failures.
+
+
+
+ hive.stats.retries.wait
+ 3000
+ The base waiting window (in milliseconds) before the next retry. The actual wait time is calculated by baseWindow * failures baseWindow * (failure 1) * (random number between [0.0,1.0]).
+
+
+ hive.stats.collect.rawdatasize
+ true
+ should the raw data size be collected when analyzing tables
+
+
+ hive.client.stats.counters
+
+
+ Subset of counters that should be of interest for hive.client.stats.publishers (when one wants to limit their publishing).
+ Non-display names should be used
+
+
+
+ hive.stats.reliable
+ false
+
+ Whether queries will fail because stats cannot be collected completely accurately.
+ If this is set to true, reading/writing from/into a partition may fail because the stats
+ could not be computed accurately.
+
+
+
+ hive.stats.gather.num.threads
+ 10
+
+ Number of threads used by partialscan/noscan analyze command for partitioned tables.
+ This is applicable only for file formats that implement StatsProvidingRecordReader (like ORC).
+
+
+
+ hive.stats.collect.tablekeys
+ false
+
+ Whether join and group by keys on tables are derived and maintained in the QueryPlan.
+ This is useful to identify how tables are accessed and to determine if they should be bucketed.
+
+
+
+ hive.stats.collect.scancols
+ false
+
+ Whether column accesses are tracked in the QueryPlan.
+ This is useful to identify how tables are accessed and to determine if there are wasted columns that can be trimmed.
+
+
+
+ hive.stats.ndv.error
+ 20.0
+
+ Standard error expressed in percentage. Provides a tradeoff between accuracy and compute cost.
+ A lower value for error indicates higher accuracy and a higher compute cost.
+
+
+
+ hive.stats.key.prefix.max.length
+ 150
+
+ Determines if when the prefix of the key used for intermediate stats collection
+ exceeds a certain length, a hash of the key is used instead. If the value < 0 then hashing
+
+
+
+ hive.stats.key.prefix.reserve.length
+ 24
+
+ Reserved length for postfix of stats key. Currently only meaningful for counter type which should
+ keep length of full stats key smaller than max length configured by hive.stats.key.prefix.max.length.
+ For counter type, it should be bigger than the length of LB spec if exists.
+
+
+
+ hive.stats.max.variable.length
+ 100
+
+ To estimate the size of data flowing through operators in Hive/Tez(for reducer estimation etc.),
+ average row size is multiplied with the total number of rows coming out of each operator.
+ Average row size is computed from average column size of all columns in the row. In the absence
+ of column statistics, for variable length columns (like string, bytes etc.), this value will be
+ used. For fixed length columns their corresponding Java equivalent sizes are used
+ (float - 4 bytes, double - 8 bytes etc.).
+
+
+
+ hive.stats.list.num.entries
+ 10
+
+ To estimate the size of data flowing through operators in Hive/Tez(for reducer estimation etc.),
+ average row size is multiplied with the total number of rows coming out of each operator.
+ Average row size is computed from average column size of all columns in the row. In the absence
+ of column statistics and for variable length complex columns like list, the average number of
+ entries/values can be specified using this config.
+
+
+
+ hive.stats.map.num.entries
+ 10
+
+ To estimate the size of data flowing through operators in Hive/Tez(for reducer estimation etc.),
+ average row size is multiplied with the total number of rows coming out of each operator.
+ Average row size is computed from average column size of all columns in the row. In the absence
+ of column statistics and for variable length complex columns like map, the average number of
+ entries/values can be specified using this config.
+
+
+
+ hive.stats.map.parallelism
+ 1
+
+ Hive/Tez optimizer estimates the data size flowing through each of the operators.
+ For GROUPBY operator, to accurately compute the data size map-side parallelism needs to
+ be known. By default, this value is set to 1 since optimizer is not aware of the number of
+ mappers during compile-time. This Hive config can be used to specify the number of mappers
+ to be used for data size computation of GROUPBY operator.
+
+
+
+ hive.stats.fetch.partition.stats
+ true
+
+ Annotation of operator tree with statistics information requires partition level basic
+ statistics like number of rows, data size and file size. Partition statistics are fetched from
+ metastore. Fetching partition statistics for each needed partition can be expensive when the
+ number of partitions is high. This flag can be used to disable fetching of partition statistics
+ from metastore. When this flag is disabled, Hive will make calls to filesystem to get file sizes
+ and will estimate the number of rows from row schema.
+
+
+
+ hive.stats.fetch.column.stats
+ false
+
+ Annotation of operator tree with statistics information requires column statistics.
+ Column statistics are fetched from metastore. Fetching column statistics for each needed column
+ can be expensive when the number of columns is high. This flag can be used to disable fetching
+ of column statistics from metastore.
+
+
+
+ hive.stats.join.factor
+ 1.1
+
+ Hive/Tez optimizer estimates the data size flowing through each of the operators. JOIN operator
+ uses column statistics to estimate the number of rows flowing out of it and hence the data size.
+ In the absence of column statistics, this factor determines the amount of rows that flows out
+ of JOIN operator.
+
+
+
+ hive.stats.deserialization.factor
+ 1.0
+
+ Hive/Tez optimizer estimates the data size flowing through each of the operators. In the absence
+ of basic statistics like number of rows and data size, file size is used to estimate the number
+ of rows and data size. Since files in tables/partitions are serialized (and optionally
+ compressed) the estimates of number of rows and data size cannot be reliably determined.
+ This factor is multiplied with the file size to account for serialization and compression.
+
+
+
+ hive.support.concurrency
+ false
+
+ Whether Hive supports concurrency control or not.
+ A ZooKeeper instance must be up and running when using zookeeper Hive lock manager
+
+
+
+ hive.lock.manager
+ org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager
+
+
+
+ hive.lock.numretries
+ 100
+ The number of times you want to try to get all the locks
+
+
+ hive.unlock.numretries
+ 10
+ The number of times you want to retry to do one unlock
+
+
+ hive.lock.sleep.between.retries
+ 60
+ The sleep time (in seconds) between various retries
+
+
+ hive.lock.mapred.only.operation
+ false
+
+ This param is to control whether or not only do lock on queries
+ that need to execute at least one mapred job.
+
+
+
+ hive.zookeeper.quorum
+
+ The list of ZooKeeper servers to talk to. This is only needed for read/write locks.
+
+
+ hive.zookeeper.client.port
+ 2181
+ The port of ZooKeeper servers to talk to. This is only needed for read/write locks.
+
+
+ hive.zookeeper.session.timeout
+ 600000
+
+ ZooKeeper client's session timeout. The client is disconnected, and as a result, all locks released,
+ if a heartbeat is not sent in the timeout.
+
+
+
+ hive.zookeeper.namespace
+ hive_zookeeper_namespace
+ The parent node under which all ZooKeeper nodes are created.
+
+
+ hive.zookeeper.clean.extra.nodes
+ false
+ Clean extra nodes at the end of the session.
+
+
+ hive.txn.manager
+ org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager
+
+
+
+ hive.txn.timeout
+ 300
+ time after which transactions are declared aborted if the client has not sent a heartbeat, in seconds.
+
+
+ hive.txn.max.open.batch
+ 1000
+
+ Maximum number of transactions that can be fetched in one call to open_txns().
+ Increasing this will decrease the number of delta files created when
+ streaming data into Hive. But it will also increase the number of
+ open transactions at any given time, possibly impacting read performance.
+
+
+
+ hive.compactor.initiator.on
+ false
+ Whether to run the compactor's initiator thread in this metastore instance or not.
+
+
+ hive.compactor.worker.threads
+ 0
+ Number of compactor worker threads to run on this metastore instance.
+
+
+ hive.compactor.worker.timeout
+ 86400
+
+ Time in seconds, before a given compaction in working state is declared a failure
+ and returned to the initiated state.
+
+
+
+ hive.compactor.check.interval
+ 300
+
+ Time in seconds between checks to see if any partitions need compacted.
+ This should be kept high because each check for compaction requires many calls against the NameNode.
+
+
+
+ hive.compactor.delta.num.threshold
+ 10
+
+ Number of delta files that must exist in a directory before the compactor will attempt
+ a minor compaction.
+
+
+
+ hive.compactor.delta.pct.threshold
+ 0.1
+ Percentage (by size) of base that deltas can be before major compaction is initiated.
+
+
+ hive.compactor.abortedtxn.threshold
+ 1000
+
+ Number of aborted transactions involving a particular table or partition before major
+ compaction is initiated.
+
+
+
+ hive.hbase.wal.enabled
+ true
+
+ Whether writes to HBase should be forced to the write-ahead log.
+ Disabling this improves HBase write performance at the risk of lost writes in case of a crash.
+
+
+
+ hive.hbase.generatehfiles
+ false
+ True when HBaseStorageHandler should generate hfiles instead of operate against the online table.
+
+
+ hive.archive.enabled
+ false
+ Whether archiving operations are permitted
+
+
+ hive.optimize.index.groupby
+ false
+ Whether to enable optimization of group-by queries using Aggregate indexes.
+
+
+ hive.outerjoin.supports.filters
+ true
+
+
+
+ hive.fetch.task.conversion
+ minimal
+
+ Some select queries can be converted to single FETCH task minimizing latency.
+ Currently the query should be single sourced not having any subquery and should not have
+ any aggregations or distincts (which incurs RS), lateral views and joins.
+ 1. minimal : SELECT STAR, FILTER on partition columns, LIMIT only
+ 2. more : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)
+
+
+
+ hive.fetch.task.conversion.threshold
+ -1
+
+ Input threshold for applying hive.fetch.task.conversion. If target table is native, input length
+ is calculated by summation of file lengths. If it's not native, storage handler for the table
+ can optionally implement org.apache.hadoop.hive.ql.metadata.InputEstimator interface.
+
+
+
+ hive.fetch.task.aggr
+ false
+
+ Aggregation queries with no group-by clause (for example, select count(*) from src) execute
+ final aggregations in single reduce task. If this is set true, Hive delegates final aggregation
+ stage to fetch task, possibly decreasing the query time.
+
+
+
+ hive.compute.query.using.stats
+ false
+
+ When set to true Hive will answer a few queries like count(1) purely using stats
+ stored in metastore. For basic stats collection turn on the config hive.stats.autogather to true.
+ For more advanced stats collection need to run analyze table queries.
+
+
+
+ hive.fetch.output.serde
+ org.apache.hadoop.hive.serde2.DelimitedJSONSerDe
+ The SerDe used by FetchTask to serialize the fetch output.
+
+
+ hive.cache.expr.evaluation
+ true
+ If true, evaluation result of deterministic expression referenced twice or more will be cached.
+
+
+ hive.variable.substitute
+ true
+ This enables substitution using syntax like ${var} ${system:var} and ${env:var}.
+
+
+ hive.variable.substitute.depth
+ 40
+ The maximum replacements the substitution engine will do.
+
+
+ hive.conf.validation
+ true
+ Enables type checking for registered Hive configurations
+
+
+ hive.semantic.analyzer.hook
+
+
+
+
+ hive.security.authorization.enabled
+ false
+ enable or disable the Hive client authorization
+
+
+ hive.security.authorization.manager
+ org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider
+
+ The Hive client authorization manager class name. The user defined authorization class should implement
+ interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.
+
+
+
+ hive.security.authenticator.manager
+ org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator
+
+ hive client authenticator manager class name. The user defined authenticator should implement
+ interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.
+
+
+
+ hive.security.metastore.authorization.manager
+ org.apache.hadoop.hive.ql.security.authorization.DefaultHiveMetastoreAuthorizationProvider
+
+ authorization manager class name to be used in the metastore for authorization.
+ The user defined authorization class should implement interface
+ org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider.
+
+
+
+ hive.security.metastore.authenticator.manager
+ org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator
+
+ authenticator manager class name to be used in the metastore for authentication.
+ The user defined authenticator should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.
+
+
+
+ hive.security.authorization.createtable.user.grants
+
+
+ the privileges automatically granted to some users whenever a table gets created.
+ An example like "userX,userY:select;userZ:create" will grant select privilege to userX and userY,
+ and grant create privilege to userZ whenever a new table created.
+
+
+
+ hive.security.authorization.createtable.group.grants
+
+
+ the privileges automatically granted to some groups whenever a table gets created.
+ An example like "groupX,groupY:select;groupZ:create" will grant select privilege to groupX and groupY,
+ and grant create privilege to groupZ whenever a new table created.
+
+
+
+ hive.security.authorization.createtable.role.grants
+
+
+ the privileges automatically granted to some roles whenever a table gets created.
+ An example like "roleX,roleY:select;roleZ:create" will grant select privilege to roleX and roleY,
+ and grant create privilege to roleZ whenever a new table created.
+
+
+
+ hive.security.authorization.createtable.owner.grants
+
+
+ the privileges automatically granted to the owner whenever a table gets created.
+ An example like "select,drop" will grant select and drop privilege to the owner of the table
+
+
+
+ hive.security.authorization.sqlstd.confwhitelist
+
+ interal variable. List of modifiable configurations by user.
+
+
+ hive.cli.print.header
+ false
+ Whether to print the names of the columns in query output.
+
+
+ hive.error.on.empty.partition
+ false
+ Whether to throw an exception if dynamic partition insert generates empty results.
+
+
+ hive.index.compact.file
+
+ internal variable
+
+
+ hive.index.blockfilter.file
+
+ internal variable
+
+
+ hive.index.compact.file.ignore.hdfs
+ false
+
+ When true the HDFS location stored in the index file will be ignored at runtime.
+ If the data got moved or the name of the cluster got changed, the index data should still be usable.
+
+
+
+ hive.exim.uri.scheme.whitelist
+ hdfs,pfile
+ A comma separated list of acceptable URI schemes for import and export.
+
+
+ hive.mapper.cannot.span.multiple.partitions
+ false
+
+
+
+ hive.rework.mapredwork
+ false
+
+ should rework the mapred work or not.
+ This is first introduced by SymlinkTextInputFormat to replace symlink files with real paths at compile time.
+
+
+
+ hive.exec.concatenate.check.index
+ true
+
+ If this is set to true, Hive will throw error when doing
+ 'alter table tbl_name [partSpec] concatenate' on a table/partition
+ that has indexes on it. The reason the user want to set this to true
+ is because it can help user to avoid handling all index drop, recreation,
+ rebuild work. This is very helpful for tables with thousands of partitions.
+
+
+
+ hive.io.exception.handlers
+
+
+ A list of io exception handler class names. This is used
+ to construct a list exception handlers to handle exceptions thrown
+ by record readers
+
+
+
+ hive.log4j.file
+
+
+ Hive log4j configuration file.
+ If the property is not set, then logging will be initialized using hive-log4j.properties found on the classpath.
+ If the property is set, the value must be a valid URI (java.net.URI, e.g. "file:///tmp/my-logging.properties"),
+ which you can then extract a URL from and pass to PropertyConfigurator.configure(URL).
+
+
+
+ hive.exec.log4j.file
+
+
+ Hive log4j configuration file for execution mode(sub command).
+ If the property is not set, then logging will be initialized using hive-exec-log4j.properties found on the classpath.
+ If the property is set, the value must be a valid URI (java.net.URI, e.g. "file:///tmp/my-logging.properties"),
+ which you can then extract a URL from and pass to PropertyConfigurator.configure(URL).
+
+
+
+ hive.autogen.columnalias.prefix.label
+ _c
+
+ String used as a prefix when auto generating column alias.
+ By default the prefix label will be appended with a column position number to form the column alias.
+ Auto generation would happen if an aggregate function is used in a select clause without an explicit alias.
+
+
+
+ hive.autogen.columnalias.prefix.includefuncname
+ false
+ Whether to include function name in the column alias auto generated by Hive.
+
+
+ hive.exec.perf.logger
+ org.apache.hadoop.hive.ql.log.PerfLogger
+
+ The class responsible for logging client side performance metrics.
+ Must be a subclass of org.apache.hadoop.hive.ql.log.PerfLogger
+
+
+
+ hive.start.cleanup.scratchdir
+ false
+ To cleanup the Hive scratchdir when starting the Hive Server
+
+
+ hive.insert.into.multilevel.dirs
+ false
+
+ Where to insert into multilevel directories like
+ "insert directory '/HIVEFT25686/chinna/' from table"
+
+
+
+ hive.warehouse.subdir.inherit.perms
+ false
+
+ Set this to true if the the table directories should inherit the
+ permission of the warehouse or database directory instead of being created
+ with the permissions derived from dfs umask
+
+
+
+ hive.insert.into.external.tables
+ true
+ whether insert into external tables is allowed
+
+
+ hive.exec.driver.run.hooks
+
+ A comma separated list of hooks which implement HiveDriverRunHook. Will be run at the beginning and end of Driver.run, these will be run in the order specified.
+
+
+ hive.ddl.output.format
+
+
+ The data format to use for DDL output. One of "text" (for human
+ readable text) or "json" (for a json object).
+
+
+
+ hive.entity.separator
+ @
+ Separator used to construct names of tables and partitions. For example, dbname@tablename@partitionname
+
+
+ hive.display.partition.cols.separately
+ true
+
+ In older Hive version (0.10 and earlier) no distinction was made between
+ partition columns or non-partition columns while displaying columns in describe
+ table. From 0.12 onwards, they are displayed separately. This flag will let you
+ get old behavior, if desired. See, test-case in patch for HIVE-6689.
+
+
+
+ hive.server2.max.start.attempts
+ 30
+
+ This number of times HiveServer2 will attempt to start before exiting, sleeping 60 seconds between retries.
+ The default of 30 will keep trying for 30 minutes.
+
+
+
+ hive.server2.transport.mode
+ binary
+ Server transport mode. "binary" or "http"
+
+
+ hive.server2.thrift.http.port
+ 10001
+ Port number when in HTTP mode.
+
+
+ hive.server2.thrift.http.path
+ cliservice
+ Path component of URL endpoint when in HTTP mode.
+
+
+ hive.server2.thrift.http.min.worker.threads
+ 5
+ Minimum number of worker threads when in HTTP mode.
+
+
+ hive.server2.thrift.http.max.worker.threads
+ 500
+ Maximum number of worker threads when in HTTP mode.
+
+
+ hive.server2.thrift.port
+ 10000
+
+ Port number of HiveServer2 Thrift interface.
+ Can be overridden by setting $HIVE_SERVER2_THRIFT_PORT
+
+
+
+ hive.server2.thrift.bind.host
+
+
+ Bind host on which to run the HiveServer2 Thrift interface.
+ Can be overridden by setting $HIVE_SERVER2_THRIFT_BIND_HOST
+
+
+
+ hive.server2.thrift.sasl.qop
+ auth
+
+ Sasl QOP value; Set it to one of following values to enable higher levels of
+ protection for HiveServer2 communication with clients.
+ "auth" - authentication only (default)
+ "auth-int" - authentication plus integrity protection
+ "auth-conf" - authentication plus integrity and confidentiality protection
+ This is applicable only if HiveServer2 is configured to use Kerberos authentication.
+
+
+
+ hive.server2.thrift.min.worker.threads
+ 5
+ Minimum number of Thrift worker threads
+
+
+ hive.server2.thrift.max.worker.threads
+ 500
+ Maximum number of Thrift worker threads
+
+
+ hive.server2.async.exec.threads
+ 100
+ Number of threads in the async thread pool for HiveServer2
+
+
+ hive.server2.async.exec.shutdown.timeout
+ 10
+ Time (in seconds) for which HiveServer2 shutdown will wait for async
+
+
+ hive.server2.async.exec.wait.queue.size
+ 100
+
+ Size of the wait queue for async thread pool in HiveServer2.
+ After hitting this limit, the async thread pool will reject new requests.
+
+
+
+ hive.server2.async.exec.keepalive.time
+ 10
+
+ Time (in seconds) that an idle HiveServer2 async thread (from the thread pool) will wait
+ for a new task to arrive before terminating
+
+
+
+ hive.server2.long.polling.timeout
+ 5000
+
+ Time in milliseconds that HiveServer2 will wait,
+ before responding to asynchronous calls that use long polling
+
+
+
+ hive.server2.authentication
+ NONE
+
+ Client authentication types.
+ NONE: no authentication check
+ LDAP: LDAP/AD based authentication
+ KERBEROS: Kerberos/GSSAPI authentication
+ CUSTOM: Custom authentication provider
+ (Use with property hive.server2.custom.authentication.class)
+
+
+
+ hive.server2.allow.user.substitution
+ true
+ Allow alternate user to be specified as part of HiveServer2 open connection request.
+
+
+ hive.server2.authentication.kerberos.keytab
+
+ Kerberos keytab file for server principal
+
+
+ hive.server2.authentication.kerberos.principal
+
+ Kerberos server principal
+
+
+ hive.server2.authentication.spnego.keytab
+
+
+ keytab file for SPNego principal, optional,
+ typical value would look like /etc/security/keytabs/spnego.service.keytab,
+ This keytab would be used by HiveServer2 when Kerberos security is enabled and
+ HTTP transport mode is used.
+ This needs to be set only if SPNEGO is to be used in authentication.
+ SPNego authentication would be honored only if valid
+ hive.server2.authentication.spnego.principal
+ and
+ hive.server2.authentication.spnego.keytab
+ are specified.
+
+
+
+ hive.server2.authentication.spnego.principal
+
+
+ SPNego service principal, optional,
+ typical value would look like HTTP/_HOST@EXAMPLE.COM
+ SPNego service principal would be used by HiveServer2 when Kerberos security is enabled
+ and HTTP transport mode is used.
+ This needs to be set only if SPNEGO is to be used in authentication.
+
+
+
+ hive.server2.authentication.ldap.url
+
+ LDAP connection URL
+
+
+ hive.server2.authentication.ldap.baseDN
+
+ LDAP base DN
+
+
+ hive.server2.authentication.ldap.Domain
+
+
+
+
+ hive.server2.custom.authentication.class
+
+
+ Custom authentication class. Used when property
+ 'hive.server2.authentication' is set to 'CUSTOM'. Provided class
+ must be a proper implementation of the interface
+ org.apache.hive.service.auth.PasswdAuthenticationProvider. HiveServer2
+ will call its Authenticate(user, passed) method to authenticate requests.
+ The implementation may optionally extend Hadoop's
+ org.apache.hadoop.conf.Configured class to grab Hive's Configuration object.
+
+
+
+ hive.server2.authentication.pam.services
+
+
+ List of the underlying pam services that should be used when auth type is PAM
+ A file with the same name must exist in /etc/pam.d
+
+
+
+ hive.server2.enable.doAs
+ true
+
+ Setting this property to true will have HiveServer2 execute
+ Hive operations as the user making the calls to it.
+
+
+
+ hive.server2.table.type.mapping
+ CLASSIC
+
+ This setting reflects how HiveServer2 will report the table types for JDBC and other
+ client implementations that retrieve the available tables and supported table types
+ HIVE : Exposes Hive's native table types like MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW
+ CLASSIC : More generic types like TABLE and VIEW
+
+
+
+ hive.server2.session.hook
+
+
+
+
+ hive.server2.use.SSL
+ false
+
+
+
+ hive.server2.keystore.path
+
+
+
+
+ hive.server2.keystore.password
+
+
+
+
+ hive.security.command.whitelist
+ set,reset,dfs,add,delete,compile
+ Comma separated list of non-SQL Hive commands users are authorized to execute
+
+
+ hive.conf.restricted.list
+ hive.security.authenticator.manager,hive.security.authorization.manager
+ Comma separated list of configuration options which are immutable at runtime
+
+
+ hive.multi.insert.move.tasks.share.dependencies
+ false
+
+ If this is set all move tasks for tables/partitions (not directories) at the end of a
+ multi-insert query will only begin once the dependencies for all these move tasks have been
+ met.
+ Advantages: If concurrency is enabled, the locks will only be released once the query has
+ finished, so with this config enabled, the time when the table/partition is
+ generated will be much closer to when the lock on it is released.
+ Disadvantages: If concurrency is not enabled, with this disabled, the tables/partitions which
+ are produced by this query and finish earlier will be available for querying
+ much earlier. Since the locks are only released once the query finishes, this
+ does not apply if concurrency is enabled.
+
+
+
+ hive.exec.infer.bucket.sort
+ false
+
+ If this is set, when writing partitions, the metadata will include the bucketing/sorting
+ properties with which the data was written if any (this will not overwrite the metadata
+ inherited from the table if the table is bucketed/sorted)
+
+
+
+ hive.exec.infer.bucket.sort.num.buckets.power.two
+ false
+
+ If this is set, when setting the number of reducers for the map reduce task which writes the
+ final output files, it will choose a number which is a power of two, unless the user specifies
+ the number of reducers to use using mapred.reduce.tasks. The number of reducers
+ may be set to a power of two, only to be followed by a merge task meaning preventing
+ anything from being inferred.
+ With hive.exec.infer.bucket.sort set to true:
+ Advantages: If this is not set, the number of buckets for partitions will seem arbitrary,
+ which means that the number of mappers used for optimized joins, for example, will
+ be very low. With this set, since the number of buckets used for any partition is
+ a power of two, the number of mappers used for optimized joins will be the least
+ number of buckets used by any partition being joined.
+ Disadvantages: This may mean a much larger or much smaller number of reducers being used in the
+ final map reduce job, e.g. if a job was originally going to take 257 reducers,
+ it will now take 512 reducers, similarly if the max number of reducers is 511,
+ and a job was going to use this many, it will now use 256 reducers.
+
+
+
+ hive.merge.current.job.concatenate.list.bucketing
+ true
+
+
+
+ hive.merge.current.job.concatenate.list.bucketing.depth
+ 0
+
+
+
+ hive.optimize.listbucketing
+ false
+ Enable list bucketing optimizer. Default value is false so that we disable it by default.
+
+
+ hive.server.read.socket.timeout
+ 10
+ Timeout for the HiveServer to close the connection if no response from the client in N seconds, defaults to 10 seconds.
+
+
+ hive.server.tcp.keepalive
+ true
+ Whether to enable TCP keepalive for the Hive Server. Keepalive will prevent accumulation of half-open connections.
+
+
+ hive.decode.partition.name
+ false
+ Whether to show the unquoted partition names in query results.
+
+
+ hive.execution.engine
+ mr
+ Chooses execution engine. Options are: mr (Map reduce, default) or tez (hadoop 2 only)
+
+
+ hive.jar.directory
+
+
+ This is the location hive in tez mode will look for to find a site wide
+ installed hive instance.
+
+
+
+ hive.user.install.directory
+ hdfs:///user/
+
+ If hive (in tez mode only) cannot find a usable hive jar in "hive.jar.directory",
+ it will upload the hive jar to "hive.user.install.directory/user.name"
+ and use it to run queries.
+
+
+
+ hive.vectorized.execution.enabled
+ false
+
+ This flag should be set to true to enable vectorized mode of query execution.
+ The default value is false.
+
+
+
+ hive.vectorized.groupby.checkinterval
+ 100000
+ Number of entries added to the group by aggregation hash before a recomputation of average entry size is performed.
+
+
+ hive.vectorized.groupby.maxentries
+ 1000000
+
+ Max number of entries in the vector group by aggregation hashtables.
+ Exceeding this will trigger a flush irrelevant of memory pressure condition.
+
+
+
+ hive.vectorized.groupby.flush.percent
+ 0.1
+ Percent of entries in the group by aggregation hash flushed when the memory threshold is exceeded.
+
+
+ hive.typecheck.on.insert
+ true
+
+
+
+ hive.rpc.query.plan
+ false
+ Whether to send the query plan via local resource or RPC
+
+
+ hive.compute.splits.in.am
+ true
+ Whether to generate the splits locally or in the AM (tez only)
+
+
+ hive.prewarm.enabled
+ false
+ Enables container prewarm for Tez (Hadoop 2 only)
+
+
+ hive.prewarm.numcontainers
+ 10
+ Controls the number of containers to prewarm for Tez (Hadoop 2 only)
+
+
+ hive.stageid.rearrange
+ none
+
+
+
+ hive.explain.dependency.append.tasktype
+ false
+
+
+
+ hive.counters.group.name
+ HIVE
+ The name of counter group for internal Hive variables (CREATED_FILE, FATAL_ERROR, etc.)
+
+
+ hive.server2.tez.default.queues
+
+
+ A list of comma separated values corresponding to YARN queues of the same name.
+ When HiveServer2 is launched in Tez mode, this configuration needs to be set
+ for multiple Tez sessions to run in parallel on the cluster.
+
+
+
+ hive.server2.tez.sessions.per.default.queue
+ 1
+
+ A positive integer that determines the number of Tez sessions that should be
+ launched on each of the queues specified by "hive.server2.tez.default.queues".
+ Determines the parallelism on each queue.
+
+
+
+ hive.server2.tez.initialize.default.sessions
+ false
+
+ This flag is used in HiveServer2 to enable a user to use HiveServer2 without
+ turning on Tez for HiveServer2. The user could potentially want to run queries
+ over Tez without the pool of sessions.
+
+
+
+ hive.support.quoted.identifiers
+ column
+
+ Whether to use quoted identifier. 'none' ot 'column' can be used.
+ none: default(past) behavior. Implies only alphaNumeric and underscore are valid characters in identifiers.
+ column: implies column names can contain any character.
+
+
+
+ hive.users.in.admin.role
+
+
+ Comma separated list of users who are in admin role for bootstrapping.
+ More users can be added in ADMIN role later.
+
+
+
+ hive.compat
+ 0.12
+
+ Enable (configurable) deprecated behaviors by setting desired level of backward compatibility.
+ Setting to 0.12:
+ Maintains division behavior: int / int = double
+
+
+
+ hive.convert.join.bucket.mapjoin.tez
+ false
+
+ Whether joins can be automatically converted to bucket map joins in hive
+ when tez is used as the execution engine.
+
+
+
+ hive.exec.check.crossproducts
+ true
+ Check if a plan contains a Cross Product. If there is one, output a warning to the Session's console.
+
+
+ hive.localize.resource.wait.interval
+ 5000
+ Time in milliseconds to wait for another thread to localize the same resource for hive-tez.
+
+
+ hive.localize.resource.num.wait.attempts
+ 5
+ The number of attempts waiting for localizing a resource in hive-tez.
+
+
+ hive.tez.auto.reducer.parallelism
+ false
+
+ Turn on Tez' auto reducer parallelism feature. When enabled, Hive will still estimate data sizes
+ and set parallelism estimates. Tez will sample source vertices' output sizes and adjust the estimates at runtime as
+ necessary.
+
+
+
+ hive.tez.max.partition.factor
+ 2.0
+ When auto reducer parallelism is enabled this factor will be used to over-partition data in shuffle edges.
+
+
+ hive.tez.min.partition.factor
+ 0.25
+
+ When auto reducer parallelism is enabled this factor will be used to put a lower limit to the number
+ of reducers that tez specifies.
+
+
+
+ hive.tez.dynamic.partition.pruning
+ true
+ When dynamic pruning is enabled, joins on partition keys will be processed by sending events from the processing vertices to the tez application master. These events will be used to prune unnecessary partitions.
+
+
+ hive.tez.dynamic.partition.pruning.max.event.size
+ 1048576
+ Maximum size of events sent by processors in dynamic pruning. If this size is crossed no pruning will take place.
+
+
+ hive.tez.dynamic.parition.pruning.max.data.size
+ 104857600
+ Maximum total data size of events in dynamic pruning.
+
+
diff --git itests/qtest/testconfiguration.properties itests/qtest/testconfiguration.properties
index d0df420..6fd1096 100644
--- itests/qtest/testconfiguration.properties
+++ itests/qtest/testconfiguration.properties
@@ -1,5 +1,5 @@
minimr.query.files=stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q,udf_using.q,empty_dir_in_table.q,temp_table_external.q
minimr.query.negative.files=cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q,udf_local_resource.q
-minitez.query.files=tez_fsstat.q,mapjoin_decimal.q,tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q,tez_union.q,bucket_map_join_tez1.q,bucket_map_join_tez2.q,tez_schema_evolution.q,tez_join_hash.q,tez_bmj_schema_evolution.q
+minitez.query.files=tez_fsstat.q,mapjoin_decimal.q,tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q,tez_union.q,bucket_map_join_tez1.q,bucket_map_join_tez2.q,tez_schema_evolution.q,tez_join_hash.q,tez_bmj_schema_evolution.q,dynamic_partition_pruning.q
minitez.query.files.shared=cross_product_check_1.q,cross_product_check_2.q,dynpart_sort_opt_vectorization.q,dynpart_sort_optimization.q,orc_analyze.q,join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q,stats_counter.q,stats_noscan_1.q,stats_counter_partitioned.q,union2.q,union3.q,union4.q,union5.q,union6.q,union7.q,union8.q,union9.q,transform1.q,transform2.q,transform_ppr1.q,transform_ppr2.q,script_env_var1.q,script_env_var2.q,script_pipe.q,scriptfile1.q,metadataonly1.q,temp_table.q,vectorized_ptf.q,optimize_nullscan.q
beeline.positive.exclude=add_part_exist.q,alter1.q,alter2.q,alter4.q,alter5.q,alter_rename_partition.q,alter_rename_partition_authorization.q,archive.q,archive_corrupt.q,archive_multi.q,archive_mr_1806.q,archive_multi_mr_1806.q,authorization_1.q,authorization_2.q,authorization_4.q,authorization_5.q,authorization_6.q,authorization_7.q,ba_table1.q,ba_table2.q,ba_table3.q,ba_table_udfs.q,binary_table_bincolserde.q,binary_table_colserde.q,cluster.q,columnarserde_create_shortcut.q,combine2.q,constant_prop.q,create_nested_type.q,create_or_replace_view.q,create_struct_table.q,create_union_table.q,database.q,database_location.q,database_properties.q,ddltime.q,describe_database_json.q,drop_database_removes_partition_dirs.q,escape1.q,escape2.q,exim_00_nonpart_empty.q,exim_01_nonpart.q,exim_02_00_part_empty.q,exim_02_part.q,exim_03_nonpart_over_compat.q,exim_04_all_part.q,exim_04_evolved_parts.q,exim_05_some_part.q,exim_06_one_part.q,exim_07_all_part_over_nonoverlap.q,exim_08_nonpart_rename.q,exim_09_part_spec_nonoverlap.q,exim_10_external_managed.q,exim_11_managed_external.q,exim_12_external_location.q,exim_13_managed_location.q,exim_14_managed_location_over_existing.q,exim_15_external_part.q,exim_16_part_external.q,exim_17_part_managed.q,exim_18_part_external.q,exim_19_00_part_external_location.q,exim_19_part_external_location.q,exim_20_part_managed_location.q,exim_21_export_authsuccess.q,exim_22_import_exist_authsuccess.q,exim_23_import_part_authsuccess.q,exim_24_import_nonexist_authsuccess.q,global_limit.q,groupby_complex_types.q,groupby_complex_types_multi_single_reducer.q,index_auth.q,index_auto.q,index_auto_empty.q,index_bitmap.q,index_bitmap1.q,index_bitmap2.q,index_bitmap3.q,index_bitmap_auto.q,index_bitmap_rc.q,index_compact.q,index_compact_1.q,index_compact_2.q,index_compact_3.q,index_stale_partitioned.q,init_file.q,input16.q,input16_cc.q,input46.q,input_columnarserde.q,input_dynamicserde.q,input_lazyserde.q,input_testxpath3.q,input_testxpath4.q,insert2_overwrite_partitions.q,insertexternal1.q,join_thrift.q,lateral_view.q,load_binary_data.q,load_exist_part_authsuccess.q,load_nonpart_authsuccess.q,load_part_authsuccess.q,loadpart_err.q,lock1.q,lock2.q,lock3.q,lock4.q,merge_dynamic_partition.q,multi_insert.q,multi_insert_move_tasks_share_dependencies.q,null_column.q,ppd_clusterby.q,query_with_semi.q,rename_column.q,sample6.q,sample_islocalmode_hook.q,set_processor_namespaces.q,show_tables.q,source.q,split_sample.q,str_to_map.q,transform1.q,udaf_collect_set.q,udaf_context_ngrams.q,udaf_histogram_numeric.q,udaf_ngrams.q,udaf_percentile_approx.q,udf_array.q,udf_bitmap_and.q,udf_bitmap_or.q,udf_explode.q,udf_format_number.q,udf_map.q,udf_map_keys.q,udf_map_values.q,udf_max.q,udf_min.q,udf_named_struct.q,udf_percentile.q,udf_printf.q,udf_sentences.q,udf_sort_array.q,udf_split.q,udf_struct.q,udf_substr.q,udf_translate.q,udf_union.q,udf_xpath.q,udtf_stack.q,view.q,virtual_column.q
diff --git metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
old mode 100644
new mode 100755
diff --git ql/if/queryplan.thrift ql/if/queryplan.thrift
index eafbe5a..6026d94 100644
--- ql/if/queryplan.thrift
+++ ql/if/queryplan.thrift
@@ -56,6 +56,7 @@ enum OperatorType {
PTF,
MUX,
DEMUX,
+ EVENT,
}
struct Operator {
diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.cpp ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
index 96dbb29..b9e04e2 100644
--- ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
+++ ql/src/gen/thrift/gen-cpp/queryplan_types.cpp
@@ -51,7 +51,8 @@ int _kOperatorTypeValues[] = {
OperatorType::HASHTABLEDUMMY,
OperatorType::PTF,
OperatorType::MUX,
- OperatorType::DEMUX
+ OperatorType::DEMUX,
+ OperatorType::EVENT
};
const char* _kOperatorTypeNames[] = {
"JOIN",
@@ -74,9 +75,10 @@ const char* _kOperatorTypeNames[] = {
"HASHTABLEDUMMY",
"PTF",
"MUX",
- "DEMUX"
+ "DEMUX",
+ "EVENT"
};
-const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(21, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(22, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
int _kTaskTypeValues[] = {
TaskType::MAP,
diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.h ql/src/gen/thrift/gen-cpp/queryplan_types.h
index 634dd55..30ef711 100644
--- ql/src/gen/thrift/gen-cpp/queryplan_types.h
+++ ql/src/gen/thrift/gen-cpp/queryplan_types.h
@@ -56,7 +56,8 @@ struct OperatorType {
HASHTABLEDUMMY = 17,
PTF = 18,
MUX = 19,
- DEMUX = 20
+ DEMUX = 20,
+ EVENT = 21
};
};
diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java
index aa094ee..6f23575 100644
--- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java
+++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java
@@ -32,7 +32,8 @@
HASHTABLEDUMMY(17),
PTF(18),
MUX(19),
- DEMUX(20);
+ DEMUX(20),
+ EVENT(21);
private final int value;
@@ -95,6 +96,8 @@ public static OperatorType findByValue(int value) {
return MUX;
case 20:
return DEMUX;
+ case 21:
+ return EVENT;
default:
return null;
}
diff --git ql/src/gen/thrift/gen-php/Types.php ql/src/gen/thrift/gen-php/Types.php
index 5164b2c..4ed7fcc 100644
--- ql/src/gen/thrift/gen-php/Types.php
+++ ql/src/gen/thrift/gen-php/Types.php
@@ -56,6 +56,7 @@ final class OperatorType {
const PTF = 18;
const MUX = 19;
const DEMUX = 20;
+ const EVENT = 21;
static public $__names = array(
0 => 'JOIN',
1 => 'MAPJOIN',
@@ -78,6 +79,7 @@ final class OperatorType {
18 => 'PTF',
19 => 'MUX',
20 => 'DEMUX',
+ 21 => 'EVENT',
);
}
diff --git ql/src/gen/thrift/gen-py/queryplan/ttypes.py ql/src/gen/thrift/gen-py/queryplan/ttypes.py
index 2a3f745..96777fa 100644
--- ql/src/gen/thrift/gen-py/queryplan/ttypes.py
+++ ql/src/gen/thrift/gen-py/queryplan/ttypes.py
@@ -66,6 +66,7 @@ class OperatorType:
PTF = 18
MUX = 19
DEMUX = 20
+ EVENT = 21
_VALUES_TO_NAMES = {
0: "JOIN",
@@ -89,6 +90,7 @@ class OperatorType:
18: "PTF",
19: "MUX",
20: "DEMUX",
+ 21: "EVENT",
}
_NAMES_TO_VALUES = {
@@ -113,6 +115,7 @@ class OperatorType:
"PTF": 18,
"MUX": 19,
"DEMUX": 20,
+ "EVENT": 21,
}
class TaskType:
diff --git ql/src/gen/thrift/gen-rb/queryplan_types.rb ql/src/gen/thrift/gen-rb/queryplan_types.rb
index 35e1f0f..449becf 100644
--- ql/src/gen/thrift/gen-rb/queryplan_types.rb
+++ ql/src/gen/thrift/gen-rb/queryplan_types.rb
@@ -42,8 +42,9 @@ module OperatorType
PTF = 18
MUX = 19
DEMUX = 20
- VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX"}
- VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX]).freeze
+ EVENT = 21
+ VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX", 21 => "EVENT"}
+ VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX, EVENT]).freeze
end
module TaskType
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
new file mode 100644
index 0000000..527a645
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
@@ -0,0 +1,141 @@
+/**
+ * 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.exec;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.tez.TezContext;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
+import org.apache.hadoop.hive.ql.plan.api.OperatorType;
+import org.apache.hadoop.hive.serde2.Serializer;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+
+@SuppressWarnings({ "deprecation", "serial" })
+public class AppMasterEventOperator extends Operator {
+
+ private transient Serializer serializer;
+ private transient DataOutputBuffer buffer;
+ private transient boolean hasReachedMaxSize = false;
+ private transient long MAX_SIZE;
+
+ @Override
+ public void initializeOp(Configuration hconf) throws HiveException {
+ MAX_SIZE = HiveConf.getLongVar(hconf, ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING_MAX_EVENT_SIZE);
+ serializer =
+ (Serializer) ReflectionUtils.newInstance(conf.getTable().getDeserializerClass(), null);
+ initDataBuffer(false);
+ }
+
+ private void initDataBuffer(boolean skipPruning) throws HiveException {
+ buffer = new DataOutputBuffer();
+ try {
+ // where does this go to?
+ buffer.writeUTF(((TezContext) TezContext.get()).getTezProcessorContext().getTaskVertexName());
+
+ // add any other header info
+ getConf().writeEventHeader(buffer);
+
+ // write byte to say whether to skip pruning or not
+ buffer.writeBoolean(skipPruning);
+ } catch (IOException e) {
+ throw new HiveException(e);
+ }
+ }
+
+ @Override
+ public void processOp(Object row, int tag) throws HiveException {
+ if (hasReachedMaxSize) {
+ return;
+ }
+
+ ObjectInspector rowInspector = inputObjInspectors[0];
+ try {
+ Writable writableRow = serializer.serialize(row, rowInspector);
+ writableRow.write(buffer);
+ if (buffer.getLength() > MAX_SIZE) {
+ LOG.info("Disabling dynamic pruning. Buffer size for pruning: " + buffer.getLength());
+ hasReachedMaxSize = true;
+ buffer = null;
+ }
+ } catch (Exception e) {
+ throw new HiveException(e);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("AppMasterEvent: " + row);
+ }
+ forward(row, rowInspector);
+ }
+
+ @Override
+ public void closeOp(boolean abort) throws HiveException {
+ if (!abort) {
+ TezContext context = (TezContext) TezContext.get();
+
+ String vertexName = getConf().getVertexName();
+ String inputName = getConf().getInputName();
+
+ byte[] payload = null;
+
+ if (hasReachedMaxSize) {
+ initDataBuffer(true);
+ }
+
+ payload = new byte[buffer.getLength()];
+ System.arraycopy(buffer.getData(), 0, payload, 0, buffer.getLength());
+
+ Event event =
+ InputInitializerEvent.create(vertexName, inputName,
+ ByteBuffer.wrap(payload, 0, payload.length));
+
+ LOG.info("Sending Tez event to vertex = " + vertexName + ", input = " + inputName
+ + ". Payload size = " + payload.length);
+
+ context.getTezProcessorContext().sendEvents(Collections.singletonList(event));
+ }
+ }
+
+ @Override
+ public OperatorType getType() {
+ return OperatorType.EVENT;
+ }
+
+ /**
+ * @return the name of the operator
+ */
+ @Override
+ public String getName() {
+ return getOperatorName();
+ }
+
+ static public String getOperatorName() {
+ return "EVENT";
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
index 2bcb481..8946221 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
@@ -29,13 +29,15 @@
import org.apache.hadoop.hive.ql.exec.vector.VectorLimitOperator;
import org.apache.hadoop.hive.ql.exec.vector.VectorMapJoinOperator;
import org.apache.hadoop.hive.ql.exec.vector.VectorReduceSinkOperator;
-import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator;
import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator;
import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
import org.apache.hadoop.hive.ql.plan.CollectDesc;
import org.apache.hadoop.hive.ql.plan.DemuxDesc;
import org.apache.hadoop.hive.ql.plan.DummyStoreDesc;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.ExtractDesc;
import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
@@ -64,6 +66,7 @@
* OperatorFactory.
*
*/
+@SuppressWarnings({ "rawtypes", "unchecked" })
public final class OperatorFactory {
private static final List opvec;
private static final List vectorOpvec;
@@ -101,6 +104,10 @@
DemuxOperator.class));
opvec.add(new OpTuple(MuxDesc.class,
MuxOperator.class));
+ opvec.add(new OpTuple(AppMasterEventDesc.class,
+ AppMasterEventOperator.class));
+ opvec.add(new OpTuple(DynamicPruningEventDesc.class,
+ AppMasterEventOperator.class));
}
static {
@@ -119,9 +126,9 @@
private static final class OpTuple {
private final Class descClass;
- private final Class extends Operator> opClass;
+ private final Class extends Operator>> opClass;
- public OpTuple(Class descClass, Class extends Operator> opClass) {
+ public OpTuple(Class descClass, Class extends Operator>> opClass) {
this.descClass = descClass;
this.opClass = opClass;
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java
index 0d0770f..0d3366b 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java
@@ -39,9 +39,9 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.split.TezMapReduceSplitsGrouper;
import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
-import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.VertexLocationHint;
@@ -97,7 +97,7 @@ public void initialize() {
@Override
public void onVertexStarted(Map> completions) {
int numTasks = context.getVertexNumTasks(context.getVertexName());
- List scheduledTasks =
+ List scheduledTasks =
new ArrayList(numTasks);
for (int i = 0; i < numTasks; ++i) {
scheduledTasks.add(new VertexManagerPluginContext.TaskWithLocationHint(new Integer(i), null));
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
index f9fbf22..9e1929a 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java
@@ -18,9 +18,21 @@
package org.apache.hadoop.hive.ql.exec.tez;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import javolution.testing.AssertionException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -30,7 +42,17 @@
import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
import org.apache.hadoop.hive.ql.plan.MapWork;
import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputFormat;
import org.apache.hadoop.mapred.InputSplit;
@@ -46,12 +68,12 @@
import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto;
import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.events.InputInitializerEvent;
import org.apache.tez.runtime.api.InputInitializer;
import org.apache.tez.runtime.api.InputInitializerContext;
import org.apache.tez.runtime.api.InputSpecUpdate;
+import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.Lists;
@@ -63,11 +85,19 @@
* making sure that splits from different partitions are only grouped if they
* are of the same schema, format and serde
*/
+@SuppressWarnings("deprecation")
public class HiveSplitGenerator extends InputInitializer {
private static final Log LOG = LogFactory.getLog(HiveSplitGenerator.class);
private static final SplitGrouper grouper = new SplitGrouper();
+ private final Map> sourceInfoMap =
+ new HashMap>();
+ private InputInitializerContext context;
+ private final BytesWritable writable = new BytesWritable();
+ private final BlockingQueue queue =
+ new LinkedBlockingQueue();
+ private int sourceInfoCount = 0;
public HiveSplitGenerator(InputInitializerContext initializerContext) {
super(initializerContext);
@@ -77,6 +107,8 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) {
public List initialize() throws Exception {
InputInitializerContext rootInputContext = getContext();
+ context = rootInputContext;
+
MRInputUserPayloadProto userPayloadProto =
MRInputHelpers.parseMRInputPayload(rootInputContext.getInputUserPayload());
@@ -90,6 +122,32 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) {
JobConf jobConf = new JobConf(conf);
ShimLoader.getHadoopShims().getMergedCredentials(jobConf);
+ MapWork work = Utilities.getMapWork(jobConf);
+
+ for (String s : work.getEventSourceTableDescMap().keySet()) {
+ List tables = work.getEventSourceTableDescMap().get(s);
+ List columnNames = work.getEventSourceColumnNameMap().get(s);
+
+ Iterator it = columnNames.iterator();
+ for (TableDesc t: tables) {
+ ++sourceInfoCount;
+ String columnName = it.next();
+ SourceInfo si = new SourceInfo(t, columnName, jobConf);
+ if (!sourceInfoMap.containsKey(s)) {
+ sourceInfoMap.put(s, new ArrayList());
+ }
+ List sis = sourceInfoMap.get(s);
+ sis.add(si);
+ }
+ }
+
+ LOG.info("Waiting for events (" + sourceInfoCount + " items) ...");
+ // synchronous event processing loop. Won't return until all events have
+ // been processed.
+ this.processEvents();
+ this.prunePartitions(work);
+ LOG.info("Ok to proceed.");
+
InputSplitInfoMem inputSplitInfo = null;
String realInputFormatName = conf.get("mapred.input.format.class");
boolean groupingEnabled = userPayloadProto.getGroupingEnabled();
@@ -138,6 +196,53 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) {
return createEventList(sendSerializedEvents, inputSplitInfo);
}
+ private void prunePartitions(MapWork work) {
+ for (String source : this.sourceInfoMap.keySet()) {
+ for (SourceInfo si: this.sourceInfoMap.get(source)) {
+
+ if (si.skipPruning) {
+ // in this case we've determined that there's too much data
+ // to prune dynamically.
+ LOG.info("Skip pruning on " + source + ", column " + si.columnName);
+ continue;
+ }
+
+ Set values = si.values;
+ String columnName = si.columnName;
+
+ if (values == null || values.isEmpty()) {
+ // can happen when there are too many values and we decided to skip
+ // pruning.
+ LOG.info("No values received. Skipping dynamic pruning for: " + source);
+ continue;
+ }
+
+ Iterator it = work.getPathToPartitionInfo().keySet().iterator();
+ while (it.hasNext()) {
+ String p = it.next();
+ PartitionDesc desc = work.getPathToPartitionInfo().get(p);
+ Map spec = desc.getPartSpec();
+ if (spec == null) {
+ throw new AssertionException("No partition spec found in dynamic pruning");
+ }
+
+ String partValue = spec.get(columnName);
+ if (partValue == null) {
+ throw new AssertionException("Could not find partition value for column: " + columnName);
+ }
+
+ if (!values.contains(partValue)) {
+ LOG.info("Pruning path: " + p);
+ it.remove();
+ work.getPathToAliases().remove(p);
+ work.getPaths().remove(p);
+ work.getPartitionDescs().remove(desc);
+ }
+ }
+ }
+ }
+ }
+
public static Multimap generateGroupedSplits(JobConf jobConf,
Configuration conf, InputSplit[] splits, float waves, int availableSlots)
throws Exception {
@@ -189,10 +294,6 @@ public HiveSplitGenerator(InputInitializerContext initializerContext) {
return groupedSplits;
}
- @Override
- public void handleInputInitializerEvent(List events) throws Exception {
- }
-
private List createEventList(boolean sendSerializedEvents, InputSplitInfoMem inputSplitInfo) {
List events = Lists.newArrayListWithCapacity(inputSplitInfo.getNumTasks() + 1);
@@ -220,4 +321,157 @@ public void handleInputInitializerEvent(List events) thro
}
return events;
}
+
+ private static class SourceInfo {
+ public final Deserializer deserializer;
+ public final StructObjectInspector soi;
+ public final StructField field;
+ public final ObjectInspector fieldInspector;
+ public final Set values = new HashSet();
+ public boolean skipPruning = false;
+ public final String columnName;
+
+ public SourceInfo(TableDesc table, String columnName, JobConf jobConf) throws SerDeException {
+
+ this.columnName = columnName;
+
+ deserializer = ReflectionUtils.newInstance(table.getDeserializerClass(), null);
+ deserializer.initialize(jobConf, table.getProperties());
+
+ ObjectInspector inspector = deserializer.getObjectInspector();
+ LOG.debug("Type of obj insp: " + inspector.getTypeName());
+
+ soi = (StructObjectInspector) inspector;
+ List extends StructField> fields = soi.getAllStructFieldRefs();
+ if (fields.size() > 1) {
+ LOG.error("expecting single field in input");
+ }
+
+ field = fields.get(0);
+
+ fieldInspector =
+ ObjectInspectorUtils.getStandardObjectInspector(field.getFieldObjectInspector());
+ }
+ }
+
+ @Override
+ public void handleInputInitializerEvent(List events) throws Exception {
+ for (InputInitializerEvent e : events) {
+ queue.put(e);
+ }
+ }
+
+ private void processEvents() throws SerDeException, IOException, InterruptedException {
+ int eventCount = 0;
+ int neededEvents = getExpectedNumberOfEvents();
+
+ while (neededEvents > eventCount) {
+ InputInitializerEvent event = queue.take();
+ LOG.info("Input event: " + event.getTargetInputName() + ", " + event.getTargetVertexName()
+ + ", " + (event.getUserPayload().limit() - event.getUserPayload().position()));
+ processPayload(event.getUserPayload());
+ eventCount += 1;
+ neededEvents = getExpectedNumberOfEvents();
+ LOG.info("Needed events: " + neededEvents + ", received events: " + eventCount);
+ }
+ }
+
+ private int getExpectedNumberOfEvents() throws InterruptedException {
+ int neededEvents = 0;
+
+ boolean notInitialized;
+ do {
+ neededEvents = 0;
+ notInitialized = false;
+ for (String s : sourceInfoMap.keySet()) {
+ int multiplier = sourceInfoMap.get(s).size();
+ int taskNum = context.getVertexNumTasks(s);
+ LOG.info("Vertex " + s + " has " + taskNum + " events.");
+ if (taskNum < 0) {
+ notInitialized = true;
+ Thread.sleep(10);
+ continue;
+ }
+ neededEvents += (taskNum * multiplier);
+ }
+ } while(notInitialized);
+
+ return neededEvents;
+ }
+
+ private String processPayload(ByteBuffer payload) throws SerDeException, IOException {
+ DataInputStream in = new DataInputStream(new ByteBufferBackedInputStream(payload));
+ String sourceName = in.readUTF();
+ String columnName = in.readUTF();
+ boolean skip = in.readBoolean();
+
+ LOG.info("Source of event: " + sourceName);
+
+ List infos = this.sourceInfoMap.get(sourceName);
+ if (infos == null) {
+ in.close();
+ throw new AssertionException("no source info for event source: " + sourceName);
+ }
+
+ SourceInfo info = null;
+ for (SourceInfo si : infos) {
+ if (columnName.equals(si.columnName)) {
+ info = si;
+ }
+ }
+ if (info == null) {
+ in.close();
+ throw new AssertionException("no source info for column: " + columnName);
+ }
+
+ info.skipPruning = skip;
+
+ boolean isExhausted = false;
+ do {
+ try {
+ writable.readFields(in);
+
+ Object row = info.deserializer.deserialize(writable);
+
+ String value =
+ PrimitiveObjectInspectorUtils.getString(info.soi.getStructFieldData(row, info.field),
+ (PrimitiveObjectInspector) info.fieldInspector);
+ info.values.add(value);
+
+ } catch (IOException e) {
+ LOG.debug("deser: ", e);
+ isExhausted = true;
+ }
+ } while (!isExhausted);
+ in.close();
+ return sourceName;
+ }
+
+ private static class ByteBufferBackedInputStream extends InputStream {
+
+ ByteBuffer buf;
+
+ public ByteBufferBackedInputStream(ByteBuffer buf) {
+ this.buf = buf;
+ }
+
+ @Override
+ public int read() throws IOException {
+ if (!buf.hasRemaining()) {
+ return -1;
+ }
+ return buf.get() & 0xFF;
+ }
+
+ @Override
+ public int read(byte[] bytes, int off, int len) throws IOException {
+ if (!buf.hasRemaining()) {
+ return -1;
+ }
+
+ len = Math.min(len, buf.remaining());
+ buf.get(bytes, off, len);
+ return len;
+ }
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
index 61c20ba..f05f491 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
@@ -115,7 +115,8 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep
l4j.info(mapOp.dump(0));
MapredContext.init(true, new JobConf(jconf));
- ((TezContext)MapredContext.get()).setInputs(inputs);
+ ((TezContext) MapredContext.get()).setInputs(inputs);
+ ((TezContext) MapredContext.get()).setTezProcessorContext(processorContext);
mapOp.setExecContext(execContext);
mapOp.initializeLocalWork(jconf);
mapOp.initialize(jconf, null);
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
index e278572..990a4f1 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
@@ -138,7 +138,7 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep
try {
keyTableDesc = redWork.getKeyDesc();
- inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc
+ inputKeyDeserializer = ReflectionUtils.newInstance(keyTableDesc
.getDeserializerClass(), null);
SerDeUtils.initializeSerDe(inputKeyDeserializer, null, keyTableDesc.getProperties(), null);
keyObjectInspector = inputKeyDeserializer.getObjectInspector();
@@ -150,7 +150,7 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep
keyStructInspector = (StructObjectInspector)keyObjectInspector;
batches = new VectorizedRowBatch[maxTags];
valueStructInspectors = new StructObjectInspector[maxTags];
- valueStringWriters = (List[])new List[maxTags];
+ valueStringWriters = new List[maxTags];
keysColumnOffset = keyStructInspector.getAllStructFieldRefs().size();
buffer = new DataOutputBuffer();
}
@@ -213,7 +213,8 @@ void init(JobConf jconf, ProcessorContext processorContext, MRTaskReporter mrRep
}
MapredContext.init(false, new JobConf(jconf));
- ((TezContext)MapredContext.get()).setInputs(inputs);
+ ((TezContext) MapredContext.get()).setInputs(inputs);
+ ((TezContext) MapredContext.get()).setTezProcessorContext(processorContext);
// initialize reduce operator tree
try {
@@ -304,7 +305,7 @@ void run() throws Exception {
Map tag2input = redWork.getTagToInput();
ArrayList shuffleInputs = new ArrayList();
for(String inpStr : tag2input.values()){
- shuffleInputs.add((LogicalInput)inputs.get(inpStr));
+ shuffleInputs.add(inputs.get(inpStr));
}
return shuffleInputs;
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java
index 01d68ab..010a6f4 100644
--- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezContext.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.tez.runtime.api.LogicalInput;
import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.ProcessorContext;
/**
* TezContext contains additional context only available with Tez
@@ -31,9 +32,11 @@
// all the inputs for the tez processor
private Map inputs;
-
+
private Map outputs;
+ private ProcessorContext processorContext;
+
public TezContext(boolean isMap, JobConf jobConf) {
super(isMap, jobConf);
}
@@ -41,7 +44,7 @@ public TezContext(boolean isMap, JobConf jobConf) {
public void setInputs(Map inputs) {
this.inputs = inputs;
}
-
+
public void setOutputs(Map outputs) {
this.outputs = outputs;
}
@@ -52,11 +55,19 @@ public LogicalInput getInput(String name) {
}
return inputs.get(name);
}
-
+
public LogicalOutput getOutput(String name) {
if (outputs == null) {
return null;
}
return outputs.get(name);
}
+
+ public void setTezProcessorContext(ProcessorContext processorContext) {
+ this.processorContext = processorContext;
+ }
+
+ public ProcessorContext getTezProcessorContext() {
+ return processorContext;
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index d42e1f7..18e1f68 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hive.ql.optimizer;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -27,6 +28,7 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
import org.apache.hadoop.hive.ql.exec.GroupByOperator;
import org.apache.hadoop.hive.ql.exec.JoinOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
@@ -39,6 +41,7 @@
import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
import org.apache.hadoop.hive.ql.parse.ParseContext;
import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
@@ -75,7 +78,7 @@ public Object process(Node nd, Stack stack,
}
JoinOperator joinOp = (JoinOperator) nd;
- // if we have traits, and table info is present in the traits, we know the
+ // if we have traits, and table info is present in the traits, we know the
// exact number of buckets. Else choose the largest number of estimated
// reducers from the parent operators.
int numBuckets = -1;
@@ -83,13 +86,13 @@ public Object process(Node nd, Stack stack,
if (context.conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN_BUCKET_MAPJOIN_TEZ)) {
for (Operator extends OperatorDesc>parentOp : joinOp.getParentOperators()) {
if (parentOp.getOpTraits().getNumBuckets() > 0) {
- numBuckets = (numBuckets < parentOp.getOpTraits().getNumBuckets()) ?
- parentOp.getOpTraits().getNumBuckets() : numBuckets;
+ numBuckets = (numBuckets < parentOp.getOpTraits().getNumBuckets()) ?
+ parentOp.getOpTraits().getNumBuckets() : numBuckets;
}
if (parentOp instanceof ReduceSinkOperator) {
ReduceSinkOperator rs = (ReduceSinkOperator)parentOp;
- estimatedBuckets = (estimatedBuckets < rs.getConf().getNumReducers()) ?
+ estimatedBuckets = (estimatedBuckets < rs.getConf().getNumReducers()) ?
rs.getConf().getNumReducers() : estimatedBuckets;
}
}
@@ -106,7 +109,7 @@ public Object process(Node nd, Stack stack,
LOG.info("Estimated number of buckets " + numBuckets);
int mapJoinConversionPos = mapJoinConversionPos(joinOp, context, numBuckets);
if (mapJoinConversionPos < 0) {
- // we cannot convert to bucket map join, we cannot convert to
+ // we cannot convert to bucket map join, we cannot convert to
// map join either based on the size
return null;
}
@@ -148,7 +151,7 @@ private void setAllChildrenTraitsToNull(Operator extends OperatorDesc> current
}
}
- private boolean convertJoinBucketMapJoin(JoinOperator joinOp, OptimizeTezProcContext context,
+ private boolean convertJoinBucketMapJoin(JoinOperator joinOp, OptimizeTezProcContext context,
int bigTablePosition) throws SemanticException {
TezBucketJoinProcCtx tezBucketJoinProcCtx = new TezBucketJoinProcCtx(context.conf);
@@ -158,7 +161,7 @@ private boolean convertJoinBucketMapJoin(JoinOperator joinOp, OptimizeTezProcCon
return false;
}
- MapJoinOperator mapJoinOp =
+ MapJoinOperator mapJoinOp =
convertJoinMapJoin(joinOp, context, bigTablePosition);
MapJoinDesc joinDesc = mapJoinOp.getConf();
joinDesc.setBucketMapJoin(true);
@@ -169,7 +172,7 @@ private boolean convertJoinBucketMapJoin(JoinOperator joinOp, OptimizeTezProcCon
mapJoinOp.setOpTraits(opTraits);
setNumberOfBucketsOnChildren(mapJoinOp);
- // Once the conversion is done, we can set the partitioner to bucket cols on the small table
+ // Once the conversion is done, we can set the partitioner to bucket cols on the small table
Map bigTableBucketNumMapping = new HashMap();
bigTableBucketNumMapping.put(joinDesc.getBigTableAlias(), tezBucketJoinProcCtx.getNumBuckets());
joinDesc.setBigTableBucketNumMapping(bigTableBucketNumMapping);
@@ -191,13 +194,13 @@ private void setNumberOfBucketsOnChildren(Operator extends OperatorDesc> curre
/*
* We perform the following checks to see if we can convert to a bucket map join
- * 1. If the parent reduce sink of the big table side has the same emit key cols as
+ * 1. If the parent reduce sink of the big table side has the same emit key cols as
* its parent, we can create a bucket map join eliminating the reduce sink.
- * 2. If we have the table information, we can check the same way as in Mapreduce to
+ * 2. If we have the table information, we can check the same way as in Mapreduce to
* determine if we can perform a Bucket Map Join.
*/
- private boolean checkConvertJoinBucketMapJoin(JoinOperator joinOp,
- OptimizeTezProcContext context, int bigTablePosition,
+ private boolean checkConvertJoinBucketMapJoin(JoinOperator joinOp,
+ OptimizeTezProcContext context, int bigTablePosition,
TezBucketJoinProcCtx tezBucketJoinProcCtx) throws SemanticException {
// bail on mux-operator because mux operator masks the emit keys of the
// constituent reduce sinks
@@ -210,7 +213,7 @@ private boolean checkConvertJoinBucketMapJoin(JoinOperator joinOp,
ReduceSinkOperator rs = (ReduceSinkOperator) joinOp.getParentOperators().get(bigTablePosition);
/*
* this is the case when the big table is a sub-query and is probably
- * already bucketed by the join column in say a group by operation
+ * already bucketed by the join column in say a group by operation
*/
List> colNames = rs.getParentOperators().get(0).getOpTraits().getBucketColNames();
if ((colNames != null) && (colNames.isEmpty() == false)) {
@@ -233,7 +236,7 @@ private boolean checkConvertJoinBucketMapJoin(JoinOperator joinOp,
break;
}
}
-
+
if (colCount == rs.getOpTraits().getBucketColNames().get(0).size()) {
// all keys matched.
int numBuckets = parentOfParent.getOpTraits().getNumBuckets();
@@ -255,7 +258,7 @@ private boolean checkConvertJoinBucketMapJoin(JoinOperator joinOp,
return false;
}
- public int mapJoinConversionPos(JoinOperator joinOp, OptimizeTezProcContext context,
+ public int mapJoinConversionPos(JoinOperator joinOp, OptimizeTezProcContext context,
int buckets) {
Set bigTableCandidateSet = MapJoinProcessor.
getBigTableCandidates(joinOp.getConf().getConds());
@@ -344,9 +347,9 @@ public int mapJoinConversionPos(JoinOperator joinOp, OptimizeTezProcContext cont
* for tez.
*/
- public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcContext context,
+ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcContext context,
int bigTablePosition) throws SemanticException {
- // bail on mux operator because currently the mux operator masks the emit keys
+ // bail on mux operator because currently the mux operator masks the emit keys
// of the constituent reduce sinks.
for (Operator extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
if (parentOp instanceof MuxOperator) {
@@ -363,6 +366,19 @@ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcCo
Operator extends OperatorDesc> parentBigTableOp
= mapJoinOp.getParentOperators().get(bigTablePosition);
if (parentBigTableOp instanceof ReduceSinkOperator) {
+ for (Operator> p : parentBigTableOp.getParentOperators()) {
+ // we might have generated a dynamic partition operator chain. Since
+ // we're removing the reduce sink we nee do remove that too.
+ Set> dynamicPartitionOperators = new HashSet>();
+ for (Operator> c : p.getChildOperators()) {
+ if (hasDynamicPartitionBroadcast(c)) {
+ dynamicPartitionOperators.add(c);
+ }
+ }
+ for (Operator> c : dynamicPartitionOperators) {
+ p.removeChild(c);
+ }
+ }
mapJoinOp.getParentOperators().remove(bigTablePosition);
if (!(mapJoinOp.getParentOperators().contains(
parentBigTableOp.getParentOperators().get(0)))) {
@@ -380,4 +396,16 @@ public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcCo
return mapJoinOp;
}
+
+ private boolean hasDynamicPartitionBroadcast(Operator> op) {
+ if (op instanceof AppMasterEventOperator && op.getConf() instanceof DynamicPruningEventDesc) {
+ return true;
+ }
+ for (Operator> c : op.getChildOperators()) {
+ if (hasDynamicPartitionBroadcast(c)) {
+ return true;
+ }
+ }
+ return false;
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruner.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruner.java
new file mode 100644
index 0000000..43494fd
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruner.java
@@ -0,0 +1,316 @@
+/**
+ * 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.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
+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.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+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.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc;
+import org.apache.hadoop.hive.ql.plan.FilterDesc;
+import org.apache.hadoop.hive.ql.plan.GroupByDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+import org.apache.hadoop.hive.ql.plan.SelectDesc;
+
+public class DynamicPartitionPruner implements NodeProcessor {
+
+ public static class DynamicPartitionPrunerProc implements NodeProcessor {
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+ ExprNodeDynamicListDesc desc = (ExprNodeDynamicListDesc) nd;
+ DynamicPartitionPrunerContext context = (DynamicPartitionPrunerContext) procCtx;
+
+ ExprNodeDesc parent = (ExprNodeDesc) stack.get(stack.size() - 2);
+ ExprNodeDesc grandParent = stack.size() >= 3 ? (ExprNodeDesc) stack.get(stack.size() - 3) : null;
+
+ ExprNodeDesc greatGrandParent =
+ stack.size() >= 4 ? (ExprNodeDesc) stack.get(stack.size() - 4) : null;
+
+ context.addSyntheticNode(desc, parent, grandParent, greatGrandParent,
+ (ReduceSinkOperator) desc.getSource());
+
+ return context;
+ }
+ }
+
+ private static class DynamicListContext {
+ public ExprNodeDynamicListDesc desc;
+ public ExprNodeDesc parent;
+ public ExprNodeDesc grandParent;
+ public ExprNodeDesc greatGrandParent;
+ public ReduceSinkOperator generator;
+
+ public DynamicListContext(ExprNodeDynamicListDesc desc, ExprNodeDesc parent,
+ ExprNodeDesc grandParent, ExprNodeDesc greatGrandParent, ReduceSinkOperator generator) {
+ LOG.debug("Desc: " + desc);
+ this.desc = desc;
+
+ LOG.debug("Parent: " + parent);
+ this.parent = parent;
+
+ LOG.debug("Grandparent: " + grandParent);
+ this.grandParent = grandParent;
+
+ LOG.debug("Greatgrandparent: " + greatGrandParent);
+
+ LOG.debug("Generator: " + generator);
+ this.generator = generator;
+ }
+ }
+
+ private static class DynamicPartitionPrunerContext implements NodeProcessorCtx,
+ Iterable {
+ public boolean isChildOfTableScan = false;
+ public List dynLists = new ArrayList();
+
+ public void addSyntheticNode(ExprNodeDynamicListDesc desc, ExprNodeDesc parent,
+ ExprNodeDesc grandParent, ExprNodeDesc greatGrandParent, ReduceSinkOperator generator) {
+ dynLists.add(new DynamicListContext(desc, parent, grandParent, greatGrandParent, generator));
+ }
+
+ @Override
+ public Iterator iterator() {
+ return dynLists.iterator();
+ }
+ }
+
+ static final private Log LOG = LogFactory.getLog(DynamicPartitionPruner.class.getName());
+
+ private String extractColName(ExprNodeDesc root) {
+ if (root instanceof ExprNodeColumnDesc) {
+ return ((ExprNodeColumnDesc) root).getColumn();
+ } else {
+ if (root.getChildren() == null) {
+ return null;
+ }
+
+ String column = null;
+ for (ExprNodeDesc d: root.getChildren()) {
+ String candidate = extractColName(d);
+ if (column != null && candidate != null) {
+ return null;
+ } else if (candidate != null) {
+ column = candidate;
+ }
+ }
+ return column;
+ }
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
+ throws SemanticException {
+ OptimizeTezProcContext context = (OptimizeTezProcContext) procCtx;
+ ParseContext parseContext = context.parseContext;
+
+ FilterOperator filter = (FilterOperator) nd;
+ FilterDesc desc = filter.getConf();
+ boolean filterRemoved = false;
+
+ if (!parseContext.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) {
+ // nothing to do when the optimization is off
+ return null;
+ }
+
+ DynamicPartitionPrunerContext removerContext = new DynamicPartitionPrunerContext();
+ removerContext.isChildOfTableScan =
+ filter.getParentOperators().size() == 1
+ && (filter.getParentOperators().get(0) instanceof TableScanOperator);
+
+ LOG.debug("Parent: " + filter.getParentOperators().get(0));
+ LOG.debug("Filter: " + desc.getPredicateString());
+ LOG.debug("isChildOfTableScan: " + removerContext.isChildOfTableScan);
+
+ do {
+ // collect the synthetic pruning conditions
+ removerContext.dynLists.clear();
+ walkExprTree(desc.getPredicate(), removerContext);
+
+ for (DynamicListContext ctx : removerContext) {
+
+ String column = extractColName(ctx.parent);
+
+ if (removerContext.isChildOfTableScan && column != null) {
+
+ TableScanOperator ts = (TableScanOperator) filter.getParentOperators().get(0);
+ Table table = parseContext.getTopToTable().get(ts);
+
+ if (table != null && table.isPartitionKey(column)) {
+ LOG.info("Dynamic partitioning: " + table.getCompleteName() + "." + column);
+ generateEventOperatorPlan(ctx, parseContext, ts, column);
+ } else {
+ LOG.debug("Column " + column + " is not a partition column");
+ }
+ }
+
+ // // no grand parent means the synthetic condition is the only thing in
+ // // the
+ // // filter. let's remove it.
+ // if (ctx.grandParent == null) {
+ // filter.getParentOperators().get(0).removeChildAndAdoptItsChildren(filter);
+ // filterRemoved = true;
+ // break;
+ // }
+ //
+ // // if the grandParent is an and expr we can just remove the synthetic
+ // // expr
+ // if (ctx.grandParent instanceof ExprNodeGenericFuncDesc
+ // && ((ExprNodeGenericFuncDesc) ctx.grandParent).getGenericUDF()
+ // instanceof GenericUDFOPAnd) {
+ // int i = ctx.grandParent.getChildren().indexOf(ctx.parent);
+ // ctx.grandParent.getChildren().remove(i);
+ //
+ // if (ctx.grandParent.getChildren().size() == 1) {
+ // if (ctx.greatGrandParent == null) {
+ // filter.getConf().setPredicate(ctx.parent);
+ // } else {
+ // int j = ctx.greatGrandParent.getChildren().indexOf(ctx.grandParent);
+ // ctx.greatGrandParent.getChildren().remove(j);
+ // ctx.greatGrandParent.getChildren().add(j, ctx.parent);
+ // }
+ // break;
+ // }
+ // }
+
+ // otherwise we just replace it with "true"
+ ExprNodeDesc constNode = new ExprNodeConstantDesc(ctx.parent.getTypeInfo(), true);
+ if (ctx.grandParent == null) {
+ desc.setPredicate(constNode);
+ } else {
+ int i = ctx.grandParent.getChildren().indexOf(ctx.parent);
+ ctx.grandParent.getChildren().remove(i);
+ ctx.grandParent.getChildren().add(i, constNode);
+ }
+ }
+ } while (!filterRemoved && removerContext.dynLists.size() > 0);
+
+ return false;
+ }
+
+ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext parseContext,
+ TableScanOperator ts, String column) {
+
+ // we will put a fork in the plan at the source of the reduce sink
+ Operator extends OperatorDesc> parentOfRS = ctx.generator.getParentOperators().get(0);
+
+ // we need the expr that generated the key of the reduce sink
+ ExprNodeDesc key = ctx.generator.getConf().getKeyCols().get(ctx.desc.getKeyIndex());
+
+ List keyExprs = new ArrayList();
+ keyExprs.add(key);
+
+ // group by requires "ArrayList", don't ask.
+ ArrayList outputNames = new ArrayList();
+ outputNames.add(HiveConf.getColumnInternalName(0));
+
+ // project the relevant key column
+ SelectDesc select = new SelectDesc(keyExprs, outputNames);
+ SelectOperator selectOp =
+ (SelectOperator) OperatorFactory.getAndMakeChild(select, parentOfRS);
+
+ // do a group by on the list to dedup
+ float groupByMemoryUsage =
+ HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY);
+ float memoryThreshold =
+ HiveConf.getFloatVar(parseContext.getConf(),
+ HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD);
+
+ ArrayList groupByExprs = new ArrayList();
+ ExprNodeDesc groupByExpr =
+ new ExprNodeColumnDesc(key.getTypeInfo(), outputNames.get(0), null, false);
+ groupByExprs.add(groupByExpr);
+
+ GroupByDesc groupBy =
+ new GroupByDesc(GroupByDesc.Mode.HASH, outputNames, groupByExprs,
+ new ArrayList(), false, groupByMemoryUsage, memoryThreshold,
+ null, false, 0, true);
+
+ GroupByOperator groupByOp =
+ (GroupByOperator) OperatorFactory.getAndMakeChild(groupBy, selectOp);
+
+ // finally add the even broadcast operator
+ DynamicPruningEventDesc eventDesc = new DynamicPruningEventDesc();
+ eventDesc.setTableScan(ts);
+ eventDesc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils
+ .getFieldSchemasFromColumnList(keyExprs, "key")));
+ eventDesc.setTargetColumnName(column);
+
+ OperatorFactory.getAndMakeChild(eventDesc, groupByOp);
+ }
+
+ private Map walkExprTree(ExprNodeDesc pred, NodeProcessorCtx ctx)
+ 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 exprRules = new LinkedHashMap();
+ exprRules.put(new RuleRegExp("R1", ExprNodeDynamicListDesc.class.getName() + "%"),
+ new DynamicPartitionPrunerProc());
+
+ // The dispatcher fires the processor corresponding to the closest matching
+ // rule and passes the context along
+ Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx);
+ GraphWalker egw = new DefaultGraphWalker(disp);
+
+ List startNodes = new ArrayList();
+ startNodes.add(pred);
+
+ HashMap outputMap = new HashMap();
+ egw.startWalking(startNodes, outputMap);
+ return outputMap;
+ }
+
+}
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 d7e1fbf..20ef64d 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -36,6 +36,7 @@
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.ppd.PredicatePushDown;
import org.apache.hadoop.hive.ql.ppd.PredicateTransitivePropagate;
+import org.apache.hadoop.hive.ql.ppd.SyntheticJoinPredicate;
/**
* Implementation of the optimizer.
@@ -55,6 +56,7 @@ public void initialize(HiveConf hiveConf) {
transformations.add(new Generator());
if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) {
transformations.add(new PredicateTransitivePropagate());
+ transformations.add(new SyntheticJoinPredicate());
transformations.add(new PredicatePushDown());
transformations.add(new PartitionPruner());
transformations.add(new PartitionConditionRemover());
@@ -125,8 +127,8 @@ public void initialize(HiveConf hiveConf) {
if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES)) {
transformations.add(new StatsOptimizer());
}
- if (pctx.getContext().getExplain() ||
- HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
+ if (pctx.getContext().getExplain()
+ || !HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
transformations.add(new AnnotateWithStatistics());
transformations.add(new AnnotateWithOpTraits());
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
index b639a2a..e633fdc 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerExpressionOperatorFactory.java
@@ -186,8 +186,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
return ((ExprNodeNullDesc) nd).clone();
}
- assert (false);
- return null;
+ return new ExprNodeConstantDesc(((ExprNodeDesc)nd).getTypeInfo(), null);
}
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index 9eb81aa..fdce5c2 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -26,8 +26,6 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
@@ -42,7 +40,6 @@
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.plan.BaseWork;
import org.apache.hadoop.hive.ql.plan.ColStatistics;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc;
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
@@ -63,7 +60,7 @@
/* (non-Javadoc)
* This processor addresses the RS-MJ case that occurs in tez on the small/hash
- * table side of things. The work that RS will be a part of must be connected
+ * table side of things. The work that RS will be a part of must be connected
* to the MJ work via be a broadcast edge.
* We should not walk down the tree when we encounter this pattern because:
* the type of work (map work or reduce work) needs to be determined
@@ -134,7 +131,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext,
String prefix = Utilities.ReduceField.KEY.toString();
for (String keyCol : keyCols) {
ExprNodeDesc realCol = parentRS.getColumnExprMap().get(prefix + "." + keyCol);
- ColStatistics cs = StatsUtils.getColStatisticsFromExpression(null, stats, realCol);
+ ColStatistics cs =
+ StatsUtils.getColStatisticsFromExpression(context.conf, stats, realCol);
if (cs == null || cs.getCountDistint() <= 0) {
maxKeyCount = Long.MAX_VALUE;
break;
@@ -185,7 +183,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext,
TezWork tezWork = context.currentTask.getWork();
LOG.debug("connecting "+parentWork.getName()+" with "+myWork.getName());
tezWork.connect(parentWork, myWork, edgeProp);
-
+
ReduceSinkOperator r = null;
if (parentRS.getConf().getOutputName() != null) {
LOG.debug("Cloning reduce sink for multi-child broadcast edge");
@@ -211,8 +209,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext,
}
linkWorkMap.put(parentWork, edgeProp);
context.linkOpWithWorkMap.put(mapJoinOp, linkWorkMap);
-
- List reduceSinks
+
+ List reduceSinks
= context.linkWorkWithReduceSinkMap.get(parentWork);
if (reduceSinks == null) {
reduceSinks = new ArrayList();
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
new file mode 100644
index 0000000..02957a6
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/RemoveDynamicPruningBySize.java
@@ -0,0 +1,72 @@
+/**
+ * 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.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+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.parse.OptimizeTezProcContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
+
+/**
+ * SetReducerParallelism determines how many reducers should be run for a given
+ * reduce sink.
+ */
+public class RemoveDynamicPruningBySize implements NodeProcessor {
+
+ static final private Log LOG = LogFactory.getLog(RemoveDynamicPruningBySize.class.getName());
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procContext,
+ Object... nodeOutputs)
+ throws SemanticException {
+
+ OptimizeTezProcContext context = (OptimizeTezProcContext) procContext;
+
+ AppMasterEventOperator event = (AppMasterEventOperator) nd;
+ AppMasterEventDesc desc = event.getConf();
+
+ if (desc.getStatistics().getDataSize() > context.conf
+ .getLongVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE)) {
+ Operator> child = event;
+ Operator> curr = event;
+
+ while (curr.getChildOperators().size() <= 1) {
+ child = curr;
+ curr = curr.getParentOperators().get(0);
+ }
+ // at this point we've found the fork in the op pipeline that has the
+ // pruning as a child plan.
+ LOG.info("Disabling dynamic pruning for: "
+ + ((DynamicPruningEventDesc) desc).getTableScan().getName()
+ + ". Expected data size is too big: " + desc.getStatistics().getDataSize());
+ curr.removeChild(child);
+ }
+ return false;
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
index 625af85..bf0ad4f 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SetReducerParallelism.java
@@ -42,7 +42,6 @@
static final private Log LOG = LogFactory.getLog(SetReducerParallelism.class.getName());
- @SuppressWarnings("unchecked")
@Override
public Object process(Node nd, Stack stack,
NodeProcessorCtx procContext, Object... nodeOutputs)
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
index e46e144..cbd4e6c 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/pcr/PcrExprProcFactory.java
@@ -396,8 +396,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
return new NodeInfoWrapper(WalkState.CONSTANT, null,
(ExprNodeDesc) nd);
}
- assert (false);
- return null;
+ return new NodeInfoWrapper(WalkState.UNKNOWN, null, (ExprNodeDesc)nd);
}
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index 8513f99..39f6a73 100644
--- ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -26,29 +26,27 @@
import java.util.Map;
import java.util.Set;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.Task;
import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.exec.tez.TezTask;
import org.apache.hadoop.hive.ql.hooks.ReadEntity;
import org.apache.hadoop.hive.ql.hooks.WriteEntity;
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.plan.BaseWork;
import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.MoveWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
-import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
-import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
import org.apache.hadoop.hive.ql.plan.TezWork;
/**
@@ -133,6 +131,7 @@
// remember which reducesinks we've already connected
public final Set connectedReduceSinks;
+ public final Set eventOperatorSet;
@SuppressWarnings("unchecked")
public GenTezProcContext(HiveConf conf, ParseContext parseContext,
@@ -165,6 +164,7 @@ public GenTezProcContext(HiveConf conf, ParseContext parseContext,
this.linkedFileSinks = new LinkedHashMap>();
this.fileSinkSet = new LinkedHashSet();
this.connectedReduceSinks = new LinkedHashSet();
+ this.eventOperatorSet = new LinkedHashSet();
rootTasks.add(currentTask);
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
index deccb16..6aa1091 100644
--- ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
+++ ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
@@ -23,30 +23,33 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
-import java.util.List;
import java.util.LinkedList;
+import java.util.List;
import java.util.Map;
import java.util.Set;
-import org.apache.hadoop.fs.Path;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
import org.apache.hadoop.hive.ql.exec.FetchTask;
-import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.MapWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.ReduceWork;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
import org.apache.hadoop.hive.ql.plan.TezEdgeProperty;
import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
import org.apache.hadoop.hive.ql.plan.TezWork;
@@ -119,12 +122,12 @@ public ReduceWork createReduceWork(GenTezProcContext context, Operator> root,
int maxReducers = context.conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS);
// min we allow tez to pick
- int minPartition = Math.max(1, (int) (reduceSink.getConf().getNumReducers()
+ int minPartition = Math.max(1, (int) (reduceSink.getConf().getNumReducers()
* minPartitionFactor));
minPartition = (minPartition > maxReducers) ? maxReducers : minPartition;
// max we allow tez to pick
- int maxPartition = (int) (reduceSink.getConf().getNumReducers() * maxPartitionFactor);
+ int maxPartition = (int) (reduceSink.getConf().getNumReducers() * maxPartitionFactor);
maxPartition = (maxPartition > maxReducers) ? maxReducers : maxPartition;
reduceWork.setMinReduceTasks(minPartition);
@@ -328,4 +331,69 @@ public void processFileSink(GenTezProcContext context, FileSinkOperator fileSink
}
}
}
+
+ /**
+ * processAppMasterEvent sets up the event descriptor and the MapWork
+ * correctly.
+ *
+ * @param procCtx
+ * @param event
+ */
+ public void processAppMasterEvent(GenTezProcContext procCtx, AppMasterEventOperator event) {
+ DynamicPruningEventDesc eventDesc = (DynamicPruningEventDesc)event.getConf();
+ TableScanOperator ts = eventDesc.getTableScan();
+ MapWork work = (MapWork) procCtx.rootToWorkMap.get(ts);
+ if (work == null) {
+ throw new AssertionError("No work found for tablescan" + ts.toString());
+ }
+
+ String sourceName = getEnclosingWork(event, procCtx).getName();
+
+ // store the vertex name in the operator pipeline
+ eventDesc.setVertexName(work.getName());
+ eventDesc.setInputName(work.getAliases().get(0));
+
+ // store table descriptor in map-work
+ if (!work.getEventSourceTableDescMap().containsKey(sourceName)) {
+ work.getEventSourceTableDescMap().put(sourceName, new LinkedList());
+ }
+ List tables = work.getEventSourceTableDescMap().get(sourceName);
+ tables.add(event.getConf().getTable());
+
+ // store column name in map-work
+ if (!work.getEventSourceColumnNameMap().containsKey(sourceName)) {
+ work.getEventSourceColumnNameMap().put(sourceName, new LinkedList());
+ }
+ List columns = work.getEventSourceColumnNameMap().get(sourceName);
+ columns.add(eventDesc.getTargetColumnName());
+ }
+
+ /**
+ * getEncosingWork finds the BaseWork any given operator belongs to.
+ */
+ public BaseWork getEnclosingWork(Operator> op, GenTezProcContext procCtx) {
+ List> ops = new ArrayList>();
+ findRoots(op, ops);
+ for (Operator> r : ops) {
+ BaseWork work = procCtx.rootToWorkMap.get(r);
+ if (work != null) {
+ return work;
+ }
+ }
+ return null;
+ }
+
+ /*
+ * findRoots returns all root operators (in ops) that result in operator op
+ */
+ private void findRoots(Operator> op, List> ops) {
+ List> parents = op.getParentOperators();
+ if (parents == null || parents.isEmpty()) {
+ ops.add(op);
+ return;
+ }
+ for (Operator> p : parents) {
+ findRoots(p, ops);
+ }
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
index 0807d65..ee71971 100644
--- ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
+++ ql/src/java/org/apache/hadoop/hive/ql/parse/OptimizeTezProcContext.java
@@ -23,13 +23,18 @@
import java.util.Set;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
import org.apache.hadoop.hive.ql.hooks.ReadEntity;
import org.apache.hadoop.hive.ql.hooks.WriteEntity;
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
/**
* OptimizeTezProcContext. OptimizeTezProcContext maintains information
* about the current operator plan as we walk the operator tree
@@ -47,19 +52,23 @@
public final Set visitedReduceSinks
= new HashSet();
+ public final Multimap eventOpToTableScanMap =
+ HashMultimap.create();
+
// rootOperators are all the table scan operators in sequence
// of traversal
- public final Deque> rootOperators;
+ public Deque> rootOperators;
- @SuppressWarnings("unchecked")
- public OptimizeTezProcContext(HiveConf conf, ParseContext parseContext,
- Set inputs, Set outputs,
- Deque> rootOperators) {
+ public OptimizeTezProcContext(HiveConf conf, ParseContext parseContext, Set inputs,
+ Set outputs) {
this.conf = conf;
this.parseContext = parseContext;
this.inputs = inputs;
this.outputs = outputs;
- this.rootOperators = rootOperators;
+ }
+
+ public void setRootOperators(Deque> roots) {
+ this.rootOperators = roots;
}
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index d58c59d..5a42139 100644
--- ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -21,20 +21,24 @@
import java.util.ArrayList;
import java.util.Deque;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Stack;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
import org.apache.hadoop.hive.ql.exec.ConditionalTask;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
import org.apache.hadoop.hive.ql.exec.JoinOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
@@ -56,16 +60,21 @@
import org.apache.hadoop.hive.ql.lib.Rule;
import org.apache.hadoop.hive.ql.lib.RuleRegExp;
import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate;
import org.apache.hadoop.hive.ql.optimizer.ConvertJoinMapJoin;
+import org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruner;
import org.apache.hadoop.hive.ql.optimizer.ReduceSinkMapJoinProc;
+import org.apache.hadoop.hive.ql.optimizer.RemoveDynamicPruningBySize;
import org.apache.hadoop.hive.ql.optimizer.SetReducerParallelism;
import org.apache.hadoop.hive.ql.optimizer.physical.CrossProductCheck;
import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer;
import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer;
import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
-import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
+import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
+import org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics;
import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
import org.apache.hadoop.hive.ql.plan.MapWork;
import org.apache.hadoop.hive.ql.plan.MoveWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -85,7 +94,7 @@ public TezCompiler() {
@Override
public void init(HiveConf conf, LogHelper console, Hive db) {
super.init(conf, console, db);
-
+
// Tez requires us to use RPC for the query plan
HiveConf.setBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN, true);
@@ -98,33 +107,193 @@ public void init(HiveConf conf, LogHelper console, Hive db) {
protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs,
Set outputs) throws SemanticException {
- // Sequence of TableScan operators to be walked
+ // Create the context for the walker
+ OptimizeTezProcContext procCtx = new OptimizeTezProcContext(conf, pCtx, inputs, outputs);
+
+ // setup dynamic partition pruning where possible
+ runDynamicPartitionPruning(procCtx, inputs, outputs);
+
+ // setup stats in the operator plan
+ runStatsAnnotation(procCtx);
+
+ // run the optimizations that use stats for optimization
+ runStatsDependentOptimizations(procCtx, inputs, outputs);
+
+ // after the stats phase we might have some cyclic dependencies that we need
+ // to take care of.
+ runCycleAnalysisForPartitionPruning(procCtx, inputs, outputs);
+
+ }
+
+ private void runCycleAnalysisForPartitionPruning(OptimizeTezProcContext procCtx,
+ Set inputs, Set outputs) throws SemanticException {
+ boolean cycleFree = false;
+ while (!cycleFree) {
+ cycleFree = true;
+ Set>> components = getComponents(procCtx);
+ for (Set> component : components) {
+ LOG.debug("Component: ");
+ for (Operator> co : component) {
+ LOG.debug("Operator: " + co.getName() + ", " + co.getIdentifier());
+ }
+ if (component.size() != 1) {
+ LOG.info("Found cycle in operator plan...");
+ cycleFree = false;
+ removeEventOperator(component);
+ }
+ }
+ LOG.info("Cycle free: " + cycleFree);
+ }
+ }
+
+ private void removeEventOperator(Set> component) {
+ AppMasterEventOperator victim = null;
+ for (Operator> o : component) {
+ if (o instanceof AppMasterEventOperator) {
+ if (victim == null
+ || o.getConf().getStatistics().getDataSize() < victim.getConf().getStatistics()
+ .getDataSize()) {
+ victim = (AppMasterEventOperator) o;
+ }
+ }
+ }
+
+ Operator> child = victim;
+ Operator> curr = victim;
+
+ while (curr.getChildOperators().size() <= 1) {
+ child = curr;
+ curr = curr.getParentOperators().get(0);
+ }
+
+ // at this point we've found the fork in the op pipeline that has the
+ // pruning as a child plan.
+ LOG.info("Disabling dynamic pruning for: "
+ + ((DynamicPruningEventDesc) victim.getConf()).getTableScan().toString()
+ + ". Needed to break cyclic dependency");
+ curr.removeChild(child);
+ }
+
+ // Tarjan's algo
+ private Set>> getComponents(OptimizeTezProcContext procCtx) {
Deque> deque = new LinkedList>();
- deque.addAll(pCtx.getTopOps().values());
+ deque.addAll(procCtx.parseContext.getTopOps().values());
- // Create the context for the walker
- OptimizeTezProcContext procCtx
- = new OptimizeTezProcContext(conf, pCtx, inputs, outputs, deque);
+ AtomicInteger index = new AtomicInteger();
+ Map, Integer> indexes = new HashMap, Integer>();
+ Map, Integer> lowLinks = new HashMap, Integer>();
+ Stack> nodes = new Stack>();
+ Set>> components = new HashSet>>();
+
+ for (Operator> o : deque) {
+ if (!indexes.containsKey(o)) {
+ connect(o, index, nodes, indexes, lowLinks, components);
+ }
+ }
+
+ return components;
+ }
+
+ private void connect(Operator> o, AtomicInteger index, Stack> nodes,
+ Map, Integer> indexes, Map, Integer> lowLinks,
+ Set>> components) {
+
+ indexes.put(o, index.get());
+ lowLinks.put(o, index.get());
+ index.incrementAndGet();
+ nodes.push(o);
+
+ List> children;
+ if (o instanceof AppMasterEventOperator) {
+ children = new ArrayList>();
+ children.addAll(o.getChildOperators());
+ TableScanOperator ts = ((DynamicPruningEventDesc) o.getConf()).getTableScan();
+ LOG.debug("Adding special edge: " + o.getName() + " --> " + ts.toString());
+ children.add(ts);
+ } else {
+ children = o.getChildOperators();
+ }
+
+ for (Operator> child : children) {
+ if (!indexes.containsKey(child)) {
+ connect(child, index, nodes, indexes, lowLinks, components);
+ lowLinks.put(child, Math.min(lowLinks.get(o), lowLinks.get(child)));
+ } else if (nodes.contains(child)) {
+ lowLinks.put(o, Math.min(lowLinks.get(o), indexes.get(child)));
+ }
+ }
+
+ if (lowLinks.get(o).equals(indexes.get(o))) {
+ Set> component = new HashSet>();
+ components.add(component);
+ Operator> current;
+ do {
+ current = nodes.pop();
+ component.add(current);
+ } while (current != o);
+ }
+ }
+
+ private void runStatsAnnotation(OptimizeTezProcContext procCtx) throws SemanticException {
+ new AnnotateWithStatistics().transform(procCtx.parseContext);
+ }
+
+ private void runStatsDependentOptimizations(OptimizeTezProcContext procCtx,
+ Set inputs, Set outputs) throws SemanticException {
+
+ // Sequence of TableScan operators to be walked
+ Deque> deque = new LinkedList>();
+ deque.addAll(procCtx.parseContext.getTopOps().values());
// create a walker which walks the tree in a DFS manner while maintaining
// the operator stack.
Map opRules = new LinkedHashMap();
- opRules.put(new RuleRegExp(new String("Set parallelism - ReduceSink"),
+ opRules.put(new RuleRegExp("Set parallelism - ReduceSink",
ReduceSinkOperator.getOperatorName() + "%"),
new SetReducerParallelism());
- opRules.put(new RuleRegExp(new String("Convert Join to Map-join"),
+ opRules.put(new RuleRegExp("Convert Join to Map-join",
JoinOperator.getOperatorName() + "%"), new ConvertJoinMapJoin());
+ opRules.put(
+ new RuleRegExp("Remove dynamic pruning by size",
+ AppMasterEventOperator.getOperatorName() + "%"),
+ new RemoveDynamicPruningBySize());
+
// The dispatcher fires the processor corresponding to the closest matching
// rule and passes the context along
Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
List topNodes = new ArrayList();
- topNodes.addAll(pCtx.getTopOps().values());
+ topNodes.addAll(procCtx.parseContext.getTopOps().values());
GraphWalker ogw = new ForwardWalker(disp);
ogw.startWalking(topNodes, null);
}
+ private void runDynamicPartitionPruning(OptimizeTezProcContext procCtx, Set inputs,
+ Set outputs) throws SemanticException {
+
+ // Sequence of TableScan operators to be walked
+ Deque> deque = new LinkedList>();
+ deque.addAll(procCtx.parseContext.getTopOps().values());
+
+ Map opRules = new LinkedHashMap();
+ opRules.put(
+ new RuleRegExp(new String("Dynamic Partition Pruning"), FilterOperator.getOperatorName()
+ + "%"), new DynamicPartitionPruner());
+
+ // The dispatcher fires the processor corresponding to the closest matching
+ // rule and passes the context along
+ Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
+ List topNodes = new ArrayList();
+ topNodes.addAll(procCtx.parseContext.getTopOps().values());
+ GraphWalker ogw = new ForwardWalker(disp);
+ ogw.startWalking(topNodes, null);
+
+ // need a new run of the constant folding because we might have created lots
+ // of "and true and true" conditions.
+ new ConstantPropagate().transform(procCtx.parseContext);
+ }
+
@Override
protected void generateTaskTree(List> rootTasks, ParseContext pCtx,
List> mvTask, Set inputs, Set outputs)
@@ -157,17 +326,30 @@ protected void generateTaskTree(List> rootTasks, Pa
TableScanOperator.getOperatorName() + "%"),
new ProcessAnalyzeTable(GenTezUtils.getUtils()));
- opRules.put(new RuleRegExp("Remember union",
- UnionOperator.getOperatorName() + "%"), new NodeProcessor()
- {
+ opRules.put(new RuleRegExp("Remember union", UnionOperator.getOperatorName() + "%"),
+ new NodeProcessor() {
+ @Override
+ public Object process(Node n, Stack s, NodeProcessorCtx procCtx, Object... os)
+ throws SemanticException {
+ GenTezProcContext context = (GenTezProcContext) procCtx;
+ UnionOperator union = (UnionOperator) n;
+
+ // simply need to remember that we've seen a union.
+ context.currentUnionOperators.add(union);
+ return null;
+ }
+ });
+
+ opRules.put(new RuleRegExp("AppMasterEventOperator", AppMasterEventOperator.getOperatorName()
+ + "%"), new NodeProcessor() {
@Override
- public Object process(Node n, Stack s,
- NodeProcessorCtx procCtx, Object... os) throws SemanticException {
+ public Object process(Node n, Stack s, NodeProcessorCtx procCtx, Object... os)
+ throws SemanticException {
GenTezProcContext context = (GenTezProcContext) procCtx;
- UnionOperator union = (UnionOperator) n;
+ AppMasterEventOperator event = (AppMasterEventOperator) n;
- // simply need to remember that we've seen a union.
- context.currentUnionOperators.add(union);
+ // simply need to remember that we've seen an event operator.
+ context.eventOperatorSet.add(event);
return null;
}
});
@@ -185,10 +367,17 @@ public Object process(Node n, Stack s,
GenTezUtils.getUtils().removeUnionOperators(conf, procCtx, w);
}
- // finally make sure the file sink operators are set up right
+ // then we make sure the file sink operators are set up right
for (FileSinkOperator fileSink: procCtx.fileSinkSet) {
GenTezUtils.getUtils().processFileSink(procCtx, fileSink);
}
+
+ // and finally we hook up any events that need to be sent to the tez AM
+ LOG.debug("There are " + procCtx.eventOperatorSet.size() + " app master events.");
+ for (AppMasterEventOperator event : procCtx.eventOperatorSet) {
+ LOG.debug("Handling AppMasterEventOperator: " + event);
+ GenTezUtils.getUtils().processAppMasterEvent(procCtx, event);
+ }
}
@Override
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java
new file mode 100644
index 0000000..264f959
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/AppMasterEventDesc.java
@@ -0,0 +1,63 @@
+/**
+ * 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.plan;
+
+import java.io.IOException;
+
+import org.apache.hadoop.io.DataOutputBuffer;
+
+
+@SuppressWarnings("serial")
+@Explain(displayName = "Application Master Event Operator")
+public class AppMasterEventDesc extends AbstractOperatorDesc {
+
+ private TableDesc table;
+ private String vertexName;
+ private String inputName;
+
+ @Explain(displayName = "Target Vertex")
+ public String getVertexName() {
+ return vertexName;
+ }
+
+ @Explain(displayName = "Target Input")
+ public String getInputName() {
+ return inputName;
+ }
+
+ public void setInputName(String inputName) {
+ this.inputName = inputName;
+ }
+
+ public void setVertexName(String vertexName) {
+ this.vertexName = vertexName;
+ }
+
+ public TableDesc getTable() {
+ return table;
+ }
+
+ public void setTable(TableDesc table) {
+ this.table = table;
+ }
+
+ public void writeEventHeader(DataOutputBuffer buffer) throws IOException {
+ // nothing to add
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java
new file mode 100644
index 0000000..664470f
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/DynamicPruningEventDesc.java
@@ -0,0 +1,58 @@
+/**
+ * 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.plan;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+@SuppressWarnings("serial")
+@Explain(displayName = "Dynamic Partitioning Event Operator")
+public class DynamicPruningEventDesc extends AppMasterEventDesc {
+
+ // column in the target table that will be pruned against
+ private String targetColumnName;
+
+ // tableScan is only available during compile
+ private transient TableScanOperator tableScan;
+
+ public TableScanOperator getTableScan() {
+ return tableScan;
+ }
+
+ public void setTableScan(TableScanOperator tableScan) {
+ this.tableScan = tableScan;
+ }
+
+ @Explain(displayName = "Target column")
+ public String getTargetColumnName() {
+ return targetColumnName;
+ }
+
+ public void setTargetColumnName(String columnName) {
+ this.targetColumnName = columnName;
+ }
+
+ @Override
+ public void writeEventHeader(DataOutputBuffer buffer) throws IOException {
+ super.writeEventHeader(buffer);
+ buffer.writeUTF(targetColumnName);
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
new file mode 100644
index 0000000..efcf2ab
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
@@ -0,0 +1,86 @@
+/**
+ * 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.plan;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+/**
+ * Dummy desc only for populating TOK_ALLCOLREF and should not be used
+ * outside of TypeCheckProcFactory
+ */
+public class ExprNodeDynamicListDesc extends ExprNodeDesc {
+
+ Operator extends OperatorDesc> source;
+ int keyIndex;
+
+ public ExprNodeDynamicListDesc() {
+ }
+
+ public ExprNodeDynamicListDesc(TypeInfo typeInfo, Operator extends OperatorDesc> source, int keyIndex) {
+ super(typeInfo);
+ this.source = source;
+ this.keyIndex = keyIndex;
+ }
+
+ public void setSource(Operator extends OperatorDesc> source) {
+ this.source = source;
+ }
+
+ public Operator extends OperatorDesc> getSource() {
+ return source;
+ }
+
+ public void setKeyIndex(int keyIndex) {
+ this.keyIndex = keyIndex;
+ }
+
+ public int getKeyIndex() {
+ return this.keyIndex;
+ }
+
+ @Override
+ public ExprNodeDesc clone() {
+ ExprNodeDynamicListDesc clone = new ExprNodeDynamicListDesc(typeInfo, source, keyIndex);
+ return clone;
+ }
+
+ @Override
+ public boolean isSame(Object o) {
+ if (o instanceof ExprNodeDynamicListDesc) {
+ return source.equals(((ExprNodeDynamicListDesc)o).getSource());
+ }
+ return false;
+ }
+
+ @Override
+ public String getExprString() {
+ return source.toString();
+ }
+
+ @Override
+ public String toString() {
+ return source.toString();
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index b4278d3..efb0ece 100644
--- ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -26,9 +26,9 @@
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
-import java.util.Set;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -116,6 +116,12 @@
private boolean useOneNullRowInputFormat;
+ // used for dynamic partitioning
+ private Map> eventSourceTableDescMap =
+ new LinkedHashMap>();
+ private Map> eventSourceColumnNameMap =
+ new LinkedHashMap>();
+
public MapWork() {}
public MapWork(String name) {
@@ -525,4 +531,20 @@ public void logPathToAliases() {
}
}
}
+
+ public void setEventSourceTableDescMap(Map> map) {
+ this.eventSourceTableDescMap = map;
+ }
+
+ public Map> getEventSourceTableDescMap() {
+ return eventSourceTableDescMap;
+ }
+
+ public void setEventSourceColumnNameMap(Map> map) {
+ this.eventSourceColumnNameMap = map;
+ }
+
+ public Map> getEventSourceColumnNameMap() {
+ return eventSourceColumnNameMap;
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
new file mode 100644
index 0000000..b1091d0
--- /dev/null
+++ ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
@@ -0,0 +1,267 @@
+/**
+ * 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.ppd;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
+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.TableScanOperator;
+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.PreOrderWalker;
+import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.optimizer.Transform;
+import org.apache.hadoop.hive.ql.parse.OpParseContext;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.RowResolver;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.FilterDesc;
+import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
+import org.apache.hadoop.hive.ql.plan.JoinDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+
+/**
+ * propagates filters to other aliases based on join condition
+ */
+public class SyntheticJoinPredicate implements Transform {
+
+ private static transient Log LOG = LogFactory.getLog(SyntheticJoinPredicate.class.getName());
+
+ @Override
+ public ParseContext transform(ParseContext pctx) throws SemanticException {
+
+ if (!pctx.getConf().getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")
+ || !pctx.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) {
+ return pctx;
+ }
+
+ Map opRules = new LinkedHashMap();
+ opRules.put(new RuleRegExp("R1", "(" +
+ TableScanOperator.getOperatorName() + "%" + ".*" +
+ ReduceSinkOperator.getOperatorName() + "%" +
+ JoinOperator.getOperatorName() + "%)"), new JoinSynthetic());
+
+ // The dispatcher fires the processor corresponding to the closest matching
+ // rule and passes the context along
+ SyntheticContext context = new SyntheticContext(pctx);
+ Dispatcher disp = new DefaultRuleDispatcher(null, opRules, context);
+ GraphWalker ogw = new PreOrderWalker(disp);
+
+ // Create a list of topop nodes
+ List topNodes = new ArrayList();
+ topNodes.addAll(pctx.getTopOps().values());
+ ogw.startWalking(topNodes, null);
+
+ return pctx;
+ }
+
+ // insert filter operator between target(child) and input(parent)
+ private static Operator createFilter(Operator> target, Operator> parent,
+ RowResolver parentRR, ExprNodeDesc filterExpr) {
+ Operator filter = OperatorFactory.get(new FilterDesc(filterExpr, false),
+ new RowSchema(parentRR.getColumnInfos()));
+ filter.setParentOperators(new ArrayList>());
+ filter.setChildOperators(new ArrayList>());
+ filter.getParentOperators().add(parent);
+ filter.getChildOperators().add(target);
+ parent.replaceChild(target, filter);
+ target.replaceParent(parent, filter);
+ return filter;
+ }
+
+ private static class SyntheticContext implements NodeProcessorCtx {
+
+ ParseContext parseContext;
+
+ public SyntheticContext(ParseContext pCtx) {
+ parseContext = pCtx;
+ }
+
+ public ParseContext getParseContext() {
+ return parseContext;
+ }
+ }
+
+ private static class JoinSynthetic implements NodeProcessor {
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+
+ ParseContext pCtx = ((SyntheticContext) procCtx).getParseContext();
+
+ @SuppressWarnings("unchecked")
+ CommonJoinOperator join = (CommonJoinOperator) nd;
+
+ ReduceSinkOperator source = (ReduceSinkOperator) stack.get(stack.size() - 2);
+ int srcPos = join.getParentOperators().indexOf(source);
+
+ List> parents = join.getParentOperators();
+
+ int[][] targets = getTargets(join);
+
+ Operator extends OperatorDesc> parent = source.getParentOperators().get(0);
+ RowResolver parentRR = pCtx.getOpParseCtx().get(parent).getRowResolver();
+
+ for (int targetPos: targets[srcPos]) {
+ if (srcPos == targetPos) {
+ continue;
+ }
+
+ LOG.info("Synthetic predicate: "+srcPos+" --> "+targetPos);
+ ReduceSinkOperator target = (ReduceSinkOperator) parents.get(targetPos);
+ List sourceKeys = source.getConf().getKeyCols();
+ List targetKeys = target.getConf().getKeyCols();
+
+ if (sourceKeys.size() < 1) {
+ continue;
+ }
+
+ ExprNodeDesc syntheticExpr = null;
+
+ for (int i = 0; i < sourceKeys.size(); ++i) {
+ List inArgs = new ArrayList();
+ inArgs.add(sourceKeys.get(i));
+
+ ExprNodeDynamicListDesc dynamicExpr = new ExprNodeDynamicListDesc(targetKeys.get(i).getTypeInfo(), target, i);
+ inArgs.add(dynamicExpr);
+
+ ExprNodeDesc syntheticInExpr = ExprNodeGenericFuncDesc.newInstance(
+ FunctionRegistry.getFunctionInfo("in").getGenericUDF(), inArgs);
+
+ if (syntheticExpr != null) {
+ List andArgs = new ArrayList();
+ andArgs.add(syntheticExpr);
+ andArgs.add(syntheticInExpr);
+
+ syntheticExpr = ExprNodeGenericFuncDesc.newInstance(
+ FunctionRegistry.getFunctionInfo("and").getGenericUDF(), andArgs);
+ } else {
+ syntheticExpr = syntheticInExpr;
+ }
+ }
+
+ Operator newFilter = createFilter(source, parent, parentRR, syntheticExpr);
+ pCtx.getOpParseCtx().put(newFilter, new OpParseContext(parentRR));
+ parent = newFilter;
+ }
+
+ return null;
+ }
+
+ // calculate filter propagation directions for each alias
+ // L<->R for inner/semi join, L->R for left outer join, R->L for right outer join
+ private int[][] getTargets(CommonJoinOperator join) {
+ JoinCondDesc[] conds = join.getConf().getConds();
+
+ int aliases = conds.length + 1;
+ Vectors vector = new Vectors(aliases);
+ for (JoinCondDesc cond : conds) {
+ int left = cond.getLeft();
+ int right = cond.getRight();
+ switch (cond.getType()) {
+ case JoinDesc.INNER_JOIN:
+ case JoinDesc.LEFT_SEMI_JOIN:
+ vector.add(left, right);
+ vector.add(right, left);
+ break;
+ case JoinDesc.LEFT_OUTER_JOIN:
+ vector.add(left, right);
+ break;
+ case JoinDesc.RIGHT_OUTER_JOIN:
+ vector.add(right, left);
+ break;
+ case JoinDesc.FULL_OUTER_JOIN:
+ break;
+ }
+ }
+ int[][] result = new int[aliases][];
+ for (int pos = 0 ; pos < aliases; pos++) {
+ // find all targets recursively
+ result[pos] = vector.traverse(pos);
+ }
+ return result;
+ }
+ }
+
+ private static class Vectors {
+
+ private final Set[] vector;
+
+ @SuppressWarnings("unchecked")
+ public Vectors(int length) {
+ vector = new Set[length];
+ }
+
+ public void add(int from, int to) {
+ if (vector[from] == null) {
+ vector[from] = new HashSet();
+ }
+ vector[from].add(to);
+ }
+
+ public int[] traverse(int pos) {
+ Set targets = new HashSet();
+ traverse(targets, pos);
+ return toArray(targets);
+ }
+
+ private int[] toArray(Set values) {
+ int index = 0;
+ int[] result = new int[values.size()];
+ for (int value : values) {
+ result[index++] = value;
+ }
+ return result;
+ }
+
+ private void traverse(Set targets, int pos) {
+ if (vector[pos] == null) {
+ return;
+ }
+ for (int target : vector[pos]) {
+ if (targets.add(target)) {
+ traverse(targets, target);
+ }
+ }
+ }
+ }
+}
diff --git ql/src/test/queries/clientpositive/dynamic_partition_pruning.q ql/src/test/queries/clientpositive/dynamic_partition_pruning.q
new file mode 100644
index 0000000..5fc9c64
--- /dev/null
+++ ql/src/test/queries/clientpositive/dynamic_partition_pruning.q
@@ -0,0 +1,135 @@
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.optimize.metadataonly=false;
+
+select distinct ds from srcpart;
+select distinct hr from srcpart;
+
+EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds;
+create table srcpart_date as select ds as ds, ds as date from srcpart group by ds;
+create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr;
+create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr;
+create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr;
+
+-- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+set hive.tez.dynamic.partition.pruning=false;
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+set hive.tez.dynamic.partition.pruning=true;
+select count(*) from srcpart where ds = '2008-04-08';
+
+-- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+set hive.tez.dynamic.partition.pruning=false;
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+set hive.tez.dynamic.partition.pruning=true;
+select count(*) from srcpart where hr = 11 and ds = '2008-04-08';
+
+-- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+set hive.tez.dynamic.partition.pruning=false;
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+set hive.tez.dynamic.partition.pruning=true;
+select count(*) from srcpart where ds = '2008-04-08' and hr = 11;
+
+-- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+set hive.tez.dynamic.partition.pruning=false;
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+set hive.tez.dynamic.partition.pruning=true;
+select count(*) from srcpart where ds = 'I DONT EXIST';
+
+-- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+set hive.tez.dynamic.partition.pruning=false;
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+set hive.tez.dynamic.partition.pruning=true;
+select count(*) from srcpart where hr = 11;
+
+-- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08';
+select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08';
+select count(*) from srcpart where ds = '2008-04-08';
+
+-- non-equi join
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr);
+select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr);
+
+-- old style join syntax
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr;
+select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr;
+
+-- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+
+-- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask = true;
+set hive.auto.convert.join.noconditionaltask.size = 10000000;
+
+-- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+select count(*) from srcpart where ds = '2008-04-08';
+
+-- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11;
+select count(*) from srcpart where hr = 11 and ds = '2008-04-08';
+
+-- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11;
+select count(*) from srcpart where ds = '2008-04-08' and hr = 11;
+
+-- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST';
+
+-- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11;
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11;
+select count(*) from srcpart where hr = 11;
+
+-- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08';
+select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08';
+select count(*) from srcpart where ds = '2008-04-08';
+
+-- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+
+-- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08';
+
+drop table srcpart_date;
+drop table srcpart_hour;
+drop table srcpart_date_hour;
+drop table srcpart_double_hour;
diff --git ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
new file mode 100644
index 0000000..2423d92
--- /dev/null
+++ ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out
@@ -0,0 +1,3587 @@
+PREHOOK: query: select distinct ds from srcpart
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select distinct ds from srcpart
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+2008-04-08
+2008-04-09
+PREHOOK: query: select distinct hr from srcpart
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select distinct hr from srcpart
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+11
+12
+PREHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds
+PREHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds
+POSTHOOK: type: CREATETABLE_AS_SELECT
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-4 depends on stages: Stage-2, Stage-0
+ Stage-3 depends on stages: Stage-4
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: ds (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string), _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.srcpart_date
+
+ Stage: Stage-2
+ Dependency Collection
+
+ Stage: Stage-4
+ Create Table Operator:
+ Create Table
+ columns: ds string, date string
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+ serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: srcpart_date
+
+ Stage: Stage-3
+ Stats-Aggr Operator
+
+ Stage: Stage-0
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: create table srcpart_date as select ds as ds, ds as date from srcpart group by ds
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: query: create table srcpart_date as select ds as ds, ds as date from srcpart group by ds
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_date
+PREHOOK: query: create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: query: create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_hour
+PREHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_date_hour
+PREHOOK: query: create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: query: create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_double_hour
+PREHOOK: query: -- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+1000
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+1000
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+1000
+PREHOOK: query: -- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+ Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ value expressions: hr (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_hour
+ Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: hr (type: string)
+ sort order: +
+ Map-reduce partition columns: hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col2}
+ 1
+ outputColumnNames: _col3
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 4
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+PREHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+POSTHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+ Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ value expressions: hr (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_hour
+ Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: hr (type: string)
+ sort order: +
+ Map-reduce partition columns: hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col2}
+ 1
+ outputColumnNames: _col3
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 4
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+PREHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+POSTHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+500
+PREHOOK: query: -- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date_hour
+ Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date_hour
+ Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+500
+PREHOOK: query: -- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+0
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+0
+PREHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+#### A masked pattern was here ####
+0
+PREHOOK: query: -- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: UDFToDouble(hr) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: UDFToDouble(hr) (type: double)
+ sort order: +
+ Map-reduce partition columns: UDFToDouble(hr) (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 5.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: 5.0 (type: double)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: double)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: (hr * 2) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: (hr * 2) (type: double)
+ sort order: +
+ Map-reduce partition columns: (hr * 2) (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 11.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: 11.0 (type: double)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: double)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: UDFToDouble(hr) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: UDFToDouble(hr) (type: double)
+ sort order: +
+ Map-reduce partition columns: UDFToDouble(hr) (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 5.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: (hr * 2) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: (hr * 2) (type: double)
+ sort order: +
+ Map-reduce partition columns: (hr * 2) (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 11.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: select count(*) from srcpart where hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+#### A masked pattern was here ####
+1000
+PREHOOK: query: -- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 5 <- Map 4 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Target column: ds
+ Target Vertex: Map 4
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: ds (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Reducer 5
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+1000
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+1000
+Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: -- non-equi join
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- non-equi join
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ value expressions: ds (type: string), hr (type: string)
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date_hour
+ Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((date = '2008-04-08') and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE
+ value expressions: ds (type: string), hr (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col2} {VALUE._col3}
+ 1 {VALUE._col0} {VALUE._col2}
+ outputColumnNames: _col2, _col3, _col7, _col9
+ Statistics: Num rows: 1 Data size: 29 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((_col2 = _col7) or (_col3 = _col9)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+1500
+PREHOOK: query: -- old style join syntax
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr
+PREHOOK: type: QUERY
+POSTHOOK: query: -- old style join syntax
+EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date_hour
+ Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {KEY.reducesinkkey0} {KEY.reducesinkkey1}
+ 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1}
+ outputColumnNames: _col2, _col3, _col7, _col9
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: -- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ value expressions: date (type: string)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Left Outer Join0 to 1
+ condition expressions:
+ 0
+ 1 {VALUE._col0}
+ outputColumnNames: _col8
+ Statistics: Num rows: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col8 = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (date = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Left Outer Join0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: -- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ value expressions: date (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Outer Join 0 to 1
+ condition expressions:
+ 0
+ 1 {VALUE._col0}
+ outputColumnNames: _col8
+ Statistics: Num rows: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col8 = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: -- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- single column, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 ds (type: string)
+ 1 ds (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+1000
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+1000
+PREHOOK: query: -- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- multiple sources, single key
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {hr}
+ 1
+ keys:
+ 0 ds (type: string)
+ 1 ds (type: string)
+ outputColumnNames: _col3
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 _col3 (type: string)
+ 1 hr (type: string)
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 28129 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_hour
+ Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: hr (type: string)
+ sort order: +
+ Map-reduce partition columns: hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = '2008-04-08')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+PREHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr)
+where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+POSTHOOK: Input: default@srcpart_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+500
+PREHOOK: query: -- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- multiple columns single source
+EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 ds (type: string), hr (type: string)
+ 1 ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date_hour
+ Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string), hr (type: string)
+ sort order: ++
+ Map-reduce partition columns: ds (type: string), hr (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date_hour
+#### A masked pattern was here ####
+500
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+#### A masked pattern was here ####
+500
+PREHOOK: query: -- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- empty set
+EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 ds (type: string)
+ 1 ds (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_date
+#### A masked pattern was here ####
+0
+PREHOOK: query: -- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: -- expressions
+EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: UDFToDouble(hr) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 UDFToDouble(hr) (type: double)
+ 1 5.0 (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 5.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: 5.0 (type: double)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: double)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: (hr * 2) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 (hr * 2) (type: double)
+ 1 11.0 (type: double)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_double_hour
+ Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (hr is not null and (hr = 11.0)) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: 11.0 (type: double)
+ sort order: +
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: 11.0 (type: double)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: double)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Target column: hr
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_double_hour
+#### A masked pattern was here ####
+0
+PREHOOK: query: select count(*) from srcpart where hr = 11
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where hr = 11
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+#### A masked pattern was here ####
+1000
+PREHOOK: query: -- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- parent is reduce tasks
+EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Reducer 4 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 4 <- Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 ds (type: string)
+ 1 _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 0 Data size: 25572 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: ds (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 11624 Basic stats: PARTIAL Column stats: COMPLETE
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Reducer 4
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Target column: ds
+ Target Vertex: Map 1
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+1000
+PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+#### A masked pattern was here ####
+1000
+PREHOOK: query: -- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- left join
+EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Left Outer Join0 to 1
+ condition expressions:
+ 0
+ 1 {date}
+ keys:
+ 0 ds (type: string)
+ 1 ds (type: string)
+ outputColumnNames: _col8
+ Statistics: Num rows: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col8 = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ value expressions: date (type: string)
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Dynamic Partitioning Event Operator
+ Target Input: srcpart
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 2 <- Map 1 (BROADCAST_EDGE)
+ Reducer 3 <- Map 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (date = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Left Outer Join0 to 1
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 ds (type: string)
+ 1 ds (type: string)
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: -- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- full outer
+EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_date
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE
+ value expressions: date (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Outer Join 0 to 1
+ condition expressions:
+ 0
+ 1 {VALUE._col0}
+ outputColumnNames: _col8
+ Statistics: Num rows: 2 Data size: 46 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col8 = '2008-04-08') (type: boolean)
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 23 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: drop table srcpart_date
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@srcpart_date
+PREHOOK: Output: default@srcpart_date
+POSTHOOK: query: drop table srcpart_date
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@srcpart_date
+POSTHOOK: Output: default@srcpart_date
+PREHOOK: query: drop table srcpart_hour
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@srcpart_hour
+PREHOOK: Output: default@srcpart_hour
+POSTHOOK: query: drop table srcpart_hour
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@srcpart_hour
+POSTHOOK: Output: default@srcpart_hour
+PREHOOK: query: drop table srcpart_date_hour
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@srcpart_date_hour
+PREHOOK: Output: default@srcpart_date_hour
+POSTHOOK: query: drop table srcpart_date_hour
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@srcpart_date_hour
+POSTHOOK: Output: default@srcpart_date_hour
+PREHOOK: query: drop table srcpart_double_hour
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@srcpart_double_hour
+PREHOOK: Output: default@srcpart_double_hour
+POSTHOOK: query: drop table srcpart_double_hour
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@srcpart_double_hour
+POSTHOOK: Output: default@srcpart_double_hour
diff --git ql/src/test/results/clientpositive/tez/join_synthetic.q.out ql/src/test/results/clientpositive/tez/join_synthetic.q.out
new file mode 100644
index 0000000..8eea2c9
--- /dev/null
+++ ql/src/test/results/clientpositive/tez/join_synthetic.q.out
@@ -0,0 +1,497 @@
+PREHOOK: query: EXPLAIN select s1.key, s2.k from src s1 join (select key as k, value as v from src) s2 on s1.key = s2.k and s2.k = 5
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select s1.key, s2.k from src s1 join (select key as k, value as v from src) s2 on s1.key = s2.k and s2.k = 5
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: s1
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((key = 5) and key is not null) and true) (type: boolean)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((key = 5) and key is not null) and true) (type: boolean)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {KEY.reducesinkkey0}
+ 1 {KEY.reducesinkkey0}
+ outputColumnNames: _col0, _col4
+ Statistics: Num rows: 7 Data size: 771 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 7 Data size: 771 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 7 Data size: 771 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: EXPLAIN select * from srcpart where ds = 5
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select * from srcpart where ds = 5
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 116 Data size: 23248 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (ds = 5) (type: boolean)
+ Statistics: Num rows: 116 Data size: 23248 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string), value (type: string), ds (type: string), hr (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 116 Data size: 23248 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 116 Data size: 23248 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: EXPLAIN select srcpart.key, s2.k from srcpart join (select key as k, value as v from src) s2 on srcpart.ds = s2.k and s2.k = 5
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select srcpart.key, s2.k from srcpart join (select key as k, value as v from src) s2 on srcpart.ds = s2.k and s2.k = 5
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 232 Data size: 23248 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((ds = 5) and true) (type: boolean)
+ Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (((key = 5) and key is not null) and true) (type: boolean)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 7 Data size: 701 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Application Master Event Operator
+ Target Input: srcpart
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1 {KEY.reducesinkkey0}
+ outputColumnNames: _col0, _col6
+ Statistics: Num rows: 127 Data size: 12786 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col6 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 127 Data size: 12786 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 127 Data size: 12786 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: create table srcpart_filtered_ds as select * from srcpart where ds = '2008-04-08'
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: query: create table srcpart_filtered_ds as select * from srcpart where ds = '2008-04-08'
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@srcpart_filtered_ds
+PREHOOK: query: create table srcpart_filtered_hr as select * from srcpart where hr = 11
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: query: create table srcpart_filtered_hr as select * from srcpart where hr = 11
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Output: default@srcpart_filtered_hr
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) where srcpart_filtered_ds.key > 0
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) where srcpart_filtered_ds.key > 0
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_filtered_ds
+ Statistics: Num rows: 1000 Data size: 24624 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((ds is not null and true) and (key > 0)) (type: boolean)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
+ Group By Operator
+ keys: ds (type: string)
+ mode: hash
+ outputColumnNames: ds
+ Application Master Event Operator
+ Target Input: srcpart
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 91 Data size: 2247 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 91 Data size: 2247 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 3
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) where srcpart_filtered_ds.key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_filtered_ds
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) where srcpart_filtered_ds.key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_filtered_ds
+#### A masked pattern was here ####
+994000
+PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) join srcpart_filtered_hr on (srcpart.hr = srcpart_filtered_hr.hr) where srcpart_filtered_ds.key > 0 and srcpart_filtered_hr.key > 0
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) join srcpart_filtered_hr on (srcpart.hr = srcpart_filtered_hr.hr) where srcpart_filtered_ds.key > 0 and srcpart_filtered_hr.key > 0
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+ Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: srcpart
+ Statistics: Num rows: 0 Data size: 23248 Basic stats: PARTIAL Column stats: COMPLETE
+ Filter Operator
+ predicate: (true and true) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE
+ value expressions: hr (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_filtered_hr
+ Statistics: Num rows: 1000 Data size: 24624 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((hr is not null and true) and (key > 0)) (type: boolean)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: hr (type: string)
+ sort order: +
+ Map-reduce partition columns: hr (type: string)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: hr (type: string)
+ outputColumnNames: hr
+ Group By Operator
+ keys: hr (type: string)
+ mode: hash
+ outputColumnNames: hr
+ Application Master Event Operator
+ Target Input: srcpart
+ Target column: hr
+ Target Vertex: Map 1
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: srcpart_filtered_ds
+ Statistics: Num rows: 1000 Data size: 24624 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((ds is not null and true) and (key > 0)) (type: boolean)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: ds (type: string)
+ sort order: +
+ Map-reduce partition columns: ds (type: string)
+ Statistics: Num rows: 83 Data size: 2043 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ds (type: string)
+ outputColumnNames: ds
+ Group By Operator
+ keys: ds (type: string)
+ mode: hash
+ outputColumnNames: ds
+ Application Master Event Operator
+ Target Input: srcpart
+ Target column: ds
+ Target Vertex: Map 1
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col2}
+ 1
+ outputColumnNames: _col3
+ Statistics: Num rows: 91 Data size: 2247 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 91 Data size: 2247 Basic stats: COMPLETE Column stats: NONE
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0
+ 1
+ Statistics: Num rows: 100 Data size: 2471 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ Statistics: Num rows: 100 Data size: 2471 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: bigint)
+ Reducer 4
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) join srcpart_filtered_hr on (srcpart.hr = srcpart_filtered_hr.hr) where srcpart_filtered_ds.key > 0 and srcpart_filtered_hr.key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Input: default@srcpart_filtered_ds
+PREHOOK: Input: default@srcpart_filtered_hr
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from srcpart join srcpart_filtered_ds on (srcpart.ds = srcpart_filtered_ds.ds) join srcpart_filtered_hr on (srcpart.hr = srcpart_filtered_hr.hr) where srcpart_filtered_ds.key > 0 and srcpart_filtered_hr.key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Input: default@srcpart_filtered_ds
+POSTHOOK: Input: default@srcpart_filtered_hr
+#### A masked pattern was here ####
+494018000
+PREHOOK: query: drop table srcpart_filtered
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table srcpart_filtered
+POSTHOOK: type: DROPTABLE
diff --git service/src/gen/thrift/gen-py/TCLIService/TCLIService-remote service/src/gen/thrift/gen-py/TCLIService/TCLIService-remote
old mode 100644
new mode 100755
diff --git service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
old mode 100644
new mode 100755