aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2015-06-12 23:06:31 -0700
committerReynold Xin <rxin@databricks.com>2015-06-12 23:06:31 -0700
commitd46f8e5d4b5c1278e0fae3ad133b2229ac01b197 (patch)
tree7ec124d4ac2ff365a9675d89113cbfee1e8abad8
parent6e9c3ff1ecaf12a0126d83f27f5a4153ae420a34 (diff)
downloadspark-d46f8e5d4b5c1278e0fae3ad133b2229ac01b197.tar.gz
spark-d46f8e5d4b5c1278e0fae3ad133b2229ac01b197.tar.bz2
spark-d46f8e5d4b5c1278e0fae3ad133b2229ac01b197.zip
[SPARK-7186] [SQL] Decouple internal Row from external Row
Currently, we use o.a.s.sql.Row both internally and externally. The external interface is wider than what the internal needs because it is designed to facilitate end-user programming. This design has proven to be very error prone and cumbersome for internal Row implementations. As a first step, we create an InternalRow interface in the catalyst module, which is identical to the current Row interface. And we switch all internal operators/expressions to use this InternalRow instead. When we need to expose Row, we convert the InternalRow implementation into Row for users. For all public API, we use Row (for example, data source APIs), which will be converted into/from InternalRow by CatalystTypeConverters. For all internal data sources (Json, Parquet, JDBC, Hive), we use InternalRow for better performance, casted into Row in buildScan() (without change the public API). When create a PhysicalRDD, we cast them back to InternalRow. cc rxin marmbrus JoshRosen Author: Davies Liu <davies@databricks.com> Closes #6792 from davies/internal_row and squashes the following commits: f2abd13 [Davies Liu] fix scalastyle a7e025c [Davies Liu] move InternalRow into catalyst 30db8ba [Davies Liu] Merge branch 'master' of github.com:apache/spark into internal_row 7cbced8 [Davies Liu] separate Row and InternalRow
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/BaseMutableRow.java (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/BaseMutableRow.java)0
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/BaseRow.java (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/BaseRow.java)5
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java8
-rw-r--r--sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala54
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala57
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala11
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala5
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala12
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala104
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala51
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala66
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala73
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala15
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala11
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala17
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala15
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala13
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala15
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala9
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala9
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala28
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala20
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala23
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala10
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala12
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala12
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala9
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala7
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala8
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala22
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala4
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala16
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala4
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala10
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateUtilsSuite.scala1
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala18
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala26
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala7
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala65
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala27
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala13
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala14
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala24
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala18
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala13
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala20
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala20
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala21
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala46
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala13
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala11
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala11
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala12
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala43
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala25
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala8
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala22
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala9
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala18
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala14
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala13
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala16
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala38
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala18
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala14
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala111
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala7
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala74
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala19
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala27
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala9
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala6
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala5
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala26
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala57
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala7
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala27
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala8
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala6
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala18
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala6
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala12
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala68
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala16
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala10
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala32
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala24
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala9
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala3
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala2
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala4
132 files changed, 1160 insertions, 973 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/BaseMutableRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/BaseMutableRow.java
index acec2bf452..acec2bf452 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/BaseMutableRow.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/BaseMutableRow.java
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/BaseRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/BaseRow.java
index e91daf17f8..611e02d8fb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/BaseRow.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/BaseRow.java
@@ -25,10 +25,11 @@ import java.util.List;
import scala.collection.Seq;
import scala.collection.mutable.ArraySeq;
+import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.GenericRow;
import org.apache.spark.sql.types.StructType;
-public abstract class BaseRow implements Row {
+public abstract class BaseRow extends InternalRow {
@Override
final public int length() {
@@ -176,7 +177,7 @@ public abstract class BaseRow implements Row {
}
@Override
- public Row copy() {
+ public InternalRow copy() {
final int n = size();
Object[] arr = new Object[n];
for (int i = 0; i < n; i++) {
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java
index 299ff3728a..b23e0efc83 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions;
import java.util.Arrays;
import java.util.Iterator;
-import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.PlatformDependent;
@@ -107,7 +107,7 @@ public final class UnsafeFixedWidthAggregationMap {
* @param enablePerfMetrics if true, performance metrics will be recorded (has minor perf impact)
*/
public UnsafeFixedWidthAggregationMap(
- Row emptyAggregationBuffer,
+ InternalRow emptyAggregationBuffer,
StructType aggregationBufferSchema,
StructType groupingKeySchema,
TaskMemoryManager memoryManager,
@@ -125,7 +125,7 @@ public final class UnsafeFixedWidthAggregationMap {
/**
* Convert a Java object row into an UnsafeRow, allocating it into a new long array.
*/
- private static long[] convertToUnsafeRow(Row javaRow, StructType schema) {
+ private static long[] convertToUnsafeRow(InternalRow javaRow, StructType schema) {
final UnsafeRowConverter converter = new UnsafeRowConverter(schema);
final long[] unsafeRow = new long[converter.getSizeRequirement(javaRow)];
final long writtenLength =
@@ -138,7 +138,7 @@ public final class UnsafeFixedWidthAggregationMap {
* Return the aggregation buffer for the current group. For efficiency, all calls to this method
* return the same object.
*/
- public UnsafeRow getAggregationBuffer(Row groupingKey) {
+ public UnsafeRow getAggregationBuffer(InternalRow groupingKey) {
final int groupingKeySize = groupingKeyToUnsafeRowConverter.getSizeRequirement(groupingKey);
// Make sure that the buffer is large enough to hold the key. If it's not, grow it:
if (groupingKeySize > groupingKeyConversionScratchSpace.length) {
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
index 143acc9f5e..aec88c9241 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
@@ -26,7 +26,7 @@ import java.util.Set;
import scala.collection.Seq;
import scala.collection.mutable.ArraySeq;
-import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.BaseMutableRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.StructType;
@@ -334,7 +334,7 @@ public final class UnsafeRow extends BaseMutableRow {
@Override
- public Row copy() {
+ public InternalRow copy() {
throw new UnsupportedOperationException();
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
index 7e4b11a495..6175456c58 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.HashMap
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.DateUtils
+import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -105,7 +106,7 @@ object CatalystTypeConverters {
/**
* Given a Catalyst row, convert the value at column `column` to its Scala equivalent.
*/
- final def toScala(row: Row, column: Int): ScalaOutputType = {
+ final def toScala(row: InternalRow, column: Int): ScalaOutputType = {
if (row.isNullAt(column)) null.asInstanceOf[ScalaOutputType] else toScalaImpl(row, column)
}
@@ -125,20 +126,20 @@ object CatalystTypeConverters {
* Given a Catalyst row, convert the value at column `column` to its Scala equivalent.
* This method will only be called on non-null columns.
*/
- protected def toScalaImpl(row: Row, column: Int): ScalaOutputType
+ protected def toScalaImpl(row: InternalRow, column: Int): ScalaOutputType
}
private object IdentityConverter extends CatalystTypeConverter[Any, Any, Any] {
override def toCatalystImpl(scalaValue: Any): Any = scalaValue
override def toScala(catalystValue: Any): Any = catalystValue
- override def toScalaImpl(row: Row, column: Int): Any = row(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Any = row(column)
}
private case class UDTConverter(
udt: UserDefinedType[_]) extends CatalystTypeConverter[Any, Any, Any] {
override def toCatalystImpl(scalaValue: Any): Any = udt.serialize(scalaValue)
override def toScala(catalystValue: Any): Any = udt.deserialize(catalystValue)
- override def toScalaImpl(row: Row, column: Int): Any = toScala(row(column))
+ override def toScalaImpl(row: InternalRow, column: Int): Any = toScala(row(column))
}
/** Converter for arrays, sequences, and Java iterables. */
@@ -170,7 +171,7 @@ object CatalystTypeConverters {
}
}
- override def toScalaImpl(row: Row, column: Int): Seq[Any] =
+ override def toScalaImpl(row: InternalRow, column: Int): Seq[Any] =
toScala(row(column).asInstanceOf[Seq[Any]])
}
@@ -209,16 +210,16 @@ object CatalystTypeConverters {
}
}
- override def toScalaImpl(row: Row, column: Int): Map[Any, Any] =
+ override def toScalaImpl(row: InternalRow, column: Int): Map[Any, Any] =
toScala(row(column).asInstanceOf[Map[Any, Any]])
}
private case class StructConverter(
- structType: StructType) extends CatalystTypeConverter[Any, Row, Row] {
+ structType: StructType) extends CatalystTypeConverter[Any, Row, InternalRow] {
private[this] val converters = structType.fields.map { f => getConverterForType(f.dataType) }
- override def toCatalystImpl(scalaValue: Any): Row = scalaValue match {
+ override def toCatalystImpl(scalaValue: Any): InternalRow = scalaValue match {
case row: Row =>
val ar = new Array[Any](row.size)
var idx = 0
@@ -239,7 +240,7 @@ object CatalystTypeConverters {
new GenericRowWithSchema(ar, structType)
}
- override def toScala(row: Row): Row = {
+ override def toScala(row: InternalRow): Row = {
if (row == null) {
null
} else {
@@ -253,7 +254,8 @@ object CatalystTypeConverters {
}
}
- override def toScalaImpl(row: Row, column: Int): Row = toScala(row(column).asInstanceOf[Row])
+ override def toScalaImpl(row: InternalRow, column: Int): Row =
+ toScala(row(column).asInstanceOf[InternalRow])
}
private object StringConverter extends CatalystTypeConverter[Any, String, Any] {
@@ -266,14 +268,14 @@ object CatalystTypeConverters {
case str: String => str
case utf8: UTF8String => utf8.toString()
}
- override def toScalaImpl(row: Row, column: Int): String = row(column).toString
+ override def toScalaImpl(row: InternalRow, column: Int): String = row(column).toString
}
private object DateConverter extends CatalystTypeConverter[Date, Date, Any] {
override def toCatalystImpl(scalaValue: Date): Int = DateUtils.fromJavaDate(scalaValue)
override def toScala(catalystValue: Any): Date =
if (catalystValue == null) null else DateUtils.toJavaDate(catalystValue.asInstanceOf[Int])
- override def toScalaImpl(row: Row, column: Int): Date = toScala(row.getInt(column))
+ override def toScalaImpl(row: InternalRow, column: Int): Date = toScala(row.getInt(column))
}
private object TimestampConverter extends CatalystTypeConverter[Timestamp, Timestamp, Any] {
@@ -282,7 +284,8 @@ object CatalystTypeConverters {
override def toScala(catalystValue: Any): Timestamp =
if (catalystValue == null) null
else DateUtils.toJavaTimestamp(catalystValue.asInstanceOf[Long])
- override def toScalaImpl(row: Row, column: Int): Timestamp = toScala(row.getLong(column))
+ override def toScalaImpl(row: InternalRow, column: Int): Timestamp =
+ toScala(row.getLong(column))
}
private object BigDecimalConverter extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] {
@@ -292,10 +295,11 @@ object CatalystTypeConverters {
case d: Decimal => d
}
override def toScala(catalystValue: Decimal): JavaBigDecimal = catalystValue.toJavaBigDecimal
- override def toScalaImpl(row: Row, column: Int): JavaBigDecimal = row.get(column) match {
- case d: JavaBigDecimal => d
- case d: Decimal => d.toJavaBigDecimal
- }
+ override def toScalaImpl(row: InternalRow, column: Int): JavaBigDecimal =
+ row.get(column) match {
+ case d: JavaBigDecimal => d
+ case d: Decimal => d.toJavaBigDecimal
+ }
}
private abstract class PrimitiveConverter[T] extends CatalystTypeConverter[T, Any, Any] {
@@ -304,31 +308,31 @@ object CatalystTypeConverters {
}
private object BooleanConverter extends PrimitiveConverter[Boolean] {
- override def toScalaImpl(row: Row, column: Int): Boolean = row.getBoolean(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Boolean = row.getBoolean(column)
}
private object ByteConverter extends PrimitiveConverter[Byte] {
- override def toScalaImpl(row: Row, column: Int): Byte = row.getByte(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Byte = row.getByte(column)
}
private object ShortConverter extends PrimitiveConverter[Short] {
- override def toScalaImpl(row: Row, column: Int): Short = row.getShort(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Short = row.getShort(column)
}
private object IntConverter extends PrimitiveConverter[Int] {
- override def toScalaImpl(row: Row, column: Int): Int = row.getInt(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Int = row.getInt(column)
}
private object LongConverter extends PrimitiveConverter[Long] {
- override def toScalaImpl(row: Row, column: Int): Long = row.getLong(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Long = row.getLong(column)
}
private object FloatConverter extends PrimitiveConverter[Float] {
- override def toScalaImpl(row: Row, column: Int): Float = row.getFloat(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Float = row.getFloat(column)
}
private object DoubleConverter extends PrimitiveConverter[Double] {
- override def toScalaImpl(row: Row, column: Int): Double = row.getDouble(column)
+ override def toScalaImpl(row: InternalRow, column: Int): Double = row.getDouble(column)
}
/**
@@ -382,7 +386,7 @@ object CatalystTypeConverters {
case d: BigDecimal => BigDecimalConverter.toCatalyst(d)
case d: JavaBigDecimal => BigDecimalConverter.toCatalyst(d)
case seq: Seq[Any] => seq.map(convertToCatalyst)
- case r: Row => Row(r.toSeq.map(convertToCatalyst): _*)
+ case r: Row => InternalRow(r.toSeq.map(convertToCatalyst): _*)
case arr: Array[Any] => arr.toSeq.map(convertToCatalyst).toArray
case m: Map[Any, Any] =>
m.map { case (k, v) => (convertToCatalyst(k), convertToCatalyst(v)) }.toMap
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
new file mode 100644
index 0000000000..e3c2cc2433
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala
@@ -0,0 +1,57 @@
+/*
+ * 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.spark.sql.catalyst
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.GenericRow
+
+/**
+ * An abstract class for row used internal in Spark SQL, which only contain the columns as
+ * internal types.
+ */
+abstract class InternalRow extends Row {
+ // A default implementation to change the return type
+ override def copy(): InternalRow = {this}
+}
+
+object InternalRow {
+ def unapplySeq(row: InternalRow): Some[Seq[Any]] = Some(row.toSeq)
+
+ /**
+ * This method can be used to construct a [[Row]] with the given values.
+ */
+ def apply(values: Any*): InternalRow = new GenericRow(values.toArray)
+
+ /**
+ * This method can be used to construct a [[Row]] from a [[Seq]] of values.
+ */
+ def fromSeq(values: Seq[Any]): InternalRow = new GenericRow(values.toArray)
+
+ def fromTuple(tuple: Product): InternalRow = fromSeq(tuple.productIterator.toSeq)
+
+ /**
+ * Merge multiple rows into a single row, one after another.
+ */
+ def merge(rows: InternalRow*): InternalRow = {
+ // TODO: Improve the performance of this if used in performance critical part.
+ new GenericRow(rows.flatMap(_.toSeq).toArray)
+ }
+
+ /** Returns an empty row. */
+ val empty = apply()
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index bbb150c1e8..5de188d418 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.{errors, trees}
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
@@ -67,7 +68,7 @@ case class UnresolvedAttribute(nameParts: Seq[String])
override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName)
// Unresolved attributes are transient at compile time and don't get evaluated during execution.
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"'$name"
@@ -85,7 +86,7 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E
override lazy val resolved = false
// Unresolved functions are transient at compile time and don't get evaluated during execution.
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"'$name(${children.mkString(",")})"
@@ -107,7 +108,7 @@ trait Star extends NamedExpression with trees.LeafNode[Expression] {
override lazy val resolved = false
// Star gets expanded at runtime so we never evaluate a Star.
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression]
@@ -166,7 +167,7 @@ case class MultiAlias(child: Expression, names: Seq[String])
override lazy val resolved = false
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"$child AS $names"
@@ -200,7 +201,7 @@ case class UnresolvedExtractValue(child: Expression, extraction: Expression)
override def nullable: Boolean = throw new UnresolvedException(this, "nullable")
override lazy val resolved = false
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"$child[$extraction]"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index fcadf9595e..c4dd11a451 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -21,7 +21,7 @@ import org.apache.spark.Logging
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
import org.apache.spark.sql.types._
-import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst.{InternalRow, trees}
/**
* A bound reference points to a specific slot in the input tuple, allowing the actual value
@@ -33,7 +33,7 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
override def toString: String = s"input[$ordinal]"
- override def eval(input: Row): Any = input(ordinal)
+ override def eval(input: InternalRow): Any = input(ordinal)
override def name: String = s"i[$ordinal]"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 4c7123fcb7..afbf30af33 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -21,6 +21,7 @@ import java.sql.{Date, Timestamp}
import java.text.{DateFormat, SimpleDateFormat}
import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.catalyst.util.DateUtils
import org.apache.spark.sql.types._
@@ -393,7 +394,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
}
// TODO: Could be faster?
val newRow = new GenericMutableRow(from.fields.size)
- buildCast[Row](_, row => {
+ buildCast[catalyst.InternalRow](_, row => {
var i = 0
while (i < row.length) {
val v = row(i)
@@ -425,7 +426,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
private[this] lazy val cast: Any => Any = cast(child.dataType, dataType)
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evaluated = child.eval(input)
if (evaluated == null) null else cast(evaluated)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 0b9f621fed..61de34bfa4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedAttribute}
-import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.types._
@@ -59,7 +59,7 @@ abstract class Expression extends TreeNode[Expression] {
def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator))
/** Returns the result of evaluating this expression on a given input Row */
- def eval(input: Row = null): Any
+ def eval(input: InternalRow = null): Any
/**
* Returns an [[GeneratedExpressionCode]], which contains Java source code that
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala
index a1e0819e8a..16f3ccc3d6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import scala.collection.Map
-import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.{catalyst, AnalysisException}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.types._
@@ -105,8 +105,8 @@ case class GetStructField(child: Expression, field: StructField, ordinal: Int)
override def foldable: Boolean = child.foldable
override def toString: String = s"$child.${field.name}"
- override def eval(input: Row): Any = {
- val baseValue = child.eval(input).asInstanceOf[Row]
+ override def eval(input: catalyst.InternalRow): Any = {
+ val baseValue = child.eval(input).asInstanceOf[catalyst.InternalRow]
if (baseValue == null) null else baseValue(ordinal)
}
}
@@ -125,8 +125,8 @@ case class GetArrayStructFields(
override def foldable: Boolean = child.foldable
override def toString: String = s"$child.${field.name}"
- override def eval(input: Row): Any = {
- val baseValue = child.eval(input).asInstanceOf[Seq[Row]]
+ override def eval(input: catalyst.InternalRow): Any = {
+ val baseValue = child.eval(input).asInstanceOf[Seq[catalyst.InternalRow]]
if (baseValue == null) null else {
baseValue.map { row =>
if (row == null) null else row(ordinal)
@@ -146,7 +146,7 @@ abstract class ExtractValueWithOrdinal extends ExtractValue {
override def toString: String = s"$child[$ordinal]"
override def children: Seq[Expression] = child :: ordinal :: Nil
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val value = child.eval(input)
if (value == null) {
null
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
index 8cae548279..d6806f78ab 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
+
/**
* A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions.
@@ -30,7 +32,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection {
// null check is required for when Kryo invokes the no-arg constructor.
protected val exprArray = if (expressions != null) expressions.toArray else null
- def apply(input: Row): Row = {
+ def apply(input: catalyst.InternalRow): catalyst.InternalRow = {
val outputArray = new Array[Any](exprArray.length)
var i = 0
while (i < exprArray.length) {
@@ -55,14 +57,14 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu
private[this] val exprArray = expressions.toArray
private[this] var mutableRow: MutableRow = new GenericMutableRow(exprArray.size)
- def currentValue: Row = mutableRow
+ def currentValue: catalyst.InternalRow = mutableRow
override def target(row: MutableRow): MutableProjection = {
mutableRow = row
this
}
- override def apply(input: Row): Row = {
+ override def apply(input: catalyst.InternalRow): catalyst.InternalRow = {
var i = 0
while (i < exprArray.length) {
mutableRow(i) = exprArray(i).eval(input)
@@ -76,31 +78,31 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu
* A mutable wrapper that makes two rows appear as a single concatenated row. Designed to
* be instantiated once per thread and reused.
*/
-class JoinedRow extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -142,7 +144,7 @@ class JoinedRow extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -176,31 +178,31 @@ class JoinedRow extends Row {
* Row will be referenced, increasing the opportunity for the JIT to play tricks. This sounds
* crazy but in benchmarks it had noticeable effects.
*/
-class JoinedRow2 extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow2 extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -242,7 +244,7 @@ class JoinedRow2 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -270,31 +272,31 @@ class JoinedRow2 extends Row {
/**
* JIT HACK: Replace with macros
*/
-class JoinedRow3 extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow3 extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -336,7 +338,7 @@ class JoinedRow3 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -364,31 +366,31 @@ class JoinedRow3 extends Row {
/**
* JIT HACK: Replace with macros
*/
-class JoinedRow4 extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow4 extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -430,7 +432,7 @@ class JoinedRow4 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -458,31 +460,31 @@ class JoinedRow4 extends Row {
/**
* JIT HACK: Replace with macros
*/
-class JoinedRow5 extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow5 extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -524,7 +526,7 @@ class JoinedRow5 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
@@ -552,31 +554,31 @@ class JoinedRow5 extends Row {
/**
* JIT HACK: Replace with macros
*/
-class JoinedRow6 extends Row {
- private[this] var row1: Row = _
- private[this] var row2: Row = _
+class JoinedRow6 extends catalyst.InternalRow {
+ private[this] var row1: catalyst.InternalRow = _
+ private[this] var row2: catalyst.InternalRow = _
- def this(left: Row, right: Row) = {
+ def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = {
this()
row1 = left
row2 = right
}
/** Updates this JoinedRow to used point at two new base rows. Returns itself. */
- def apply(r1: Row, r2: Row): Row = {
+ def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = {
row1 = r1
row2 = r2
this
}
/** Updates this JoinedRow by updating its left base row. Returns itself. */
- def withLeft(newLeft: Row): Row = {
+ def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = {
row1 = newLeft
this
}
/** Updates this JoinedRow by updating its right base row. Returns itself. */
- def withRight(newRight: Row): Row = {
+ def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = {
row2 = newRight
this
}
@@ -618,7 +620,7 @@ class JoinedRow6 extends Row {
override def getAs[T](i: Int): T =
if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length)
- override def copy(): Row = {
+ override def copy(): catalyst.InternalRow = {
val totalSize = row1.length + row2.length
val copiedValues = new Array[Any](totalSize)
var i = 0
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
index 5b45347872..40f235fc19 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.types.DataType
@@ -45,7 +46,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
val func = function.asInstanceOf[($anys) => Any]
$childs
$converters
- (input: Row) => {
+ (input: InternalRow) => {
func(
$evals)
}
@@ -57,7 +58,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
private[this] val f = children.size match {
case 0 =>
val func = function.asInstanceOf[() => Any]
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func()
}
@@ -65,7 +66,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
val func = function.asInstanceOf[(Any) => Any]
val child0 = children(0)
lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)))
}
@@ -76,7 +77,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
val child1 = children(1)
lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType)
lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)))
@@ -90,7 +91,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType)
lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType)
lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -107,7 +108,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType)
lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType)
lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -127,7 +128,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType)
lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType)
lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -150,7 +151,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType)
lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType)
lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -176,7 +177,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType)
lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType)
lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -205,7 +206,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType)
lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType)
lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -237,7 +238,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType)
lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType)
lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -272,7 +273,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType)
lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType)
lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -310,7 +311,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType)
lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType)
lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -351,7 +352,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType)
lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType)
lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -395,7 +396,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType)
lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType)
lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -442,7 +443,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType)
lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType)
lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -492,7 +493,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType)
lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType)
lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -545,7 +546,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType)
lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType)
lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -601,7 +602,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType)
lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType)
lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -660,7 +661,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType)
lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType)
lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -722,7 +723,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType)
lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType)
lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -787,7 +788,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType)
lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType)
lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -855,7 +856,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType)
lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType)
lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -926,7 +927,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType)
lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType)
lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType)
- (input: Row) => {
+ (input: catalyst.InternalRow) => {
func(
converter0(child0.eval(input)),
converter1(child1.eval(input)),
@@ -955,6 +956,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
// scalastyle:on
private[this] val converter = CatalystTypeConverters.createToCatalystConverter(dataType)
- override def eval(input: Row): Any = converter(f(input))
+ override def eval(input: catalyst.InternalRow): Any = converter(f(input))
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index 99340a14c9..8a34355999 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst.{InternalRow, trees}
import org.apache.spark.sql.types.DataType
abstract sealed class SortDirection
@@ -36,7 +36,7 @@ case class SortOrder(child: Expression, direction: SortDirection) extends Expres
override def nullable: Boolean = child.nullable
// SortOrder itself is never evaluated.
- override def eval(input: Row = null): Any =
+ override def eval(input: InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"$child ${if (direction == Ascending) "ASC" else "DESC"}"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
index 98eda61a80..05aab34559 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala
@@ -222,7 +222,7 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR
override def isNullAt(i: Int): Boolean = values(i).isNull
- override def copy(): Row = {
+ override def copy(): InternalRow = {
val newValues = new Array[Any](values.length)
var i = 0
while (i < values.length) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
index 5350123bf4..d771e454b5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
@@ -48,7 +48,7 @@ class UnsafeRowConverter(fieldTypes: Array[DataType]) {
/**
* Compute the amount of space, in bytes, required to encode the given row.
*/
- def getSizeRequirement(row: Row): Int = {
+ def getSizeRequirement(row: InternalRow): Int = {
var fieldNumber = 0
var variableLengthFieldSize: Int = 0
while (fieldNumber < writers.length) {
@@ -68,7 +68,7 @@ class UnsafeRowConverter(fieldTypes: Array[DataType]) {
* @param baseOffset the base offset of the destination address
* @return the number of bytes written. This should be equal to `getSizeRequirement(row)`.
*/
- def writeRow(row: Row, baseObject: Object, baseOffset: Long): Long = {
+ def writeRow(row: InternalRow, baseObject: Object, baseOffset: Long): Long = {
unsafeRow.pointTo(baseObject, baseOffset, writers.length, null)
var fieldNumber = 0
var appendCursor: Int = fixedLengthSize
@@ -99,12 +99,12 @@ private abstract class UnsafeColumnWriter {
* used for calculating where variable-length data should be written
* @return the number of variable-length bytes written
*/
- def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int
+ def write(source: InternalRow, target: UnsafeRow, column: Int, appendCursor: Int): Int
/**
* Return the number of bytes that are needed to write this variable-length value.
*/
- def getSize(source: Row, column: Int): Int
+ def getSize(source: InternalRow, column: Int): Int
}
private object UnsafeColumnWriter {
@@ -140,72 +140,108 @@ private object StringUnsafeColumnWriter extends StringUnsafeColumnWriter
private abstract class PrimitiveUnsafeColumnWriter extends UnsafeColumnWriter {
// Primitives don't write to the variable-length region:
- def getSize(sourceRow: Row, column: Int): Int = 0
+ def getSize(sourceRow: InternalRow, column: Int): Int = 0
}
private class NullUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setNullAt(column)
0
}
}
private class BooleanUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setBoolean(column, source.getBoolean(column))
0
}
}
private class ByteUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setByte(column, source.getByte(column))
0
}
}
private class ShortUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setShort(column, source.getShort(column))
0
}
}
private class IntUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setInt(column, source.getInt(column))
0
}
}
private class LongUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setLong(column, source.getLong(column))
0
}
}
private class FloatUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setFloat(column, source.getFloat(column))
0
}
}
private class DoubleUnsafeColumnWriter private() extends PrimitiveUnsafeColumnWriter {
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
target.setDouble(column, source.getDouble(column))
0
}
}
private class StringUnsafeColumnWriter private() extends UnsafeColumnWriter {
- def getSize(source: Row, column: Int): Int = {
+ def getSize(source: InternalRow, column: Int): Int = {
val numBytes = source.get(column).asInstanceOf[UTF8String].getBytes.length
8 + ByteArrayMethods.roundNumberOfBytesToNearestWord(numBytes)
}
- override def write(source: Row, target: UnsafeRow, column: Int, appendCursor: Int): Int = {
+ override def write(
+ source: InternalRow,
+ target: UnsafeRow,
+ column: Int,
+ appendCursor: Int): Int = {
val value = source.get(column).asInstanceOf[UTF8String]
val baseObject = target.getBaseObject
val baseOffset = target.getBaseOffset
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index 0266084a6d..f9e8150a68 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -19,9 +19,10 @@ package org.apache.spark.sql.catalyst.expressions
import com.clearspring.analytics.stream.cardinality.HyperLogLog
-import org.apache.spark.sql.types._
-import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.errors.TreeNodeException
+import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.types._
import org.apache.spark.util.collection.OpenHashSet
abstract class AggregateExpression extends Expression {
@@ -37,7 +38,7 @@ abstract class AggregateExpression extends Expression {
* [[AggregateExpression.eval]] should never be invoked because [[AggregateExpression]]'s are
* replaced with a physical aggregate operator at runtime.
*/
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
}
@@ -80,7 +81,7 @@ abstract class AggregateFunction
override def nullable: Boolean = base.nullable
override def dataType: DataType = base.dataType
- def update(input: Row): Unit
+ def update(input: catalyst.InternalRow): Unit
// Do we really need this?
override def newInstance(): AggregateFunction = {
@@ -108,7 +109,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr
val currentMin: MutableLiteral = MutableLiteral(null, expr.dataType)
val cmp = GreaterThan(currentMin, expr)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
if (currentMin.value == null) {
currentMin.value = expr.eval(input)
} else if (cmp.eval(input) == true) {
@@ -116,7 +117,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr
}
}
- override def eval(input: Row): Any = currentMin.value
+ override def eval(input: catalyst.InternalRow): Any = currentMin.value
}
case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
@@ -139,7 +140,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr
val currentMax: MutableLiteral = MutableLiteral(null, expr.dataType)
val cmp = LessThan(currentMax, expr)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
if (currentMax.value == null) {
currentMax.value = expr.eval(input)
} else if (cmp.eval(input) == true) {
@@ -147,7 +148,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr
}
}
- override def eval(input: Row): Any = currentMax.value
+ override def eval(input: catalyst.InternalRow): Any = currentMax.value
}
case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
@@ -205,14 +206,14 @@ case class CollectHashSetFunction(
@transient
val distinctValue = new InterpretedProjection(expr)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = distinctValue(input)
if (!evaluatedExpr.anyNull) {
seen.add(evaluatedExpr)
}
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
seen
}
}
@@ -238,7 +239,7 @@ case class CombineSetsAndCountFunction(
val seen = new OpenHashSet[Any]()
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]]
val inputIterator = inputSetEval.iterator
while (inputIterator.hasNext) {
@@ -246,7 +247,7 @@ case class CombineSetsAndCountFunction(
}
}
- override def eval(input: Row): Any = seen.size.toLong
+ override def eval(input: catalyst.InternalRow): Any = seen.size.toLong
}
/** The data type of ApproxCountDistinctPartition since its output is a HyperLogLog object. */
@@ -453,7 +454,7 @@ case class CombineSetsAndSumFunction(
val seen = new OpenHashSet[Any]()
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]]
val inputIterator = inputSetEval.iterator
while (inputIterator.hasNext) {
@@ -461,8 +462,8 @@ case class CombineSetsAndSumFunction(
}
}
- override def eval(input: Row): Any = {
- val casted = seen.asInstanceOf[OpenHashSet[Row]]
+ override def eval(input: catalyst.InternalRow): Any = {
+ val casted = seen.asInstanceOf[OpenHashSet[catalyst.InternalRow]]
if (casted.size == 0) {
null
} else {
@@ -524,7 +525,7 @@ case class AverageFunction(expr: Expression, base: AggregateExpression)
private def addFunction(value: Any) = Add(sum,
Cast(Literal.create(value, expr.dataType), calcType))
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
if (count == 0L) {
null
} else {
@@ -541,7 +542,7 @@ case class AverageFunction(expr: Expression, base: AggregateExpression)
}
}
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = expr.eval(input)
if (evaluatedExpr != null) {
count += 1
@@ -555,14 +556,14 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag
var count: Long = _
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = expr.eval(input)
if (evaluatedExpr != null) {
count += 1L
}
}
- override def eval(input: Row): Any = count
+ override def eval(input: catalyst.InternalRow): Any = count
}
case class ApproxCountDistinctPartitionFunction(
@@ -574,14 +575,14 @@ case class ApproxCountDistinctPartitionFunction(
private val hyperLogLog = new HyperLogLog(relativeSD)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = expr.eval(input)
if (evaluatedExpr != null) {
hyperLogLog.offer(evaluatedExpr)
}
}
- override def eval(input: Row): Any = hyperLogLog
+ override def eval(input: catalyst.InternalRow): Any = hyperLogLog
}
case class ApproxCountDistinctMergeFunction(
@@ -593,12 +594,12 @@ case class ApproxCountDistinctMergeFunction(
private val hyperLogLog = new HyperLogLog(relativeSD)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = expr.eval(input)
hyperLogLog.addAll(evaluatedExpr.asInstanceOf[HyperLogLog])
}
- override def eval(input: Row): Any = hyperLogLog.cardinality()
+ override def eval(input: catalyst.InternalRow): Any = hyperLogLog.cardinality()
}
case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
@@ -619,11 +620,11 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr
private val addFunction =
Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero))
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
sum.update(addFunction, input)
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
expr.dataType match {
case DecimalType.Fixed(_, _) =>
Cast(sum, dataType).eval(null)
@@ -652,7 +653,7 @@ case class CombineSumFunction(expr: Expression, base: AggregateExpression)
private val addFunction =
Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero))
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val result = expr.eval(input)
// partial sum result can be null only when no input rows present
if(result != null) {
@@ -660,7 +661,7 @@ case class CombineSumFunction(expr: Expression, base: AggregateExpression)
}
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
expr.dataType match {
case DecimalType.Fixed(_, _) =>
Cast(sum, dataType).eval(null)
@@ -676,14 +677,14 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression)
private val seen = new scala.collection.mutable.HashSet[Any]()
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = expr.eval(input)
if (evaluatedExpr != null) {
seen += evaluatedExpr
}
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
if (seen.size == 0) {
null
} else {
@@ -707,14 +708,14 @@ case class CountDistinctFunction(
@transient
val distinctValue = new InterpretedProjection(expr)
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
val evaluatedExpr = distinctValue(input)
if (!evaluatedExpr.anyNull) {
seen.add(evaluatedExpr)
}
}
- override def eval(input: Row): Any = seen.size.toLong
+ override def eval(input: catalyst.InternalRow): Any = seen.size.toLong
}
case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
@@ -722,13 +723,13 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag
var result: Any = null
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
if (result == null) {
result = expr.eval(input)
}
}
- override def eval(input: Row): Any = result
+ override def eval(input: catalyst.InternalRow): Any = result
}
case class LastFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction {
@@ -736,11 +737,11 @@ case class LastFunction(expr: Expression, base: AggregateExpression) extends Agg
var result: Any = null
- override def update(input: Row): Unit = {
+ override def update(input: catalyst.InternalRow): Unit = {
result = input
}
- override def eval(input: Row): Any = {
- if (result != null) expr.eval(result.asInstanceOf[Row]) else null
+ override def eval(input: catalyst.InternalRow): Any = {
+ if (result != null) expr.eval(result.asInstanceOf[catalyst.InternalRow]) else null
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 124274c942..0ba2ff75aa 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -17,8 +17,9 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
-import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.types._
@@ -29,7 +30,7 @@ abstract class UnaryArithmetic extends UnaryExpression {
override def nullable: Boolean = child.nullable
override def dataType: DataType = child.dataType
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE = child.eval(input)
if (evalE == null) {
null
@@ -124,7 +125,7 @@ abstract class BinaryArithmetic extends BinaryExpression {
protected def checkTypesInternal(t: DataType): TypeCheckResult
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE1 = left.eval(input)
if(evalE1 == null) {
null
@@ -219,7 +220,7 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic
case it: IntegralType => it.integral.asInstanceOf[Integral[Any]].quot
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE2 = right.eval(input)
if (evalE2 == null || evalE2 == 0) {
null
@@ -279,7 +280,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet
case i: FractionalType => i.asIntegral.asInstanceOf[Integral[Any]]
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE2 = right.eval(input)
if (evalE2 == null || evalE2 == 0) {
null
@@ -330,7 +331,7 @@ case class MaxOf(left: Expression, right: Expression) extends BinaryArithmetic {
private lazy val ordering = TypeUtils.getOrdering(dataType)
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE1 = left.eval(input)
val evalE2 = right.eval(input)
if (evalE1 == null) {
@@ -384,7 +385,7 @@ case class MinOf(left: Expression, right: Expression) extends BinaryArithmetic {
private lazy val ordering = TypeUtils.getOrdering(dataType)
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE1 = left.eval(input)
val evalE2 = right.eval(input)
if (evalE1 == null) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
index 536e477330..244a06638f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
@@ -24,6 +24,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader}
import org.codehaus.janino.ClassBodyEvaluator
import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -34,7 +35,7 @@ class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int]
class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long]
/**
- * Java source for evaluating an [[Expression]] given a [[Row]] of input.
+ * Java source for evaluating an [[Expression]] given a [[catalyst.InternalRow]] of input.
*
* @param code The sequence of statements required to evaluate the expression.
* @param isNull A term that holds a boolean value representing whether the expression evaluated
@@ -183,13 +184,13 @@ class CodeGenContext {
}
/**
- * List of data types that have special accessors and setters in [[Row]].
+ * List of data types that have special accessors and setters in [[catalyst.InternalRow]].
*/
val nativeTypes =
Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType)
/**
- * Returns true if the data type has a special accessor and setter in [[Row]].
+ * Returns true if the data type has a special accessor and setter in [[catalyst.InternalRow]].
*/
def isNativeType(dt: DataType): Boolean = nativeTypes.contains(dt)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
index ed3df547d1..35cb954c54 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions.codegen
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions._
// MutableProjection is not accessible in Java
@@ -24,7 +25,7 @@ abstract class BaseMutableProjection extends MutableProjection {}
/**
* Generates byte code that produces a [[MutableRow]] object that can update itself based on a new
- * input [[Row]] for a fixed set of [[Expression Expressions]].
+ * input [[catalyst.InternalRow]] for a fixed set of [[Expression Expressions]].
*/
object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] {
@@ -47,7 +48,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu
"""
}.mkString("\n")
val code = s"""
- import org.apache.spark.sql.Row;
+ import org.apache.spark.sql.catalyst.InternalRow;
public SpecificProjection generate($exprType[] expr) {
return new SpecificProjection(expr);
@@ -69,12 +70,12 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu
}
/* Provide immutable access to the last projected row. */
- public Row currentValue() {
- return mutableRow;
+ public InternalRow currentValue() {
+ return (InternalRow) mutableRow;
}
public Object apply(Object _i) {
- Row i = (Row) _i;
+ InternalRow i = (InternalRow) _i;
$projectionCode
return mutableRow;
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
index 56ecc5fc06..db5d570aeb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
@@ -19,15 +19,15 @@ package org.apache.spark.sql.catalyst.expressions.codegen
import org.apache.spark.Logging
import org.apache.spark.annotation.Private
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{catalyst, Row}
import org.apache.spark.sql.catalyst.expressions._
/**
* Inherits some default implementation for Java from `Ordering[Row]`
*/
@Private
-class BaseOrdering extends Ordering[Row] {
- def compare(a: Row, b: Row): Int = {
+class BaseOrdering extends Ordering[catalyst.InternalRow] {
+ def compare(a: catalyst.InternalRow, b: catalyst.InternalRow): Int = {
throw new UnsupportedOperationException
}
}
@@ -36,7 +36,8 @@ class BaseOrdering extends Ordering[Row] {
* Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of
* [[Expression Expressions]].
*/
-object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] with Logging {
+object GenerateOrdering
+ extends CodeGenerator[Seq[SortOrder], Ordering[catalyst.InternalRow]] with Logging {
import scala.reflect.runtime.universe._
protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] =
@@ -45,7 +46,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] =
in.map(BindReferences.bindReference(_, inputSchema))
- protected def create(ordering: Seq[SortOrder]): Ordering[Row] = {
+ protected def create(ordering: Seq[SortOrder]): Ordering[catalyst.InternalRow] = {
val a = newTermName("a")
val b = newTermName("b")
val ctx = newCodeGenContext()
@@ -75,7 +76,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
}.mkString("\n")
val code = s"""
- import org.apache.spark.sql.Row;
+ import org.apache.spark.sql.catalyst.InternalRow;
public SpecificOrdering generate($exprType[] expr) {
return new SpecificOrdering(expr);
@@ -90,8 +91,8 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit
}
@Override
- public int compare(Row a, Row b) {
- Row i = null; // Holds current row being evaluated.
+ public int compare(InternalRow a, InternalRow b) {
+ InternalRow i = null; // Holds current row being evaluated.
$comparisons
return 0;
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala
index 4a547b5ce9..9e191dc2e9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala
@@ -17,30 +17,31 @@
package org.apache.spark.sql.catalyst.expressions.codegen
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions._
/**
* Interface for generated predicate
*/
abstract class Predicate {
- def eval(r: Row): Boolean
+ def eval(r: catalyst.InternalRow): Boolean
}
/**
- * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[Row]].
+ * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[InternalRow]].
*/
-object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] {
+object GeneratePredicate extends CodeGenerator[Expression, (catalyst.InternalRow) => Boolean] {
protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in)
protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression =
BindReferences.bindReference(in, inputSchema)
- protected def create(predicate: Expression): ((Row) => Boolean) = {
+ protected def create(predicate: Expression): ((catalyst.InternalRow) => Boolean) = {
val ctx = newCodeGenContext()
val eval = predicate.gen(ctx)
val code = s"""
- import org.apache.spark.sql.Row;
+ import org.apache.spark.sql.catalyst.InternalRow;
public SpecificPredicate generate($exprType[] expr) {
return new SpecificPredicate(expr);
@@ -53,7 +54,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] {
}
@Override
- public boolean eval(Row i) {
+ public boolean eval(InternalRow i) {
${eval.code}
return !${eval.isNull} && ${eval.primitive};
}
@@ -65,6 +66,6 @@ object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] {
// fetch the only one method `generate(Expression[])`
val m = c.getDeclaredMethods()(0)
val p = m.invoke(c.newInstance(), ctx.references.toArray).asInstanceOf[Predicate]
- (r: Row) => p.eval(r)
+ (r: catalyst.InternalRow) => p.eval(r)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
index 9b906c3ff5..8b5dc194be 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
@@ -27,9 +27,10 @@ import org.apache.spark.sql.types._
abstract class BaseProject extends Projection {}
/**
- * Generates bytecode that produces a new [[Row]] object based on a fixed set of input
- * [[Expression Expressions]] and a given input [[Row]]. The returned [[Row]] object is custom
- * generated based on the output types of the [[Expression]] to avoid boxing of primitive values.
+ * Generates bytecode that produces a new [[InternalRow]] object based on a fixed set of input
+ * [[Expression Expressions]] and a given input [[InternalRow]]. The returned [[InternalRow]]
+ * object is custom generated based on the output types of the [[Expression]] to avoid boxing of
+ * primitive values.
*/
object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
import scala.reflect.runtime.universe._
@@ -146,7 +147,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
}.mkString("\n")
val code = s"""
- import org.apache.spark.sql.Row;
+ import org.apache.spark.sql.catalyst.InternalRow;
public SpecificProjection generate($exprType[] expr) {
return new SpecificProjection(expr);
@@ -161,7 +162,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
@Override
public Object apply(Object r) {
- return new SpecificRow(expressions, (Row) r);
+ return new SpecificRow(expressions, (InternalRow) r);
}
}
@@ -169,7 +170,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] {
$columns
- public SpecificRow($exprType[] expressions, Row i) {
+ public SpecificRow($exprType[] expressions, InternalRow i) {
$initColumns
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index 6398b8f9e4..a6913cc03c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.types._
@@ -41,7 +42,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression {
override def nullable: Boolean = false
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
children.map(_.eval(input))
}
@@ -69,7 +70,7 @@ case class CreateStruct(children: Seq[NamedExpression]) extends Expression {
override def nullable: Boolean = false
- override def eval(input: Row): Any = {
- Row(children.map(_.eval(input)): _*)
+ override def eval(input: catalyst.InternalRow): Any = {
+ InternalRow(children.map(_.eval(input)): _*)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala
index 72b9f23456..a119c31300 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.types.{BooleanType, DataType}
@@ -42,7 +43,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi
override def dataType: DataType = trueValue.dataType
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
if (true == predicate.eval(input)) {
trueValue.eval(input)
} else {
@@ -137,7 +138,7 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike {
}
/** Written in imperative fashion for performance considerations. */
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val len = branchesArr.length
var i = 0
// If all branches fail and an elseVal is not provided, the whole statement
@@ -229,7 +230,7 @@ case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseW
}
/** Written in imperative fashion for performance considerations. */
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evaluatedKey = key.eval(input)
val len = branchesArr.length
var i = 0
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
index 8ab6d977dd..de8b66bc3b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala
@@ -17,7 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
+import org.apache.spark.sql.catalyst
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.types._
/** Return the unscaled Long value of a Decimal, assuming it fits in a Long */
@@ -28,7 +29,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression {
override def nullable: Boolean = child.nullable
override def toString: String = s"UnscaledValue($child)"
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val childResult = child.eval(input)
if (childResult == null) {
null
@@ -50,7 +51,7 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un
override def nullable: Boolean = child.nullable
override def toString: String = s"MakeDecimal($child,$precision,$scale)"
- override def eval(input: Row): Decimal = {
+ override def eval(input: catalyst.InternalRow): Decimal = {
val childResult = child.eval(input)
if (childResult == null) {
null
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index b6191eafba..a80c255a29 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import scala.collection.Map
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees}
import org.apache.spark.sql.types._
@@ -53,13 +54,13 @@ abstract class Generator extends Expression {
def elementTypes: Seq[(DataType, Boolean)]
/** Should be implemented by child classes to perform specific Generators. */
- override def eval(input: Row): TraversableOnce[Row]
+ override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow]
/**
* Notifies that there are no more rows to process, clean up code, and additional
* rows can be made here.
*/
- def terminate(): TraversableOnce[Row] = Nil
+ def terminate(): TraversableOnce[catalyst.InternalRow] = Nil
}
/**
@@ -67,22 +68,22 @@ abstract class Generator extends Expression {
*/
case class UserDefinedGenerator(
elementTypes: Seq[(DataType, Boolean)],
- function: Row => TraversableOnce[Row],
+ function: catalyst.InternalRow => TraversableOnce[catalyst.InternalRow],
children: Seq[Expression])
extends Generator {
@transient private[this] var inputRow: InterpretedProjection = _
- @transient private[this] var convertToScala: (Row) => Row = _
+ @transient private[this] var convertToScala: (catalyst.InternalRow) => catalyst.InternalRow = _
private def initializeConverters(): Unit = {
inputRow = new InterpretedProjection(children)
convertToScala = {
val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true)))
CatalystTypeConverters.createToScalaConverter(inputSchema)
- }.asInstanceOf[(Row => Row)]
+ }.asInstanceOf[(catalyst.InternalRow => catalyst.InternalRow)]
}
- override def eval(input: Row): TraversableOnce[Row] = {
+ override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow] = {
if (inputRow == null) {
initializeConverters()
}
@@ -108,7 +109,7 @@ case class Explode(child: Expression)
case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil
}
- override def eval(input: Row): TraversableOnce[Row] = {
+ override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow] = {
child.dataType match {
case ArrayType(_, _) =>
val inputArray = child.eval(input).asInstanceOf[Seq[Any]]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index a33007bda1..d8fff2b84d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.sql.{Date, Timestamp}
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.catalyst.util.DateUtils
@@ -87,7 +88,7 @@ case class Literal protected (value: Any, dataType: DataType) extends LeafExpres
case _ => false
}
- override def eval(input: Row): Any = value
+ override def eval(input: catalyst.InternalRow): Any = value
override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
// change the isNull and primitive to consts, to inline them
@@ -142,9 +143,9 @@ case class Literal protected (value: Any, dataType: DataType) extends LeafExpres
case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean = true)
extends LeafExpression {
- def update(expression: Expression, input: Row): Unit = {
+ def update(expression: Expression, input: catalyst.InternalRow): Unit = {
value = expression.eval(input)
}
- override def eval(input: Row): Any = value
+ override def eval(input: catalyst.InternalRow): Any = value
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
index 97e960b8d6..6f90d607dd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.types.{DataType, DoubleType}
@@ -34,7 +35,7 @@ abstract class LeafMathExpression(c: Double, name: String)
override def nullable: Boolean = false
override def toString: String = s"$name()"
- override def eval(input: Row): Any = c
+ override def eval(input: catalyst.InternalRow): Any = c
override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
s"""
@@ -60,7 +61,7 @@ abstract class UnaryMathExpression(f: Double => Double, name: String)
override def nullable: Boolean = true
override def toString: String = s"$name($child)"
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE = child.eval(input)
if (evalE == null) {
null
@@ -103,7 +104,7 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String)
override def dataType: DataType = DoubleType
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE1 = left.eval(input)
if (evalE1 == null) {
null
@@ -215,7 +216,7 @@ case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadia
case class Atan2(left: Expression, right: Expression)
extends BinaryMathExpression(math.atan2, "ATAN2") {
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
val evalE1 = left.eval(input)
if (evalE1 == null) {
null
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 2e4b9ba678..20505129e9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
@@ -114,7 +115,7 @@ case class Alias(child: Expression, name: String)(
// Alias(Generator, xx) need to be transformed into Generate(generator, ...)
override lazy val resolved = childrenResolved && !child.isInstanceOf[Generator]
- override def eval(input: Row): Any = child.eval(input)
+ override def eval(input: catalyst.InternalRow): Any = child.eval(input)
override def gen(ctx: CodeGenContext): GeneratedExpressionCode = child.gen(ctx)
@@ -230,7 +231,7 @@ case class AttributeReference(
}
// Unresolved attributes are transient at compile time and don't get evaluated during execution.
- override def eval(input: Row = null): Any =
+ override def eval(input: catalyst.InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"$name#${exprId.id}$typeSuffix"
@@ -252,7 +253,7 @@ case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[E
override def withName(newName: String): Attribute = throw new UnsupportedOperationException
override def qualifiers: Seq[String] = throw new UnsupportedOperationException
override def exprId: ExprId = throw new UnsupportedOperationException
- override def eval(input: Row): Any = throw new UnsupportedOperationException
+ override def eval(input: catalyst.InternalRow): Any = throw new UnsupportedOperationException
override def nullable: Boolean = throw new UnsupportedOperationException
override def dataType: DataType = NullType
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index c2d1a4eada..292d626f01 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
@@ -43,7 +44,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression {
this, s"Coalesce cannot have children of different types. $childTypes")
}
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
var i = 0
var result: Any = null
val childIterator = children.iterator
@@ -77,7 +78,7 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr
override def foldable: Boolean = child.foldable
override def nullable: Boolean = false
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
child.eval(input) == null
}
@@ -96,7 +97,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E
override def nullable: Boolean = false
override def toString: String = s"IS NOT NULL $child"
- override def eval(input: Row): Any = {
+ override def eval(input: catalyst.InternalRow): Any = {
child.eval(input) != null
}
@@ -118,7 +119,7 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate
private[this] val childrenArray = children.toArray
- override def eval(input: Row): Boolean = {
+ override def eval(input: catalyst.InternalRow): Boolean = {
var numNonNulls = 0
var i = 0
while (i < childrenArray.length && numNonNulls < n) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
index fbc97b2e75..c2e57b4715 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst
+import org.apache.spark.sql.catalyst
+
/**
* A set of classes that can be used to represent trees of relational expressions. A key goal of
* the expression library is to hide the details of naming and scoping from developers who want to
@@ -49,30 +51,30 @@ package org.apache.spark.sql.catalyst
*/
package object expressions {
- type Row = org.apache.spark.sql.Row
+ type InternalRow = catalyst.InternalRow
- val Row = org.apache.spark.sql.Row
+ val InternalRow = catalyst.InternalRow
/**
- * Converts a [[Row]] to another Row given a sequence of expression that define each column of the
- * new row. If the schema of the input row is specified, then the given expression will be bound
- * to that schema.
+ * Converts a [[InternalRow]] to another Row given a sequence of expression that define each
+ * column of the new row. If the schema of the input row is specified, then the given expression
+ * will be bound to that schema.
*/
- abstract class Projection extends (Row => Row)
+ abstract class Projection extends (InternalRow => InternalRow)
/**
- * Converts a [[Row]] to another Row given a sequence of expression that define each column of the
- * new row. If the schema of the input row is specified, then the given expression will be bound
- * to that schema.
+ * Converts a [[InternalRow]] to another Row given a sequence of expression that define each
+ * column of the new row. If the schema of the input row is specified, then the given expression
+ * will be bound to that schema.
*
* In contrast to a normal projection, a MutableProjection reuses the same underlying row object
* each time an input row is added. This significantly reduces the cost of calculating the
- * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()`
- * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()`
- * and hold on to the returned [[Row]] before calling `next()`.
+ * projection, but means that it is not safe to hold on to a reference to a [[InternalRow]] after
+ * `next()` has been called on the [[Iterator]] that produced it. Instead, the user must call
+ * `InternalRow.copy()` and hold on to the returned [[InternalRow]] before calling `next()`.
*/
abstract class MutableProjection extends Projection {
- def currentValue: Row
+ def currentValue: InternalRow
/** Uses the given row to store the output of the projection. */
def target(row: MutableRow): MutableProjection
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 7574d1cbda..082d72eb43 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -24,11 +24,11 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types._
object InterpretedPredicate {
- def create(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) =
+ def create(expression: Expression, inputSchema: Seq[Attribute]): (InternalRow => Boolean) =
create(BindReferences.bindReference(expression, inputSchema))
- def create(expression: Expression): (Row => Boolean) = {
- (r: Row) => expression.eval(r).asInstanceOf[Boolean]
+ def create(expression: Expression): (InternalRow => Boolean) = {
+ (r: InternalRow) => expression.eval(r).asInstanceOf[Boolean]
}
}
@@ -77,7 +77,7 @@ case class Not(child: Expression) extends UnaryExpression with Predicate with Ex
override def expectedChildTypes: Seq[DataType] = Seq(BooleanType)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
child.eval(input) match {
case null => null
case b: Boolean => !b
@@ -98,7 +98,7 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate {
override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN.
override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}"
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val evaluatedValue = value.eval(input)
list.exists(e => e.eval(input) == evaluatedValue)
}
@@ -117,7 +117,7 @@ case class InSet(value: Expression, hset: Set[Any])
override def nullable: Boolean = true // TODO: Figure out correct nullability semantics of IN.
override def toString: String = s"$value INSET ${hset.mkString("(", ",", ")")}"
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
hset.contains(value.eval(input))
}
}
@@ -129,7 +129,7 @@ case class And(left: Expression, right: Expression)
override def symbol: String = "&&"
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val l = left.eval(input)
if (l == false) {
false
@@ -178,7 +178,7 @@ case class Or(left: Expression, right: Expression)
override def symbol: String = "||"
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val l = left.eval(input)
if (l == true) {
true
@@ -235,7 +235,7 @@ abstract class BinaryComparison extends BinaryExpression with Predicate {
protected def checkTypesInternal(t: DataType): TypeCheckResult
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val evalE1 = left.eval(input)
if (evalE1 == null) {
null
@@ -288,7 +288,7 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp
override protected def checkTypesInternal(t: DataType) = TypeCheckResult.TypeCheckSuccess
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val l = left.eval(input)
val r = right.eval(input)
if (l == null && r == null) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala
index 6e4e9cb1be..7e8033307e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala
@@ -48,7 +48,7 @@ abstract class RDG(seed: Long) extends LeafExpression with Serializable {
/** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */
case class Rand(seed: Long) extends RDG(seed) {
- override def eval(input: Row): Double = rng.nextDouble()
+ override def eval(input: InternalRow): Double = rng.nextDouble()
}
object Rand {
@@ -62,7 +62,7 @@ object Rand {
/** Generate a random column with i.i.d. gaussian random distribution. */
case class Randn(seed: Long) extends RDG(seed) {
- override def eval(input: Row): Double = rng.nextGaussian()
+ override def eval(input: InternalRow): Double = rng.nextGaussian()
}
object Randn {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
index 5d2d82077f..534dac1f92 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala
@@ -21,10 +21,10 @@ import org.apache.spark.sql.types.{DataType, StructType, AtomicType}
import org.apache.spark.unsafe.types.UTF8String
/**
- * An extended interface to [[Row]] that allows the values for each column to be updated. Setting
- * a value through a primitive function implicitly marks that column as not null.
+ * An extended interface to [[InternalRow]] that allows the values for each column to be updated.
+ * Setting a value through a primitive function implicitly marks that column as not null.
*/
-trait MutableRow extends Row {
+trait MutableRow extends InternalRow {
def setNullAt(i: Int): Unit
def update(ordinal: Int, value: Any)
@@ -37,13 +37,12 @@ trait MutableRow extends Row {
def setByte(ordinal: Int, value: Byte)
def setFloat(ordinal: Int, value: Float)
def setString(ordinal: Int, value: String)
- // TODO(davies): add setDate() and setDecimal()
}
/**
* A row with no data. Calling any methods will result in an error. Can be used as a placeholder.
*/
-object EmptyRow extends Row {
+object EmptyRow extends InternalRow {
override def apply(i: Int): Any = throw new UnsupportedOperationException
override def toSeq: Seq[Any] = Seq.empty
override def length: Int = 0
@@ -57,7 +56,7 @@ object EmptyRow extends Row {
override def getByte(i: Int): Byte = throw new UnsupportedOperationException
override def getString(i: Int): String = throw new UnsupportedOperationException
override def getAs[T](i: Int): T = throw new UnsupportedOperationException
- override def copy(): Row = this
+ override def copy(): InternalRow = this
}
/**
@@ -65,7 +64,7 @@ object EmptyRow extends Row {
* the array is not copied, and thus could technically be mutated after creation, this is not
* allowed.
*/
-class GenericRow(protected[sql] val values: Array[Any]) extends Row {
+class GenericRow(protected[sql] val values: Array[Any]) extends InternalRow {
/** No-arg constructor for serialization. */
protected def this() = this(null)
@@ -154,7 +153,7 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row {
}
override def equals(o: Any): Boolean = o match {
- case other: Row =>
+ case other: InternalRow =>
if (values.length != other.length) {
return false
}
@@ -174,7 +173,7 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row {
case _ => false
}
- override def copy(): Row = this
+ override def copy(): InternalRow = this
}
class GenericRowWithSchema(values: Array[Any], override val schema: StructType)
@@ -207,15 +206,15 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow {
override def update(ordinal: Int, value: Any): Unit = { values(ordinal) = value }
- override def copy(): Row = new GenericRow(values.clone())
+ override def copy(): InternalRow = new GenericRow(values.clone())
}
-class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] {
+class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow] {
def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) =
this(ordering.map(BindReferences.bindReference(_, inputSchema)))
- def compare(a: Row, b: Row): Int = {
+ def compare(a: InternalRow, b: InternalRow): Int = {
var i = 0
while (i < ordering.size) {
val order = ordering(i)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
index 2bcb960e91..30e41677b7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext}
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode}
import org.apache.spark.sql.types._
import org.apache.spark.util.collection.OpenHashSet
@@ -57,7 +57,7 @@ case class NewSet(elementType: DataType) extends LeafExpression {
override def dataType: OpenHashSetUDT = new OpenHashSetUDT(elementType)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
new OpenHashSet[Any]()
}
@@ -87,7 +87,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression {
override def dataType: OpenHashSetUDT = set.dataType.asInstanceOf[OpenHashSetUDT]
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val itemEval = item.eval(input)
val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]]
@@ -137,7 +137,7 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres
override def symbol: String = "++="
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]]
if(leftEval != null) {
val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]]
@@ -183,7 +183,7 @@ case class CountSet(child: Expression) extends UnaryExpression {
override def dataType: DataType = LongType
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]]
if (childEval != null) {
childEval.size.toLong
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index 4f4c19526e..8ca8d22bc4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -49,7 +49,7 @@ trait StringRegexExpression extends ExpectsInputTypes {
protected def pattern(str: String) = if (cache == null) compile(str) else cache
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val l = left.eval(input)
if (l == null) {
null
@@ -121,7 +121,7 @@ trait CaseConversionExpression extends ExpectsInputTypes {
override def dataType: DataType = StringType
override def expectedChildTypes: Seq[DataType] = Seq(StringType)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val evaluated = child.eval(input)
if (evaluated == null) {
null
@@ -169,7 +169,7 @@ trait StringComparison extends ExpectsInputTypes {
override def expectedChildTypes: Seq[DataType] = Seq(StringType, StringType)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val leftEval = left.eval(input)
if(leftEval == null) {
null
@@ -262,7 +262,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression)
(start, end)
}
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val string = str.eval(input)
val po = pos.eval(input)
val ln = len.eval(input)
@@ -303,7 +303,7 @@ case class StringLength(child: Expression) extends UnaryExpression with ExpectsI
override def dataType: DataType = IntegerType
override def expectedChildTypes: Seq[DataType] = Seq(StringType)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
val string = child.eval(input)
if (string == null) null else string.asInstanceOf[UTF8String].length
}
@@ -314,5 +314,3 @@ case class StringLength(child: Expression) extends UnaryExpression with ExpectsI
defineCodeGen(ctx, ev, c => s"($c).length()")
}
}
-
-
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index 82c4d462cc..056f170539 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -74,7 +74,7 @@ case class WindowSpecDefinition(
override def toString: String = simpleString
- override def eval(input: Row): Any = throw new UnsupportedOperationException
+ override def eval(input: InternalRow): Any = throw new UnsupportedOperationException
override def nullable: Boolean = true
override def foldable: Boolean = false
override def dataType: DataType = throw new UnsupportedOperationException
@@ -259,7 +259,7 @@ trait WindowFunction extends Expression {
def reset(): Unit
- def prepareInputParameters(input: Row): AnyRef
+ def prepareInputParameters(input: InternalRow): AnyRef
def update(input: AnyRef): Unit
@@ -286,7 +286,7 @@ case class UnresolvedWindowFunction(
throw new UnresolvedException(this, "init")
override def reset(): Unit =
throw new UnresolvedException(this, "reset")
- override def prepareInputParameters(input: Row): AnyRef =
+ override def prepareInputParameters(input: InternalRow): AnyRef =
throw new UnresolvedException(this, "prepareInputParameters")
override def update(input: AnyRef): Unit =
throw new UnresolvedException(this, "update")
@@ -297,7 +297,7 @@ case class UnresolvedWindowFunction(
override def get(index: Int): Any =
throw new UnresolvedException(this, "get")
// Unresolved functions are transient at compile time and don't get evaluated during execution.
- override def eval(input: Row = null): Any =
+ override def eval(input: InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def toString: String = s"'$name(${children.mkString(",")})"
@@ -316,7 +316,7 @@ case class UnresolvedWindowExpression(
override lazy val resolved = false
// Unresolved functions are transient at compile time and don't get evaluated during execution.
- override def eval(input: Row = null): Any =
+ override def eval(input: InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
}
@@ -327,7 +327,7 @@ case class WindowExpression(
override def children: Seq[Expression] =
windowFunction :: windowSpec :: Nil
- override def eval(input: Row): Any =
+ override def eval(input: InternalRow): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
override def dataType: DataType = windowFunction.dataType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
index e3e070f0ff..2c946cd12f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
@@ -17,10 +17,9 @@
package org.apache.spark.sql.catalyst.plans.logical
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.{CatalystTypeConverters, analysis}
import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.types.{StructType, StructField}
+import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters, analysis}
+import org.apache.spark.sql.types.{StructField, StructType}
object LocalRelation {
def apply(output: Attribute*): LocalRelation = new LocalRelation(output)
@@ -32,11 +31,11 @@ object LocalRelation {
def fromProduct(output: Seq[Attribute], data: Seq[Product]): LocalRelation = {
val schema = StructType.fromAttributes(output)
val converter = CatalystTypeConverters.createToCatalystConverter(schema)
- LocalRelation(output, data.map(converter(_).asInstanceOf[Row]))
+ LocalRelation(output, data.map(converter(_).asInstanceOf[InternalRow]))
}
}
-case class LocalRelation(output: Seq[Attribute], data: Seq[Row] = Nil)
+case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil)
extends LeafNode with analysis.MultiInstanceRelation {
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 80ba57a082..42dead7c28 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -17,8 +17,9 @@
package org.apache.spark.sql.catalyst.plans.physical
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder}
+import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
import org.apache.spark.sql.types.{DataType, IntegerType}
/**
@@ -169,7 +170,7 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int)
override def keyExpressions: Seq[Expression] = expressions
- override def eval(input: Row = null): Any =
+ override def eval(input: InternalRow = null): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
}
@@ -213,6 +214,6 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int)
override def keyExpressions: Seq[Expression] = ordering.map(_.child)
- override def eval(input: Row): Any =
+ override def eval(input: InternalRow): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
index 9a24b23024..b4d5e013f3 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala
@@ -21,7 +21,7 @@ import java.math.BigInteger
import java.sql.{Date, Timestamp}
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._
case class PrimitiveData(
@@ -257,7 +257,7 @@ class ScalaReflectionSuite extends SparkFunSuite {
test("convert PrimitiveData to catalyst") {
val data = PrimitiveData(1, 1, 1, 1, 1, 1, true)
- val convertedData = Row(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true)
+ val convertedData = InternalRow(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true)
val dataType = schemaFor[PrimitiveData].dataType
assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData)
}
@@ -267,8 +267,8 @@ class ScalaReflectionSuite extends SparkFunSuite {
val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true),
Some(primitiveData))
val dataType = schemaFor[OptionalData].dataType
- val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true,
- Row(1, 1, 1, 1, 1, 1, true))
+ val convertedData = InternalRow(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true,
+ InternalRow(1, 1, 1, 1, 1, 1, true))
assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
index 969c6cc15f..e407f6f166 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
@@ -437,14 +437,14 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
test("cast from struct") {
val struct = Literal.create(
- Row("123", "abc", "", null),
+ InternalRow("123", "abc", "", null),
StructType(Seq(
StructField("a", StringType, nullable = true),
StructField("b", StringType, nullable = true),
StructField("c", StringType, nullable = true),
StructField("d", StringType, nullable = true))))
val struct_notNull = Literal.create(
- Row("123", "abc", ""),
+ InternalRow("123", "abc", ""),
StructType(Seq(
StructField("a", StringType, nullable = false),
StructField("b", StringType, nullable = false),
@@ -457,7 +457,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("c", IntegerType, nullable = true),
StructField("d", IntegerType, nullable = true))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(123, null, null, null))
+ checkEvaluation(ret, InternalRow(123, null, null, null))
}
{
val ret = cast(struct, StructType(Seq(
@@ -474,7 +474,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("c", BooleanType, nullable = true),
StructField("d", BooleanType, nullable = true))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(true, true, false, null))
+ checkEvaluation(ret, InternalRow(true, true, false, null))
}
{
val ret = cast(struct, StructType(Seq(
@@ -491,7 +491,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("b", IntegerType, nullable = true),
StructField("c", IntegerType, nullable = true))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(123, null, null))
+ checkEvaluation(ret, InternalRow(123, null, null))
}
{
val ret = cast(struct_notNull, StructType(Seq(
@@ -506,7 +506,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("b", BooleanType, nullable = true),
StructField("c", BooleanType, nullable = true))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(true, true, false))
+ checkEvaluation(ret, InternalRow(true, true, false))
}
{
val ret = cast(struct_notNull, StructType(Seq(
@@ -514,7 +514,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("b", BooleanType, nullable = true),
StructField("c", BooleanType, nullable = false))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(true, true, false))
+ checkEvaluation(ret, InternalRow(true, true, false))
}
{
@@ -532,10 +532,10 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
test("complex casting") {
val complex = Literal.create(
- Row(
+ InternalRow(
Seq("123", "abc", ""),
Map("a" -> "123", "b" -> "abc", "c" -> ""),
- Row(0)),
+ InternalRow(0)),
StructType(Seq(
StructField("a",
ArrayType(StringType, containsNull = false), nullable = true),
@@ -555,10 +555,10 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
StructField("l", LongType, nullable = true)))))))
assert(ret.resolved === true)
- checkEvaluation(ret, Row(
+ checkEvaluation(ret, InternalRow(
Seq(123, null, null),
Map("a" -> true, "b" -> true, "c" -> false),
- Row(0L)))
+ InternalRow(0L)))
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
index bcc594cb7c..2b0f4618b2 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
@@ -27,10 +27,10 @@ import org.apache.spark.unsafe.types.UTF8String
class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
test("CreateStruct") {
- val row = Row(1, 2, 3)
+ val row = InternalRow(1, 2, 3)
val c1 = 'a.int.at(0).as("a")
val c3 = 'c.int.at(2).as("c")
- checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row)
+ checkEvaluation(CreateStruct(Seq(c1, c3)), InternalRow(1, 3), row)
}
test("complex type") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
index 4a241d3603..12d2da8b33 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
@@ -32,26 +32,26 @@ import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project}
trait ExpressionEvalHelper {
self: SparkFunSuite =>
- protected def create_row(values: Any*): Row = {
+ protected def create_row(values: Any*): InternalRow = {
new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray)
}
protected def checkEvaluation(
- expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = {
+ expression: Expression, expected: Any, inputRow: InternalRow = EmptyRow): Unit = {
checkEvaluationWithoutCodegen(expression, expected, inputRow)
checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow)
checkEvaluationWithGeneratedProjection(expression, expected, inputRow)
checkEvaluationWithOptimization(expression, expected, inputRow)
}
- protected def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = {
+ protected def evaluate(expression: Expression, inputRow: InternalRow = EmptyRow): Any = {
expression.eval(inputRow)
}
protected def checkEvaluationWithoutCodegen(
expression: Expression,
expected: Any,
- inputRow: Row = EmptyRow): Unit = {
+ inputRow: InternalRow = EmptyRow): Unit = {
val actual = try evaluate(expression, inputRow) catch {
case e: Exception => fail(s"Exception evaluating $expression", e)
}
@@ -66,7 +66,7 @@ trait ExpressionEvalHelper {
protected def checkEvaluationWithGeneratedMutableProjection(
expression: Expression,
expected: Any,
- inputRow: Row = EmptyRow): Unit = {
+ inputRow: InternalRow = EmptyRow): Unit = {
val plan = try {
GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)()
@@ -92,7 +92,7 @@ trait ExpressionEvalHelper {
protected def checkEvaluationWithGeneratedProjection(
expression: Expression,
expected: Any,
- inputRow: Row = EmptyRow): Unit = {
+ inputRow: InternalRow = EmptyRow): Unit = {
val ctx = GenerateProjection.newCodeGenContext()
lazy val evaluated = expression.gen(ctx)
@@ -128,7 +128,7 @@ trait ExpressionEvalHelper {
protected def checkEvaluationWithOptimization(
expression: Expression,
expected: Any,
- inputRow: Row = EmptyRow): Unit = {
+ inputRow: InternalRow = EmptyRow): Unit = {
val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation)
val optimizedPlan = DefaultOptimizer.execute(plan)
checkEvaluationWithoutCodegen(optimizedPlan.expressions.head, expected, inputRow)
@@ -137,7 +137,7 @@ trait ExpressionEvalHelper {
protected def checkDoubleEvaluation(
expression: Expression,
expected: Spread[Double],
- inputRow: Row = EmptyRow): Unit = {
+ inputRow: InternalRow = EmptyRow): Unit = {
val actual = try evaluate(expression, inputRow) catch {
case e: Exception => fail(s"Exception evaluating $expression", e)
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala
index 72bbc4efeb..7aae2bbd8a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMapSuite.scala
@@ -37,7 +37,7 @@ class UnsafeFixedWidthAggregationMapSuite
private val groupKeySchema = StructType(StructField("product", StringType) :: Nil)
private val aggBufferSchema = StructType(StructField("salePrice", IntegerType) :: Nil)
- private def emptyAggregationBuffer: Row = new GenericRow(Array[Any](0))
+ private def emptyAggregationBuffer: InternalRow = new GenericRow(Array[Any](0))
private var memoryManager: TaskMemoryManager = null
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
index 61722f1ffa..577c7a0de0 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala
@@ -86,7 +86,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers {
DoubleType)
val converter = new UnsafeRowConverter(fieldTypes)
- val rowWithAllNullColumns: Row = {
+ val rowWithAllNullColumns: InternalRow = {
val r = new SpecificMutableRow(fieldTypes)
for (i <- 0 to fieldTypes.length - 1) {
r.setNullAt(i)
@@ -117,7 +117,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers {
// If we have an UnsafeRow with columns that are initially non-null and we null out those
// columns, then the serialized row representation should be identical to what we would get by
// creating an entirely null row via the converter
- val rowWithNoNullColumns: Row = {
+ val rowWithNoNullColumns: InternalRow = {
val r = new SpecificMutableRow(fieldTypes)
r.setNullAt(0)
r.setBoolean(1, false)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala
index 6841bd9890..54e8c6462e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala
@@ -17,10 +17,10 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
-import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
@@ -37,13 +37,11 @@ class ConvertToLocalRelationSuite extends PlanTest {
test("Project on LocalRelation should be turned into a single LocalRelation") {
val testRelation = LocalRelation(
LocalRelation('a.int, 'b.int).output,
- Row(1, 2) ::
- Row(4, 5) :: Nil)
+ InternalRow(1, 2) :: InternalRow(4, 5) :: Nil)
val correctAnswer = LocalRelation(
LocalRelation('a1.int, 'b1.int).output,
- Row(1, 3) ::
- Row(4, 6) :: Nil)
+ InternalRow(1, 3) :: InternalRow(4, 6) :: Nil)
val projectOnLocal = testRelation.select(
UnresolvedAttribute("a").as("a1"),
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
index 8ec79c3d4d..bda217935c 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
@@ -28,7 +28,7 @@ case class Dummy(optKey: Option[Expression]) extends Expression {
override def nullable: Boolean = true
override def dataType: NullType = NullType
override lazy val resolved = true
- override def eval(input: Row): Any = null.asInstanceOf[Any]
+ override def eval(input: InternalRow): Any = null.asInstanceOf[Any]
}
case class ComplexPlan(exprs: Seq[Seq[Expression]])
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateUtilsSuite.scala
index a4245545ff..4d8fe4ac5e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateUtilsSuite.scala
@@ -21,7 +21,6 @@ import java.sql.Timestamp
import org.apache.spark.SparkFunSuite
-
class DateUtilsSuite extends SparkFunSuite {
test("timestamp") {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index f041fd397b..f1acdfeea5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql
import java.io.CharArrayWriter
import java.util.Properties
-import scala.collection.JavaConversions._
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
@@ -33,7 +32,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.analysis.{MultiAlias, ResolvedStar, UnresolvedAttribute, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.analysis.{MultiAlias, ResolvedStar, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{Filter, _}
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
@@ -1032,7 +1031,8 @@ class DataFrame private[sql](
val names = schema.toAttributes.map(_.name)
val rowFunction =
- f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema).asInstanceOf[Row]))
+ f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema)
+ .asInstanceOf[InternalRow]))
val generator = UserDefinedGenerator(elementTypes, rowFunction, input.map(_.expr))
Generate(generator, join = true, outer = false,
@@ -1058,8 +1058,9 @@ class DataFrame private[sql](
val elementTypes = attributes.map { attr => (attr.dataType, attr.nullable) }
val names = attributes.map(_.name)
- def rowFunction(row: Row): TraversableOnce[Row] = {
- f(row(0).asInstanceOf[A]).map(o => Row(CatalystTypeConverters.convertToCatalyst(o, dataType)))
+ def rowFunction(row: Row): TraversableOnce[InternalRow] = {
+ f(row(0).asInstanceOf[A]).map(o =>
+ catalyst.InternalRow(CatalystTypeConverters.convertToCatalyst(o, dataType)))
}
val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil)
@@ -1221,7 +1222,7 @@ class DataFrame private[sql](
val outputCols = (if (cols.isEmpty) numericColumns.map(_.prettyString) else cols).toList
- val ret: Seq[Row] = if (outputCols.nonEmpty) {
+ val ret: Seq[InternalRow] = if (outputCols.nonEmpty) {
val aggExprs = statistics.flatMap { case (_, colToAgg) =>
outputCols.map(c => Column(Cast(colToAgg(Column(c).expr), StringType)).as(c))
}
@@ -1230,11 +1231,12 @@ class DataFrame private[sql](
// Pivot the data so each summary is one row
row.grouped(outputCols.size).toSeq.zip(statistics).map {
- case (aggregation, (statistic, _)) => Row(statistic :: aggregation.toList: _*)
+ case (aggregation, (statistic, _)) =>
+ catalyst.InternalRow(statistic :: aggregation.toList: _*)
}
} else {
// If there are no output columns, just output a single column that contains the stats.
- statistics.map { case (name, _) => Row(name) }
+ statistics.map { case (name, _) => catalyst.InternalRow(name) }
}
// All columns are string type
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
index b44d4c86ac..1828ed1aab 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -245,7 +245,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) {
JsonRDD.nullTypeToStringType(
JsonRDD.inferSchema(jsonRDD, 1.0, columnNameOfCorruptJsonRecord)))
val rowRDD = JsonRDD.jsonStringToRow(jsonRDD, appliedSchema, columnNameOfCorruptJsonRecord)
- sqlContext.createDataFrame(rowRDD, appliedSchema, needsConversion = false)
+ sqlContext.internalCreateDataFrame(rowRDD, appliedSchema)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 5f758adf3d..22d0e50e4e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -31,7 +31,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst._
+import org.apache.spark.sql.catalyst.{InternalRow, _}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.errors.DialectException
@@ -486,15 +486,27 @@ class SQLContext(@transient val sparkContext: SparkContext)
// schema differs from the existing schema on any field data type.
val catalystRows = if (needsConversion) {
val converter = CatalystTypeConverters.createToCatalystConverter(schema)
- rowRDD.map(converter(_).asInstanceOf[Row])
+ rowRDD.map(converter(_).asInstanceOf[InternalRow])
} else {
- rowRDD
+ rowRDD.map{r: Row => InternalRow.fromSeq(r.toSeq)}
}
val logicalPlan = LogicalRDD(schema.toAttributes, catalystRows)(self)
DataFrame(this, logicalPlan)
}
/**
+ * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be
+ * converted to Catalyst rows.
+ */
+ private[sql]
+ def internalCreateDataFrame(catalystRows: RDD[InternalRow], schema: StructType) = {
+ // TODO: use MutableProjection when rowRDD is another DataFrame and the applied
+ // schema differs from the existing schema on any field data type.
+ val logicalPlan = LogicalRDD(schema.toAttributes, catalystRows)(self)
+ DataFrame(this, logicalPlan)
+ }
+
+ /**
* :: DeveloperApi ::
* Creates a [[DataFrame]] from an [[JavaRDD]] containing [[Row]]s using the given schema.
* It is important to make sure that the structure of every [[Row]] of the provided RDD matches
@@ -531,7 +543,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
extractors.zip(attributeSeq).map { case (e, attr) =>
CatalystTypeConverters.convertToCatalyst(e.invoke(row), attr.dataType)
}.toArray[Any]
- ) : Row
+ ) : InternalRow
}
}
DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this))
@@ -886,7 +898,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
protected[sql] val planner = new SparkPlanner
@transient
- protected[sql] lazy val emptyResult = sparkContext.parallelize(Seq.empty[Row], 1)
+ protected[sql] lazy val emptyResult = sparkContext.parallelize(Seq.empty[InternalRow], 1)
/**
* Prepares a planned SparkPlan for execution by inserting shuffle operations as needed.
@@ -953,7 +965,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
lazy val executedPlan: SparkPlan = prepareForExecution.execute(sparkPlan)
/** Internal version of the RDD. Avoids copies and has no schema */
- lazy val toRdd: RDD[Row] = executedPlan.execute()
+ lazy val toRdd: RDD[InternalRow] = executedPlan.execute()
protected def stringOrError[A](f: => A): String =
try f.toString catch { case e: Throwable => e.toString }
@@ -1035,7 +1047,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
}
val rowRdd = convertedRdd.mapPartitions { iter =>
- iter.map { m => new GenericRow(m): Row}
+ iter.map { m => new GenericRow(m): InternalRow}
}
DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
index aa10af400c..cc7506dec1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -18,8 +18,7 @@
package org.apache.spark.sql.columnar
import java.nio.{ByteBuffer, ByteOrder}
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.columnar.ColumnBuilder._
import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder}
import org.apache.spark.sql.types._
@@ -33,7 +32,7 @@ private[sql] trait ColumnBuilder {
/**
* Appends `row(ordinal)` to the column builder.
*/
- def appendFrom(row: Row, ordinal: Int)
+ def appendFrom(row: InternalRow, ordinal: Int)
/**
* Column statistics information
@@ -68,7 +67,7 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId)
}
- override def appendFrom(row: Row, ordinal: Int): Unit = {
+ override def appendFrom(row: InternalRow, ordinal: Int): Unit = {
buffer = ensureFreeSpace(buffer, columnType.actualSize(row, ordinal))
columnType.append(row, ordinal, buffer)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
index 11c79c865f..1bce214d1d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.columnar
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference}
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -53,7 +53,7 @@ private[sql] sealed trait ColumnStats extends Serializable {
/**
* Gathers statistics information from `row(ordinal)`.
*/
- def gatherStats(row: Row, ordinal: Int): Unit = {
+ def gatherStats(row: InternalRow, ordinal: Int): Unit = {
if (row.isNullAt(ordinal)) {
nullCount += 1
// 4 bytes for null position
@@ -66,23 +66,23 @@ private[sql] sealed trait ColumnStats extends Serializable {
* Column statistics represented as a single row, currently including closed lower bound, closed
* upper bound and null count.
*/
- def collectedStatistics: Row
+ def collectedStatistics: InternalRow
}
/**
* A no-op ColumnStats only used for testing purposes.
*/
private[sql] class NoopColumnStats extends ColumnStats {
- override def gatherStats(row: Row, ordinal: Int): Unit = super.gatherStats(row, ordinal)
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = super.gatherStats(row, ordinal)
- override def collectedStatistics: Row = Row(null, null, nullCount, count, 0L)
+ override def collectedStatistics: InternalRow = InternalRow(null, null, nullCount, count, 0L)
}
private[sql] class BooleanColumnStats extends ColumnStats {
protected var upper = false
protected var lower = true
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getBoolean(ordinal)
@@ -92,14 +92,15 @@ private[sql] class BooleanColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class ByteColumnStats extends ColumnStats {
protected var upper = Byte.MinValue
protected var lower = Byte.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getByte(ordinal)
@@ -109,14 +110,15 @@ private[sql] class ByteColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class ShortColumnStats extends ColumnStats {
protected var upper = Short.MinValue
protected var lower = Short.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getShort(ordinal)
@@ -126,14 +128,15 @@ private[sql] class ShortColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class LongColumnStats extends ColumnStats {
protected var upper = Long.MinValue
protected var lower = Long.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getLong(ordinal)
@@ -143,14 +146,15 @@ private[sql] class LongColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class DoubleColumnStats extends ColumnStats {
protected var upper = Double.MinValue
protected var lower = Double.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getDouble(ordinal)
@@ -160,14 +164,15 @@ private[sql] class DoubleColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class FloatColumnStats extends ColumnStats {
protected var upper = Float.MinValue
protected var lower = Float.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getFloat(ordinal)
@@ -177,14 +182,15 @@ private[sql] class FloatColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class FixedDecimalColumnStats extends ColumnStats {
protected var upper: Decimal = null
protected var lower: Decimal = null
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row(ordinal).asInstanceOf[Decimal]
@@ -194,14 +200,15 @@ private[sql] class FixedDecimalColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class IntColumnStats extends ColumnStats {
protected var upper = Int.MinValue
protected var lower = Int.MaxValue
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row.getInt(ordinal)
@@ -211,14 +218,15 @@ private[sql] class IntColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class StringColumnStats extends ColumnStats {
protected var upper: UTF8String = null
protected var lower: UTF8String = null
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
val value = row(ordinal).asInstanceOf[UTF8String]
@@ -228,7 +236,8 @@ private[sql] class StringColumnStats extends ColumnStats {
}
}
- override def collectedStatistics: Row = Row(lower, upper, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(lower, upper, nullCount, count, sizeInBytes)
}
private[sql] class DateColumnStats extends IntColumnStats
@@ -236,23 +245,25 @@ private[sql] class DateColumnStats extends IntColumnStats
private[sql] class TimestampColumnStats extends LongColumnStats
private[sql] class BinaryColumnStats extends ColumnStats {
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
sizeInBytes += BINARY.actualSize(row, ordinal)
}
}
- override def collectedStatistics: Row = Row(null, null, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(null, null, nullCount, count, sizeInBytes)
}
private[sql] class GenericColumnStats extends ColumnStats {
- override def gatherStats(row: Row, ordinal: Int): Unit = {
+ override def gatherStats(row: InternalRow, ordinal: Int): Unit = {
super.gatherStats(row, ordinal)
if (!row.isNullAt(ordinal)) {
sizeInBytes += GENERIC.actualSize(row, ordinal)
}
}
- override def collectedStatistics: Row = Row(null, null, nullCount, count, sizeInBytes)
+ override def collectedStatistics: InternalRow =
+ InternalRow(null, null, nullCount, count, sizeInBytes)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
index 3db26fad2b..761f427b8c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
@@ -19,21 +19,16 @@ package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
-import org.apache.spark.{Accumulable, Accumulator, Accumulators}
-import org.apache.spark.sql.catalyst.expressions
-
import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
-import org.apache.spark.SparkContext
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
import org.apache.spark.sql.execution.{LeafNode, SparkPlan}
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.{Accumulable, Accumulator, Accumulators}
private[sql] object InMemoryRelation {
def apply(
@@ -45,7 +40,7 @@ private[sql] object InMemoryRelation {
new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child, tableName)()
}
-private[sql] case class CachedBatch(buffers: Array[Array[Byte]], stats: Row)
+private[sql] case class CachedBatch(buffers: Array[Array[Byte]], stats: InternalRow)
private[sql] case class InMemoryRelation(
output: Seq[Attribute],
@@ -56,12 +51,12 @@ private[sql] case class InMemoryRelation(
tableName: Option[String])(
private var _cachedColumnBuffers: RDD[CachedBatch] = null,
private var _statistics: Statistics = null,
- private var _batchStats: Accumulable[ArrayBuffer[Row], Row] = null)
+ private var _batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] = null)
extends LogicalPlan with MultiInstanceRelation {
- private val batchStats: Accumulable[ArrayBuffer[Row], Row] =
+ private val batchStats: Accumulable[ArrayBuffer[InternalRow], InternalRow] =
if (_batchStats == null) {
- child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row])
+ child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[InternalRow])
} else {
_batchStats
}
@@ -151,7 +146,7 @@ private[sql] case class InMemoryRelation(
rowCount += 1
}
- val stats = Row.merge(columnBuilders.map(_.columnStats.collectedStatistics) : _*)
+ val stats = InternalRow.merge(columnBuilders.map(_.columnStats.collectedStatistics) : _*)
batchStats += stats
CachedBatch(columnBuilders.map(_.build().array()), stats)
@@ -267,7 +262,7 @@ private[sql] case class InMemoryColumnarTableScan(
private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
if (enableAccumulators) {
readPartitions.setValue(0)
readBatches.setValue(0)
@@ -296,7 +291,7 @@ private[sql] case class InMemoryColumnarTableScan(
val nextRow = new SpecificMutableRow(requestedColumnDataTypes)
- def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]): Iterator[Row] = {
+ def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]): Iterator[InternalRow] = {
val rows = cacheBatches.flatMap { cachedBatch =>
// Build column accessors
val columnAccessors = requestedColumnIndices.map { batchColumnIndex =>
@@ -306,15 +301,15 @@ private[sql] case class InMemoryColumnarTableScan(
}
// Extract rows via column accessors
- new Iterator[Row] {
+ new Iterator[InternalRow] {
private[this] val rowLen = nextRow.length
- override def next(): Row = {
+ override def next(): InternalRow = {
var i = 0
while (i < rowLen) {
columnAccessors(i).extractTo(nextRow, i)
i += 1
}
- if (attributes.isEmpty) Row.empty else nextRow
+ if (attributes.isEmpty) InternalRow.empty else nextRow
}
override def hasNext: Boolean = columnAccessors(0).hasNext
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
index f1f494ac26..ba47bc783f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.columnar
import java.nio.{ByteBuffer, ByteOrder}
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
/**
* A stackable trait used for building byte buffer for a column containing null values. Memory
@@ -52,7 +52,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder {
super.initialize(initialSize, columnName, useCompression)
}
- abstract override def appendFrom(row: Row, ordinal: Int): Unit = {
+ abstract override def appendFrom(row: InternalRow, ordinal: Int): Unit = {
columnStats.gatherStats(row, ordinal)
if (row.isNullAt(ordinal)) {
nulls = ColumnBuilder.ensureFreeSpace(nulls, 4)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
index 8e2a1af6da..39b21ddb47 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar.compression
import java.nio.{ByteBuffer, ByteOrder}
import org.apache.spark.Logging
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder}
import org.apache.spark.sql.types.AtomicType
@@ -66,7 +66,7 @@ private[sql] trait CompressibleColumnBuilder[T <: AtomicType]
encoder.compressionRatio < 0.8
}
- private def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ private def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
var i = 0
while (i < compressionEncoders.length) {
compressionEncoders(i).gatherCompressibilityStats(row, ordinal)
@@ -74,7 +74,7 @@ private[sql] trait CompressibleColumnBuilder[T <: AtomicType]
}
}
- abstract override def appendFrom(row: Row, ordinal: Int): Unit = {
+ abstract override def appendFrom(row: InternalRow, ordinal: Int): Unit = {
super.appendFrom(row, ordinal)
if (!row.isNullAt(ordinal)) {
gatherCompressibilityStats(row, ordinal)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
index 17c2d9b111..4eaec6d853 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
@@ -18,14 +18,13 @@
package org.apache.spark.sql.columnar.compression
import java.nio.{ByteBuffer, ByteOrder}
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.MutableRow
import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType}
import org.apache.spark.sql.types.AtomicType
private[sql] trait Encoder[T <: AtomicType] {
- def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {}
+ def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {}
def compressedSize: Int
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
index 534ae90ddb..5abc1259a1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
@@ -22,8 +22,7 @@ import java.nio.ByteBuffer
import scala.collection.mutable
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.runtimeMirror
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow}
import org.apache.spark.sql.columnar._
import org.apache.spark.sql.types._
@@ -96,7 +95,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme {
override def compressedSize: Int = _compressedSize
- override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ override def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
val value = columnType.getField(row, ordinal)
val actualSize = columnType.actualSize(row, ordinal)
_uncompressedSize += actualSize
@@ -217,7 +216,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme {
// to store dictionary element count.
private var dictionarySize = 4
- override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ override def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
val value = columnType.getField(row, ordinal)
if (!overflow) {
@@ -310,7 +309,7 @@ private[sql] case object BooleanBitSet extends CompressionScheme {
class Encoder extends compression.Encoder[BooleanType.type] {
private var _uncompressedSize = 0
- override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ override def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
_uncompressedSize += BOOLEAN.defaultSize
}
@@ -404,7 +403,7 @@ private[sql] case object IntDelta extends CompressionScheme {
private var prevValue: Int = _
- override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ override def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
val value = row.getInt(ordinal)
val delta = value - prevValue
@@ -484,7 +483,7 @@ private[sql] case object LongDelta extends CompressionScheme {
private var prevValue: Long = _
- override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {
+ override def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = {
val value = row.getLong(ordinal)
val delta = value - prevValue
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
index 8d16749697..6e8a5ef18a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
@@ -20,12 +20,10 @@ package org.apache.spark.sql.execution
import java.util.HashMap
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.SQLContext
/**
* :: DeveloperApi ::
@@ -121,11 +119,11 @@ case class Aggregate(
}
}
- protected override def doExecute(): RDD[Row] = attachTree(this, "execute") {
+ protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
if (groupingExpressions.isEmpty) {
child.execute().mapPartitions { iter =>
val buffer = newAggregateBuffer()
- var currentRow: Row = null
+ var currentRow: InternalRow = null
while (iter.hasNext) {
currentRow = iter.next()
var i = 0
@@ -147,10 +145,10 @@ case class Aggregate(
}
} else {
child.execute().mapPartitions { iter =>
- val hashTable = new HashMap[Row, Array[AggregateFunction]]
+ val hashTable = new HashMap[InternalRow, Array[AggregateFunction]]
val groupingProjection = new InterpretedMutableProjection(groupingExpressions, child.output)
- var currentRow: Row = null
+ var currentRow: InternalRow = null
while (iter.hasNext) {
currentRow = iter.next()
val currentGroup = groupingProjection(currentRow)
@@ -167,7 +165,7 @@ case class Aggregate(
}
}
- new Iterator[Row] {
+ new Iterator[InternalRow] {
private[this] val hashTableIter = hashTable.entrySet().iterator()
private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length)
private[this] val resultProjection =
@@ -177,7 +175,7 @@ case class Aggregate(
override final def hasNext: Boolean = hashTableIter.hasNext
- override final def next(): Row = {
+ override final def next(): InternalRow = {
val currentEntry = hashTableIter.next()
val currentGroup = currentEntry.getKey
val currentBuffer = currentEntry.getValue
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 6fa7ccc6cc..c9a188309a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -17,19 +17,19 @@
package org.apache.spark.sql.execution
-import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner, SparkEnv}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.{RDD, ShuffledRDD}
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.shuffle.unsafe.UnsafeShuffleManager
+import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.types.DataType
-import org.apache.spark.sql.{SQLContext, Row}
import org.apache.spark.util.MutablePair
+import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner, SparkEnv}
/**
* :: DeveloperApi ::
@@ -157,7 +157,7 @@ case class Exchange(
serializer
}
- protected override def doExecute(): RDD[Row] = attachTree(this , "execute") {
+ protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") {
newPartitioning match {
case HashPartitioning(expressions, numPartitions) =>
val keySchema = expressions.map(_.dataType).toArray
@@ -173,11 +173,11 @@ case class Exchange(
} else {
child.execute().mapPartitions { iter =>
val hashExpressions = newMutableProjection(expressions, child.output)()
- val mutablePair = new MutablePair[Row, Row]()
+ val mutablePair = new MutablePair[InternalRow, InternalRow]()
iter.map(r => mutablePair.update(hashExpressions(r), r))
}
}
- val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part)
+ val shuffled = new ShuffledRDD[InternalRow, InternalRow, InternalRow](rdd, part)
shuffled.setSerializer(serializer)
shuffled.map(_._2)
@@ -190,7 +190,7 @@ case class Exchange(
// Internally, RangePartitioner runs a job on the RDD that samples keys to compute
// partition bounds. To get accurate samples, we need to copy the mutable keys.
val rddForSampling = childRdd.mapPartitions { iter =>
- val mutablePair = new MutablePair[Row, Null]()
+ val mutablePair = new MutablePair[InternalRow, Null]()
iter.map(row => mutablePair.update(row.copy(), null))
}
// TODO: RangePartitioner should take an Ordering.
@@ -202,12 +202,12 @@ case class Exchange(
childRdd.mapPartitions { iter => iter.map(row => (row.copy(), null))}
} else {
childRdd.mapPartitions { iter =>
- val mutablePair = new MutablePair[Row, Null]()
+ val mutablePair = new MutablePair[InternalRow, Null]()
iter.map(row => mutablePair.update(row, null))
}
}
- val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part)
+ val shuffled = new ShuffledRDD[InternalRow, Null, Null](rdd, part)
shuffled.setSerializer(serializer)
shuffled.map(_._1)
@@ -217,14 +217,16 @@ case class Exchange(
val partitioner = new HashPartitioner(1)
val rdd = if (needToCopyObjectsBeforeShuffle(partitioner, serializer)) {
- child.execute().mapPartitions { iter => iter.map(r => (null, r.copy())) }
+ child.execute().mapPartitions {
+ iter => iter.map(r => (null, r.copy()))
+ }
} else {
child.execute().mapPartitions { iter =>
- val mutablePair = new MutablePair[Null, Row]()
+ val mutablePair = new MutablePair[Null, InternalRow]()
iter.map(r => mutablePair.update(null, r))
}
}
- val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner)
+ val shuffled = new ShuffledRDD[Null, InternalRow, InternalRow](rdd, partitioner)
shuffled.setSerializer(serializer)
shuffled.map(_._2)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index f931dc95ef..da27a753a7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.CatalystTypeConverters
+import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics}
@@ -31,7 +31,7 @@ import org.apache.spark.sql.{Row, SQLContext}
*/
@DeveloperApi
object RDDConversions {
- def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[Row] = {
+ def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = {
data.mapPartitions { iterator =>
val numColumns = outputTypes.length
val mutableRow = new GenericMutableRow(numColumns)
@@ -51,7 +51,7 @@ object RDDConversions {
/**
* Convert the objects inside Row into the types Catalyst expected.
*/
- def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[Row] = {
+ def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = {
data.mapPartitions { iterator =>
val numColumns = outputTypes.length
val mutableRow = new GenericMutableRow(numColumns)
@@ -70,7 +70,9 @@ object RDDConversions {
}
/** Logical plan node for scanning data from an RDD. */
-private[sql] case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext)
+private[sql] case class LogicalRDD(
+ output: Seq[Attribute],
+ rdd: RDD[InternalRow])(sqlContext: SQLContext)
extends LogicalPlan with MultiInstanceRelation {
override def children: Seq[LogicalPlan] = Nil
@@ -91,13 +93,15 @@ private[sql] case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlCon
}
/** Physical plan node for scanning data from an RDD. */
-private[sql] case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode {
- protected override def doExecute(): RDD[Row] = rdd
+private[sql] case class PhysicalRDD(
+ output: Seq[Attribute],
+ rdd: RDD[InternalRow]) extends LeafNode {
+ protected override def doExecute(): RDD[InternalRow] = rdd
}
/** Logical plan node for scanning data from a local collection. */
private[sql]
-case class LogicalLocalTable(output: Seq[Attribute], rows: Seq[Row])(sqlContext: SQLContext)
+case class LogicalLocalTable(output: Seq[Attribute], rows: Seq[InternalRow])(sqlContext: SQLContext)
extends LogicalPlan with MultiInstanceRelation {
override def children: Seq[LogicalPlan] = Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
index 4b601c1192..42a0c1be4f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
@@ -19,10 +19,9 @@ package org.apache.spark.sql.execution
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical.{UnknownPartitioning, Partitioning}
+import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
/**
* Apply the all of the GroupExpressions to every input row, hence we will get
@@ -43,7 +42,7 @@ case class Expand(
// as UNKNOWN partitioning
override def outputPartitioning: Partitioning = UnknownPartitioning(0)
- protected override def doExecute(): RDD[Row] = attachTree(this, "execute") {
+ protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") {
child.execute().mapPartitions { iter =>
// TODO Move out projection objects creation and transfer to
// workers via closure. However we can't assume the Projection
@@ -51,14 +50,14 @@ case class Expand(
// create the projections within each of the partition processing.
val groups = projections.map(ee => newProjection(ee, child.output)).toArray
- new Iterator[Row] {
- private[this] var result: Row = _
+ new Iterator[InternalRow] {
+ private[this] var result: InternalRow = _
private[this] var idx = -1 // -1 means the initial state
- private[this] var input: Row = _
+ private[this] var input: InternalRow = _
override final def hasNext: Boolean = (-1 < idx && idx < groups.length) || iter.hasNext
- override final def next(): Row = {
+ override final def next(): InternalRow = {
if (idx <= 0) {
// in the initial (-1) or beginning(0) of a new input row, fetch the next input tuple
input = iter.next()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
index dd02c1f457..c1665f78a9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
@@ -25,12 +25,12 @@ import org.apache.spark.sql.catalyst.expressions._
* For lazy computing, be sure the generator.terminate() called in the very last
* TODO reusing the CompletionIterator?
*/
-private[execution] sealed case class LazyIterator(func: () => TraversableOnce[Row])
- extends Iterator[Row] {
+private[execution] sealed case class LazyIterator(func: () => TraversableOnce[InternalRow])
+ extends Iterator[InternalRow] {
lazy val results = func().toIterator
override def hasNext: Boolean = results.hasNext
- override def next(): Row = results.next()
+ override def next(): InternalRow = results.next()
}
/**
@@ -58,11 +58,11 @@ case class Generate(
val boundGenerator = BindReferences.bindReference(generator, child.output)
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
// boundGenerator.terminate() should be triggered after all of the rows in the partition
if (join) {
child.execute().mapPartitions { iter =>
- val generatorNullRow = Row.fromSeq(Seq.fill[Any](generator.elementTypes.size)(null))
+ val generatorNullRow = InternalRow.fromSeq(Seq.fill[Any](generator.elementTypes.size)(null))
val joinedRow = new JoinedRow
iter.flatMap { row =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
index 1c40a9209f..ba2c8f53d7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala
@@ -66,7 +66,7 @@ case class GeneratedAggregate(
override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val aggregatesToCompute = aggregateExpressions.flatMap { a =>
a.collect { case agg: AggregateExpression => agg}
}
@@ -273,7 +273,7 @@ case class GeneratedAggregate(
if (groupingExpressions.isEmpty) {
// TODO: Codegening anything other than the updateProjection is probably over kill.
val buffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow]
- var currentRow: Row = null
+ var currentRow: InternalRow = null
updateProjection.target(buffer)
while (iter.hasNext) {
@@ -295,19 +295,19 @@ case class GeneratedAggregate(
)
while (iter.hasNext) {
- val currentRow: Row = iter.next()
- val groupKey: Row = groupProjection(currentRow)
+ val currentRow: InternalRow = iter.next()
+ val groupKey: InternalRow = groupProjection(currentRow)
val aggregationBuffer = aggregationMap.getAggregationBuffer(groupKey)
updateProjection.target(aggregationBuffer)(joinedRow(aggregationBuffer, currentRow))
}
- new Iterator[Row] {
+ new Iterator[InternalRow] {
private[this] val mapIterator = aggregationMap.iterator()
private[this] val resultProjection = resultProjectionBuilder()
def hasNext: Boolean = mapIterator.hasNext
- def next(): Row = {
+ def next(): InternalRow = {
val entry = mapIterator.next()
val result = resultProjection(joinedRow(entry.key, entry.value))
if (hasNext) {
@@ -326,9 +326,9 @@ case class GeneratedAggregate(
if (unsafeEnabled) {
log.info("Not using Unsafe-based aggregator because it is not supported for this schema")
}
- val buffers = new java.util.HashMap[Row, MutableRow]()
+ val buffers = new java.util.HashMap[InternalRow, MutableRow]()
- var currentRow: Row = null
+ var currentRow: InternalRow = null
while (iter.hasNext) {
currentRow = iter.next()
val currentGroup = groupProjection(currentRow)
@@ -342,13 +342,13 @@ case class GeneratedAggregate(
updateProjection.target(currentBuffer)(joinedRow(currentBuffer, currentRow))
}
- new Iterator[Row] {
+ new Iterator[InternalRow] {
private[this] val resultIterator = buffers.entrySet.iterator()
private[this] val resultProjection = resultProjectionBuilder()
def hasNext: Boolean = resultIterator.hasNext
- def next(): Row = {
+ def next(): InternalRow = {
val currentGroup = resultIterator.next()
resultProjection(joinedRow(currentGroup.getKey, currentGroup.getValue))
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
index 03bee80ad7..cd341180b6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala
@@ -19,18 +19,20 @@ package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.CatalystTypeConverters
+import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.expressions.Attribute
/**
* Physical plan node for scanning data from a local collection.
*/
-private[sql] case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNode {
+private[sql] case class LocalTableScan(
+ output: Seq[Attribute],
+ rows: Seq[InternalRow]) extends LeafNode {
private lazy val rdd = sqlContext.sparkContext.parallelize(rows)
- protected override def doExecute(): RDD[Row] = rdd
+ protected override def doExecute(): RDD[InternalRow] = rdd
override def executeCollect(): Array[Row] = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 435ac01117..7739a9f949 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -23,6 +23,7 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope}
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees}
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.physical._
@@ -79,11 +80,11 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil)
/**
- * Returns the result of this query as an RDD[Row] by delegating to doExecute
+ * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute
* after adding query plan information to created RDDs for visualization.
* Concrete implementations of SparkPlan should override doExecute instead.
*/
- final def execute(): RDD[Row] = {
+ final def execute(): RDD[InternalRow] = {
RDDOperationScope.withScope(sparkContext, nodeName, false, true) {
doExecute()
}
@@ -91,9 +92,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
/**
* Overridden by concrete implementations of SparkPlan.
- * Produces the result of the query as an RDD[Row]
+ * Produces the result of the query as an RDD[InternalRow]
*/
- protected def doExecute(): RDD[Row]
+ protected def doExecute(): RDD[InternalRow]
/**
* Runs this query returning the result as an array.
@@ -117,7 +118,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
val childRDD = execute().map(_.copy())
- val buf = new ArrayBuffer[Row]
+ val buf = new ArrayBuffer[InternalRow]
val totalParts = childRDD.partitions.length
var partsScanned = 0
while (buf.size < n && partsScanned < totalParts) {
@@ -140,7 +141,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
val sc = sqlContext.sparkContext
val res =
- sc.runJob(childRDD, (it: Iterator[Row]) => it.take(left).toArray, p, allowLocal = false)
+ sc.runJob(childRDD, (it: Iterator[InternalRow]) => it.take(left).toArray, p,
+ allowLocal = false)
res.foreach(buf ++= _.take(n - buf.size))
partsScanned += numPartsToTry
@@ -175,7 +177,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
protected def newPredicate(
- expression: Expression, inputSchema: Seq[Attribute]): (Row) => Boolean = {
+ expression: Expression, inputSchema: Seq[Attribute]): (InternalRow) => Boolean = {
if (codegenEnabled) {
GeneratePredicate.generate(expression, inputSchema)
} else {
@@ -183,7 +185,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
}
}
- protected def newOrdering(order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[Row] = {
+ protected def newOrdering(
+ order: Seq[SortOrder],
+ inputSchema: Seq[Attribute]): Ordering[InternalRow] = {
if (codegenEnabled) {
GenerateOrdering.generate(order, inputSchema)
} else {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 7a1331a391..422992d019 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -203,7 +203,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
}
protected lazy val singleRowRdd =
- sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1)
+ sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): InternalRow), 1)
object TakeOrdered extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
index c4327ce262..fd6f1d7ae1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala
@@ -20,9 +20,8 @@ package org.apache.spark.sql.execution
import java.util
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, ClusteredDistribution, Partitioning}
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning}
import org.apache.spark.util.collection.CompactBuffer
/**
@@ -112,16 +111,16 @@ case class Window(
}
}
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().mapPartitions { iter =>
- new Iterator[Row] {
+ new Iterator[InternalRow] {
// Although input rows are grouped based on windowSpec.partitionSpec, we need to
// know when we have a new partition.
// This is to manually construct an ordering that can be used to compare rows.
// TODO: We may want to have a newOrdering that takes BoundReferences.
// So, we can take advantave of code gen.
- private val partitionOrdering: Ordering[Row] =
+ private val partitionOrdering: Ordering[InternalRow] =
RowOrdering.forSchema(windowSpec.partitionSpec.map(_.dataType))
// This is used to project expressions for the partition specification.
@@ -137,13 +136,13 @@ case class Window(
// The number of buffered rows in the inputRowBuffer (the size of the current partition).
var partitionSize: Int = 0
// The buffer used to buffer rows in a partition.
- var inputRowBuffer: CompactBuffer[Row] = _
+ var inputRowBuffer: CompactBuffer[InternalRow] = _
// The partition key of the current partition.
- var currentPartitionKey: Row = _
+ var currentPartitionKey: InternalRow = _
// The partition key of next partition.
- var nextPartitionKey: Row = _
+ var nextPartitionKey: InternalRow = _
// The first row of next partition.
- var firstRowInNextPartition: Row = _
+ var firstRowInNextPartition: InternalRow = _
// Indicates if this partition is the last one in the iter.
var lastPartition: Boolean = false
@@ -316,7 +315,7 @@ case class Window(
!lastPartition || (rowPosition < partitionSize)
}
- override final def next(): Row = {
+ override final def next(): InternalRow = {
if (hasNext) {
if (rowPosition == partitionSize) {
// All rows of this buffer have been consumed.
@@ -353,7 +352,7 @@ case class Window(
// Fetch the next partition.
private def fetchNextPartition(): Unit = {
// Create a new buffer for input rows.
- inputRowBuffer = new CompactBuffer[Row]()
+ inputRowBuffer = new CompactBuffer[InternalRow]()
// We already have the first row for this partition
// (recorded in firstRowInNextPartition). Add it back.
inputRowBuffer += firstRowInNextPartition
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index fb42072f9d..7aedd630e3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -17,16 +17,17 @@
package org.apache.spark.sql.execution
-import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.{RDD, ShuffledRDD}
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.util.{CompletionIterator, MutablePair}
import org.apache.spark.util.collection.ExternalSorter
+import org.apache.spark.util.{CompletionIterator, MutablePair}
+import org.apache.spark.{HashPartitioner, SparkEnv}
/**
* :: DeveloperApi ::
@@ -37,7 +38,7 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends
@transient lazy val buildProjection = newMutableProjection(projectList, child.output)
- protected override def doExecute(): RDD[Row] = child.execute().mapPartitions { iter =>
+ protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter =>
val resuableProjection = buildProjection()
iter.map(resuableProjection)
}
@@ -52,9 +53,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends
case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
- @transient lazy val conditionEvaluator: (Row) => Boolean = newPredicate(condition, child.output)
+ @transient lazy val conditionEvaluator: (InternalRow) => Boolean =
+ newPredicate(condition, child.output)
- protected override def doExecute(): RDD[Row] = child.execute().mapPartitions { iter =>
+ protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter =>
iter.filter(conditionEvaluator)
}
@@ -83,7 +85,7 @@ case class Sample(
override def output: Seq[Attribute] = child.output
// TODO: How to pick seed?
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
if (withReplacement) {
child.execute().map(_.copy()).sample(withReplacement, upperBound - lowerBound, seed)
} else {
@@ -99,7 +101,8 @@ case class Sample(
case class Union(children: Seq[SparkPlan]) extends SparkPlan {
// TODO: attributes output by union should be distinct for nullability purposes
override def output: Seq[Attribute] = children.head.output
- protected override def doExecute(): RDD[Row] = sparkContext.union(children.map(_.execute()))
+ protected override def doExecute(): RDD[InternalRow] =
+ sparkContext.union(children.map(_.execute()))
}
/**
@@ -124,19 +127,19 @@ case class Limit(limit: Int, child: SparkPlan)
override def executeCollect(): Array[Row] = child.executeTake(limit)
- protected override def doExecute(): RDD[Row] = {
- val rdd: RDD[_ <: Product2[Boolean, Row]] = if (sortBasedShuffleOn) {
+ protected override def doExecute(): RDD[InternalRow] = {
+ val rdd: RDD[_ <: Product2[Boolean, InternalRow]] = if (sortBasedShuffleOn) {
child.execute().mapPartitions { iter =>
iter.take(limit).map(row => (false, row.copy()))
}
} else {
child.execute().mapPartitions { iter =>
- val mutablePair = new MutablePair[Boolean, Row]()
+ val mutablePair = new MutablePair[Boolean, InternalRow]()
iter.take(limit).map(row => mutablePair.update(false, row))
}
}
val part = new HashPartitioner(1)
- val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part)
+ val shuffled = new ShuffledRDD[Boolean, InternalRow, InternalRow](rdd, part)
shuffled.setSerializer(new SparkSqlSerializer(child.sqlContext.sparkContext.getConf))
shuffled.mapPartitions(_.take(limit).map(_._2))
}
@@ -157,7 +160,8 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
private val ord: RowOrdering = new RowOrdering(sortOrder, child.output)
- private def collectData(): Array[Row] = child.execute().map(_.copy()).takeOrdered(limit)(ord)
+ private def collectData(): Array[InternalRow] =
+ child.execute().map(_.copy()).takeOrdered(limit)(ord)
override def executeCollect(): Array[Row] = {
val converter = CatalystTypeConverters.createToScalaConverter(schema)
@@ -166,7 +170,7 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
// TODO: Terminal split should be implemented differently from non-terminal split.
// TODO: Pick num splits based on |limit|.
- protected override def doExecute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1)
+ protected override def doExecute(): RDD[InternalRow] = sparkContext.makeRDD(collectData(), 1)
override def outputOrdering: Seq[SortOrder] = sortOrder
}
@@ -186,7 +190,7 @@ case class Sort(
override def requiredChildDistribution: Seq[Distribution] =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
- protected override def doExecute(): RDD[Row] = attachTree(this, "sort") {
+ protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") {
child.execute().mapPartitions( { iterator =>
val ordering = newOrdering(sortOrder, child.output)
iterator.map(_.copy()).toArray.sorted(ordering).iterator
@@ -214,14 +218,14 @@ case class ExternalSort(
override def requiredChildDistribution: Seq[Distribution] =
if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil
- protected override def doExecute(): RDD[Row] = attachTree(this, "sort") {
+ protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") {
child.execute().mapPartitions( { iterator =>
val ordering = newOrdering(sortOrder, child.output)
- val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering))
+ val sorter = new ExternalSorter[InternalRow, Null, InternalRow](ordering = Some(ordering))
sorter.insertAll(iterator.map(r => (r.copy, null)))
val baseIterator = sorter.iterator.map(_._1)
// TODO(marmbrus): The complex type signature below thwarts inference for no reason.
- CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop())
+ CompletionIterator[InternalRow, Iterator[InternalRow]](baseIterator, sorter.stop())
}, preservesPartitioning = true)
}
@@ -239,7 +243,7 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: SparkPlan)
extends UnaryNode {
override def output: Seq[Attribute] = child.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().map(_.copy()).coalesce(numPartitions, shuffle)
}
}
@@ -254,7 +258,7 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: SparkPlan)
case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = left.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
left.execute().map(_.copy()).subtract(right.execute().map(_.copy()))
}
}
@@ -268,7 +272,7 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode {
case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = children.head.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
left.execute().map(_.copy()).intersection(right.execute().map(_.copy()))
}
}
@@ -283,5 +287,5 @@ case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode {
case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan {
def children: Seq[SparkPlan] = child :: Nil
- protected override def doExecute(): RDD[Row] = child.execute()
+ protected override def doExecute(): RDD[InternalRow] = child.execute()
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 49b361e96b..653792ea2e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -20,13 +20,13 @@ package org.apache.spark.sql.execution
import org.apache.spark.Logging
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.CatalystTypeConverters
+import org.apache.spark.sql.catalyst.{InternalRow, CatalystTypeConverters}
import org.apache.spark.sql.catalyst.errors.TreeNodeException
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext}
+import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext}
/**
* A logical command that is executed for its side-effects. `RunnableCommand`s are
@@ -64,9 +64,9 @@ private[sql] case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan
override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val converted = sideEffectResult.map(r =>
- CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[Row])
+ CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[InternalRow])
sqlContext.sparkContext.parallelize(converted, 1)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
index 83c1f65d5c..3ee4033bae 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.unsafe.types.UTF8String
@@ -25,7 +26,7 @@ import scala.collection.mutable.HashSet
import org.apache.spark.{AccumulatorParam, Accumulator}
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.{SQLConf, SQLContext, DataFrame, Row}
+import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
import org.apache.spark.sql.types._
@@ -126,11 +127,11 @@ package object debug {
}
}
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().mapPartitions { iter =>
- new Iterator[Row] {
+ new Iterator[InternalRow] {
def hasNext: Boolean = iter.hasNext
- def next(): Row = {
+ def next(): InternalRow = {
val currentRow = iter.next()
tupleCount += 1
var i = 0
@@ -155,7 +156,7 @@ package object debug {
def typeCheck(data: Any, schema: DataType): Unit = (data, schema) match {
case (null, _) =>
- case (row: Row, StructType(fields)) =>
+ case (row: InternalRow, StructType(fields)) =>
row.toSeq.zip(fields.map(_.dataType)).foreach { case(d, t) => typeCheck(d, t) }
case (s: Seq[_], ArrayType(elemType, _)) =>
s.foreach(typeCheck(_, elemType))
@@ -196,7 +197,7 @@ package object debug {
def children: List[SparkPlan] = child :: Nil
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().map { row =>
try typeCheck(row, child.schema) catch {
case e: Exception =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala
index e228a60c90..68914cf85c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/MonotonicallyIncreasingID.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.expressions
import org.apache.spark.TaskContext
-import org.apache.spark.sql.catalyst.expressions.{Row, LeafExpression}
+import org.apache.spark.sql.catalyst.expressions.{InternalRow, LeafExpression}
import org.apache.spark.sql.types.{LongType, DataType}
/**
@@ -43,7 +43,7 @@ private[sql] case class MonotonicallyIncreasingID() extends LeafExpression {
override def dataType: DataType = LongType
- override def eval(input: Row): Long = {
+ override def eval(input: InternalRow): Long = {
val currentCount = count
count += 1
(TaskContext.get().partitionId().toLong << 33) + currentCount
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala
index 1272793f88..12c2eed0d6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/expressions/SparkPartitionID.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.expressions
import org.apache.spark.TaskContext
-import org.apache.spark.sql.catalyst.expressions.{LeafExpression, Row}
+import org.apache.spark.sql.catalyst.expressions.{LeafExpression, InternalRow}
import org.apache.spark.sql.types.{IntegerType, DataType}
@@ -31,5 +31,5 @@ private[sql] case object SparkPartitionID extends LeafExpression {
override def dataType: DataType = IntegerType
- override def eval(input: Row): Int = TaskContext.get().partitionId()
+ override def eval(input: InternalRow): Int = TaskContext.get().partitionId()
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
index b8b12be875..2d2e1b92b8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala
@@ -17,16 +17,15 @@
package org.apache.spark.sql.execution.joins
-import org.apache.spark.rdd.RDD
-import org.apache.spark.util.ThreadUtils
-
import scala.concurrent._
import scala.concurrent.duration._
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.catalyst.expressions.{Row, Expression}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.expressions.{Expression, InternalRow}
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, Partitioning, UnspecifiedDistribution}
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
+import org.apache.spark.util.ThreadUtils
/**
* :: DeveloperApi ::
@@ -61,12 +60,12 @@ case class BroadcastHashJoin(
@transient
private val broadcastFuture = future {
// Note that we use .execute().collect() because we don't want to convert data to Scala types
- val input: Array[Row] = buildPlan.execute().map(_.copy()).collect()
+ val input: Array[InternalRow] = buildPlan.execute().map(_.copy()).collect()
val hashed = HashedRelation(input.iterator, buildSideKeyGenerator, input.length)
sparkContext.broadcast(hashed)
}(BroadcastHashJoin.broadcastHashJoinExecutionContext)
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val broadcastRelation = Await.result(broadcastFuture, timeout)
streamedPlan.execute().mapPartitions { streamedIter =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala
index a32e5fc4f7..044964f3a3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastLeftSemiJoinHash.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row}
+import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
/**
@@ -38,10 +38,10 @@ case class BroadcastLeftSemiJoinHash(
override def output: Seq[Attribute] = left.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val buildIter = buildPlan.execute().map(_.copy()).collect().toIterator
- val hashSet = new java.util.HashSet[Row]()
- var currentRow: Row = null
+ val hashSet = new java.util.HashSet[InternalRow]()
+ var currentRow: InternalRow = null
// Create a Hash set of buildKeys
while (buildIter.hasNext) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
index caad3dfbe1..0b2cf8e12a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
@@ -61,13 +61,14 @@ case class BroadcastNestedLoopJoin(
@transient private lazy val boundCondition =
newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output)
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val broadcastedRelation =
- sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq)
+ sparkContext.broadcast(broadcast.execute().map(_.copy())
+ .collect().toIndexedSeq)
/** All rows that either match both-way, or rows from streamed joined with nulls. */
val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter =>
- val matchedRows = new CompactBuffer[Row]
+ val matchedRows = new CompactBuffer[InternalRow]
// TODO: Use Spark's BitSet.
val includedBroadcastTuples =
new scala.collection.mutable.BitSet(broadcastedRelation.value.size)
@@ -118,8 +119,8 @@ case class BroadcastNestedLoopJoin(
val leftNulls = new GenericMutableRow(left.output.size)
val rightNulls = new GenericMutableRow(right.output.size)
/** Rows from broadcasted joined with nulls. */
- val broadcastRowsWithNulls: Seq[Row] = {
- val buf: CompactBuffer[Row] = new CompactBuffer()
+ val broadcastRowsWithNulls: Seq[InternalRow] = {
+ val buf: CompactBuffer[InternalRow] = new CompactBuffer()
var i = 0
val rel = broadcastedRelation.value
while (i < rel.length) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
index 191c00cb55..261b472415 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow}
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
@@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
override def output: Seq[Attribute] = left.output ++ right.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val leftResults = left.execute().map(_.copy())
val rightResults = right.execute().map(_.copy())
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
index 851de16855..3a4196a90d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala
@@ -49,11 +49,13 @@ trait HashJoin {
@transient protected lazy val streamSideKeyGenerator: () => MutableProjection =
newMutableProjection(streamedKeys, streamedPlan.output)
- protected def hashJoin(streamIter: Iterator[Row], hashedRelation: HashedRelation): Iterator[Row] =
+ protected def hashJoin(
+ streamIter: Iterator[InternalRow],
+ hashedRelation: HashedRelation): Iterator[InternalRow] =
{
- new Iterator[Row] {
- private[this] var currentStreamedRow: Row = _
- private[this] var currentHashMatches: CompactBuffer[Row] = _
+ new Iterator[InternalRow] {
+ private[this] var currentStreamedRow: InternalRow = _
+ private[this] var currentHashMatches: CompactBuffer[InternalRow] = _
private[this] var currentMatchPosition: Int = -1
// Mutable per row objects.
@@ -65,7 +67,7 @@ trait HashJoin {
(currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) ||
(streamIter.hasNext && fetchNext())
- override final def next(): Row = {
+ override final def next(): InternalRow = {
val ret = buildSide match {
case BuildRight => joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition))
case BuildLeft => joinRow(currentHashMatches(currentMatchPosition), currentStreamedRow)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala
index c21a453115..19aef9978e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala
@@ -68,26 +68,29 @@ case class HashOuterJoin(
}
}
- @transient private[this] lazy val DUMMY_LIST = Seq[Row](null)
- @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row]
+ @transient private[this] lazy val DUMMY_LIST = Seq[InternalRow](null)
+ @transient private[this] lazy val EMPTY_LIST = Seq.empty[InternalRow]
@transient private[this] lazy val leftNullRow = new GenericRow(left.output.length)
@transient private[this] lazy val rightNullRow = new GenericRow(right.output.length)
@transient private[this] lazy val boundCondition =
- condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true)
+ condition.map(
+ newPredicate(_, left.output ++ right.output)).getOrElse((row: InternalRow) => true)
// TODO we need to rewrite all of the iterators with our own implementation instead of the Scala
// iterator for performance purpose.
private[this] def leftOuterIterator(
- key: Row, joinedRow: JoinedRow, rightIter: Iterable[Row]): Iterator[Row] = {
- val ret: Iterable[Row] = {
+ key: InternalRow,
+ joinedRow: JoinedRow,
+ rightIter: Iterable[InternalRow]): Iterator[InternalRow] = {
+ val ret: Iterable[InternalRow] = {
if (!key.anyNull) {
val temp = rightIter.collect {
case r if boundCondition(joinedRow.withRight(r)) => joinedRow.copy()
}
if (temp.size == 0) {
- joinedRow.withRight(rightNullRow).copy :: Nil
+ joinedRow.withRight(rightNullRow).copy.asInstanceOf[InternalRow] :: Nil
} else {
temp
}
@@ -99,12 +102,15 @@ case class HashOuterJoin(
}
private[this] def rightOuterIterator(
- key: Row, leftIter: Iterable[Row], joinedRow: JoinedRow): Iterator[Row] = {
+ key: InternalRow,
+ leftIter: Iterable[InternalRow],
+ joinedRow: JoinedRow): Iterator[InternalRow] = {
- val ret: Iterable[Row] = {
+ val ret: Iterable[InternalRow] = {
if (!key.anyNull) {
val temp = leftIter.collect {
- case l if boundCondition(joinedRow.withLeft(l)) => joinedRow.copy
+ case l if boundCondition(joinedRow.withLeft(l)) =>
+ joinedRow.copy
}
if (temp.size == 0) {
joinedRow.withLeft(leftNullRow).copy :: Nil
@@ -119,14 +125,14 @@ case class HashOuterJoin(
}
private[this] def fullOuterIterator(
- key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row],
- joinedRow: JoinedRow): Iterator[Row] = {
+ key: InternalRow, leftIter: Iterable[InternalRow], rightIter: Iterable[InternalRow],
+ joinedRow: JoinedRow): Iterator[InternalRow] = {
if (!key.anyNull) {
// Store the positions of records in right, if one of its associated row satisfy
// the join condition.
val rightMatchedSet = scala.collection.mutable.Set[Int]()
- leftIter.iterator.flatMap[Row] { l =>
+ leftIter.iterator.flatMap[InternalRow] { l =>
joinedRow.withLeft(l)
var matched = false
rightIter.zipWithIndex.collect {
@@ -157,24 +163,25 @@ case class HashOuterJoin(
joinedRow(leftNullRow, r).copy()
}
} else {
- leftIter.iterator.map[Row] { l =>
+ leftIter.iterator.map[InternalRow] { l =>
joinedRow(l, rightNullRow).copy()
- } ++ rightIter.iterator.map[Row] { r =>
+ } ++ rightIter.iterator.map[InternalRow] { r =>
joinedRow(leftNullRow, r).copy()
}
}
}
private[this] def buildHashTable(
- iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, CompactBuffer[Row]] = {
- val hashTable = new JavaHashMap[Row, CompactBuffer[Row]]()
+ iter: Iterator[InternalRow],
+ keyGenerator: Projection): JavaHashMap[InternalRow, CompactBuffer[InternalRow]] = {
+ val hashTable = new JavaHashMap[InternalRow, CompactBuffer[InternalRow]]()
while (iter.hasNext) {
val currentRow = iter.next()
val rowKey = keyGenerator(currentRow)
var existingMatchList = hashTable.get(rowKey)
if (existingMatchList == null) {
- existingMatchList = new CompactBuffer[Row]()
+ existingMatchList = new CompactBuffer[InternalRow]()
hashTable.put(rowKey, existingMatchList)
}
@@ -184,7 +191,7 @@ case class HashOuterJoin(
hashTable
}
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val joinedRow = new JoinedRow()
left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) =>
// TODO this probably can be replaced by external sort (sort merged join?)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
index ab84c123e0..e18c817975 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.joins
import java.io.{ObjectInput, ObjectOutput, Externalizable}
import java.util.{HashMap => JavaHashMap}
-import org.apache.spark.sql.catalyst.expressions.{Projection, Row}
+import org.apache.spark.sql.catalyst.expressions.{Projection, InternalRow}
import org.apache.spark.sql.execution.SparkSqlSerializer
import org.apache.spark.util.collection.CompactBuffer
@@ -30,7 +30,7 @@ import org.apache.spark.util.collection.CompactBuffer
* object.
*/
private[joins] sealed trait HashedRelation {
- def get(key: Row): CompactBuffer[Row]
+ def get(key: InternalRow): CompactBuffer[InternalRow]
// This is a helper method to implement Externalizable, and is used by
// GeneralHashedRelation and UniqueKeyHashedRelation
@@ -54,12 +54,12 @@ private[joins] sealed trait HashedRelation {
* A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values.
*/
private[joins] final class GeneralHashedRelation(
- private var hashTable: JavaHashMap[Row, CompactBuffer[Row]])
+ private var hashTable: JavaHashMap[InternalRow, CompactBuffer[InternalRow]])
extends HashedRelation with Externalizable {
def this() = this(null) // Needed for serialization
- override def get(key: Row): CompactBuffer[Row] = hashTable.get(key)
+ override def get(key: InternalRow): CompactBuffer[InternalRow] = hashTable.get(key)
override def writeExternal(out: ObjectOutput): Unit = {
writeBytes(out, SparkSqlSerializer.serialize(hashTable))
@@ -75,17 +75,18 @@ private[joins] final class GeneralHashedRelation(
* A specialized [[HashedRelation]] that maps key into a single value. This implementation
* assumes the key is unique.
*/
-private[joins] final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[Row, Row])
+private[joins]
+final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[InternalRow, InternalRow])
extends HashedRelation with Externalizable {
def this() = this(null) // Needed for serialization
- override def get(key: Row): CompactBuffer[Row] = {
+ override def get(key: InternalRow): CompactBuffer[InternalRow] = {
val v = hashTable.get(key)
if (v eq null) null else CompactBuffer(v)
}
- def getValue(key: Row): Row = hashTable.get(key)
+ def getValue(key: InternalRow): InternalRow = hashTable.get(key)
override def writeExternal(out: ObjectOutput): Unit = {
writeBytes(out, SparkSqlSerializer.serialize(hashTable))
@@ -103,13 +104,13 @@ private[joins] final class UniqueKeyHashedRelation(private var hashTable: JavaHa
private[joins] object HashedRelation {
def apply(
- input: Iterator[Row],
+ input: Iterator[InternalRow],
keyGenerator: Projection,
sizeEstimate: Int = 64): HashedRelation = {
// TODO: Use Spark's HashMap implementation.
- val hashTable = new JavaHashMap[Row, CompactBuffer[Row]](sizeEstimate)
- var currentRow: Row = null
+ val hashTable = new JavaHashMap[InternalRow, CompactBuffer[InternalRow]](sizeEstimate)
+ var currentRow: InternalRow = null
// Whether the join key is unique. If the key is unique, we can convert the underlying
// hash map into one specialized for this.
@@ -122,7 +123,7 @@ private[joins] object HashedRelation {
if (!rowKey.anyNull) {
val existingMatchList = hashTable.get(rowKey)
val matchList = if (existingMatchList == null) {
- val newMatchList = new CompactBuffer[Row]()
+ val newMatchList = new CompactBuffer[InternalRow]()
hashTable.put(rowKey, newMatchList)
newMatchList
} else {
@@ -134,7 +135,7 @@ private[joins] object HashedRelation {
}
if (keyIsUnique) {
- val uniqHashTable = new JavaHashMap[Row, Row](hashTable.size)
+ val uniqHashTable = new JavaHashMap[InternalRow, InternalRow](hashTable.size)
val iter = hashTable.entrySet().iterator()
while (iter.hasNext) {
val entry = iter.next()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala
index 036423e6fa..2a6d4d1ab0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala
@@ -47,7 +47,7 @@ case class LeftSemiJoinBNL(
@transient private lazy val boundCondition =
newPredicate(condition.getOrElse(Literal(true)), left.output ++ right.output)
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val broadcastedRelation =
sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala
index 8ad27eae80..20d74270af 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, InternalRow}
import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
@@ -42,10 +42,10 @@ case class LeftSemiJoinHash(
override def output: Seq[Attribute] = left.output
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) =>
- val hashSet = new java.util.HashSet[Row]()
- var currentRow: Row = null
+ val hashSet = new java.util.HashSet[InternalRow]()
+ var currentRow: InternalRow = null
// Create a Hash set of buildKeys
while (buildIter.hasNext) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala
index 219525d9d8..5439e10a60 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.joins
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning}
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
@@ -43,7 +43,7 @@ case class ShuffledHashJoin(
override def requiredChildDistribution: Seq[ClusteredDistribution] =
ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) =>
val hashed = HashedRelation(buildIter, buildSideKeyGenerator)
hashJoin(streamIter, hashed)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
index 1a39fb4b96..2abe65a718 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
@@ -21,9 +21,7 @@ import java.util.NoSuchElementException
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
import org.apache.spark.util.collection.CompactBuffer
@@ -60,29 +58,29 @@ case class SortMergeJoin(
private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] =
keys.map(SortOrder(_, Ascending))
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val leftResults = left.execute().map(_.copy())
val rightResults = right.execute().map(_.copy())
leftResults.zipPartitions(rightResults) { (leftIter, rightIter) =>
- new Iterator[Row] {
+ new Iterator[InternalRow] {
// Mutable per row objects.
private[this] val joinRow = new JoinedRow5
- private[this] var leftElement: Row = _
- private[this] var rightElement: Row = _
- private[this] var leftKey: Row = _
- private[this] var rightKey: Row = _
- private[this] var rightMatches: CompactBuffer[Row] = _
+ private[this] var leftElement: InternalRow = _
+ private[this] var rightElement: InternalRow = _
+ private[this] var leftKey: InternalRow = _
+ private[this] var rightKey: InternalRow = _
+ private[this] var rightMatches: CompactBuffer[InternalRow] = _
private[this] var rightPosition: Int = -1
private[this] var stop: Boolean = false
- private[this] var matchKey: Row = _
+ private[this] var matchKey: InternalRow = _
// initialize iterator
initialize()
override final def hasNext: Boolean = nextMatchingPair()
- override final def next(): Row = {
+ override final def next(): InternalRow = {
if (hasNext) {
// we are using the buffered right rows and run down left iterator
val joinedRow = joinRow(leftElement, rightMatches(rightPosition))
@@ -145,7 +143,7 @@ case class SortMergeJoin(
fetchLeft()
}
}
- rightMatches = new CompactBuffer[Row]()
+ rightMatches = new CompactBuffer[InternalRow]()
if (stop) {
stop = false
// iterate the right side to buffer all rows that matches
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
index 2b45a83d14..1ce150ceaf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala
@@ -29,7 +29,8 @@ import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.python.{PythonBroadcast, PythonRDD}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Row, _}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.Rule
@@ -56,7 +57,7 @@ private[spark] case class PythonUDF(
def nullable: Boolean = true
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
throw new UnsupportedOperationException("PythonUDFs can not be directly evaluated.")
}
}
@@ -241,7 +242,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child:
def children: Seq[SparkPlan] = child :: Nil
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
val childResults = child.execute().map(_.copy())
val parent = childResults.mapPartitions { iter =>
@@ -276,7 +277,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child:
val row = new GenericMutableRow(1)
iter.map { result =>
row(0) = EvaluatePython.fromJava(result, udf.dataType)
- row: Row
+ row: InternalRow
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
index c41c21c0ee..8df1da037c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/FrequentItems.scala
@@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.stat
import scala.collection.mutable.{Map => MutableMap}
import org.apache.spark.Logging
-import org.apache.spark.sql.{Column, DataFrame, Row}
+import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.types.{ArrayType, StructField, StructType}
+import org.apache.spark.sql.{Column, DataFrame}
private[sql] object FrequentItems extends Logging {
@@ -110,7 +111,7 @@ private[sql] object FrequentItems extends Logging {
}
)
val justItems = freqItems.map(m => m.baseMap.keys.toSeq)
- val resultRow = Row(justItems : _*)
+ val resultRow = InternalRow(justItems : _*)
// append frequent Items to the column name for easy debugging
val outputCols = colInfo.map { v =>
StructField(v._1 + "_freqItems", ArrayType(v._2, false))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala
index e75e6681c5..667fc70cff 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala
@@ -24,7 +24,7 @@ import org.apache.commons.lang3.StringUtils
import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow}
+import org.apache.spark.sql.catalyst.expressions.{InternalRow, SpecificMutableRow}
import org.apache.spark.sql.catalyst.util.DateUtils
import org.apache.spark.sql.types._
import org.apache.spark.sql.sources._
@@ -211,7 +211,7 @@ private[sql] object JDBCRDD extends Logging {
fqTable: String,
requiredColumns: Array[String],
filters: Array[Filter],
- parts: Array[Partition]): RDD[Row] = {
+ parts: Array[Partition]): RDD[InternalRow] = {
val dialect = JdbcDialects.get(url)
val quotedColumns = requiredColumns.map(colName => dialect.quoteIdentifier(colName))
new JDBCRDD(
@@ -240,7 +240,7 @@ private[sql] class JDBCRDD(
filters: Array[Filter],
partitions: Array[Partition],
properties: Properties)
- extends RDD[Row](sc, Nil) {
+ extends RDD[InternalRow](sc, Nil) {
/**
* Retrieve the list of partitions corresponding to this RDD.
@@ -348,12 +348,12 @@ private[sql] class JDBCRDD(
/**
* Runs the SQL query against the JDBC driver.
*/
- override def compute(thePart: Partition, context: TaskContext): Iterator[Row] = new Iterator[Row]
- {
+ override def compute(thePart: Partition, context: TaskContext): Iterator[InternalRow] =
+ new Iterator[InternalRow] {
var closed = false
var finished = false
var gotNext = false
- var nextValue: Row = null
+ var nextValue: InternalRow = null
context.addTaskCompletionListener{ context => close() }
val part = thePart.asInstanceOf[JDBCPartition]
@@ -375,7 +375,7 @@ private[sql] class JDBCRDD(
val conversions = getConversions(schema)
val mutableRow = new SpecificMutableRow(schema.fields.map(x => x.dataType))
- def getNext(): Row = {
+ def getNext(): InternalRow = {
if (rs.next()) {
var i = 0
while (i < conversions.length) {
@@ -443,7 +443,7 @@ private[sql] class JDBCRDD(
mutableRow
} else {
finished = true
- null.asInstanceOf[Row]
+ null.asInstanceOf[InternalRow]
}
}
@@ -486,7 +486,7 @@ private[sql] class JDBCRDD(
!finished
}
- override def next(): Row = {
+ override def next(): InternalRow = {
if (!hasNext) {
throw new NoSuchElementException("End of stream")
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala
index 30f9190d45..4d3aac464c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala
@@ -23,10 +23,9 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.Partition
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext}
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode}
/**
* Instructions on how to partition the table among workers.
@@ -138,7 +137,7 @@ private[sql] case class JDBCRelation(
table,
requiredColumns,
filters,
- parts)
+ parts).map(_.asInstanceOf[Row])
}
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
index c772cd1f53..69bf13e1e5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala
@@ -22,10 +22,10 @@ import java.io.IOException
import org.apache.hadoop.fs.Path
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.sources._
-import org.apache.spark.sql.types.{StructField, StructType}
-import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode}
private[sql] class DefaultSource
@@ -154,12 +154,12 @@ private[sql] class JSONRelation(
JacksonParser(
baseRDD(),
schema,
- sqlContext.conf.columnNameOfCorruptRecord)
+ sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
} else {
JsonRDD.jsonStringToRow(
baseRDD(),
schema,
- sqlContext.conf.columnNameOfCorruptRecord)
+ sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}
}
@@ -168,12 +168,12 @@ private[sql] class JSONRelation(
JacksonParser(
baseRDD(),
StructType.fromAttributes(requiredColumns),
- sqlContext.conf.columnNameOfCorruptRecord)
+ sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
} else {
JsonRDD.jsonStringToRow(
baseRDD(),
StructType.fromAttributes(requiredColumns),
- sqlContext.conf.columnNameOfCorruptRecord)
+ sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala
index 325f54b680..1e6b1198d2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonGenerator.scala
@@ -21,7 +21,7 @@ import scala.collection.Map
import com.fasterxml.jackson.core._
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
private[sql] object JacksonGenerator {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala
index f16075ce58..817e8a20b3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala
@@ -35,7 +35,7 @@ private[sql] object JacksonParser {
def apply(
json: RDD[String],
schema: StructType,
- columnNameOfCorruptRecords: String): RDD[Row] = {
+ columnNameOfCorruptRecords: String): RDD[InternalRow] = {
parseJson(json, schema, columnNameOfCorruptRecords)
}
@@ -130,7 +130,10 @@ private[sql] object JacksonParser {
*
* Fields in the json that are not defined in the requested schema will be dropped.
*/
- private def convertObject(factory: JsonFactory, parser: JsonParser, schema: StructType): Row = {
+ private def convertObject(
+ factory: JsonFactory,
+ parser: JsonParser,
+ schema: StructType): InternalRow = {
val row = new GenericMutableRow(schema.length)
while (nextUntil(parser, JsonToken.END_OBJECT)) {
schema.getFieldIndex(parser.getCurrentName) match {
@@ -176,9 +179,9 @@ private[sql] object JacksonParser {
private def parseJson(
json: RDD[String],
schema: StructType,
- columnNameOfCorruptRecords: String): RDD[Row] = {
+ columnNameOfCorruptRecords: String): RDD[InternalRow] = {
- def failedRecord(record: String): Seq[Row] = {
+ def failedRecord(record: String): Seq[InternalRow] = {
// create a row even if no corrupt record column is present
val row = new GenericMutableRow(schema.length)
for (corruptIndex <- schema.getFieldIndex(columnNameOfCorruptRecords)) {
@@ -202,7 +205,7 @@ private[sql] object JacksonParser {
// convertField wrap an object into a single value array when necessary.
convertField(factory, parser, ArrayType(schema)) match {
case null => failedRecord(record)
- case list: Seq[Row @unchecked] => list
+ case list: Seq[InternalRow @unchecked] => list
case _ =>
sys.error(
s"Failed to parse record $record. Please make sure that each line of the file " +
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
index e4acf1ddaf..44594c5080 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala
@@ -38,7 +38,7 @@ private[sql] object JsonRDD extends Logging {
private[sql] def jsonStringToRow(
json: RDD[String],
schema: StructType,
- columnNameOfCorruptRecords: String): RDD[Row] = {
+ columnNameOfCorruptRecords: String): RDD[InternalRow] = {
parseJson(json, columnNameOfCorruptRecords).map(parsed => asRow(parsed, schema))
}
@@ -434,7 +434,7 @@ private[sql] object JsonRDD extends Logging {
}
}
- private def asRow(json: Map[String, Any], schema: StructType): Row = {
+ private def asRow(json: Map[String, Any], schema: StructType): InternalRow = {
// TODO: Reuse the row instead of creating a new one for every record.
val row = new GenericMutableRow(schema.fields.length)
schema.fields.zipWithIndex.foreach {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
index ab9f878d1e..4da5e96b82 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala
@@ -79,7 +79,7 @@ private[sql] object CatalystConverter {
// TODO: consider using Array[T] for arrays to avoid boxing of primitive types
type ArrayScalaType[T] = Seq[T]
- type StructScalaType[T] = Row
+ type StructScalaType[T] = InternalRow
type MapScalaType[K, V] = Map[K, V]
protected[parquet] def createConverter(
@@ -240,7 +240,7 @@ private[parquet] abstract class CatalystConverter extends GroupConverter {
*
* @return
*/
- def getCurrentRecord: Row = throw new UnsupportedOperationException
+ def getCurrentRecord: InternalRow = throw new UnsupportedOperationException
/**
* Read a decimal value from a Parquet Binary into "dest". Only supports decimals that fit in
@@ -275,7 +275,7 @@ private[parquet] abstract class CatalystConverter extends GroupConverter {
/**
* A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
- * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object.
+ * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object.
*
* @param schema The corresponding Catalyst schema in the form of a list of attributes.
*/
@@ -284,7 +284,7 @@ private[parquet] class CatalystGroupConverter(
protected[parquet] val index: Int,
protected[parquet] val parent: CatalystConverter,
protected[parquet] var current: ArrayBuffer[Any],
- protected[parquet] var buffer: ArrayBuffer[Row])
+ protected[parquet] var buffer: ArrayBuffer[InternalRow])
extends CatalystConverter {
def this(schema: Array[FieldType], index: Int, parent: CatalystConverter) =
@@ -293,7 +293,7 @@ private[parquet] class CatalystGroupConverter(
index,
parent,
current = null,
- buffer = new ArrayBuffer[Row](
+ buffer = new ArrayBuffer[InternalRow](
CatalystArrayConverter.INITIAL_ARRAY_SIZE))
/**
@@ -309,7 +309,7 @@ private[parquet] class CatalystGroupConverter(
override val size = schema.size
- override def getCurrentRecord: Row = {
+ override def getCurrentRecord: InternalRow = {
assert(isRootConverter, "getCurrentRecord should only be called in root group converter!")
// TODO: use iterators if possible
// Note: this will ever only be called in the root converter when the record has been
@@ -347,7 +347,7 @@ private[parquet] class CatalystGroupConverter(
/**
* A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
- * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
+ * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object. Note that his
* converter is optimized for rows of primitive types (non-nested records).
*/
private[parquet] class CatalystPrimitiveRowConverter(
@@ -373,7 +373,7 @@ private[parquet] class CatalystPrimitiveRowConverter(
override val parent = null
// Should be only called in root group converter!
- override def getCurrentRecord: Row = current
+ override def getCurrentRecord: InternalRow = current
override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
index 272608d4e2..39360e1331 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
@@ -46,7 +46,7 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SQLConf
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row, _}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, InternalRow, _}
import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
import org.apache.spark.sql.types.StructType
import org.apache.spark.{Logging, SerializableWritable, TaskContext}
@@ -54,7 +54,7 @@ import org.apache.spark.{Logging, SerializableWritable, TaskContext}
/**
* :: DeveloperApi ::
* Parquet table scan operator. Imports the file that backs the given
- * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``.
+ * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[InternalRow]``.
*/
private[sql] case class ParquetTableScan(
attributes: Seq[Attribute],
@@ -77,7 +77,7 @@ private[sql] case class ParquetTableScan(
}
}.toArray
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
import org.apache.parquet.filter2.compat.FilterCompat.FilterPredicateCompat
val sc = sqlContext.sparkContext
@@ -125,7 +125,7 @@ private[sql] case class ParquetTableScan(
sc,
classOf[FilteringParquetRowInputFormat],
classOf[Void],
- classOf[Row],
+ classOf[InternalRow],
conf)
if (requestedPartitionOrdinals.nonEmpty) {
@@ -154,9 +154,9 @@ private[sql] case class ParquetTableScan(
.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow))
if (primitiveRow) {
- new Iterator[Row] {
+ new Iterator[InternalRow] {
def hasNext: Boolean = iter.hasNext
- def next(): Row = {
+ def next(): InternalRow = {
// We are using CatalystPrimitiveRowConverter and it returns a SpecificMutableRow.
val row = iter.next()._2.asInstanceOf[SpecificMutableRow]
@@ -173,12 +173,12 @@ private[sql] case class ParquetTableScan(
} else {
// Create a mutable row since we need to fill in values from partition columns.
val mutableRow = new GenericMutableRow(outputSize)
- new Iterator[Row] {
+ new Iterator[InternalRow] {
def hasNext: Boolean = iter.hasNext
- def next(): Row = {
+ def next(): InternalRow = {
// We are using CatalystGroupConverter and it returns a GenericRow.
// Since GenericRow is not mutable, we just cast it to a Row.
- val row = iter.next()._2.asInstanceOf[Row]
+ val row = iter.next()._2.asInstanceOf[InternalRow]
var i = 0
while (i < row.size) {
@@ -258,7 +258,7 @@ private[sql] case class InsertIntoParquetTable(
/**
* Inserts all rows into the Parquet file.
*/
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
// TODO: currently we do not check whether the "schema"s are compatible
// That means if one first creates a table and then INSERTs data with
// and incompatible schema the execution will fail. It would be nice
@@ -321,13 +321,13 @@ private[sql] case class InsertIntoParquetTable(
* @param conf A [[org.apache.hadoop.conf.Configuration]].
*/
private def saveAsHadoopFile(
- rdd: RDD[Row],
+ rdd: RDD[InternalRow],
path: String,
conf: Configuration) {
val job = new Job(conf)
val keyType = classOf[Void]
job.setOutputKeyClass(keyType)
- job.setOutputValueClass(classOf[Row])
+ job.setOutputValueClass(classOf[InternalRow])
NewFileOutputFormat.setOutputPath(job, new Path(path))
val wrappedConf = new SerializableWritable(job.getConfiguration)
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
@@ -342,7 +342,7 @@ private[sql] case class InsertIntoParquetTable(
.findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1
}
- def writeShard(context: TaskContext, iter: Iterator[Row]): Int = {
+ def writeShard(context: TaskContext, iter: Iterator[InternalRow]): Int = {
/* "reduce task" <split #> <attempt # = spark task #> */
val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
context.attemptNumber)
@@ -381,7 +381,7 @@ private[sql] case class InsertIntoParquetTable(
* to imported ones.
*/
private[parquet] class AppendingParquetOutputFormat(offset: Int)
- extends org.apache.parquet.hadoop.ParquetOutputFormat[Row] {
+ extends org.apache.parquet.hadoop.ParquetOutputFormat[InternalRow] {
// override to accept existing directories as valid output directory
override def checkOutputSpecs(job: JobContext): Unit = {}
var committer: OutputCommitter = null
@@ -434,25 +434,25 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int)
* RecordFilter we want to use.
*/
private[parquet] class FilteringParquetRowInputFormat
- extends org.apache.parquet.hadoop.ParquetInputFormat[Row] with Logging {
+ extends org.apache.parquet.hadoop.ParquetInputFormat[InternalRow] with Logging {
private var fileStatuses = Map.empty[Path, FileStatus]
override def createRecordReader(
inputSplit: InputSplit,
- taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = {
+ taskAttemptContext: TaskAttemptContext): RecordReader[Void, InternalRow] = {
import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter
- val readSupport: ReadSupport[Row] = new RowReadSupport()
+ val readSupport: ReadSupport[InternalRow] = new RowReadSupport()
val filter = ParquetInputFormat.getFilter(ContextUtil.getConfiguration(taskAttemptContext))
if (!filter.isInstanceOf[NoOpFilter]) {
- new ParquetRecordReader[Row](
+ new ParquetRecordReader[InternalRow](
readSupport,
filter)
} else {
- new ParquetRecordReader[Row](readSupport)
+ new ParquetRecordReader[InternalRow](readSupport)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
index c62c592b3f..a8775a2a8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
@@ -28,7 +28,7 @@ import org.apache.parquet.io.api._
import org.apache.parquet.schema.MessageType
import org.apache.spark.Logging
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
import org.apache.spark.sql.catalyst.util.DateUtils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -39,12 +39,12 @@ import org.apache.spark.unsafe.types.UTF8String
*@param root The root group converter for the record.
*/
private[parquet] class RowRecordMaterializer(root: CatalystConverter)
- extends RecordMaterializer[Row] {
+ extends RecordMaterializer[InternalRow] {
def this(parquetSchema: MessageType, attributes: Seq[Attribute]) =
this(CatalystConverter.createRootConverter(parquetSchema, attributes))
- override def getCurrentRecord: Row = root.getCurrentRecord
+ override def getCurrentRecord: InternalRow = root.getCurrentRecord
override def getRootConverter: GroupConverter = root.asInstanceOf[GroupConverter]
}
@@ -52,13 +52,13 @@ private[parquet] class RowRecordMaterializer(root: CatalystConverter)
/**
* A `parquet.hadoop.api.ReadSupport` for Row objects.
*/
-private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging {
+private[parquet] class RowReadSupport extends ReadSupport[InternalRow] with Logging {
override def prepareForRead(
conf: Configuration,
stringMap: java.util.Map[String, String],
fileSchema: MessageType,
- readContext: ReadContext): RecordMaterializer[Row] = {
+ readContext: ReadContext): RecordMaterializer[InternalRow] = {
log.debug(s"preparing for read with Parquet file schema $fileSchema")
// Note: this very much imitates AvroParquet
val parquetSchema = readContext.getRequestedSchema
@@ -133,7 +133,7 @@ private[parquet] object RowReadSupport {
/**
* A `parquet.hadoop.api.WriteSupport` for Row objects.
*/
-private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
+private[parquet] class RowWriteSupport extends WriteSupport[InternalRow] with Logging {
private[parquet] var writer: RecordConsumer = null
private[parquet] var attributes: Array[Attribute] = null
@@ -157,7 +157,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
log.debug(s"preparing for write with schema $attributes")
}
- override def write(record: Row): Unit = {
+ override def write(record: InternalRow): Unit = {
val attributesSize = attributes.size
if (attributesSize > record.size) {
throw new IndexOutOfBoundsException(
@@ -322,7 +322,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
// Optimized for non-nested rows
private[parquet] class MutableRowWriteSupport extends RowWriteSupport {
- override def write(record: Row): Unit = {
+ override def write(record: InternalRow): Unit = {
val attributesSize = attributes.size
if (attributesSize > record.size) {
throw new IndexOutOfBoundsException(
@@ -345,7 +345,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport {
private def consumeType(
ctype: DataType,
- record: Row,
+ record: InternalRow,
index: Int): Unit = {
ctype match {
case StringType => writer.addBinary(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
index 7af4eb1ca4..bc27a9b67a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
@@ -34,15 +34,15 @@ import org.apache.parquet.hadoop._
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.parquet.hadoop.util.ContextUtil
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.{Partition => SparkPartition, SerializableWritable, Logging, SparkException}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.RDD._
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql._
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{DataType, StructType}
-import org.apache.spark.sql.{Row, SQLConf, SQLContext}
import org.apache.spark.util.Utils
private[sql] class DefaultSource extends HadoopFsRelationProvider {
@@ -60,7 +60,7 @@ private[sql] class DefaultSource extends HadoopFsRelationProvider {
private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext)
extends OutputWriter {
- private val recordWriter: RecordWriter[Void, Row] = {
+ private val recordWriter: RecordWriter[Void, InternalRow] = {
val conf = context.getConfiguration
val outputFormat = {
// When appending new Parquet files to an existing Parquet file directory, to avoid
@@ -93,7 +93,7 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext
}
}
- new ParquetOutputFormat[Row]() {
+ new ParquetOutputFormat[InternalRow]() {
// Here we override `getDefaultWorkFile` for two reasons:
//
// 1. To allow appending. We need to generate output file name based on the max available
@@ -112,7 +112,7 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext
outputFormat.getRecordWriter(context)
}
- override def write(row: Row): Unit = recordWriter.write(null, row)
+ override def write(row: Row): Unit = recordWriter.write(null, row.asInstanceOf[InternalRow])
override def close(): Unit = recordWriter.close(context)
}
@@ -286,7 +286,7 @@ private[sql] class ParquetRelation2(
initLocalJobFuncOpt = Some(initLocalJobFuncOpt),
inputFormatClass = classOf[FilteringParquetRowInputFormat],
keyClass = classOf[Void],
- valueClass = classOf[Row]) {
+ valueClass = classOf[InternalRow]) {
val cacheMetadata = useMetadataCache
@@ -331,7 +331,7 @@ private[sql] class ParquetRelation2(
new SqlNewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
}
}
- }.values
+ }.values.map(_.asInstanceOf[Row])
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
index edda3f2017..4cf67439b9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala
@@ -17,48 +17,48 @@
package org.apache.spark.sql.sources
-import org.apache.spark.{Logging, SerializableWritable, TaskContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.{MapPartitionsRDD, RDD, UnionRDD}
+import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.sql.{SaveMode, Strategy, execution, sources}
import org.apache.spark.util.Utils
import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.{Logging, SerializableWritable, TaskContext}
/**
* A Strategy for planning scans over data sources defined using the sources API.
*/
private[sql] object DataSourceStrategy extends Strategy with Logging {
def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match {
- case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: CatalystScan)) =>
+ case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan)) =>
pruneFilterProjectRaw(
l,
- projectList,
+ projects,
filters,
- (a, f) => t.buildScan(a, f)) :: Nil
+ (a, f) => toCatalystRDD(l, a, t.buildScan(a, f))) :: Nil
- case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: PrunedFilteredScan)) =>
+ case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan)) =>
pruneFilterProject(
l,
- projectList,
+ projects,
filters,
- (a, f) => t.buildScan(a, f)) :: Nil
+ (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil
- case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: PrunedScan)) =>
+ case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan)) =>
pruneFilterProject(
l,
- projectList,
+ projects,
filters,
- (a, _) => t.buildScan(a)) :: Nil
+ (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil
// Scanning partitioned HadoopFsRelation
- case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: HadoopFsRelation))
+ case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation))
if t.partitionSpec.partitionColumns.nonEmpty =>
val selectedPartitions = prunePartitions(filters, t.partitionSpec).toArray
@@ -80,13 +80,13 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
buildPartitionedTableScan(
l,
- projectList,
+ projects,
pushedFilters,
t.partitionSpec.partitionColumns,
selectedPartitions) :: Nil
// Scanning non-partitioned HadoopFsRelation
- case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: HadoopFsRelation)) =>
+ case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation)) =>
// See buildPartitionedTableScan for the reason that we need to create a shard
// broadcast HadoopConf.
val sharedHadoopConf = SparkHadoopUtil.get.conf
@@ -94,12 +94,13 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
t.sqlContext.sparkContext.broadcast(new SerializableWritable(sharedHadoopConf))
pruneFilterProject(
l,
- projectList,
+ projects,
filters,
- (a, f) => t.buildScan(a, f, t.paths, confBroadcast)) :: Nil
+ (a, f) =>
+ toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f, t.paths, confBroadcast))) :: Nil
case l @ LogicalRelation(t: TableScan) =>
- createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil
+ execution.PhysicalRDD(l.output, toCatalystRDD(l, t.buildScan())) :: Nil
case i @ logical.InsertIntoTable(
l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty =>
@@ -119,7 +120,6 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
filters: Seq[Expression],
partitionColumns: StructType,
partitions: Array[Partition]) = {
- val output = projections.map(_.toAttribute)
val relation = logicalRelation.relation.asInstanceOf[HadoopFsRelation]
// Because we are creating one RDD per partition, we need to have a shared HadoopConf.
@@ -138,23 +138,23 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
logicalRelation,
projections,
filters,
- (requiredColumns, filters) => {
+ (columns: Seq[Attribute], filters) => {
val partitionColNames = partitionColumns.fieldNames
// Don't scan any partition columns to save I/O. Here we are being optimistic and
// assuming partition columns data stored in data files are always consistent with those
// partition values encoded in partition directory paths.
- val nonPartitionColumns = requiredColumns.filterNot(partitionColNames.contains)
+ val needed = columns.filterNot(a => partitionColNames.contains(a.name))
val dataRows =
- relation.buildScan(nonPartitionColumns, filters, Array(dir), confBroadcast)
+ relation.buildScan(needed.map(_.name).toArray, filters, Array(dir), confBroadcast)
// Merges data values with partition values.
mergeWithPartitionValues(
relation.schema,
- requiredColumns,
+ columns.map(_.name).toArray,
partitionColNames,
partitionValues,
- dataRows)
+ toCatalystRDD(logicalRelation, needed, dataRows))
})
scan.execute()
@@ -167,15 +167,15 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
new UnionRDD(relation.sqlContext.sparkContext, perPartitionRows)
}
- createPhysicalRDD(logicalRelation.relation, output, unionedRows)
+ execution.PhysicalRDD(projections.map(_.toAttribute), unionedRows)
}
private def mergeWithPartitionValues(
schema: StructType,
requiredColumns: Array[String],
partitionColumns: Array[String],
- partitionValues: Row,
- dataRows: RDD[Row]): RDD[Row] = {
+ partitionValues: InternalRow,
+ dataRows: RDD[InternalRow]): RDD[InternalRow] = {
val nonPartitionColumns = requiredColumns.filterNot(partitionColumns.contains)
// If output columns contain any partition column(s), we need to merge scanned data
@@ -186,13 +186,13 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
val i = partitionColumns.indexOf(name)
if (i != -1) {
// If yes, gets column value from partition values.
- (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => {
+ (mutableRow: MutableRow, dataRow: InternalRow, ordinal: Int) => {
mutableRow(ordinal) = partitionValues(i)
}
} else {
// Otherwise, inherits the value from scanned data.
val i = nonPartitionColumns.indexOf(name)
- (mutableRow: MutableRow, dataRow: expressions.Row, ordinal: Int) => {
+ (mutableRow: MutableRow, dataRow: InternalRow, ordinal: Int) => {
mutableRow(ordinal) = dataRow(i)
}
}
@@ -201,7 +201,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
// Since we know for sure that this closure is serializable, we can avoid the overhead
// of cleaning a closure for each RDD by creating our own MapPartitionsRDD. Functionally
// this is equivalent to calling `dataRows.mapPartitions(mapPartitionsFunc)` (SPARK-7718).
- val mapPartitionsFunc = (_: TaskContext, _: Int, iterator: Iterator[Row]) => {
+ val mapPartitionsFunc = (_: TaskContext, _: Int, iterator: Iterator[InternalRow]) => {
val dataTypes = requiredColumns.map(schema(_).dataType)
val mutableRow = new SpecificMutableRow(dataTypes)
iterator.map { dataRow =>
@@ -210,7 +210,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
mergers(i)(mutableRow, dataRow, i)
i += 1
}
- mutableRow.asInstanceOf[expressions.Row]
+ mutableRow.asInstanceOf[InternalRow]
}
}
@@ -256,26 +256,26 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
// Based on Public API.
protected def pruneFilterProject(
relation: LogicalRelation,
- projectList: Seq[NamedExpression],
+ projects: Seq[NamedExpression],
filterPredicates: Seq[Expression],
- scanBuilder: (Array[String], Array[Filter]) => RDD[Row]) = {
+ scanBuilder: (Seq[Attribute], Array[Filter]) => RDD[InternalRow]) = {
pruneFilterProjectRaw(
relation,
- projectList,
+ projects,
filterPredicates,
(requestedColumns, pushedFilters) => {
- scanBuilder(requestedColumns.map(_.name).toArray, selectFilters(pushedFilters).toArray)
+ scanBuilder(requestedColumns, selectFilters(pushedFilters).toArray)
})
}
// Based on Catalyst expressions.
protected def pruneFilterProjectRaw(
relation: LogicalRelation,
- projectList: Seq[NamedExpression],
+ projects: Seq[NamedExpression],
filterPredicates: Seq[Expression],
- scanBuilder: (Seq[Attribute], Seq[Expression]) => RDD[Row]) = {
+ scanBuilder: (Seq[Attribute], Seq[Expression]) => RDD[InternalRow]) = {
- val projectSet = AttributeSet(projectList.flatMap(_.references))
+ val projectSet = AttributeSet(projects.flatMap(_.references))
val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
val filterCondition = filterPredicates.reduceLeftOption(expressions.And)
@@ -283,38 +283,47 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes.
}}
- if (projectList.map(_.toAttribute) == projectList &&
- projectSet.size == projectList.size &&
+ if (projects.map(_.toAttribute) == projects &&
+ projectSet.size == projects.size &&
filterSet.subsetOf(projectSet)) {
// When it is possible to just use column pruning to get the right projection and
// when the columns of this projection are enough to evaluate all filter conditions,
// just do a scan followed by a filter, with no extra project.
val requestedColumns =
- projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above.
+ projects.asInstanceOf[Seq[Attribute]] // Safe due to if above.
.map(relation.attributeMap) // Match original case of attributes.
- val scan = createPhysicalRDD(relation.relation, projectList.map(_.toAttribute),
- scanBuilder(requestedColumns, pushedFilters))
+ val scan = execution.PhysicalRDD(projects.map(_.toAttribute),
+ scanBuilder(requestedColumns, pushedFilters))
filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)
} else {
val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq
- val scan = createPhysicalRDD(relation.relation, requestedColumns,
+ val scan = execution.PhysicalRDD(requestedColumns,
scanBuilder(requestedColumns, pushedFilters))
- execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan))
+ execution.Project(projects, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan))
}
}
- private[this] def createPhysicalRDD(
- relation: BaseRelation,
+ /**
+ * Convert RDD of Row into RDD of InternalRow with objects in catalyst types
+ */
+ private[this] def toCatalystRDD(
+ relation: LogicalRelation,
output: Seq[Attribute],
- rdd: RDD[Row]): SparkPlan = {
- val converted = if (relation.needConversion) {
- execution.RDDConversions.rowToRowRdd(rdd, output.map(_.dataType))
+ rdd: RDD[Row]): RDD[InternalRow] = {
+ if (relation.relation.needConversion) {
+ execution.RDDConversions.rowToRowRdd(rdd.asInstanceOf[RDD[Row]], output.map(_.dataType))
} else {
- rdd
+ rdd.map(_.asInstanceOf[InternalRow])
}
- execution.PhysicalRDD(output, converted)
+ }
+
+ /**
+ * Convert RDD of Row into RDD of InternalRow with objects in catalyst types
+ */
+ private[this] def toCatalystRDD(relation: LogicalRelation, rdd: RDD[Row]): RDD[InternalRow] = {
+ toCatalystRDD(relation, relation.output, rdd)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala
index 7a2b5b949d..c6f535dde7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala
@@ -25,12 +25,11 @@ import scala.util.Try
import org.apache.hadoop.fs.Path
import org.apache.hadoop.util.Shell
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
import org.apache.spark.sql.types._
-private[sql] case class Partition(values: Row, path: String)
+private[sql] case class Partition(values: InternalRow, path: String)
private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition])
@@ -100,7 +99,7 @@ private[sql] object PartitioningUtils {
// Finally, we create `Partition`s based on paths and resolved partition values.
val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map {
case (PartitionValues(_, literals), (path, _)) =>
- Partition(Row.fromSeq(literals.map(_.value)), path.toString)
+ Partition(InternalRow.fromSeq(literals.map(_.value)), path.toString)
}
PartitionSpec(StructType(fields), partitions)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala
index c94199bfcd..1763cee419 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext, SaveMode}
+import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode}
private[sql] case class InsertIntoDataSource(
logicalRelation: LogicalRelation,
@@ -44,18 +44,17 @@ private[sql] case class InsertIntoDataSource(
overwrite: Boolean)
extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val relation = logicalRelation.relation.asInstanceOf[InsertableRelation]
val data = DataFrame(sqlContext, query)
// Apply the schema of the existing table to the new data.
- val df = sqlContext.createDataFrame(
- data.queryExecution.toRdd, logicalRelation.schema, needsConversion = false)
+ val df = sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, logicalRelation.schema)
relation.insert(df, overwrite)
// Invalidate the cache.
sqlContext.cacheManager.invalidateCache(logicalRelation)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -65,7 +64,7 @@ private[sql] case class InsertIntoHadoopFsRelation(
mode: SaveMode)
extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
require(
relation.paths.length == 1,
s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}")
@@ -90,7 +89,7 @@ private[sql] case class InsertIntoHadoopFsRelation(
if (doInsertion) {
val job = new Job(hadoopConf)
job.setOutputKeyClass(classOf[Void])
- job.setOutputValueClass(classOf[Row])
+ job.setOutputValueClass(classOf[InternalRow])
FileOutputFormat.setOutputPath(job, qualifiedOutputPath)
// We create a DataFrame by applying the schema of relation to the data to make sure.
@@ -103,10 +102,8 @@ private[sql] case class InsertIntoHadoopFsRelation(
val project = Project(
relation.schema.map(field => new UnresolvedAttribute(Seq(field.name))), query)
- sqlContext.createDataFrame(
- DataFrame(sqlContext, project).queryExecution.toRdd,
- relation.schema,
- needsConversion = false)
+ sqlContext.internalCreateDataFrame(
+ DataFrame(sqlContext, project).queryExecution.toRdd, relation.schema)
}
val partitionColumns = relation.partitionColumns.fieldNames
@@ -119,7 +116,7 @@ private[sql] case class InsertIntoHadoopFsRelation(
}
}
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
private def insert(writerContainer: BaseWriterContainer, df: DataFrame): Unit = {
@@ -141,22 +138,19 @@ private[sql] case class InsertIntoHadoopFsRelation(
throw new SparkException("Job aborted.", cause)
}
- def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = {
+ def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = {
// If anything below fails, we should abort the task.
try {
writerContainer.executorSideSetup(taskContext)
- if (needsConversion) {
- val converter = CatalystTypeConverters.createToScalaConverter(dataSchema)
- while (iterator.hasNext) {
- val row = converter(iterator.next()).asInstanceOf[Row]
- writerContainer.outputWriterForRow(row).write(row)
- }
+ val converter = if (needsConversion) {
+ CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row]
} else {
- while (iterator.hasNext) {
- val row = iterator.next()
- writerContainer.outputWriterForRow(row).write(row)
- }
+ r: InternalRow => r.asInstanceOf[Row]
+ }
+ while (iterator.hasNext) {
+ val row = converter(iterator.next())
+ writerContainer.outputWriterForRow(row).write(row)
}
writerContainer.commitTask()
@@ -210,32 +204,28 @@ private[sql] case class InsertIntoHadoopFsRelation(
throw new SparkException("Job aborted.", cause)
}
- def writeRows(taskContext: TaskContext, iterator: Iterator[Row]): Unit = {
+ def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = {
// If anything below fails, we should abort the task.
try {
writerContainer.executorSideSetup(taskContext)
val partitionProj = newProjection(codegenEnabled, partitionOutput, output)
val dataProj = newProjection(codegenEnabled, dataOutput, output)
-
- if (needsConversion) {
- val converter = CatalystTypeConverters.createToScalaConverter(dataSchema)
- while (iterator.hasNext) {
- val row = iterator.next()
- val partitionPart = partitionProj(row)
- val dataPart = dataProj(row)
- val convertedDataPart = converter(dataPart).asInstanceOf[Row]
- writerContainer.outputWriterForRow(partitionPart).write(convertedDataPart)
- }
+ val dataConverter: InternalRow => Row = if (needsConversion) {
+ CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row]
} else {
- val partitionSchema = StructType.fromAttributes(partitionOutput)
- val converter = CatalystTypeConverters.createToScalaConverter(partitionSchema)
- while (iterator.hasNext) {
- val row = iterator.next()
- val partitionPart = converter(partitionProj(row)).asInstanceOf[Row]
- val dataPart = dataProj(row)
- writerContainer.outputWriterForRow(partitionPart).write(dataPart)
- }
+ r: InternalRow => r.asInstanceOf[Row]
+ }
+ val partitionSchema = StructType.fromAttributes(partitionOutput)
+ val partConverter: InternalRow => Row =
+ CatalystTypeConverters.createToScalaConverter(partitionSchema)
+ .asInstanceOf[InternalRow => Row]
+
+ while (iterator.hasNext) {
+ val row = iterator.next()
+ val partitionPart = partConverter(partitionProj(row))
+ val dataPart = dataConverter(dataProj(row))
+ writerContainer.outputWriterForRow(partitionPart).write(dataPart)
}
writerContainer.commitTask()
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
index 20afd60cb7..01c67db232 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala
@@ -26,7 +26,7 @@ import org.apache.spark.Logging
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.sql.catalyst.AbstractSparkSQLParser
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InternalRow}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.types._
@@ -404,7 +404,7 @@ private[sql] case class CreateTempTableUsing(
provider: String,
options: Map[String, String]) extends RunnableCommand {
- def run(sqlContext: SQLContext): Seq[Row] = {
+ def run(sqlContext: SQLContext): Seq[InternalRow] = {
val resolved = ResolvedDataSource(
sqlContext, userSpecifiedSchema, Array.empty[String], provider, options)
sqlContext.registerDataFrameAsTable(
@@ -421,7 +421,7 @@ private[sql] case class CreateTempTableUsingAsSelect(
options: Map[String, String],
query: LogicalPlan) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val df = DataFrame(sqlContext, query)
val resolved = ResolvedDataSource(sqlContext, provider, partitionColumns, mode, options, df)
sqlContext.registerDataFrameAsTable(
@@ -434,7 +434,7 @@ private[sql] case class CreateTempTableUsingAsSelect(
private[sql] case class RefreshTable(databaseName: String, tableName: String)
extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
// Refresh the given table's metadata first.
sqlContext.catalog.refreshTable(databaseName, tableName)
@@ -453,7 +453,7 @@ private[sql] case class RefreshTable(databaseName: String, tableName: String)
sqlContext.cacheManager.cacheQuery(df, Some(tableName))
}
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
index d1547fb1e4..27534a1f48 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala
@@ -28,7 +28,8 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
import org.apache.spark.SerializableWritable
-import org.apache.spark.sql.{Row, _}
+import org.apache.spark.sql.execution.RDDConversions
+import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection
import org.apache.spark.sql.types.StructType
@@ -195,6 +196,8 @@ abstract class BaseRelation {
* java.lang.String -> UTF8String
* java.lang.Decimal -> Decimal
*
+ * If `needConversion` is `false`, buildScan() should return an [[RDD]] of [[InternalRow]]
+ *
* Note: The internal representation is not stable across releases and thus data sources outside
* of Spark SQL should leave this as true.
*
@@ -443,7 +446,7 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio
val castedValues = partitionSchema.zip(literals).map { case (field, literal) =>
Cast(literal, field.dataType).eval()
}
- p.copy(values = Row.fromSeq(castedValues))
+ p.copy(values = InternalRow.fromSeq(castedValues))
}
PartitionSpec(partitionSchema, castedPartitions)
}
@@ -579,15 +582,21 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio
BoundReference(dataSchema.fieldIndex(col), field.dataType, field.nullable)
}.toSeq
- buildScan(inputFiles).mapPartitions { rows =>
+ val rdd = buildScan(inputFiles)
+ val converted =
+ if (needConversion) {
+ RDDConversions.rowToRowRdd(rdd, dataSchema.fields.map(_.dataType))
+ } else {
+ rdd.map(_.asInstanceOf[InternalRow])
+ }
+ converted.mapPartitions { rows =>
val buildProjection = if (codegenEnabled) {
GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes)
} else {
() => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes)
}
-
val mutableProjection = buildProjection()
- rows.map(mutableProjection)
+ rows.map(r => mutableProjection(r).asInstanceOf[Row])
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index 17a3cec48b..eb3e913322 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -94,7 +94,7 @@ class CachedTableSuite extends QueryTest {
}
test("too big for memory") {
- val data = "*" * 10000
+ val data = "*" * 1000
ctx.sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF()
.registerTempTable("bigData")
ctx.table("bigData").persist(StorageLevel.MEMORY_AND_DISK)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
index 16836628cb..1f37455dd0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
@@ -18,25 +18,28 @@
package org.apache.spark.sql.columnar
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.types._
class ColumnStatsSuite extends SparkFunSuite {
- testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0))
- testColumnStats(classOf[ShortColumnStats], SHORT, Row(Short.MaxValue, Short.MinValue, 0))
- testColumnStats(classOf[IntColumnStats], INT, Row(Int.MaxValue, Int.MinValue, 0))
- testColumnStats(classOf[LongColumnStats], LONG, Row(Long.MaxValue, Long.MinValue, 0))
- testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0))
- testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0))
- testColumnStats(classOf[FixedDecimalColumnStats], FIXED_DECIMAL(15, 10), Row(null, null, 0))
- testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0))
- testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0))
- testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(Long.MaxValue, Long.MinValue, 0))
+ testColumnStats(classOf[ByteColumnStats], BYTE, InternalRow(Byte.MaxValue, Byte.MinValue, 0))
+ testColumnStats(classOf[ShortColumnStats], SHORT, InternalRow(Short.MaxValue, Short.MinValue, 0))
+ testColumnStats(classOf[IntColumnStats], INT, InternalRow(Int.MaxValue, Int.MinValue, 0))
+ testColumnStats(classOf[LongColumnStats], LONG, InternalRow(Long.MaxValue, Long.MinValue, 0))
+ testColumnStats(classOf[FloatColumnStats], FLOAT, InternalRow(Float.MaxValue, Float.MinValue, 0))
+ testColumnStats(classOf[DoubleColumnStats], DOUBLE,
+ InternalRow(Double.MaxValue, Double.MinValue, 0))
+ testColumnStats(classOf[FixedDecimalColumnStats],
+ FIXED_DECIMAL(15, 10), InternalRow(null, null, 0))
+ testColumnStats(classOf[StringColumnStats], STRING, InternalRow(null, null, 0))
+ testColumnStats(classOf[DateColumnStats], DATE, InternalRow(Int.MaxValue, Int.MinValue, 0))
+ testColumnStats(classOf[TimestampColumnStats], TIMESTAMP,
+ InternalRow(Long.MaxValue, Long.MinValue, 0))
def testColumnStats[T <: AtomicType, U <: ColumnStats](
columnStatsClass: Class[U],
columnType: NativeColumnType[T],
- initialStatistics: Row): Unit = {
+ initialStatistics: InternalRow): Unit = {
val columnStatsName = columnStatsClass.getSimpleName
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
index 1bc7eb3631..7c86eae3f7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
@@ -19,14 +19,11 @@ package org.apache.spark.sql.columnar
import scala.collection.immutable.HashSet
import scala.util.Random
-
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.types.{AtomicType, DataType, Decimal}
import org.apache.spark.sql.types.{DataType, Decimal, AtomicType}
import org.apache.spark.unsafe.types.UTF8String
-
object ColumnarTestUtils {
def makeNullRow(length: Int): GenericMutableRow = {
val row = new GenericMutableRow(length)
@@ -79,9 +76,9 @@ object ColumnarTestUtils {
def makeRandomRow(
head: ColumnType[_ <: DataType, _],
- tail: ColumnType[_ <: DataType, _]*): Row = makeRandomRow(Seq(head) ++ tail)
+ tail: ColumnType[_ <: DataType, _]*): InternalRow = makeRandomRow(Seq(head) ++ tail)
- def makeRandomRow(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Row = {
+ def makeRandomRow(columnTypes: Seq[ColumnType[_ <: DataType, _]]): InternalRow = {
val row = new GenericMutableRow(columnTypes.length)
makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) =>
row(index) = value
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
index fa3b8144c0..12f95eb557 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
@@ -20,9 +20,8 @@ package org.apache.spark.sql.columnar
import java.sql.{Date, Timestamp}
import org.apache.spark.sql.TestData._
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{QueryTest, TestData}
+import org.apache.spark.sql.{QueryTest, Row, TestData}
import org.apache.spark.storage.StorageLevel.MEMORY_ONLY
class InMemoryColumnarQuerySuite extends QueryTest {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
index 20d65a74e3..f606e2133b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql.columnar.compression
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-import org.apache.spark.sql.columnar.{NoopColumnStats, BOOLEAN}
import org.apache.spark.sql.columnar.ColumnarTestUtils._
+import org.apache.spark.sql.columnar.{BOOLEAN, NoopColumnStats}
class BooleanBitSetSuite extends SparkFunSuite {
import BooleanBitSet._
@@ -32,7 +32,7 @@ class BooleanBitSetSuite extends SparkFunSuite {
// -------------
val builder = TestCompressibleColumnBuilder(new NoopColumnStats, BOOLEAN, BooleanBitSet)
- val rows = Seq.fill[Row](count)(makeRandomRow(BOOLEAN))
+ val rows = Seq.fill[InternalRow](count)(makeRandomRow(BOOLEAN))
val values = rows.map(_(0))
rows.foreach(builder.appendFrom(_, 0))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
index 45a7e8fe68..3e27f58a92 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -18,16 +18,15 @@
package org.apache.spark.sql.execution
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.{SQLConf, execution}
-import org.apache.spark.sql.functions._
import org.apache.spark.sql.TestData._
-import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin}
+import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.TestSQLContext._
import org.apache.spark.sql.test.TestSQLContext.implicits._
import org.apache.spark.sql.test.TestSQLContext.planner._
import org.apache.spark.sql.types._
+import org.apache.spark.sql.{Row, SQLConf, execution}
class PlannerSuite extends SparkFunSuite {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
index 5290c28cfc..71db6a2159 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.joins
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.expressions.{Projection, Row}
+import org.apache.spark.sql.catalyst.expressions.{Projection, InternalRow}
import org.apache.spark.util.collection.CompactBuffer
@@ -26,37 +26,37 @@ class HashedRelationSuite extends SparkFunSuite {
// Key is simply the record itself
private val keyProjection = new Projection {
- override def apply(row: Row): Row = row
+ override def apply(row: InternalRow): InternalRow = row
}
test("GeneralHashedRelation") {
- val data = Array(Row(0), Row(1), Row(2), Row(2))
+ val data = Array(InternalRow(0), InternalRow(1), InternalRow(2), InternalRow(2))
val hashed = HashedRelation(data.iterator, keyProjection)
assert(hashed.isInstanceOf[GeneralHashedRelation])
- assert(hashed.get(data(0)) == CompactBuffer[Row](data(0)))
- assert(hashed.get(data(1)) == CompactBuffer[Row](data(1)))
- assert(hashed.get(Row(10)) === null)
+ assert(hashed.get(data(0)) == CompactBuffer[InternalRow](data(0)))
+ assert(hashed.get(data(1)) == CompactBuffer[InternalRow](data(1)))
+ assert(hashed.get(InternalRow(10)) === null)
- val data2 = CompactBuffer[Row](data(2))
+ val data2 = CompactBuffer[InternalRow](data(2))
data2 += data(2)
assert(hashed.get(data(2)) == data2)
}
test("UniqueKeyHashedRelation") {
- val data = Array(Row(0), Row(1), Row(2))
+ val data = Array(InternalRow(0), InternalRow(1), InternalRow(2))
val hashed = HashedRelation(data.iterator, keyProjection)
assert(hashed.isInstanceOf[UniqueKeyHashedRelation])
- assert(hashed.get(data(0)) == CompactBuffer[Row](data(0)))
- assert(hashed.get(data(1)) == CompactBuffer[Row](data(1)))
- assert(hashed.get(data(2)) == CompactBuffer[Row](data(2)))
- assert(hashed.get(Row(10)) === null)
+ assert(hashed.get(data(0)) == CompactBuffer[InternalRow](data(0)))
+ assert(hashed.get(data(1)) == CompactBuffer[InternalRow](data(1)))
+ assert(hashed.get(data(2)) == CompactBuffer[InternalRow](data(2)))
+ assert(hashed.get(InternalRow(10)) === null)
val uniqHashed = hashed.asInstanceOf[UniqueKeyHashedRelation]
assert(uniqHashed.getValue(data(0)) == data(0))
assert(uniqHashed.getValue(data(1)) == data(1))
assert(uniqHashed.getValue(data(2)) == data(2))
- assert(uniqHashed.getValue(Row(10)) == null)
+ assert(uniqHashed.getValue(InternalRow(10)) == null)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
index 17f5f9a491..fa5d4eca05 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.sources.LogicalRelation
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{Column, DataFrame, QueryTest, SQLConf}
+import org.apache.spark.sql.{Column, DataFrame, QueryTest, Row, SQLConf}
/**
* A test suite that tests Parquet filter2 API based filter pushdown optimization.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
index 46b25859d9..fc827bc4ca 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala
@@ -35,11 +35,10 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser}
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.SparkException
+import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.ScalaReflection
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.catalyst.util.DateUtils
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf, SaveMode}
// Write support class for nested groups: ParquetWriter initializes GroupWriteSupport
// with an empty configuration (it is after all not intended to be used in this way?)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala
index 3240079483..01df189d1f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala
@@ -26,11 +26,13 @@ import scala.collection.mutable.ArrayBuffer
import com.google.common.io.Files
import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.sources.PartitioningUtils._
import org.apache.spark.sql.sources.{LogicalRelation, Partition, PartitionSpec}
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{Column, QueryTest, Row, SQLContext}
+import org.apache.spark.sql._
+import org.apache.spark.unsafe.types.UTF8String
// The data where the partitioning key exists only in the directory structure.
case class ParquetData(intField: Int, stringField: String)
@@ -114,7 +116,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructType(Seq(
StructField("a", IntegerType),
StructField("b", StringType))),
- Seq(Partition(Row(10, "hello"), "hdfs://host:9000/path/a=10/b=hello"))))
+ Seq(Partition(InternalRow(10, UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10/b=hello"))))
check(Seq(
"hdfs://host:9000/path/a=10/b=20",
@@ -124,8 +127,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", DoubleType),
StructField("b", StringType))),
Seq(
- Partition(Row(10, "20"), "hdfs://host:9000/path/a=10/b=20"),
- Partition(Row(10.5, "hello"), "hdfs://host:9000/path/a=10.5/b=hello"))))
+ Partition(InternalRow(10, UTF8String.fromString("20")),
+ "hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(10.5, UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10.5/b=hello"))))
check(Seq(
"hdfs://host:9000/path/_temporary",
@@ -143,8 +148,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", DoubleType),
StructField("b", StringType))),
Seq(
- Partition(Row(10, "20"), "hdfs://host:9000/path/a=10/b=20"),
- Partition(Row(10.5, "hello"), "hdfs://host:9000/path/a=10.5/b=hello"))))
+ Partition(InternalRow(10, UTF8String.fromString("20")),
+ "hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(10.5, UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10.5/b=hello"))))
check(Seq(
s"hdfs://host:9000/path/a=10/b=20",
@@ -154,8 +161,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", IntegerType),
StructField("b", StringType))),
Seq(
- Partition(Row(10, "20"), s"hdfs://host:9000/path/a=10/b=20"),
- Partition(Row(null, "hello"), s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello"))))
+ Partition(InternalRow(10, UTF8String.fromString("20")),
+ s"hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(null, UTF8String.fromString("hello")),
+ s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello"))))
check(Seq(
s"hdfs://host:9000/path/a=10/b=$defaultPartitionName",
@@ -165,8 +174,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", DoubleType),
StructField("b", StringType))),
Seq(
- Partition(Row(10, null), s"hdfs://host:9000/path/a=10/b=$defaultPartitionName"),
- Partition(Row(10.5, null), s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"))))
+ Partition(InternalRow(10, null), s"hdfs://host:9000/path/a=10/b=$defaultPartitionName"),
+ Partition(InternalRow(10.5, null),
+ s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"))))
check(Seq(
s"hdfs://host:9000/path1",
@@ -185,7 +195,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructType(Seq(
StructField("a", StringType),
StructField("b", StringType))),
- Seq(Partition(Row("10", "hello"), "hdfs://host:9000/path/a=10/b=hello"))))
+ Seq(Partition(InternalRow(UTF8String.fromString("10"), UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10/b=hello"))))
check(Seq(
"hdfs://host:9000/path/a=10/b=20",
@@ -195,8 +206,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", StringType),
StructField("b", StringType))),
Seq(
- Partition(Row("10", "20"), "hdfs://host:9000/path/a=10/b=20"),
- Partition(Row("10.5", "hello"), "hdfs://host:9000/path/a=10.5/b=hello"))))
+ Partition(InternalRow(UTF8String.fromString("10"), UTF8String.fromString("20")),
+ "hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(UTF8String.fromString("10.5"), UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10.5/b=hello"))))
check(Seq(
"hdfs://host:9000/path/_temporary",
@@ -214,8 +227,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", StringType),
StructField("b", StringType))),
Seq(
- Partition(Row("10", "20"), "hdfs://host:9000/path/a=10/b=20"),
- Partition(Row("10.5", "hello"), "hdfs://host:9000/path/a=10.5/b=hello"))))
+ Partition(InternalRow(UTF8String.fromString("10"), UTF8String.fromString("20")),
+ "hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(UTF8String.fromString("10.5"), UTF8String.fromString("hello")),
+ "hdfs://host:9000/path/a=10.5/b=hello"))))
check(Seq(
s"hdfs://host:9000/path/a=10/b=20",
@@ -225,8 +240,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", StringType),
StructField("b", StringType))),
Seq(
- Partition(Row("10", "20"), s"hdfs://host:9000/path/a=10/b=20"),
- Partition(Row(null, "hello"), s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello"))))
+ Partition(InternalRow(UTF8String.fromString("10"), UTF8String.fromString("20")),
+ s"hdfs://host:9000/path/a=10/b=20"),
+ Partition(InternalRow(null, UTF8String.fromString("hello")),
+ s"hdfs://host:9000/path/a=$defaultPartitionName/b=hello"))))
check(Seq(
s"hdfs://host:9000/path/a=10/b=$defaultPartitionName",
@@ -236,8 +253,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest {
StructField("a", StringType),
StructField("b", StringType))),
Seq(
- Partition(Row("10", null), s"hdfs://host:9000/path/a=10/b=$defaultPartitionName"),
- Partition(Row("10.5", null), s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"))))
+ Partition(InternalRow(UTF8String.fromString("10"), null),
+ s"hdfs://host:9000/path/a=10/b=$defaultPartitionName"),
+ Partition(InternalRow(UTF8String.fromString("10.5"), null),
+ s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"))))
check(Seq(
s"hdfs://host:9000/path1",
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index de0107a361..be3b34d5b9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -20,15 +20,13 @@ package org.apache.spark.sql.parquet
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{SQLConf, QueryTest}
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.{QueryTest, Row, SQLConf}
/**
* A test suite that tests various Parquet queries.
*/
class ParquetQuerySuiteBase extends QueryTest with ParquetTest {
lazy val sqlContext = org.apache.spark.sql.test.TestSQLContext
- import sqlContext.implicits._
import sqlContext.sql
test("simple select queries") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
index 51d22b6a13..5fc53f7012 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
@@ -19,7 +19,9 @@ package org.apache.spark.sql.sources
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
class DDLScanSource extends RelationProvider {
override def createRelation(
@@ -56,9 +58,12 @@ case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlCo
)
))
+ override def needConversion: Boolean = false
override def buildScan(): RDD[Row] = {
- sqlContext.sparkContext.parallelize(from to to).map(e => Row(s"people$e", e * 2))
+ sqlContext.sparkContext.parallelize(from to to).map { e =>
+ InternalRow(UTF8String.fromString(s"people$e"), e * 2)
+ }
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
index 5d4ecd8108..4a4d8d2548 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala
@@ -19,9 +19,13 @@ package org.apache.spark.sql.sources
import java.sql.{Timestamp, Date}
+
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.util.DateUtils
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
class DefaultSource extends SimpleScanSource
@@ -60,10 +64,12 @@ case class AllDataTypesScan(
override def schema: StructType = userSpecifiedSchema
+ override def needConversion: Boolean = false
+
override def buildScan(): RDD[Row] = {
sqlContext.sparkContext.parallelize(from to to).map { i =>
- Row(
- s"str_$i",
+ InternalRow(
+ UTF8String.fromString(s"str_$i"),
s"str_$i".getBytes(),
i % 2 == 0,
i.toByte,
@@ -72,17 +78,18 @@ case class AllDataTypesScan(
i.toLong,
i.toFloat,
i.toDouble,
- new java.math.BigDecimal(i),
- new java.math.BigDecimal(i),
- new Date(1970, 1, 1),
- new Timestamp(20000 + i),
- s"varchar_$i",
+ Decimal(new java.math.BigDecimal(i)),
+ Decimal(new java.math.BigDecimal(i)),
+ DateUtils.fromJavaDate(new Date(1970, 1, 1)),
+ DateUtils.fromJavaTimestamp(new Timestamp(20000 + i)),
+ UTF8String.fromString(s"varchar_$i"),
Seq(i, i + 1),
- Seq(Map(s"str_$i" -> Row(i.toLong))),
+ Seq(Map(UTF8String.fromString(s"str_$i") -> InternalRow(i.toLong))),
Map(i -> i.toString),
- Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)),
+ Map(Map(UTF8String.fromString(s"str_$i") -> i.toFloat) -> InternalRow(i.toLong)),
Row(i, i.toString),
- Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1)))))
+ Row(Seq(UTF8String.fromString(s"str_$i"), UTF8String.fromString(s"str_${i + 1}")),
+ InternalRow(Seq(DateUtils.fromJavaDate(new Date(1970, 1, i + 1))))))
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index fd01a8722b..d4f1ae8ee0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -366,7 +366,7 @@ private[hive] trait HiveInspectors {
(o: Any) => {
if (o != null) {
val struct = soi.create()
- (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row].toSeq).zipped.foreach {
+ (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[InternalRow].toSeq).zipped.foreach {
(field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data))
}
struct
@@ -474,7 +474,7 @@ private[hive] trait HiveInspectors {
}
case x: SettableStructObjectInspector =>
val fieldRefs = x.getAllStructFieldRefs
- val row = a.asInstanceOf[Row]
+ val row = a.asInstanceOf[InternalRow]
// 1. create the pojo (most likely) object
val result = x.create()
var i = 0
@@ -490,7 +490,7 @@ private[hive] trait HiveInspectors {
result
case x: StructObjectInspector =>
val fieldRefs = x.getAllStructFieldRefs
- val row = a.asInstanceOf[Row]
+ val row = a.asInstanceOf[InternalRow]
val result = new java.util.ArrayList[AnyRef](fieldRefs.length)
var i = 0
while (i < fieldRefs.length) {
@@ -517,7 +517,7 @@ private[hive] trait HiveInspectors {
}
def wrap(
- row: Row,
+ row: InternalRow,
inspectors: Seq[ObjectInspector],
cache: Array[AnyRef]): Array[AnyRef] = {
var i = 0
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 5a4651a887..619ef63223 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -302,7 +302,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive
val partitionColumnDataTypes = partitionSchema.map(_.dataType)
val partitions = metastoreRelation.hiveQlPartitions.map { p =>
val location = p.getLocation
- val values = Row.fromSeq(p.getValues.zip(partitionColumnDataTypes).map {
+ val values = InternalRow.fromSeq(p.getValues.zip(partitionColumnDataTypes).map {
case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null)
})
ParquetPartition(values, location)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index c6b6510645..452b7f0bcc 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
-import org.apache.spark.sql.catalyst.expressions.{Row, _}
+import org.apache.spark.sql.catalyst.expressions.{InternalRow, _}
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -137,7 +137,7 @@ private[hive] trait HiveStrategies {
val partitionLocations = partitions.map(_.getLocation)
if (partitionLocations.isEmpty) {
- PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil
+ PhysicalRDD(plan.output, sparkContext.emptyRDD[InternalRow]) :: Nil
} else {
hiveContext
.read.parquet(partitionLocations: _*)
@@ -165,7 +165,7 @@ private[hive] trait HiveStrategies {
// TODO: Remove this hack for Spark 1.3.
case iae: java.lang.IllegalArgumentException
if iae.getMessage.contains("Can not create a Path from an empty string") =>
- PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil
+ PhysicalRDD(plan.output, sparkContext.emptyRDD[InternalRow]) :: Nil
}
case _ => Nil
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index d3c82d8c2e..485810320f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -41,9 +41,9 @@ import org.apache.spark.util.Utils
* A trait for subclasses that handle table scans.
*/
private[hive] sealed trait TableReader {
- def makeRDDForTable(hiveTable: HiveTable): RDD[Row]
+ def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow]
- def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row]
+ def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow]
}
@@ -74,7 +74,7 @@ class HadoopTableReader(
private val _broadcastedHiveConf =
sc.sparkContext.broadcast(new SerializableWritable(hiveExtraConf))
- override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] =
+ override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] =
makeRDDForTable(
hiveTable,
Class.forName(
@@ -94,7 +94,7 @@ class HadoopTableReader(
def makeRDDForTable(
hiveTable: HiveTable,
deserializerClass: Class[_ <: Deserializer],
- filterOpt: Option[PathFilter]): RDD[Row] = {
+ filterOpt: Option[PathFilter]): RDD[InternalRow] = {
assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table,
since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""")
@@ -125,7 +125,7 @@ class HadoopTableReader(
deserializedHadoopRDD
}
- override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] = {
+ override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow] = {
val partitionToDeserializer = partitions.map(part =>
(part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap
makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None)
@@ -144,7 +144,7 @@ class HadoopTableReader(
def makeRDDForPartitionedTable(
partitionToDeserializer: Map[HivePartition,
Class[_ <: Deserializer]],
- filterOpt: Option[PathFilter]): RDD[Row] = {
+ filterOpt: Option[PathFilter]): RDD[InternalRow] = {
// SPARK-5068:get FileStatus and do the filtering locally when the path is not exists
def verifyPartitionPath(
@@ -243,7 +243,7 @@ class HadoopTableReader(
// Even if we don't use any partitions, we still need an empty RDD
if (hivePartitionRDDs.size == 0) {
- new EmptyRDD[Row](sc.sparkContext)
+ new EmptyRDD[InternalRow](sc.sparkContext)
} else {
new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs)
}
@@ -319,7 +319,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
rawDeser: Deserializer,
nonPartitionKeyAttrs: Seq[(Attribute, Int)],
mutableRow: MutableRow,
- tableDeser: Deserializer): Iterator[Row] = {
+ tableDeser: Deserializer): Iterator[InternalRow] = {
val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) {
rawDeser.getObjectInspector.asInstanceOf[StructObjectInspector]
@@ -391,7 +391,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging {
i += 1
}
- mutableRow: Row
+ mutableRow: InternalRow
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 87c36a8b61..0e4a2427a9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution
import org.apache.spark.annotation.Experimental
import org.apache.spark.sql.{AnalysisException, SQLContext}
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan}
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.hive.client.{HiveTable, HiveColumn}
@@ -42,7 +42,7 @@ case class CreateTableAsSelect(
def database: String = tableDesc.database
def tableName: String = tableDesc.name
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
lazy val metastoreRelation: MetastoreRelation = {
import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
@@ -89,7 +89,7 @@ case class CreateTableAsSelect(
hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd
}
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
override def argString: String = {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index 6fce69b58b..a89381000a 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -21,12 +21,10 @@ import scala.collection.JavaConversions._
import org.apache.hadoop.hive.metastore.api.FieldSchema
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
-import org.apache.spark.sql.execution.{SparkPlan, RunnableCommand}
-import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation}
-import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.hive.MetastoreRelation
/**
* Implementation for "describe [extended] table".
@@ -37,7 +35,7 @@ case class DescribeHiveTableCommand(
override val output: Seq[Attribute],
isExtended: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
// Trying to mimic the format of Hive's output. But not exactly the same.
var results: Seq[(String, String, String)] = Nil
@@ -59,7 +57,7 @@ case class DescribeHiveTableCommand(
}
results.map { case (name, dataType, comment) =>
- Row(name, dataType, comment)
+ InternalRow(name, dataType, comment)
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 60a9bb630d..87f8e3f7fc 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -1,34 +1,34 @@
-/*
- * 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.spark.sql.hive.execution
-
-import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
-import org.apache.spark.sql.execution.RunnableCommand
-import org.apache.spark.sql.hive.HiveContext
-import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.types.StringType
-
-private[hive]
-case class HiveNativeCommand(sql: String) extends RunnableCommand {
-
- override def output: Seq[AttributeReference] =
- Seq(AttributeReference("result", StringType, nullable = false)())
-
- override def run(sqlContext: SQLContext): Seq[Row] =
- sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_))
-}
+/*
+ * 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.spark.sql.hive.execution
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, InternalRow}
+import org.apache.spark.sql.execution.RunnableCommand
+import org.apache.spark.sql.hive.HiveContext
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.types.StringType
+
+private[hive]
+case class HiveNativeCommand(sql: String) extends RunnableCommand {
+
+ override def output: Seq[AttributeReference] =
+ Seq(AttributeReference("result", StringType, nullable = false)())
+
+ override def run(sqlContext: SQLContext): Seq[InternalRow] =
+ sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(InternalRow(_))
+}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
index 11ee550314..1f5e4af2e4 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
@@ -129,7 +129,7 @@ case class HiveTableScan(
}
}
- protected override def doExecute(): RDD[Row] = if (!relation.hiveQlTable.isPartitioned) {
+ protected override def doExecute(): RDD[InternalRow] = if (!relation.hiveQlTable.isPartitioned) {
hadoopReader.makeRDDForTable(relation.hiveQlTable)
} else {
hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index eeb472602b..1d306c5d10 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -30,7 +30,8 @@ import org.apache.hadoop.hive.serde2.objectinspector._
import org.apache.hadoop.mapred.{FileOutputFormat, JobConf}
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
import org.apache.spark.sql.execution.{UnaryNode, SparkPlan}
import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
import org.apache.spark.sql.hive._
@@ -60,7 +61,7 @@ case class InsertIntoHiveTable(
def output: Seq[Attribute] = child.output
def saveAsHiveFile(
- rdd: RDD[Row],
+ rdd: RDD[InternalRow],
valueClass: Class[_],
fileSinkConf: FileSinkDesc,
conf: SerializableWritable[JobConf],
@@ -82,7 +83,7 @@ case class InsertIntoHiveTable(
writerContainer.commitJob()
// Note that this function is executed on executor side
- def writeToFile(context: TaskContext, iterator: Iterator[Row]): Unit = {
+ def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = {
val serializer = newSerializer(fileSinkConf.getTableInfo)
val standardOI = ObjectInspectorUtils
.getStandardObjectInspector(
@@ -119,7 +120,7 @@ case class InsertIntoHiveTable(
*
* Note: this is run once and then kept to avoid double insertions.
*/
- protected[sql] lazy val sideEffectResult: Seq[Row] = {
+ protected[sql] lazy val sideEffectResult: Seq[InternalRow] = {
// Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer
// instances within the closure, since Serializer is not serializable while TableDesc is.
val tableDesc = table.tableDesc
@@ -250,12 +251,13 @@ case class InsertIntoHiveTable(
// however for now we return an empty list to simplify compatibility checks with hive, which
// does not return anything for insert operations.
// TODO: implement hive compatibility as rules.
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
- override def executeCollect(): Array[Row] = sideEffectResult.toArray
+ override def executeCollect(): Array[Row] =
+ sideEffectResult.toArray
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
sqlContext.sparkContext.parallelize(sideEffectResult, 1)
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
index 28792db768..9d8872aa47 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
@@ -55,7 +55,7 @@ case class ScriptTransformation(
override def otherCopyArgs: Seq[HiveContext] = sc :: Nil
- protected override def doExecute(): RDD[Row] = {
+ protected override def doExecute(): RDD[InternalRow] = {
child.execute().mapPartitions { iter =>
val cmd = List("/bin/bash", "-c", script)
val builder = new ProcessBuilder(cmd)
@@ -72,8 +72,8 @@ case class ScriptTransformation(
val (outputSerde, outputSoi) = ioschema.initOutputSerDe(output)
- val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors {
- var cacheRow: Row = null
+ val iterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors {
+ var cacheRow: InternalRow = null
var curLine: String = null
var eof: Boolean = false
@@ -90,7 +90,7 @@ case class ScriptTransformation(
}
}
- def deserialize(): Row = {
+ def deserialize(): InternalRow = {
if (cacheRow != null) return cacheRow
val mutableRow = new SpecificMutableRow(output.map(_.dataType))
@@ -120,7 +120,7 @@ case class ScriptTransformation(
}
}
- override def next(): Row = {
+ override def next(): InternalRow = {
if (!hasNext) {
throw new NoSuchElementException
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index 0ba94d7b7c..195e5752c3 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.sources._
import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext}
-import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, InternalRow}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.RunnableCommand
import org.apache.spark.sql.hive.HiveContext
@@ -39,9 +39,9 @@ import org.apache.spark.util.Utils
private[hive]
case class AnalyzeTable(tableName: String) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
sqlContext.asInstanceOf[HiveContext].analyze(tableName)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -53,7 +53,7 @@ case class DropTable(
tableName: String,
ifExists: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
val ifExistsClause = if (ifExists) "IF EXISTS " else ""
try {
@@ -70,7 +70,7 @@ case class DropTable(
hiveContext.invalidateTable(tableName)
hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")
hiveContext.catalog.unregisterTable(Seq(tableName))
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -83,7 +83,7 @@ case class AddJar(path: String) extends RunnableCommand {
schema.toAttributes
}
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
val currentClassLoader = Utils.getContextOrSparkClassLoader
@@ -99,18 +99,18 @@ case class AddJar(path: String) extends RunnableCommand {
// Add jar to executors
hiveContext.sparkContext.addJar(path)
- Seq(Row(0))
+ Seq(InternalRow(0))
}
}
private[hive]
case class AddFile(path: String) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
hiveContext.runSqlHive(s"ADD FILE $path")
hiveContext.sparkContext.addFile(path)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -123,12 +123,12 @@ case class CreateMetastoreDataSource(
allowExisting: Boolean,
managedIfNoPath: Boolean) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
if (hiveContext.catalog.tableExists(tableName :: Nil)) {
if (allowExisting) {
- return Seq.empty[Row]
+ return Seq.empty[InternalRow]
} else {
throw new AnalysisException(s"Table $tableName already exists.")
}
@@ -151,7 +151,7 @@ case class CreateMetastoreDataSource(
optionsWithPath,
isExternal)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
@@ -164,7 +164,7 @@ case class CreateMetastoreDataSourceAsSelect(
options: Map[String, String],
query: LogicalPlan) extends RunnableCommand {
- override def run(sqlContext: SQLContext): Seq[Row] = {
+ override def run(sqlContext: SQLContext): Seq[InternalRow] = {
val hiveContext = sqlContext.asInstanceOf[HiveContext]
var createMetastoreTable = false
var isExternal = true
@@ -188,7 +188,7 @@ case class CreateMetastoreDataSourceAsSelect(
s"Or, if you are using SQL CREATE TABLE, you need to drop $tableName first.")
case SaveMode.Ignore =>
// Since the table already exists and the save mode is Ignore, we will just return.
- return Seq.empty[Row]
+ return Seq.empty[InternalRow]
case SaveMode.Append =>
// Check if the specified data source match the data source of the existing table.
val resolved = ResolvedDataSource(
@@ -230,7 +230,7 @@ case class CreateMetastoreDataSourceAsSelect(
val data = DataFrame(hiveContext, query)
val df = existingSchema match {
// If we are inserting into an existing table, just use the existing schema.
- case Some(schema) => sqlContext.createDataFrame(data.queryExecution.toRdd, schema)
+ case Some(schema) => sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, schema)
case None => data
}
@@ -253,6 +253,6 @@ case class CreateMetastoreDataSourceAsSelect(
// Refresh the cache of the table in the catalog.
hiveContext.refreshTable(tableName)
- Seq.empty[Row]
+ Seq.empty[InternalRow]
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index a46ee9da90..c40dd4e4b9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -121,7 +121,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, childre
protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length)
// TODO: Finish input output types.
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
unwrap(
FunctionRegistry.invoke(method, function, conversionHelper
.convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*),
@@ -178,7 +178,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr
lazy val dataType: DataType = inspectorToDataType(returnInspector)
- override def eval(input: Row): Any = {
+ override def eval(input: InternalRow): Any = {
returnInspector // Make sure initialized.
var i = 0
@@ -345,7 +345,7 @@ private[hive] case class HiveWindowFunction(
def nullable: Boolean = true
- override def eval(input: Row): Any =
+ override def eval(input: InternalRow): Any =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
@transient
@@ -369,7 +369,7 @@ private[hive] case class HiveWindowFunction(
evaluator.reset(hiveEvaluatorBuffer)
}
- override def prepareInputParameters(input: Row): AnyRef = {
+ override def prepareInputParameters(input: InternalRow): AnyRef = {
wrap(inputProjection(input), inputInspectors, new Array[AnyRef](children.length))
}
// Add input parameters for a single row.
@@ -512,7 +512,7 @@ private[hive] case class HiveGenericUdtf(
field => (inspectorToDataType(field.getFieldObjectInspector), true)
}
- override def eval(input: Row): TraversableOnce[Row] = {
+ override def eval(input: InternalRow): TraversableOnce[InternalRow] = {
outputInspector // Make sure initialized.
val inputProjection = new InterpretedProjection(children)
@@ -522,23 +522,23 @@ private[hive] case class HiveGenericUdtf(
}
protected class UDTFCollector extends Collector {
- var collected = new ArrayBuffer[Row]
+ var collected = new ArrayBuffer[InternalRow]
override def collect(input: java.lang.Object) {
// We need to clone the input here because implementations of
// GenericUDTF reuse the same object. Luckily they are always an array, so
// it is easy to clone.
- collected += unwrap(input, outputInspector).asInstanceOf[Row]
+ collected += unwrap(input, outputInspector).asInstanceOf[InternalRow]
}
- def collectRows(): Seq[Row] = {
+ def collectRows(): Seq[InternalRow] = {
val toCollect = collected
- collected = new ArrayBuffer[Row]
+ collected = new ArrayBuffer[InternalRow]
toCollect
}
}
- override def terminate(): TraversableOnce[Row] = {
+ override def terminate(): TraversableOnce[InternalRow] = {
outputInspector // Make sure initialized.
function.close()
collector.collectRows()
@@ -578,7 +578,7 @@ private[hive] case class HiveUdafFunction(
private val buffer =
function.getNewAggregationBuffer
- override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector)
+ override def eval(input: InternalRow): Any = unwrap(function.evaluate(buffer), returnInspector)
@transient
val inputProjection = new InterpretedProjection(exprs)
@@ -586,7 +586,7 @@ private[hive] case class HiveUdafFunction(
@transient
protected lazy val cached = new Array[AnyRef](exprs.length)
- def update(input: Row): Unit = {
+ def update(input: InternalRow): Unit = {
val inputs = inputProjection(input)
function.iterate(buffer, wrap(inputs, inspectors, cached))
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
index df137e7b2b..aff0456b37 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala
@@ -28,8 +28,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
import org.apache.hadoop.io.LongWritable
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.expressions.{Literal, Row}
+import org.apache.spark.sql.catalyst.expressions.{Literal, InternalRow}
import org.apache.spark.sql.types._
+import org.apache.spark.sql.Row
class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
test("Test wrap SettableStructObjectInspector") {
@@ -45,7 +46,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
classOf[UDAFPercentile.State],
ObjectInspectorOptions.JAVA).asInstanceOf[StructObjectInspector]
- val a = unwrap(state, soi).asInstanceOf[Row]
+ val a = unwrap(state, soi).asInstanceOf[InternalRow]
val b = wrap(a, soi).asInstanceOf[UDAFPercentile.State]
val sfCounts = soi.getStructFieldRef("counts")
@@ -127,7 +128,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
}
}
- def checkValues(row1: Seq[Any], row2: Row): Unit = {
+ def checkValues(row1: Seq[Any], row2: InternalRow): Unit = {
row1.zip(row2.toSeq).foreach { case (r1, r2) =>
checkValue(r1, r2)
}
@@ -203,7 +204,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors {
})
checkValues(row,
- unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row])
+ unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[InternalRow])
checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt)))
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
index 5a5ea10e3c..a0d80dc39c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
@@ -17,10 +17,9 @@
package org.apache.spark.sql.hive
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.parquet.ParquetTest
-import org.apache.spark.sql.{QueryTest, SQLConf}
+import org.apache.spark.sql.{QueryTest, Row, SQLConf}
case class Cases(lower: String, UPPER: String)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
index 0e63d84e98..8707f9f936 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala
@@ -21,7 +21,7 @@ import java.io.File
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
index b384fb39f3..267d22c6b5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala
@@ -25,7 +25,7 @@ import org.scalatest.BeforeAndAfterAll
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.InternalRow
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.hive.test.TestHive._
import org.apache.spark.sql.hive.test.TestHive.implicits._
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index e62ac909cb..3864349cdb 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -21,8 +21,6 @@ import java.io.File
import org.scalatest.BeforeAndAfterAll
-import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD}
import org.apache.spark.sql.hive.execution.HiveTableScan
import org.apache.spark.sql.hive.test.TestHive._
@@ -30,7 +28,7 @@ import org.apache.spark.sql.hive.test.TestHive.implicits._
import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan}
import org.apache.spark.sql.sources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation}
import org.apache.spark.sql.types._
-import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf, SaveMode}
+import org.apache.spark.sql.{DataFrame, QueryTest, Row, SQLConf, SaveMode}
import org.apache.spark.util.Utils
// The data where the partitioning key exists only in the directory structure.