Apache Spark - A unified analytics engine for large-scale data processing
Go to file
Kazuaki Ishizaki f35df7d182 [SPARK-13805] [SQL] Generate code that get a value in each column from ColumnVector when ColumnarBatch is used
## What changes were proposed in this pull request?

This PR generates code that get a value in each column from ```ColumnVector``` instead of creating ```InternalRow``` when ```ColumnarBatch``` is accessed. This PR improves benchmark program by up to 15%.
This PR consists of two parts:

1. Get an ```ColumnVector ``` by using ```ColumnarBatch.column()``` method
2. Get a value of each column by using ```rdd_col${COLIDX}.getInt(ROWIDX)``` instead of ```rdd_row.getInt(COLIDX)```

This is a motivated example.
````
    sqlContext.conf.setConfString(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true")
    sqlContext.conf.setConfString(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
    val values = 10
    withTempPath { dir =>
      withTempTable("t1", "tempTable") {
        sqlContext.range(values).registerTempTable("t1")
        sqlContext.sql("select id % 2 as p, cast(id as INT) as id from t1")
          .write.partitionBy("p").parquet(dir.getCanonicalPath)
        sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("tempTable")
        sqlContext.sql("select sum(p) from tempTable").collect
      }
    }
````

The original code
````java
    ...
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         InternalRow rdd_row = rdd_batch.getRow(rdd_batchIdx++);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_row.isNullAt(0);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_row.getInt(0));
    ...
````

The code generated by this PR
````java
    /* 072 */       while (!shouldStop() && rdd_batchIdx < numRows) {
    /* 073 */         org.apache.spark.sql.execution.vectorized.ColumnVector rdd_col0 = rdd_batch.column(0);
    /* 074 */         /*** CONSUME: TungstenAggregate(key=[], functions=[(sum(cast(p#4 as bigint)),mode=Partial,isDistinct=false)], output=[sum#10L]) */
    /* 075 */         /* input[0, int] */
    /* 076 */         boolean rdd_isNull = rdd_col0.getIsNull(rdd_batchIdx);
    /* 077 */         int rdd_value = rdd_isNull ? -1 : (rdd_col0.getInt(rdd_batchIdx));
    ...
    /* 128 */         rdd_batchIdx++;
    /* 129 */       }
    /* 130 */       if (shouldStop()) return;

````
Performance
Without this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          434 /  488         36.3          27.6       1.0X
Read partition column                     302 /  346         52.1          19.2       1.4X
Read both columns                         588 /  643         26.8          37.4       0.7X
````
With this PR
````
model name	: Intel(R) Xeon(R) CPU E5-2667 v2  3.30GHz
Partitioned Table:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
Read data column                          392 /  516         40.1          24.9       1.0X
Read partition column                     256 /  318         61.4          16.3       1.5X
Read both columns                         523 /  539         30.1          33.3       0.7X
````

## How was this patch tested?
Tested by existing test suites and benchmark

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #11636 from kiszk/SPARK-13805.
2016-03-21 14:36:51 -07:00
.github [MINOR][MAINTENANCE] Fix typo for the pull request template. 2016-02-24 00:45:31 -08:00
assembly [SPARK-6363][BUILD] Make Scala 2.11 the default Scala version 2016-01-30 00:20:28 -08:00
bin [SPARK-13576][BUILD] Don't create assembly for examples. 2016-03-15 09:44:51 -07:00
build [SPARK-13324][CORE][BUILD] Update plugin, test, example dependencies for 2.x 2016-02-17 19:03:29 -08:00
common [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule 2016-03-21 07:58:57 +00:00
conf [SPARK-13264][DOC] Removed multi-byte characters in spark-env.sh.template 2016-02-11 09:30:36 +00:00
core [SPARK-14007] [SQL] Manage the memory used by hash map in shuffled hash join 2016-03-21 11:21:39 -07:00
data [SPARK-13013][DOCS] Replace example code in mllib-clustering.md using include_example 2016-03-03 09:32:47 -08:00
dev [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule 2016-03-21 07:58:57 +00:00
docs [MINOR][DOCS] Update build descriptions and commands 2016-03-18 21:32:48 -07:00
examples [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule 2016-03-21 07:58:57 +00:00
external [SPARK-14028][STREAMING][KINESIS][TESTS] Remove deprecated methods; fix two other warnings 2016-03-21 08:02:06 +00:00
graphx [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging 2016-03-17 19:23:38 +08:00
launcher [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule 2016-03-21 07:58:57 +00:00
licenses [SPARK-10833] [BUILD] Inline, organize BSD/MIT licenses in LICENSE 2015-09-28 22:56:43 -04:00
mllib [SPARK-13986][CORE][MLLIB] Remove DeveloperApi-annotations for non-publics 2016-03-21 14:57:52 +00:00
project [SPARK-13897][SQL] RelationalGroupedDataset and KeyValueGroupedDataset 2016-03-19 11:23:14 -07:00
python [SPARK-13764][SQL] Parse modes in JSON data source 2016-03-21 15:42:35 +08:00
R [MINOR][DOCS] Use spark-submit instead of sparkR to submit R script. 2016-03-19 13:23:34 +00:00
repl [SPARK-13456][SQL] fix creating encoders for case classes defined in Spark shell 2016-03-21 10:37:24 -07:00
sbin [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading issue 2016-03-14 12:22:02 -07:00
sql [SPARK-13805] [SQL] Generate code that get a value in each column from ColumnVector when ColumnarBatch is used 2016-03-21 14:36:51 -07:00
streaming [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule 2016-03-21 07:58:57 +00:00
tools [SPARK-13920][BUILD] MIMA checks should apply to @Experimental and @DeveloperAPI APIs 2016-03-15 23:25:31 -07:00
yarn [MINOR][DOCS] Add proper periods and spaces for CLI help messages and config doc. 2016-03-21 08:00:09 +00:00
.gitattributes [SPARK-3870] EOL character enforcement 2014-10-31 12:39:52 -07:00
.gitignore [SPARK-13596][BUILD] Move misc top-level build files into appropriate subdirs 2016-03-07 14:48:02 -08:00
CONTRIBUTING.md [SPARK-6889] [DOCS] CONTRIBUTING.md updates to accompany contribution doc updates 2015-04-21 22:34:31 -07:00
LICENSE [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading issue 2016-03-14 12:22:02 -07:00
NOTICE [SPARK-8725][PROJECT-INFRA] Test modules in topologically-sorted order in dev/run-tests 2016-01-26 14:20:11 -08:00
pom.xml [SPARK-13576][BUILD] Don't create assembly for examples. 2016-03-15 09:44:51 -07:00
README.md Add links howto to setup IDEs for developing spark 2015-12-04 14:43:16 +00:00
scalastyle-config.xml [SPARK-3854][BUILD] Scala style: require spaces before {. 2016-03-10 15:57:22 -08:00

Apache Spark

Spark is a fast and general cluster computing system for Big Data. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, and Spark Streaming for stream processing.

http://spark.apache.org/

Online Documentation

You can find the latest Spark documentation, including a programming guide, on the project web page and project wiki. This README file only contains basic setup instructions.

Building Spark

Spark is built using Apache Maven. To build Spark and its example programs, run:

build/mvn -DskipTests clean package

(You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at "Building Spark". For developing Spark using an IDE, see Eclipse and IntelliJ.

Interactive Scala Shell

The easiest way to start using Spark is through the Scala shell:

./bin/spark-shell

Try the following command, which should return 1000:

scala> sc.parallelize(1 to 1000).count()

Interactive Python Shell

Alternatively, if you prefer Python, you can use the Python shell:

./bin/pyspark

And run the following command, which should also return 1000:

>>> sc.parallelize(range(1000)).count()

Example Programs

Spark also comes with several sample programs in the examples directory. To run one of them, use ./bin/run-example <class> [params]. For example:

./bin/run-example SparkPi

will run the Pi example locally.

You can set the MASTER environment variable when running examples to submit examples to a cluster. This can be a mesos:// or spark:// URL, "yarn" to run on YARN, and "local" to run locally with one thread, or "local[N]" to run locally with N threads. You can also use an abbreviated class name if the class is in the examples package. For instance:

MASTER=spark://host:7077 ./bin/run-example SparkPi

Many of the example programs print usage help if no params are given.

Running Tests

Testing first requires building Spark. Once Spark is built, tests can be run using:

./dev/run-tests

Please see the guidance on how to run tests for a module, or individual tests.

A Note About Hadoop Versions

Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs.

Please refer to the build documentation at "Specifying the Hadoop Version" for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions.

Configuration

Please refer to the Configuration Guide in the online documentation for an overview on how to configure Spark.