Some json parsers are not closed. parser in JacksonParser#parseJson, for example.
Author: navis.ryu <navis@apache.org>
Closes#9130 from navis/SPARK-11124.
In Spark SQL, the Exchange planner tries to avoid unnecessary sorts in cases where the data has already been sorted by a superset of the requested sorting columns. For instance, let's say that a query calls for an operator's input to be sorted by `a.asc` and the input happens to already be sorted by `[a.asc, b.asc]`. In this case, we do not need to re-sort the input. The converse, however, is not true: if the query calls for `[a.asc, b.asc]`, then `a.asc` alone will not satisfy the ordering requirements, requiring an additional sort to be planned by Exchange.
However, the current Exchange code gets this wrong and incorrectly skips sorting when the existing output ordering is a subset of the required ordering. This is simple to fix, however.
This bug was introduced in https://github.com/apache/spark/pull/7458, so it affects 1.5.0+.
This patch fixes the bug and significantly improves the unit test coverage of Exchange's sort-planning logic.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9140 from JoshRosen/SPARK-11135.
#9084 uncovered that many tests that test spilling don't actually spill. This is a follow-up patch to fix that to ensure our unit tests actually catch potential bugs in spilling. The size of this patch is inflated by the refactoring of `ExternalSorterSuite`, which had a lot of duplicate code and logic.
Author: Andrew Or <andrew@databricks.com>
Closes#9124 from andrewor14/spilling-tests.
Actually all of the `UnaryMathExpression` doens't support the Decimal, will create follow ups for supporing it. This is the first PR which will be good to review the approach I am taking.
Author: Cheng Hao <hao.cheng@intel.com>
Closes#9086 from chenghao-intel/ceiling.
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.
The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
```scala
withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") {
withTempPath { dir =>
val path = s"${dir.getCanonicalPath}/part=1"
(1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path)
// If the "part = 1" filter gets pushed down, this query will throw an exception since
// "part" is not a valid column in the actual Parquet file
checkAnswer(
sqlContext.read.parquet(path).filter("a > 0 and (part = 0 or a > 1)"),
(2 to 3).map(i => Row(i, i.toString, 1)))
}
}
```
We expect the result to be:
```
2,1
3,1
```
But got
```
1,1
2,1
3,1
```
Author: Cheng Hao <hao.cheng@intel.com>
Closes#8916 from chenghao-intel/partition_filter.
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.
Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>
Closes#8453 from cloud-fan/table-name.
With this feature, we can track the query plan, time cost, exception during query execution for spark users.
Author: Wenchen Fan <cloud0fan@163.com>
Closes#9078 from cloud-fan/callback.
We should not stop resolving having when the having condtion is resolved, or something like `count(1)` will crash.
Author: Wenchen Fan <cloud0fan@163.com>
Closes#9105 from cloud-fan/having.
This is a first draft of the ability to construct expressions that will take a catalyst internal row and construct a Product (case class or tuple) that has fields with the correct names. Support include:
- Nested classes
- Maps
- Efficiently handling of arrays of primitive types
Not yet supported:
- Case classes that require custom collection types (i.e. List instead of Seq).
Author: Michael Armbrust <michael@databricks.com>
Closes#9100 from marmbrus/productContructor.
In the current implementation of named expressions' `ExprIds`, we rely on a per-JVM AtomicLong to ensure that expression ids are unique within a JVM. However, these expression ids will not be _globally_ unique. This opens the potential for id collisions if new expression ids happen to be created inside of tasks rather than on the driver.
There are currently a few cases where tasks allocate expression ids, which happen to be safe because those expressions are never compared to expressions created on the driver. In order to guard against the introduction of invalid comparisons between driver-created and executor-created expression ids, this patch extends `ExprId` to incorporate a UUID to identify the JVM that created the id, which prevents collisions.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9093 from JoshRosen/SPARK-11080.
This patch unifies the memory management of the storage and execution regions such that either side can borrow memory from each other. When memory pressure arises, storage will be evicted in favor of execution. To avoid regressions in cases where storage is crucial, we dynamically allocate a fraction of space for storage that execution cannot evict. Several configurations are introduced:
- **spark.memory.fraction (default 0.75)**: fraction of the heap space used for execution and storage. The lower this is, the more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation in the case of sparse, unusually large records.
- **spark.memory.storageFraction (default 0.5)**: size of the storage region within the space set aside by `spark.memory.fraction`. Cached data may only be evicted if total storage exceeds this region.
- **spark.memory.useLegacyMode (default false)**: whether to use the memory management that existed in Spark 1.5 and before. This is mainly for backward compatibility.
For a detailed description of the design, see [SPARK-10000](https://issues.apache.org/jira/browse/SPARK-10000). This patch builds on top of the `MemoryManager` interface introduced in #9000.
Author: Andrew Or <andrew@databricks.com>
Closes#9084 from andrewor14/unified-memory-manager.
Two points in this PR:
1. Originally thought was that a named R list is assumed to be a struct in SerDe. But this is problematic because some R functions will implicitly generate named lists that are not intended to be a struct when transferred by SerDe. So SerDe clients have to explicitly mark a names list as struct by changing its class from "list" to "struct".
2. SerDe is in the Spark Core module, and data of StructType is represented as GenricRow which is defined in Spark SQL module. SerDe can't import GenricRow as in maven build Spark SQL module depends on Spark Core module. So this PR adds a registration hook in SerDe to allow SQLUtils in Spark SQL module to register its functions for serialization and deserialization of StructType.
Author: Sun Rui <rui.sun@intel.com>
Closes#8794 from sun-rui/SPARK-10051.
The SQLTab will be shared by multiple sessions.
If we create multiple independent SQLContexts (not using newSession()), will still see multiple SQLTabs in the Spark UI.
Author: Davies Liu <davies@databricks.com>
Closes#9048 from davies/sqlui.
Currently, All windows function could generate wrong result in cluster sometimes.
The root cause is that AttributeReference is called in executor, then id of it may not be unique than others created in driver.
Here is the script that could reproduce the problem (run in local cluster):
```
from pyspark import SparkContext, HiveContext
from pyspark.sql.window import Window
from pyspark.sql.functions import rowNumber
sqlContext = HiveContext(SparkContext())
sqlContext.setConf("spark.sql.shuffle.partitions", "3")
df = sqlContext.range(1<<20)
df2 = df.select((df.id % 1000).alias("A"), (df.id / 1000).alias('B'))
ws = Window.partitionBy(df2.A).orderBy(df2.B)
df3 = df2.select("client", "date", rowNumber().over(ws).alias("rn")).filter("rn < 0")
assert df3.count() == 0
```
Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes#9050 from davies/wrong_window.
This PR improve the unrolling and read of complex types in columnar cache:
1) Using UnsafeProjection to do serialization of complex types, so they will not be serialized three times (two for actualSize)
2) Copy the bytes from UnsafeRow/UnsafeArrayData to ByteBuffer directly, avoiding the immediate byte[]
3) Using the underlying array in ByteBuffer to create UTF8String/UnsafeRow/UnsafeArrayData without copy.
Combine these optimizations, we can reduce the unrolling time from 25s to 21s (20% less), reduce the scanning time from 3.5s to 2.5s (28% less).
```
df = sqlContext.read.parquet(path)
t = time.time()
df.cache()
df.count()
print 'unrolling', time.time() - t
for i in range(10):
t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```
The schema is
```
root
|-- a: struct (nullable = true)
| |-- b: long (nullable = true)
| |-- c: string (nullable = true)
|-- d: array (nullable = true)
| |-- element: long (containsNull = true)
|-- e: map (nullable = true)
| |-- key: long
| |-- value: string (valueContainsNull = true)
```
Now the columnar cache depends on that UnsafeProjection support all the data types (including UDT), this PR also fix that.
Author: Davies Liu <davies@databricks.com>
Closes#9016 from davies/complex2.
For Parquet decimal columns that are encoded using plain-dictionary encoding, we can make the upper level converter aware of the dictionary, so that we can pre-instantiate all the decimals to avoid duplicated instantiation.
Note that plain-dictionary encoding isn't available for `FIXED_LEN_BYTE_ARRAY` for Parquet writer version `PARQUET_1_0`. So currently only decimals written as `INT32` and `INT64` can benefit from this optimization.
Author: Cheng Lian <lian@databricks.com>
Closes#9040 from liancheng/spark-11007.decimal-converter-dict-support.
JIRA: https://issues.apache.org/jira/browse/SPARK-10960
When accessing a column in inner select from a select with window function, `AnalysisException` will be thrown. For example, an query like this:
select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 from (select month, area, product, 1 as tmp1 from windowData) tmp
Currently, the rule `ExtractWindowExpressions` in `Analyzer` only extracts regular expressions from `WindowFunction`, `WindowSpecDefinition` and `AggregateExpression`. We need to also extract other attributes as the one in `Alias` as shown in the above query.
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#9011 from viirya/fix-window-inner-column.
SortBasedAggregationIterator uses a KVIterator interface in order to process input rows as key-value pairs, but this use of KVIterator is unnecessary, slightly complicates the code, and might hurt performance. This patch refactors this code to remove the use of this extra layer of iterator wrapping and simplifies other parts of the code in the process.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9066 from JoshRosen/sort-iterator-cleanup.
marmbrus
rxin
This patch adds a JdbcDialect class, which customizes the datatype mappings for Derby backends. The patch also adds unit tests for the new dialect, corresponding to the existing tests for other JDBC dialects.
JDBCSuite runs cleanly for me with this patch. So does JDBCWriteSuite, although it produces noise as described here: https://issues.apache.org/jira/browse/SPARK-10890
This patch is my original work, which I license to the ASF. I am a Derby contributor, so my ICLA is on file under SVN id "rhillegas": http://people.apache.org/committer-index.html
Touches the following files:
---------------------------------
org.apache.spark.sql.jdbc.JdbcDialects
Adds a DerbyDialect.
---------------------------------
org.apache.spark.sql.jdbc.JDBCSuite
Adds unit tests for the new DerbyDialect.
Author: Rick Hillegas <rhilleg@us.ibm.com>
Closes#8982 from rick-ibm/b_10855.
This patch introduces a `MemoryManager` that is the central arbiter of how much memory to grant to storage and execution. This patch is primarily concerned only with refactoring while preserving the existing behavior as much as possible.
This is the first step away from the existing rigid separation of storage and execution memory, which has several major drawbacks discussed on the [issue](https://issues.apache.org/jira/browse/SPARK-10956). It is the precursor of a series of patches that will attempt to address those drawbacks.
Author: Andrew Or <andrew@databricks.com>
Author: Josh Rosen <joshrosen@databricks.com>
Author: andrewor14 <andrew@databricks.com>
Closes#9000 from andrewor14/memory-manager.
This PR improve the sessions management by replacing the thread-local based to one SQLContext per session approach, introduce separated temporary tables and UDFs/UDAFs for each session.
A new session of SQLContext could be created by:
1) create an new SQLContext
2) call newSession() on existing SQLContext
For HiveContext, in order to reduce the cost for each session, the classloader and Hive client are shared across multiple sessions (created by newSession).
CacheManager is also shared by multiple sessions, so cache a table multiple times in different sessions will not cause multiple copies of in-memory cache.
Added jars are still shared by all the sessions, because SparkContext does not support sessions.
cc marmbrus yhuai rxin
Author: Davies Liu <davies@databricks.com>
Closes#8909 from davies/sessions.
UnsafeRow contains 3 pieces of information when pointing to some data in memory (an object, a base offset, and length). When the row is serialized with Java/Kryo serialization, the object layout in memory can change if two machines have different pointer width (Oops in JVM).
To reproduce, launch Spark using
MASTER=local-cluster[2,1,1024] bin/spark-shell --conf "spark.executor.extraJavaOptions=-XX:-UseCompressedOops"
And then run the following
scala> sql("select 1 xx").collect()
Author: Reynold Xin <rxin@databricks.com>
Closes#9030 from rxin/SPARK-10914.
This PR refactors Parquet write path to follow parquet-format spec. It's a successor of PR #7679, but with less non-essential changes.
Major changes include:
1. Replaces `RowWriteSupport` and `MutableRowWriteSupport` with `CatalystWriteSupport`
- Writes Parquet data using standard layout defined in parquet-format
Specifically, we are now writing ...
- ... arrays and maps in standard 3-level structure with proper annotations and field names
- ... decimals as `INT32` and `INT64` whenever possible, and taking `FIXED_LEN_BYTE_ARRAY` as the final fallback
- Supports legacy mode which is compatible with Spark 1.4 and prior versions
The legacy mode is by default off, and can be turned on by flipping SQL option `spark.sql.parquet.writeLegacyFormat` to `true`.
- Eliminates per value data type dispatching costs via prebuilt composed writer functions
1. Cleans up the last pieces of old Parquet support code
As pointed out by rxin previously, we probably want to rename all those `Catalyst*` Parquet classes to `Parquet*` for clarity. But I'd like to do this in a follow-up PR to minimize code review noises in this one.
Author: Cheng Lian <lian@databricks.com>
Closes#8988 from liancheng/spark-8848/standard-parquet-write-path.
In `aggregate/utils.scala`, there is a substantial amount of duplication in the expression-rewriting logic. As a prerequisite to supporting imperative aggregate functions in `TungstenAggregate`, this patch refactors this file so that the same expression-rewriting logic is used for both `SortAggregate` and `TungstenAggregate`.
In order to allow both operators to use the same rewriting logic, `TungstenAggregationIterator. generateResultProjection()` has been updated so that it first evaluates all declarative aggregate functions' `evaluateExpression`s and writes the results into a temporary buffer, and then uses this temporary buffer and the grouping expressions to evaluate the final resultExpressions. This matches the logic in SortAggregateIterator, where this two-pass approach is necessary in order to support imperative aggregates. If this change turns out to cause performance regressions, then we can look into re-implementing the single-pass evaluation in a cleaner way as part of a followup patch.
Since the rewriting logic is now shared across both operators, this patch also extracts that logic and places it in `SparkStrategies`. This makes the rewriting logic a bit easier to follow, I think.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#9015 from JoshRosen/SPARK-10988.
This PR is a first cut at code generating an encoder that takes a Scala `Product` type and converts it directly into the tungsten binary format. This is done through the addition of a new set of expression that can be used to invoke methods on raw JVM objects, extracting fields and converting the result into the required format. These can then be used directly in an `UnsafeProjection` allowing us to leverage the existing encoding logic.
According to some simple benchmarks, this can significantly speed up conversion (~4x). However, replacing CatalystConverters is deferred to a later PR to keep this PR at a reasonable size.
```scala
case class SomeInts(a: Int, b: Int, c: Int, d: Int, e: Int)
val data = SomeInts(1, 2, 3, 4, 5)
val encoder = ProductEncoder[SomeInts]
val converter = CatalystTypeConverters.createToCatalystConverter(ScalaReflection.schemaFor[SomeInts].dataType)
(1 to 5).foreach {iter =>
benchmark(s"converter $iter") {
var i = 100000000
while (i > 0) {
val res = converter(data).asInstanceOf[InternalRow]
assert(res.getInt(0) == 1)
assert(res.getInt(1) == 2)
i -= 1
}
}
benchmark(s"encoder $iter") {
var i = 100000000
while (i > 0) {
val res = encoder.toRow(data)
assert(res.getInt(0) == 1)
assert(res.getInt(1) == 2)
i -= 1
}
}
}
```
Results:
```
[info] converter 1: 7170ms
[info] encoder 1: 1888ms
[info] converter 2: 6763ms
[info] encoder 2: 1824ms
[info] converter 3: 6912ms
[info] encoder 3: 1802ms
[info] converter 4: 7131ms
[info] encoder 4: 1798ms
[info] converter 5: 7350ms
[info] encoder 5: 1912ms
```
Author: Michael Armbrust <michael@databricks.com>
Closes#9019 from marmbrus/productEncoder.
This PR refactors `HashJoinNode` to take a existing `HashedRelation`. So, we can reuse this node for both `ShuffledHashJoin` and `BroadcastHashJoin`.
https://issues.apache.org/jira/browse/SPARK-10887
Author: Yin Huai <yhuai@databricks.com>
Closes#8953 from yhuai/SPARK-10887.
In the analysis phase , while processing the rules for IN predicate, we
compare the in-list types to the lhs expression type and generate
cast operation if necessary. In the case of NULL [NOT] IN expr1 , we end up
generating cast between in list types to NULL like cast (1 as NULL) which
is not a valid cast.
The fix is to not generate such a cast if the lhs type is a NullType instead
we translate the expression to Literal(Null).
Author: Dilip Biswal <dbiswal@us.ibm.com>
Closes#8983 from dilipbiswal/spark_8654.
Its pretty hard to debug problems with expressions when you can't see all the arguments.
Before: `invoke()`
After: `invoke(inputObject#1, intField, IntegerType)`
Author: Michael Armbrust <michael@databricks.com>
Closes#9022 from marmbrus/expressionToString.
This PR addresses [SPARK-7869](https://issues.apache.org/jira/browse/SPARK-7869)
Before the patch, attempt to load the table from Postgres with JSON/JSONb datatype caused error `java.sql.SQLException: Unsupported type 1111`
Postgres data types JSON and JSONb are now mapped to String on Spark side thus they can be loaded into DF and processed on Spark side
Example
Postgres:
```
create table test_json (id int, value json);
create table test_jsonb (id int, value jsonb);
insert into test_json (id, value) values
(1, '{"field1":"value1","field2":"value2","field3":[1,2,3]}'::json),
(2, '{"field1":"value3","field2":"value4","field3":[4,5,6]}'::json),
(3, '{"field3":"value5","field4":"value6","field3":[7,8,9]}'::json);
insert into test_jsonb (id, value) values
(4, '{"field1":"value1","field2":"value2","field3":[1,2,3]}'::jsonb),
(5, '{"field1":"value3","field2":"value4","field3":[4,5,6]}'::jsonb),
(6, '{"field3":"value5","field4":"value6","field3":[7,8,9]}'::jsonb);
```
PySpark:
```
>>> import json
>>> df1 = sqlContext.read.jdbc("jdbc:postgresql://127.0.0.1:5432/test?user=testuser", "test_json")
>>> df1.map(lambda x: (x.id, json.loads(x.value))).map(lambda (id, value): (id, value.get('field3'))).collect()
[(1, [1, 2, 3]), (2, [4, 5, 6]), (3, [7, 8, 9])]
>>> df2 = sqlContext.read.jdbc("jdbc:postgresql://127.0.0.1:5432/test?user=testuser", "test_jsonb")
>>> df2.map(lambda x: (x.id, json.loads(x.value))).map(lambda (id, value): (id, value.get('field1'))).collect()
[(4, u'value1'), (5, u'value3'), (6, None)]
```
Author: 0x0FFF <programmerag@gmail.com>
Closes#8948 from 0x0FFF/SPARK-7869.
This PR improve the performance of complex types in columnar cache by using UnsafeProjection instead of KryoSerializer.
A simple benchmark show that this PR could improve the performance of scanning a cached table with complex columns by 15x (comparing to Spark 1.5).
Here is the code used to benchmark:
```
df = sc.range(1<<23).map(lambda i: Row(a=Row(b=i, c=str(i)), d=range(10), e=dict(zip(range(10), [str(i) for i in range(10)])))).toDF()
df.write.parquet("table")
```
```
df = sqlContext.read.parquet("table")
df.cache()
df.count()
t = time.time()
print df.select("*")._jdf.queryExecution().toRdd().count()
print time.time() - t
```
Author: Davies Liu <davies@databricks.com>
Closes#8971 from davies/complex.
This patch allows `Repartition` to support UnsafeRows. This is accomplished by implementing the logical `Repartition` operator in terms of `Exchange` and a new `RoundRobinPartitioning`.
Author: Josh Rosen <joshrosen@databricks.com>
Author: Liang-Chi Hsieh <viirya@appier.com>
Closes#8083 from JoshRosen/SPARK-9702.
The created decimal is wrong if using `Decimal(unscaled, precision, scale)` with unscaled > 1e18 and and precision > 18 and scale > 0.
This bug exists since the beginning.
Author: Davies Liu <davies@databricks.com>
Closes#9014 from davies/fix_decimal.
DeclarativeAggregate matches more closely with ImperativeAggregate we already have.
Author: Reynold Xin <rxin@databricks.com>
Closes#9013 from rxin/SPARK-10982.
HadoopRDD throws exception in executor, something like below.
{noformat}
5/09/17 18:51:21 INFO metastore.HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
15/09/17 18:51:21 INFO metastore.ObjectStore: ObjectStore, initialize called
15/09/17 18:51:21 WARN metastore.HiveMetaStore: Retrying creating default database after error: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
javax.jdo.JDOFatalUserException: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
at javax.jdo.JDOHelper.invokeGetPersistenceManagerFactoryOnImplementation(JDOHelper.java:1175)
at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:808)
at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:701)
at org.apache.hadoop.hive.metastore.ObjectStore.getPMF(ObjectStore.java:365)
at org.apache.hadoop.hive.metastore.ObjectStore.getPersistenceManager(ObjectStore.java:394)
at org.apache.hadoop.hive.metastore.ObjectStore.initialize(ObjectStore.java:291)
at org.apache.hadoop.hive.metastore.ObjectStore.setConf(ObjectStore.java:258)
at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
at org.apache.hadoop.hive.metastore.RawStoreProxy.<init>(RawStoreProxy.java:57)
at org.apache.hadoop.hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:66)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:593)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:571)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:620)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:461)
at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:66)
at org.apache.hadoop.hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:72)
at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:199)
at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.<init>(SessionHiveMetaStoreClient.java:74)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1521)
at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:86)
at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132)
at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104)
at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3005)
at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3024)
at org.apache.hadoop.hive.ql.metadata.Hive.getAllDatabases(Hive.java:1234)
at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:174)
at org.apache.hadoop.hive.ql.metadata.Hive.<clinit>(Hive.java:166)
at org.apache.hadoop.hive.ql.plan.PlanUtils.configureJobPropertiesForStorageHandler(PlanUtils.java:803)
at org.apache.hadoop.hive.ql.plan.PlanUtils.configureInputJobPropertiesForStorageHandler(PlanUtils.java:782)
at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:298)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
at scala.Option.map(Option.scala:145)
at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:176)
at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:220)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:216)
at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:87)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
at org.apache.spark.scheduler.Task.run(Task.scala:88)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{noformat}
Author: navis.ryu <navis@apache.org>
Closes#8804 from navis/SPARK-10679.
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.
The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.
If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.
After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#8973 from JoshRosen/tungsten-agg-comments.
This PR is mostly cosmetic and cleans up some warts in codegen (nearly all of which were inherited from the original quasiquote version).
- Add lines numbers to errors (in stacktraces when debug logging is on, and always for compile fails)
- Use a variable for input row instead of hardcoding "i" everywhere
- rename `primitive` -> `value` (since its often actually an object)
Author: Michael Armbrust <michael@databricks.com>
Closes#9006 from marmbrus/codegen-cleanup.