Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-20682][SQL] Support a new faster ORC data source based on Apache ORC #17924

Closed
wants to merge 3 commits into from
Closed

[SPARK-20682][SQL] Support a new faster ORC data source based on Apache ORC #17924

wants to merge 3 commits into from

Conversation

dongjoon-hyun
Copy link
Member

@dongjoon-hyun dongjoon-hyun commented May 9, 2017

What changes were proposed in this pull request?

Since SPARK-2883, Apache Spark supports Apache ORC inside sql/hive module with Hive dependency. This issue aims to add a new and faster ORC data source inside sql/core and to replace the old ORC data source eventually. In this issue, the latest Apache ORC 1.4.0 (released yesterday) is used.

There are four key benefits.

  • Speed: Use both Spark ColumnarBatch and ORC RowBatch together. This is faster than the current implementation in Spark.
  • Stability: Apache ORC 1.4.0 has many fixes and we can depend on ORC community more.
  • Usability: User can use ORC data sources without hive module, i.e, -Phive.
  • Maintainability: Reduce the Hive dependency and can remove old legacy code later.

The followings are two examples of comparisons in OrcReadBenchmark.scala.

Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.12.4
Intel(R) Core(TM) i7-3615QM CPU @ 2.30GHz

SQL Single Int Column Scan:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
SQL ORC Vectorized                             170 /  194         92.5          10.8       1.0X
SQL ORC MR                                     388 /  396         40.5          24.7       0.4X
HIVE ORC MR                                    488 /  496         32.3          31.0       0.3X

Partitioned Table:                       Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
SQL Read data column                           188 /  227         83.6          12.0       1.0X
SQL Read partition column                       98 /  109        161.2           6.2       1.9X
SQL Read both columns                          193 /  227         81.5          12.3       1.0X
HIVE Read data column                          530 /  530         29.7          33.7       0.4X
HIVE Read partition column                     420 /  423         37.4          26.7       0.4X
HIVE Read both columns                         558 /  562         28.2          35.5       0.3X

How was this patch tested?

Pass the Jenkins tests with newly added test suites in sql/core.

@SparkQA
Copy link

SparkQA commented May 9, 2017

Test build #76693 has finished for PR 17924 at commit 8bfd4bb.

  • This patch fails to generate documentation.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class OrcColumnarBatchReader extends RecordReader[Void, ColumnarBatch] with Logging
  • class OrcRecordIterator extends Iterator[InternalRow] with Logging

@SparkQA
Copy link

SparkQA commented May 9, 2017

Test build #76695 has finished for PR 17924 at commit 4607e0e.

  • This patch fails to generate documentation.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Retest this please.

@SparkQA
Copy link

SparkQA commented May 9, 2017

Test build #76699 has finished for PR 17924 at commit 4607e0e.

  • This patch fails to generate documentation.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented May 10, 2017

Test build #76705 has finished for PR 17924 at commit 85ef731.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

/**
* ColumnarBatch for vectorized execution by whole-stage codegen.
*/
private var columnarBatch: ColumnarBatch = _
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IIRC, @viirya also has a PR for vectorized orc reader. In that PR, we simply wrap the orc column batch to expose spark column batch interfaces, instead of writing orc column batch to spark column batch. I think that approach is more efficient.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, thank you for the comment. It sounds efficient. I'll take a look.

Copy link
Member

@viirya viirya May 10, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

More specially, we wrap Hive's ColumnVector in a batch to expose Spark's ColumnVector for constructing Spark's ColumnarBatch. So we don't need to move data from one vector format to another vector format.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Btw, the PR is at #13775.

@viirya
Copy link
Member

viirya commented May 10, 2017

From the current benchmark, seems the performance has not obvious improvement, compared with the vectorized Hive ORC reader #13775.

Maybe with more efficient batch approach as @cloud-fan suggested, it can perform better.

Besides performance, getting rid of Hive dependency on ORC datasource is a great advantage for this.

}

private val SQL_ORC_FILE_FORMAT = "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat"
private val HIVE_ORC_FILE_FORMAT = "org.apache.spark.sql.hive.orc.OrcFileFormat"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will we keep current Hive ORC datasource even this is in Spark?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to keep both versions before complete transition and for safety. Instead, we can make configurable which file format is used for orc data source string, e.g, USING ORC.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So to avoid datasource name conflict, we may change Hive ORC datasource's shortName to other than "orc".

@dongjoon-hyun
Copy link
Member Author

@cloud-fan and @viirya .

Shall we remove the vectorized part from this PR?

  • The non-vectorized ORCFileFormat is mandatory and also the performance is better than the current one.
  • After merging sql/core ORCFileFormat, many people (including @viirya and me) can work together in parallel.

How do you think about that?

@viirya
Copy link
Member

viirya commented May 10, 2017

@dongjoon-hyun It is good for me. We can reduce the size of this PR too and mitigate review job.

@dongjoon-hyun
Copy link
Member Author

Yep. Since this is an approach adding new dependency on Apache ORC, the non-vectorized PR also will need more supports(or approval) from the committers. I'll wait for more opinions at the current status for a while.

@dongjoon-hyun
Copy link
Member Author

dongjoon-hyun commented May 10, 2017

Hi, @rxin , @marmbrus , @hvanhovell , @gatorsmile , @sameeragarwal .
Could you give us your opinions on this approach in Spark SQL part, too?

@dongjoon-hyun
Copy link
Member Author

dongjoon-hyun commented May 11, 2017

Hi, All.
For further discussion and easy comparison, I made another PR (#17943) except ColumnarBatch.

@dongjoon-hyun
Copy link
Member Author

Retest this please.

@SparkQA
Copy link

SparkQA commented May 15, 2017

Test build #76920 has finished for PR 17924 at commit 85ef731.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Retest this please.

@SparkQA
Copy link

SparkQA commented Aug 14, 2017

Test build #80604 has finished for PR 17924 at commit 85ef731.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cenyuhai
Copy link
Contributor

cenyuhai commented Sep 3, 2017

@dongjoon-hyun I have a question: does this orc data sources reader support a table contains multiple file format
for example:
table/
day=2017-09-01 RCFile
day=2017-09-02 ORCFile

ParquetFileFormat doesn't support this feature.

@dongjoon-hyun
Copy link
Member Author

Hi, I didn't try that, but that's not a concept of Spark data source table. Please don't expect that. :)

@dongjoon-hyun
Copy link
Member Author

BTW, the latest version is maintained in #17980.
Recently, Spark Vector format is changed.

@dongjoon-hyun
Copy link
Member Author

Please refer the superset in #17980 .

asfgit pushed a commit that referenced this pull request Jan 9, 2018
## What changes were proposed in this pull request?

This PR adds an ORC columnar-batch reader to native `OrcFileFormat`. Since both Spark `ColumnarBatch` and ORC `RowBatch` are used together, it is faster than the current Spark implementation. This replaces the prior PR, #17924.

Also, this PR adds `OrcReadBenchmark` to show the performance improvement.

## How was this patch tested?

Pass the existing test cases.

Author: Dongjoon Hyun <[email protected]>

Closes #19943 from dongjoon-hyun/SPARK-16060.

(cherry picked from commit f44ba91)
Signed-off-by: Wenchen Fan <[email protected]>
asfgit pushed a commit that referenced this pull request Jan 9, 2018
## What changes were proposed in this pull request?

This PR adds an ORC columnar-batch reader to native `OrcFileFormat`. Since both Spark `ColumnarBatch` and ORC `RowBatch` are used together, it is faster than the current Spark implementation. This replaces the prior PR, #17924.

Also, this PR adds `OrcReadBenchmark` to show the performance improvement.

## How was this patch tested?

Pass the existing test cases.

Author: Dongjoon Hyun <[email protected]>

Closes #19943 from dongjoon-hyun/SPARK-16060.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants