├── .gitignore
├── LICENSE
├── README.md
├── build.sbt
├── docs
├── IcebergSparkSQL.bib
├── IcebergSparkSQL.org
├── IcebergSparkSQL.pdf
├── basicExample.sql
├── iceSnapshots.png
├── iceTblCommit.png
├── iceTblScan.png
├── insIcePlan.png
├── insPlan.png
└── writePath.png
├── project
├── assembly.sbt
├── build.properties
└── plugins.sbt
├── scalastyle-config.xml
└── src
├── main
└── scala
│ └── org
│ └── apache
│ └── spark
│ └── sql
│ ├── execution
│ └── datasources
│ │ └── iceberg
│ │ └── PartitioningUtils.scala
│ └── iceberg
│ ├── Config.scala
│ ├── CreateIcebergTable.scala
│ ├── IceTableScanExec.scala
│ ├── IcebergFileCommitProtocol.scala
│ ├── InsertIntoIcebergTable.scala
│ ├── parsing
│ ├── AbstractSparkSQLParser.scala
│ └── IceParser.scala
│ ├── planning
│ ├── CreateTableRules.scala
│ ├── IcebergScanStrategy.scala
│ ├── IcebergTableValidationChecks.scala
│ ├── IcebergTableWriteRule.scala
│ └── SparkSessionExtensions.scala
│ ├── table
│ ├── SparkTableOperations.scala
│ └── SparkTables.scala
│ └── utils
│ ├── ColumnDependenciesExprVisitor.scala
│ ├── DataFileUtils.scala
│ ├── DelegationUtils.scala
│ ├── ExpressionConversions.scala
│ ├── LiteralUtils.scala
│ ├── ParquetMetrics.scala
│ ├── SchemaConversions.scala
│ ├── TableUtils.scala
│ ├── Transforms.scala
│ ├── TypeConversions.scala
│ └── utils.scala
└── test
├── resources
└── store_sales
│ ├── ss_sold_date_sk=0905245
│ ├── part-00004-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00010-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00017-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00022-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00026-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── ss_sold_date_sk=0906245
│ ├── part-00016-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00018-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00020-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00023-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00029-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── ss_sold_date_sk=0907245
│ ├── part-00000-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00003-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00008-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00012-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00019-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── ss_sold_date_sk=0908245
│ ├── part-00005-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00006-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00014-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00024-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00025-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── ss_sold_date_sk=0909245
│ ├── part-00001-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00002-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00009-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00013-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00021-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── ss_sold_date_sk=0910245
│ ├── part-00007-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00011-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00015-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ ├── part-00027-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
│ └── part-00028-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
└── scala
└── org
└── apache
└── spark
└── sql
└── hive
└── test
└── iceberg
├── AbstractTest.scala
├── BasicCreateAndInsertTest.scala
├── TestIcebergHive.scala
└── TestTables.scala
/.gitignore:
--------------------------------------------------------------------------------
1 | *.class
2 | *.log
3 | *.pyc
4 | sbt/*.jar
5 | *.crc
6 |
7 | # sbt specific
8 | .cache/
9 | .history/
10 | .lib/
11 | dist/*
12 | target/
13 | lib_managed/
14 | src_managed/
15 | project/boot/
16 | project/plugins/project/
17 |
18 | # idea
19 | .idea/
20 | *.iml
21 |
22 | # Mac
23 | .DS_Store
24 |
25 | # emacs
26 | *.*~
27 |
28 | # docs
29 | docs/private/
30 | docs/*.html
31 | docs/*.tex
32 | docs/auto/
33 | *-blx.bib
34 | *.bbl
35 | *.blg
36 | *.fdb_latexmk
37 | *.fls
38 | docs/*.xml
39 | docs/*.key
40 | *.tex
41 | *.tiff
42 | docs/benchmark/*.html
43 | **/auto/
44 | docs/notes/*_ImplNotes.org
45 |
46 |
47 | derby.log
48 |
49 | #test
50 | src/test/resources/store_sales_out/
51 | src/test/scala/org/apache/spark/sql/hive/test/iceberg/temp/
52 | _SUCCESS
53 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | Apache License
2 | Version 2.0, January 2004
3 | http://www.apache.org/licenses/
4 |
5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
6 |
7 | 1. Definitions.
8 |
9 | "License" shall mean the terms and conditions for use, reproduction,
10 | and distribution as defined by Sections 1 through 9 of this document.
11 |
12 | "Licensor" shall mean the copyright owner or entity authorized by
13 | the copyright owner that is granting the License.
14 |
15 | "Legal Entity" shall mean the union of the acting entity and all
16 | other entities that control, are controlled by, or are under common
17 | control with that entity. For the purposes of this definition,
18 | "control" means (i) the power, direct or indirect, to cause the
19 | direction or management of such entity, whether by contract or
20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
21 | outstanding shares, or (iii) beneficial ownership of such entity.
22 |
23 | "You" (or "Your") shall mean an individual or Legal Entity
24 | exercising permissions granted by this License.
25 |
26 | "Source" form shall mean the preferred form for making modifications,
27 | including but not limited to software source code, documentation
28 | source, and configuration files.
29 |
30 | "Object" form shall mean any form resulting from mechanical
31 | transformation or translation of a Source form, including but
32 | not limited to compiled object code, generated documentation,
33 | and conversions to other media types.
34 |
35 | "Work" shall mean the work of authorship, whether in Source or
36 | Object form, made available under the License, as indicated by a
37 | copyright notice that is included in or attached to the work
38 | (an example is provided in the Appendix below).
39 |
40 | "Derivative Works" shall mean any work, whether in Source or Object
41 | form, that is based on (or derived from) the Work and for which the
42 | editorial revisions, annotations, elaborations, or other modifications
43 | represent, as a whole, an original work of authorship. For the purposes
44 | of this License, Derivative Works shall not include works that remain
45 | separable from, or merely link (or bind by name) to the interfaces of,
46 | the Work and Derivative Works thereof.
47 |
48 | "Contribution" shall mean any work of authorship, including
49 | the original version of the Work and any modifications or additions
50 | to that Work or Derivative Works thereof, that is intentionally
51 | submitted to Licensor for inclusion in the Work by the copyright owner
52 | or by an individual or Legal Entity authorized to submit on behalf of
53 | the copyright owner. For the purposes of this definition, "submitted"
54 | means any form of electronic, verbal, or written communication sent
55 | to the Licensor or its representatives, including but not limited to
56 | communication on electronic mailing lists, source code control systems,
57 | and issue tracking systems that are managed by, or on behalf of, the
58 | Licensor for the purpose of discussing and improving the Work, but
59 | excluding communication that is conspicuously marked or otherwise
60 | designated in writing by the copyright owner as "Not a Contribution."
61 |
62 | "Contributor" shall mean Licensor and any individual or Legal Entity
63 | on behalf of whom a Contribution has been received by Licensor and
64 | subsequently incorporated within the Work.
65 |
66 | 2. Grant of Copyright License. Subject to the terms and conditions of
67 | this License, each Contributor hereby grants to You a perpetual,
68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
69 | copyright license to reproduce, prepare Derivative Works of,
70 | publicly display, publicly perform, sublicense, and distribute the
71 | Work and such Derivative Works in Source or Object form.
72 |
73 | 3. Grant of Patent License. Subject to the terms and conditions of
74 | this License, each Contributor hereby grants to You a perpetual,
75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
76 | (except as stated in this section) patent license to make, have made,
77 | use, offer to sell, sell, import, and otherwise transfer the Work,
78 | where such license applies only to those patent claims licensable
79 | by such Contributor that are necessarily infringed by their
80 | Contribution(s) alone or by combination of their Contribution(s)
81 | with the Work to which such Contribution(s) was submitted. If You
82 | institute patent litigation against any entity (including a
83 | cross-claim or counterclaim in a lawsuit) alleging that the Work
84 | or a Contribution incorporated within the Work constitutes direct
85 | or contributory patent infringement, then any patent licenses
86 | granted to You under this License for that Work shall terminate
87 | as of the date such litigation is filed.
88 |
89 | 4. Redistribution. You may reproduce and distribute copies of the
90 | Work or Derivative Works thereof in any medium, with or without
91 | modifications, and in Source or Object form, provided that You
92 | meet the following conditions:
93 |
94 | (a) You must give any other recipients of the Work or
95 | Derivative Works a copy of this License; and
96 |
97 | (b) You must cause any modified files to carry prominent notices
98 | stating that You changed the files; and
99 |
100 | (c) You must retain, in the Source form of any Derivative Works
101 | that You distribute, all copyright, patent, trademark, and
102 | attribution notices from the Source form of the Work,
103 | excluding those notices that do not pertain to any part of
104 | the Derivative Works; and
105 |
106 | (d) If the Work includes a "NOTICE" text file as part of its
107 | distribution, then any Derivative Works that You distribute must
108 | include a readable copy of the attribution notices contained
109 | within such NOTICE file, excluding those notices that do not
110 | pertain to any part of the Derivative Works, in at least one
111 | of the following places: within a NOTICE text file distributed
112 | as part of the Derivative Works; within the Source form or
113 | documentation, if provided along with the Derivative Works; or,
114 | within a display generated by the Derivative Works, if and
115 | wherever such third-party notices normally appear. The contents
116 | of the NOTICE file are for informational purposes only and
117 | do not modify the License. You may add Your own attribution
118 | notices within Derivative Works that You distribute, alongside
119 | or as an addendum to the NOTICE text from the Work, provided
120 | that such additional attribution notices cannot be construed
121 | as modifying the License.
122 |
123 | You may add Your own copyright statement to Your modifications and
124 | may provide additional or different license terms and conditions
125 | for use, reproduction, or distribution of Your modifications, or
126 | for any such Derivative Works as a whole, provided Your use,
127 | reproduction, and distribution of the Work otherwise complies with
128 | the conditions stated in this License.
129 |
130 | 5. Submission of Contributions. Unless You explicitly state otherwise,
131 | any Contribution intentionally submitted for inclusion in the Work
132 | by You to the Licensor shall be under the terms and conditions of
133 | this License, without any additional terms or conditions.
134 | Notwithstanding the above, nothing herein shall supersede or modify
135 | the terms of any separate license agreement you may have executed
136 | with Licensor regarding such Contributions.
137 |
138 | 6. Trademarks. This License does not grant permission to use the trade
139 | names, trademarks, service marks, or product names of the Licensor,
140 | except as required for reasonable and customary use in describing the
141 | origin of the Work and reproducing the content of the NOTICE file.
142 |
143 | 7. Disclaimer of Warranty. Unless required by applicable law or
144 | agreed to in writing, Licensor provides the Work (and each
145 | Contributor provides its Contributions) on an "AS IS" BASIS,
146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
147 | implied, including, without limitation, any warranties or conditions
148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
149 | PARTICULAR PURPOSE. You are solely responsible for determining the
150 | appropriateness of using or redistributing the Work and assume any
151 | risks associated with Your exercise of permissions under this License.
152 |
153 | 8. Limitation of Liability. In no event and under no legal theory,
154 | whether in tort (including negligence), contract, or otherwise,
155 | unless required by applicable law (such as deliberate and grossly
156 | negligent acts) or agreed to in writing, shall any Contributor be
157 | liable to You for damages, including any direct, indirect, special,
158 | incidental, or consequential damages of any character arising as a
159 | result of this License or out of the use or inability to use the
160 | Work (including but not limited to damages for loss of goodwill,
161 | work stoppage, computer failure or malfunction, or any and all
162 | other commercial damages or losses), even if such Contributor
163 | has been advised of the possibility of such damages.
164 |
165 | 9. Accepting Warranty or Additional Liability. While redistributing
166 | the Work or Derivative Works thereof, You may choose to offer,
167 | and charge a fee for, acceptance of support, warranty, indemnity,
168 | or other liability obligations and/or rights consistent with this
169 | License. However, in accepting such obligations, You may act only
170 | on Your own behalf and on Your sole responsibility, not on behalf
171 | of any other Contributor, and only if You agree to indemnify,
172 | defend, and hold each Contributor harmless for any liability
173 | incurred by, or claims asserted against, such Contributor by reason
174 | of your accepting any such warranty or additional liability.
175 |
176 | END OF TERMS AND CONDITIONS
177 |
178 | APPENDIX: How to apply the Apache License to your work.
179 |
180 | To apply the Apache License to your work, attach the following
181 | boilerplate notice, with the fields enclosed by brackets "[]"
182 | replaced with your own identifying information. (Don't include
183 | the brackets!) The text should be enclosed in the appropriate
184 | comment syntax for the file format. We also recommend that a
185 | file or class name and description of purpose be included on the
186 | same "printed page" as the copyright notice for easier
187 | identification within third-party archives.
188 |
189 | Copyright [yyyy] [name of copyright owner]
190 |
191 | Licensed under the Apache License, Version 2.0 (the "License");
192 | you may not use this file except in compliance with the License.
193 | You may obtain a copy of the License at
194 |
195 | http://www.apache.org/licenses/LICENSE-2.0
196 |
197 | Unless required by applicable law or agreed to in writing, software
198 | distributed under the License is distributed on an "AS IS" BASIS,
199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
200 | See the License for the specific language governing permissions and
201 | limitations under the License.
202 |
--------------------------------------------------------------------------------
/build.sbt:
--------------------------------------------------------------------------------
1 | scalaVersion in ThisBuild := "2.11.8"
2 |
3 | crossScalaVersions := Seq("2.10.5", "2.11.8")
4 |
5 | parallelExecution in Test := false
6 |
7 | val sparkVersion = "2.4.4"
8 | val icebergVersion = "0.6.3"
9 | val scalatestVersion = "3.0.5"
10 | val icebergSQLVersion = "0.0.1-SNAPSHOT"
11 | val derbyVersion = "10.11.1.1"
12 |
13 | resolvers += "jitpack" at "https://jitpack.io"
14 |
15 | val coreDependencies = Seq(
16 | "org.apache.spark" %% "spark-core" % sparkVersion % "provided",
17 | "org.apache.spark" %% "spark-sql" % sparkVersion % "provided",
18 | "org.apache.spark" %% "spark-hive" % sparkVersion % "provided",
19 | "org.apache.derby" % "derby" % derbyVersion % "provided" force(),
20 | "com.github.netflix.iceberg" % "iceberg-api" % icebergVersion ,
21 | "com.github.netflix.iceberg" % "iceberg-common" % icebergVersion ,
22 | "com.github.netflix.iceberg" % "iceberg-core" % icebergVersion
23 | )
24 |
25 | val coreTestDependencies = Seq(
26 | "org.scalatest" %% "scalatest" % scalatestVersion % "test",
27 | "org.pegdown" % "pegdown" % "1.6.0" % "test"
28 | )
29 |
30 | lazy val commonSettings = Seq(
31 | organization := "org.rhbutani",
32 |
33 | version := icebergSQLVersion,
34 |
35 | javaOptions := Seq("-Xms1g", "-Xmx3g",
36 | "-Duser.timezone=UTC",
37 | "-Dscalac.patmat.analysisBudget=512",
38 | "-XX:MaxPermSize=256M"),
39 |
40 | // Target Java 8
41 | scalacOptions += "-target:jvm-1.8",
42 | javacOptions in compile ++= Seq("-source", "1.8", "-target", "1.8"),
43 |
44 | scalacOptions := Seq("-feature", "-deprecation"
45 | //, "-Ylog-classpath"
46 | ),
47 |
48 | licenses := Seq("Apache License, Version 2.0" ->
49 | url("http://www.apache.org/licenses/LICENSE-2.0")
50 | ),
51 |
52 | homepage := Some(url("https://github.com/hbutani/icebergSparkSQL")),
53 |
54 | publishArtifact in (Test,packageDoc) := false,
55 |
56 | pomIncludeRepository := { _ => false },
57 |
58 | test in assembly := {},
59 |
60 | pomExtra := (
61 |
62 | https://github.com/hbutani/icebergSparkSQL.git
63 | scm:git:git@github.com:hbutani/icebergSparkSQL.git
64 |
65 |
66 |
67 | Harish Butani
68 | hbutani
69 | http://github.com/hbutani
70 |
71 | ),
72 |
73 | fork in Test := false,
74 | parallelExecution in Test := false
75 | )
76 |
77 | lazy val root = project.in(file("."))
78 | // .enablePlugins(GitVersioning)
79 | .settings(commonSettings: _*)
80 | .settings(
81 | name := s"icebergSparkSQL",
82 | libraryDependencies ++= (coreDependencies ++ coreTestDependencies),
83 | publishArtifact in (Compile, packageBin) := false,
84 | publishArtifact in Test := false,
85 | testOptions in Test ++= Seq(
86 | Tests.Argument(TestFrameworks.ScalaTest, "-h", "target/test-reports"),
87 | Tests.Argument(TestFrameworks.ScalaTest, "-oDT")
88 | ),
89 | aggregate in assembly := false
90 | )
91 | .settings(addArtifact(artifact in (Compile, assembly), assembly).settings: _*)
92 |
93 |
--------------------------------------------------------------------------------
/docs/IcebergSparkSQL.bib:
--------------------------------------------------------------------------------
1 | @misc{icespec,
2 | title = {Iceberg Table Spec},
3 | howpublished={\url{https://iceberg.apache.org/spec/}}
4 | }
5 |
6 | @misc{icepres,
7 | title = {Iceberg, A modern table format for big data},
8 | howpublished={\url{https://tinyurl.com/y6z6rcg4}}
9 | }
10 |
11 | @misc{icenet,
12 | title = {Iceberg Netflix Github},
13 | howpublished={\url{https://github.com/Netflix/iceberg}}
14 | }
15 |
16 | @misc{iceapache,
17 | title = {Iceberg Apache Github},
18 | howpublished={\url{https://github.com/apache/incubator-iceberg}}
19 | }
20 |
21 | @misc{sparkdsv2,
22 | title = {Apache Spark Data Source V2},
23 | howpublished={\url{https://databricks.com/session/apache-spark-data-source-v2}}
24 | }
25 |
26 | @misc{sparkdsv2jira1,
27 | title = {Data source API v2},
28 | howpublished={\url{https://tinyurl.com/y5u576gk}}
29 | }
30 |
31 | @misc{sparkdsv2jira2,
32 | title = {Data Source V2 improvements},
33 | howpublished={\url{https://tinyurl.com/yylna72p}}
34 | }
35 |
36 | @misc{oow,
37 | title={Innovating with the Oracle Platform for Data Science and Big Data},
38 | author={Greg Pavlik and Diwakar Goel},
39 | howpublished={\url{https://tinyurl.com/y59k23bf}}
40 | }
41 |
42 | @misc{orasnap,
43 | title={Interactive Data Lake Queries At Scale},
44 | author={Peter Jeffcock},
45 | howpublished = {\url{https://tinyurl.com/y8hbyp9q}}
46 | }
47 |
48 | @misc{icesparksql,
49 | title={Iceberg Spark SQL Github},
50 | howpublished = {\url{https://github.com/hbutani/icebergSparkSQL}}
51 | }
--------------------------------------------------------------------------------
/docs/IcebergSparkSQL.pdf:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/IcebergSparkSQL.pdf
--------------------------------------------------------------------------------
/docs/basicExample.sql:
--------------------------------------------------------------------------------
1 | -- input table
2 | create table if not exists store_sales
3 | (
4 | ss_sold_time_sk int,
5 | ss_item_sk int,
6 | ss_customer_sk int,
7 | ss_cdemo_sk int,
8 | ss_hdemo_sk int,
9 | ss_addr_sk int,
10 | ss_store_sk int,
11 | ss_promo_sk int,
12 | ss_quantity int,
13 | ss_wholesale_cost decimal(7,2),
14 | ss_list_price decimal(7,2),
15 | ss_sales_price decimal(7,2),
16 | ss_ext_sales_price decimal(7,2),
17 | ss_sold_month string,
18 | ss_sold_day string,
19 | ss_sold_date_sk string
20 | )
21 | USING parquet
22 | OPTIONS (
23 | path '/Users/hbutani/sparkline/icebergSQL/src/test/resources/store_sales'
24 | )
25 | partitioned by (ss_sold_date_sk)
26 | ;
27 |
28 | msck repair table store_sales;
29 |
30 | -- managed table
31 | create table if not exists store_sales_out
32 | (
33 | ss_sold_time_sk int,
34 | ss_item_sk int,
35 | ss_customer_sk int,
36 | ss_cdemo_sk int,
37 | ss_hdemo_sk int,
38 | ss_addr_sk int,
39 | ss_store_sk int,
40 | ss_promo_sk int,
41 | ss_quantity int,
42 | ss_wholesale_cost decimal(7,2),
43 | ss_list_price decimal(7,2),
44 | ss_sales_price decimal(7,2),
45 | ss_ext_sales_price decimal(7,2),
46 | ss_sold_month string,
47 | ss_sold_day string,
48 | ss_sold_date_sk string
49 | )
50 | USING parquet
51 | OPTIONS (
52 | path '/tmp/store_sales_out',
53 | addTableManagement "true",
54 | columnDependencies "ss_sold_date_sk=ss_sold_month:truncate[2],ss_sold_date_sk=ss_sold_day:truncate[4]"
55 | )
56 | partitioned by (ss_sold_date_sk)
57 | ;
58 |
59 | -- first insert
60 | insert into store_sales_out
61 | select * from store_sales;
62 |
63 | -- show snapshots
64 | select * from `store_sales_out$snapshots`;
65 |
66 | -- count is 2109
67 | select count(*) from store_sales_out;
68 |
69 | -- query on ss_sold_date_sk='0906245'
70 | select count(*)
71 | from store_sales_out
72 | where ss_sold_date_sk='0906245';
73 |
74 | -- insert data gain
75 | insert into store_sales_out
76 | select * from store_sales;
77 |
78 | -- observe 2 snapshots
79 | select * from `store_sales_out$snapshots`;
80 |
81 | -- table now has twice number of rows
82 | select count(*) from store_sales_out;
83 |
84 | -- query as of first snapshot
85 | -- 1568658741390 is some time between 2 snapshots
86 | -- observe 2109 rows
87 | as of '1568658741390'
88 | select count(*) from store_sales_out;
89 |
90 | -- now do an insert overwrite
91 | insert overwrite table store_sales_out
92 | select * from store_sales;
93 |
94 | -- observe 3 snapshots
95 | -- latest snapshot adds x files and deletes 2x files
96 | select * from `store_sales_out$snapshots`;
97 |
98 | -- table now has 2109 rows again
99 | select count(*) from store_sales_out;
100 |
101 | -- query as of second snapshot
102 | -- 1568659495070 is some time between 2 snapshots
103 | -- observe 4218 rows
104 | as of '1568659495070'
105 | select count(*) from store_sales_out;
106 |
107 | -- SnapShot on an insert overwrite of 1 partition:
108 | insert overwrite table store_sales_out partition ( ss_sold_date_sk='0906245' )
109 | select ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_quantity,ss_wholesale_cost,ss_list_price,ss_sales_price,ss_ext_sales_price,ss_sold_month,ss_sold_day from store_sales
110 | where ss_sold_date_sk='0906245' ;
111 |
112 | -- observe 4 snapshots
113 | select * from `store_sales_out$snapshots`;
114 |
115 | -- table now has 2109 rows again
116 | select count(*) from store_sales_out;
117 |
118 | -- SnapShot on an insert overwrite of 1 partition, with source predicate: 5 files added, 5 files deleted
119 | insert overwrite table store_sales_out partition ( ss_sold_date_sk='0905245' )
120 | select ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_quantity,ss_wholesale_cost,ss_list_price,ss_sales_price,ss_ext_sales_price,ss_sold_month,ss_sold_day from store_sales
121 | where ss_sold_date_sk='0905245' and ss_item_sk < 5000;
122 |
123 | -- observe 5 snapshots
124 | select * from `store_sales_out$snapshots`;
125 |
126 | -- table now has 1877 rows again
127 | select count(*) from store_sales_out;
128 |
129 | -- now a query with ss_item_sk > 5000 on ss_sold_date_sk=0905245 should be a null scan
130 | -- observe no new job listed in Spark UI
131 | select * from store_sales_out
132 | where ss_item_sk > 5000 and ss_sold_date_sk='0905245';
133 |
--------------------------------------------------------------------------------
/docs/iceSnapshots.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/iceSnapshots.png
--------------------------------------------------------------------------------
/docs/iceTblCommit.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/iceTblCommit.png
--------------------------------------------------------------------------------
/docs/iceTblScan.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/iceTblScan.png
--------------------------------------------------------------------------------
/docs/insIcePlan.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/insIcePlan.png
--------------------------------------------------------------------------------
/docs/insPlan.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/insPlan.png
--------------------------------------------------------------------------------
/docs/writePath.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/docs/writePath.png
--------------------------------------------------------------------------------
/project/assembly.sbt:
--------------------------------------------------------------------------------
1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.9")
2 |
--------------------------------------------------------------------------------
/project/build.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Licensed to the Apache Software Foundation (ASF) under one or more
3 | # contributor license agreements. See the NOTICE file distributed with
4 | # this work for additional information regarding copyright ownership.
5 | # The ASF licenses this file to You under the Apache License, Version 2.0
6 | # (the "License"); you may not use this file except in compliance with
7 | # the License. You may obtain a copy of the License at
8 | #
9 | # http://www.apache.org/licenses/LICENSE-2.0
10 | #
11 | # Unless required by applicable law or agreed to in writing, software
12 | # distributed under the License is distributed on an "AS IS" BASIS,
13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | # See the License for the specific language governing permissions and
15 | # limitations under the License.
16 | #
17 |
18 | sbt.version=0.13.18
19 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | // You may use this file to add plugin dependencies for sbt.
2 |
3 | resolvers ++= Seq(
4 | "Central" at "https://oss.sonatype.org/content/repositories/releases/"
5 | )
6 | resolvers += "Typesafe repository" at "http://repo.typesafe.com/typesafe/releases/"
7 |
8 | addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.9.2")
9 |
10 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.11")
11 |
12 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0")
13 |
14 | addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.3.22")
15 |
16 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0")
17 |
18 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "1.0.0")
19 |
--------------------------------------------------------------------------------
/scalastyle-config.xml:
--------------------------------------------------------------------------------
1 |
17 |
19 |
20 |
21 |
22 |
23 |
24 |
25 |
26 | Scalastyle standard configuration
27 |
28 |
29 |
30 |
31 |
32 |
33 |
53 |
54 |
55 |
56 |
57 |
58 |
59 |
60 | true
61 |
62 |
63 |
64 |
65 |
66 |
67 |
68 |
69 |
70 |
71 |
72 |
73 |
74 |
75 |
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
87 |
88 |
89 |
90 |
91 |
92 |
93 |
94 |
95 |
96 |
97 |
98 |
99 |
100 |
101 |
102 |
103 |
104 |
105 |
106 |
107 |
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
118 |
119 |
120 |
121 |
122 |
123 |
124 |
125 |
126 |
127 |
128 |
129 |
130 |
131 |
132 |
133 |
134 |
135 |
136 |
137 |
138 |
139 |
140 |
141 |
142 |
143 |
144 |
145 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/execution/datasources/iceberg/PartitioningUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.execution.datasources.iceberg
19 |
20 | import org.apache.hadoop.fs.Path
21 | import org.apache.spark.sql.SparkSession
22 | import org.apache.spark.sql.catalyst.InternalRow
23 | import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
24 | import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, Cast, EqualTo, Expression, GenericInternalRow, Literal}
25 | import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
26 | import org.apache.spark.sql.execution.datasources.{PartitionPath, PartitionSpec => SparkPSec, PartitioningUtils => SparkPartitioningUtils}
27 | import org.apache.spark.sql.types.{StringType, StructType}
28 | import org.apache.spark.unsafe.types.UTF8String
29 |
30 | /**
31 | * '''Partitioning Data Structures and concepts'''
32 | *
33 | * 1. ''Partitioning Scheme:'' represents the structure of a Table's partitions. Represented
34 | * as a Spark [[StructType]]
35 | * 2. ''Partition Value or Row:'' represents the values of a Table partition. Represented
36 | * as a Spark [[InternalRow]]
37 | * 3. ''Partition Specification:'' represents a set of partitions being acted on in a ''Insert''
38 | * commad. Capture as a [[TablePartitionSpec]]
39 | * 4. ''Partition Info : '' details about a Table Partition captured as [[PartitionPath]] (this
40 | * name is confusing)
41 | * 5. ''Partition Set :'' represents details on a set of Table Partitions captured as
42 | * [[PartitionSpec]](terrible name)
43 | */
44 | object PartitioningUtils {
45 |
46 | /**
47 | * A wrapper for the [[SparkPartitioningUtils]] parsePartition function.
48 | * Used to introspect a set of filesystem paths and construct the set of
49 | * [[PartitionPath]] at these paths, and combine them into a [[PartitionSpec]]
50 | *
51 | * @param paths
52 | * @param basePaths
53 | * @param partSchema
54 | * @param sparkSession
55 | * @return
56 | */
57 | def parsePartitions(paths: Seq[Path],
58 | basePaths: Set[Path],
59 | partSchema: StructType
60 | )(implicit sparkSession: SparkSession) : SparkPSec = {
61 |
62 | val tzId = sparkSession.sessionState.conf.sessionLocalTimeZone
63 |
64 | SparkPartitioningUtils.parsePartitions(
65 | paths,
66 | typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled,
67 | basePaths = basePaths,
68 | Some(partSchema),
69 | sparkSession.sqlContext.conf.caseSensitiveAnalysis,
70 | DateTimeUtils.getTimeZone(tzId))
71 | }
72 |
73 | /**
74 | * A thin wrapper on `parsePartitions` method. This converts a filesystem path
75 | * into a ''Partition Info'', returning it as a [[PartitionPath]]
76 | * @param p
77 | * @param partSchema
78 | * @param sparkSession
79 | * @return
80 | */
81 | def parsePath(p : Path,
82 | partSchema: StructType
83 | )(implicit sparkSession: SparkSession) : PartitionPath = {
84 | val sPSpec = parsePartitions(Seq(p), Set.empty, partSchema)
85 | sPSpec.partitions.head
86 | }
87 |
88 | /**
89 | * Convert a fileSystem path into a ''Partition Value'' return it as an [[InternalRow]]
90 | * @param p
91 | * @param partSchema
92 | * @return
93 | */
94 | def partitionRowFromFullPath(p : String, partSchema : StructType) : InternalRow = {
95 | val pathString = removeFileName(p)
96 | val m = CaseInsensitiveMap(partitionValuesFromFullPath(pathString))
97 |
98 | assert(m.size == partSchema.size)
99 |
100 | val casts : Array[Cast] = for(
101 | f <- partSchema.fields
102 | ) yield {
103 | val s = m(f.name)
104 | val l = Literal.create(UTF8String.fromString(s), StringType)
105 | Cast(l, f.dataType)
106 | }
107 | val values : Seq[Any] = casts.map(c => c.eval(null))
108 | new GenericInternalRow(values.toArray)
109 |
110 | }
111 |
112 | /**
113 | * Convert a ''Partition Value'' into a [[Expression][Spark Expression]].
114 | * The Expression is a conjunct of Equalty conditions. For each partition
115 | * column we have an equality condition that the partition column equals the value in
116 | * the given partition Row/value.
117 | *
118 | * @param partSchema
119 | * @param partRow
120 | * @return
121 | */
122 | def partitionRowToPredicate(partSchema : StructType,
123 | partRow : InternalRow) : Expression = {
124 |
125 | val exprs = for((f,i) <- partSchema.zipWithIndex) yield {
126 | val attr = AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()
127 | EqualTo(attr, Literal(partRow.get(i, f.dataType)))
128 | }
129 | exprs.reduce(And)
130 | }
131 |
132 | /**
133 | * Parse a filesystem path into a ''Partition Specification:'' and return it as
134 | * [[TablePartitionSpec]]. Look for folder names of the form 'name=value' at the end of the
135 | * path and map them into entries in the [[TablePartitionSpec]]
136 | *
137 | * @param pathString
138 | * @return
139 | */
140 | def partitionValuesFromFullPath(pathString : String) : TablePartitionSpec = {
141 | var m = Map.empty[String, String]
142 | def add(col : String, value: String) : Unit =
143 | m += (col -> value)
144 |
145 | mapPartitionSpecs(pathString,
146 | (path, col, value) => add(col, value)
147 | )
148 | m
149 | }
150 |
151 | /**
152 | * Map a [[TablePartitionSpec]] into a [Expression][Spark Expression]].
153 | * The Expression is a conjunct of Equalty conditions. For each partition
154 | * column we have an equality condition that the partition column equals the value in
155 | * the given [[TablePartitionSpec]]
156 | *
157 | * @param partitionSchema
158 | * @param pSpec
159 | * @return
160 | */
161 | def expressionFromPartitionSpec(partitionSchema : StructType,
162 | pSpec : CaseInsensitiveMap[String]
163 | ) : Option[Expression] = {
164 | val exprs = for(
165 | attr <- partitionSchema.toAttributes;
166 | v <- pSpec.get(attr.name)
167 | ) yield {
168 | EqualTo(attr, Cast(Literal(UTF8String.fromString(v), StringType), attr.dataType))
169 | }
170 |
171 | exprs match {
172 | case e if e.size == 0 => None
173 | case e if e.size == 1 => Some(e.head)
174 | case _ => Some(exprs.reduce(And))
175 | }
176 | }
177 |
178 | private def mapPartitionSpecs(fullPath : String,
179 | op : (String, String, String) => Unit) : Unit = {
180 | var p = new Path(fullPath)
181 | var reachedNonPartName = false
182 |
183 | while(!reachedNonPartName) {
184 | val name = p.getName
185 | partitionSpec(name) match {
186 | case Some((col, value)) => op(name, col,value)
187 | case None => {
188 | reachedNonPartName = true
189 | }
190 | }
191 | p = p.getParent
192 | }
193 | }
194 |
195 | private def partitionSpec(fileName : String) : Option[(String,String)] = {
196 | val equalSignIndex = fileName.indexOf('=')
197 | if (equalSignIndex == -1) {
198 | None
199 | } else {
200 | val columnName = fileName.take(equalSignIndex)
201 | if (columnName.isEmpty) return None
202 | val rawColumnValue = fileName.drop(equalSignIndex + 1)
203 | if (rawColumnValue.nonEmpty) {
204 | Some((columnName, rawColumnValue))
205 | } else None
206 | }
207 | }
208 |
209 | def removeFileName(pathStr : String): String = {
210 | if (pathStr.endsWith("/")) {
211 | pathStr
212 | } else {
213 | val path = new Path(pathStr)
214 | path.getParent.toString
215 | }
216 | }
217 |
218 | }
219 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/Config.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
21 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
22 |
23 | object Config {
24 |
25 | /**
26 | * Use to specify that the system should provide additional management capabilities
27 | * for a table.
28 | */
29 | val ADD_MANAGEMENT = "addTableManagement"
30 | val DEFAULT_ADD_MANAGEMENT = "false"
31 |
32 |
33 | private def getOption(tblDesc : CatalogTable,
34 | optionName : String) : Option[String] = {
35 |
36 | def get(m : Map[String, String]) : Option[String] = {
37 | m.get(optionName).orElse(CaseInsensitiveMap(m).get(optionName))
38 | }
39 |
40 | get(tblDesc.storage.properties).orElse(get(tblDesc.properties))
41 |
42 | }
43 |
44 | def provideManagement(tblDesc : CatalogTable) : Boolean = {
45 | getOption(tblDesc, ADD_MANAGEMENT).
46 | getOrElse(DEFAULT_ADD_MANAGEMENT).toBoolean
47 | }
48 |
49 | def isManaged(tblDesc : CatalogTable) : Boolean = provideManagement(tblDesc)
50 |
51 |
52 | /**
53 | * For tables `addTableManagement=true` use this option to specify
54 | * column dependencies. This specified as a comma-sperated list of comma
55 | * separated list of column dependence. A 'column dependence' is of the form
56 | * `srcCol=destCol:transformFn`, for example
57 | * `date_col=day_col:extract[2]` where `date_col` is a string in the form `DD-MM-YYYY`.
58 | * Semantically a column dependence implies that the destCol value can be determined
59 | * from a srcCol value; the columns are in a one-to-one` or `many-to-one` relationship.
60 | * The src and dest columns can be any column (data or partition columns) of the table.
61 | *
62 | * Currently we support [[Transforms][Iceberg Transforms]] as mapping functions,
63 | * they must be a string representation of an [[Transforms][Iceberg Transforms]].
64 | * So users can relate columns based on `date` or `timestamp` elements,
65 | * based on `truncating` values or `value buckets`.
66 | *
67 | * During a table scan we will attempt to transform a predicate on the `srcCol`
68 | * into a predicate on the `destCol`. For example `date_col='09-12-2019'` will be transformed
69 | * into `day_col='09'` and applied. If the `destCol` is a partition column
70 | * this can lead to partition pruning. For example if the table is partitioned by
71 | * `day_col` then from a predicate `date_col='09-12-2019'` the inferred predicate
72 | * `day_col='09'` will lead to only partitions from the 9th day of each month to
73 | * be scanned. In case the `destCol` is a data column the inferred predicate can lead
74 | * to datafiles being pruned based on the statistics available on the column.
75 | */
76 |
77 | val COLUMN_DEPENDENCIES = "columnDependencies"
78 |
79 | def columnDependencies(tblDesc : CatalogTable) : Option[String] = {
80 | getOption(tblDesc, COLUMN_DEPENDENCIES)
81 | }
82 |
83 | }
84 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/CreateIcebergTable.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
21 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
22 | import org.apache.spark.sql.execution.SparkPlan
23 | import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, CreateDataSourceTableCommand, DataWritingCommand, RunnableCommand}
24 | import org.apache.spark.sql.iceberg.table.SparkTablesForCreate
25 | import org.apache.spark.sql.iceberg.utils.{SchemaConversions, _}
26 | import org.apache.spark.sql.{Row, SparkSession}
27 |
28 | trait IcebergTableCreation {
29 | val catalogTable: CatalogTable
30 |
31 | def createIcebergTable(sparkSession: SparkSession) : Unit = {
32 | implicit val ss = currentSparkSession
33 | import scala.collection.JavaConversions._
34 | val (schema, partSpec) = SchemaConversions.toIcebergSpec(catalogTable)
35 | val iceTables = new SparkTablesForCreate(ss, catalogTable)
36 | val tId = parseIdentifier(catalogTable.qualifiedName, ss)
37 | iceTables.create(schema, partSpec, Map.empty[String,String], catalogTable.qualifiedName)
38 | }
39 | }
40 |
41 | case class CreateIcebergTable(createTblCmd : CreateDataSourceTableCommand)
42 | extends RunnableCommand with IcebergTableCreation {
43 |
44 | val catalogTable = createTblCmd.table
45 |
46 | override def run(sparkSession: SparkSession): Seq[Row] = {
47 | val rows = createTblCmd.run(sparkSession)
48 | createIcebergTable(sparkSession)
49 | rows
50 | }
51 | }
52 |
53 |
54 | case class CreateIcebergTableAsSelect(createTblCmd : CreateDataSourceTableAsSelectCommand)
55 | extends DataWritingCommand with IcebergTableCreation {
56 |
57 | override def query: LogicalPlan = createTblCmd.query
58 |
59 | override def outputColumnNames: Seq[String] = createTblCmd.outputColumnNames
60 |
61 | override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = {
62 | val rows = createTblCmd.run(sparkSession, child)
63 | createIcebergTable(sparkSession)
64 | rows
65 | }
66 |
67 | override val catalogTable: CatalogTable = createTblCmd.table
68 | }
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/IceTableScanExec.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import com.netflix.iceberg.DataFile
21 | import com.netflix.iceberg.expressions.{ExpressionVisitors, Expression => IceExpression}
22 | import org.apache.spark.rdd.RDD
23 | import org.apache.spark.sql.catalyst.InternalRow
24 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
25 | import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
26 | import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, Literal, SortOrder}
27 | import org.apache.spark.sql.catalyst.plans.physical.Partitioning
28 | import org.apache.spark.sql.execution.datasources.PartitionDirectory
29 | import org.apache.spark.sql.execution.{CodegenSupport, FileSourceScanExec, SparkPlan, UnaryExecNode}
30 | import org.apache.spark.sql.iceberg.table.SparkTables
31 | import org.apache.spark.sql.iceberg.utils.DelegationUtils.{DelegatedField, DelegatedMethod0, DelegatedMethod1}
32 | import org.apache.spark.sql.iceberg.utils.{ColumnDependenciesExprVisitor, ExpressionConversions, TableUtils, Transforms}
33 | import com.netflix.iceberg.transforms.{Transform => IceTransform}
34 | import org.apache.spark.sql.iceberg.utils.TypeConversions.TypeMapping
35 |
36 | /**
37 | * Setup as a parent Physical Operator of a [[FileSourceScanExec]]. During execution
38 | * before handing over control to its child [[FileSourceScanExec]] operator it updates its
39 | * `selectedPartitions` member.
40 | *
41 | * This is computed based on the `partitionFilters` and `dataFilters` associated with this scan.
42 | * These are converted to an [[IceExpression]], further [[IceExpression]] are added based on
43 | * `column dependencies` defined for this table. From the current Iceberg snaphot a list of
44 | * [[DataFile]] are computed. Finally the [[FileSourceScanExec]] list of selected partitions
45 | * list is updated to remove files from [[PartitionDirectory]] instances not in this list of
46 | * DataFiles.
47 | *
48 | * @param child
49 | * @param catalogTable
50 | */
51 | case class IceTableScanExec(child: FileSourceScanExec,
52 | @transient catalogTable: CatalogTable
53 | )
54 | extends UnaryExecNode with CodegenSupport {
55 |
56 | @transient private val iceTable = {
57 | implicit val ss = sqlContext.sparkSession
58 | val iceTables = new SparkTables(ss, catalogTable)
59 | val tbId = utils.qualifiedTableIdentifier(catalogTable.identifier, ss)
60 | iceTables.load(tbId.database.getOrElse(null), tbId.table)
61 | }
62 |
63 | lazy val sparkExpressionToPushToIce: Expression = {
64 | val alwaysTrue : Expression = Literal(true)
65 | (child.partitionFilters ++ child.dataFilters).foldLeft(alwaysTrue)(And)
66 | }
67 |
68 | @transient private lazy val icebergFilter: IceExpression = {
69 | val iceExpr = ExpressionConversions.convert(sparkExpressionToPushToIce)
70 | columnDependencies.map { cds =>
71 | val exprVisitor = new ColumnDependenciesExprVisitor(iceTable.schema().asStruct(), cds)
72 | ExpressionVisitors.visit(iceExpr, exprVisitor)
73 | }.getOrElse(iceExpr)
74 | }
75 |
76 | @transient private lazy val iceDataFiles : Seq[DataFile] = {
77 | val iceScan = iceTable.newScan().
78 | useSnapshot(TableUtils.getThreadSnapShotId(iceTable)).
79 | filter(icebergFilter)
80 | import scala.collection.JavaConversions._
81 | iceScan.planFiles().map(_.file).toSeq
82 | }
83 |
84 | @transient private val selectedPartitionsDelegate =
85 | new DelegatedField[Array[PartitionDirectory], FileSourceScanExec, FileSourceScanExec](
86 | child, "org$apache$spark$sql$execution$FileSourceScanExec$$selectedPartitions")
87 |
88 | @transient private val doExecute_d =
89 | new DelegatedMethod0[SparkPlan, FileSourceScanExec, RDD[InternalRow]](child, "doExecute")
90 |
91 | @transient private val doProduce_d =
92 | new DelegatedMethod1[CodegenSupport, FileSourceScanExec, String, CodegenContext](
93 | child, "doProduce")
94 |
95 |
96 | private var updateSelectedPartitionsDone : Boolean = false
97 |
98 | private def updateSelectedPartitions : Unit = {
99 | if (!updateSelectedPartitionsDone) {
100 | child.metadata // trigger construction of child selectedPartitions
101 |
102 | val dataFiles = iceDataFiles
103 | val dFilePaths = dataFiles.map(_.path().toString).toSet
104 | val sParts = selectedPartitionsDelegate.value
105 |
106 | val updtSelParts = sParts.map { sPart =>
107 | PartitionDirectory(
108 | sPart.values,
109 | sPart.files.filter(fs => dFilePaths.contains(fs.getPath.toString))
110 | )
111 | }
112 | selectedPartitionsDelegate.value = updtSelParts
113 | updateSelectedPartitionsDone = true
114 | }
115 | }
116 |
117 | override def inputRDDs(): Seq[RDD[InternalRow]] = {
118 | updateSelectedPartitions
119 | child.inputRDDs()
120 | }
121 |
122 | override protected def doExecute(): RDD[InternalRow] = {
123 | updateSelectedPartitions
124 | doExecute_d.apply
125 | }
126 |
127 | override def output: Seq[Attribute] = child.output
128 |
129 | override protected def doProduce(ctx: CodegenContext): String = {
130 | updateSelectedPartitions
131 | child.produce(ctx, this)
132 | }
133 |
134 | override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
135 | updateSelectedPartitions
136 | parent.doConsume(ctx, input, row)
137 | }
138 |
139 | override lazy val (outputPartitioning, outputOrdering) : (Partitioning, Seq[SortOrder]) =
140 | (child.outputPartitioning, child.outputOrdering)
141 |
142 | override def doCanonicalize(): IceTableScanExec = {
143 | IceTableScanExec(child.doCanonicalize(), catalogTable)
144 | }
145 |
146 | private def columnDependencies : Option[Map[String, Map[String, IceTransform[_,_]]]] = {
147 | Config.columnDependencies(catalogTable).flatMap { colDeps =>
148 | val typMapping = TypeMapping(catalogTable.schema, iceTable.schema().asStruct())
149 | val t = Transforms.fromOption(colDeps, typMapping)(sqlContext.sparkSession)
150 | t.right.toOption
151 | }
152 | }
153 |
154 | // for testing only
155 | def showPartitionsScanned : Seq[PartitionDirectory] = {
156 | updateSelectedPartitions
157 | selectedPartitionsDelegate.value
158 | }
159 |
160 | // for testing only
161 | def getIceFilter = icebergFilter
162 |
163 | }
164 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/IcebergFileCommitProtocol.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import com.netflix.iceberg.{DataFile, PartitionSpec}
21 | import org.apache.hadoop.fs.{FileSystem, Path}
22 | import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext, TaskAttemptID}
23 | import org.apache.spark.internal.io.FileCommitProtocol
24 | import org.apache.spark.sql.iceberg.utils.{DataFileUtils, _}
25 | import org.apache.spark.sql.types.StructType
26 |
27 | import scala.collection.mutable.{ArrayBuffer, Map => MMap}
28 |
29 | /**
30 | * Provide the following function on top of the 'normal' Commit Protocol.
31 | * Commit actions are simply deferred to the 'designate' except in the following:
32 | * - track files created for each Task in a [[TaskPaths]] instance. This
33 | * tracks the temporary file location and also the location that the
34 | * file will be moved to on a commit.
35 | * - on Task Commit build an Iceberg [[DataFile]] instance. Currently only if the
36 | * file is a parquet file we will also build column level stats.
37 | * - The [[TaskCommitMessage]] we send back has a payload of [[IcebergTaskCommitMessage]],
38 | * which encapsulates the TaskCommitMessage build by the 'designate' and the
39 | * DataFile instances.
40 | * - we ignore ''deleteWithJob'' invocations, as we want to keep historical files around.
41 | * These will be removed via a `clear snapshot` command.
42 | * - on a ''commitJob'' we extract all the [[DataFile]] instances from the
43 | * [[IcebergTaskCommitMessage]] messages and expose a ''addedDataFiles'' list
44 | * which is used by [[IceTableScanExec]] to build the new Iceberg Table Snapshot.
45 | *
46 | * @param designate
47 | * @param path
48 | * @param partitionSchema
49 | * @param icePartSpec
50 | * @param iceSchema
51 | * @param iceFmt
52 | */
53 | class IcebergFileCommitProtocol(val designate: FileCommitProtocol,
54 | val path: String,
55 | val partitionSchema: StructType,
56 | val icePartSpec: PartitionSpec,
57 | val iceSchema: IceSchema,
58 | val iceFmt: IceFormat
59 | )
60 | extends FileCommitProtocol with Serializable {
61 |
62 | import org.apache.spark.sql.iceberg.IcebergFileCommitProtocol._
63 |
64 | @transient var taskFiles = MMap[TaskAttemptID, ArrayBuffer[TaskPaths]]()
65 | @transient val addedDataFiles = ArrayBuffer[DataFile]()
66 |
67 | private def setupTaskFileMap: Unit = {
68 | if (taskFiles == null) {
69 | taskFiles = MMap[TaskAttemptID, ArrayBuffer[TaskPaths]]()
70 | }
71 | }
72 |
73 | override def setupJob(jobContext: JobContext): Unit =
74 | designate.setupJob(jobContext)
75 |
76 | override def commitJob(jobContext: JobContext,
77 | taskCommits: Seq[FileCommitProtocol.TaskCommitMessage]
78 | ): Unit = {
79 | val desgMgs = taskCommits.map(m =>
80 | new FileCommitProtocol.TaskCommitMessage(
81 | m.obj.asInstanceOf[IcebergTaskCommitMessage].designatePayload
82 | )
83 | )
84 | designate.commitJob(jobContext, desgMgs)
85 | for (
86 | tasmCmtMsg <- taskCommits
87 | ) {
88 | val iceTaskMsg = tasmCmtMsg.obj.asInstanceOf[IcebergTaskCommitMessage]
89 | addedDataFiles.appendAll(iceTaskMsg.dataFiles)
90 | }
91 | }
92 |
93 | override def abortJob(jobContext: JobContext): Unit =
94 | designate.abortJob(jobContext)
95 |
96 | override def setupTask(taskContext: TaskAttemptContext): Unit = {
97 | designate.setupTask(taskContext)
98 | setupTaskFileMap
99 | taskFiles(taskContext.getTaskAttemptID) = ArrayBuffer.empty[TaskPaths]
100 | }
101 |
102 | override def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String)
103 | : String = {
104 | val taskfilePath = designate.newTaskTempFile(taskContext, dir, ext)
105 | val fNm = new Path(taskfilePath).getName
106 | val finalPath = dir.map { d =>
107 | new Path(new Path(path, d), fNm)
108 | }.getOrElse {
109 | new Path(path, fNm)
110 | }
111 |
112 | setupTaskFileMap
113 | taskFiles(taskContext.getTaskAttemptID).append(TaskPaths(taskfilePath, finalPath))
114 | taskfilePath
115 | }
116 |
117 | override def newTaskTempFileAbsPath(taskContext: TaskAttemptContext,
118 | absoluteDir: String, ext: String
119 | ): String = {
120 | val taskfilePath = designate.newTaskTempFileAbsPath(taskContext, absoluteDir, ext)
121 | val fNm = new Path(taskfilePath).getName
122 | val finalPath = new Path(absoluteDir, fNm)
123 | taskFiles(taskContext.getTaskAttemptID).append(TaskPaths(taskfilePath, finalPath))
124 | taskfilePath
125 | }
126 |
127 | override def commitTask(taskContext: TaskAttemptContext
128 | ): FileCommitProtocol.TaskCommitMessage = {
129 | val tPaths = taskFiles(taskContext.getTaskAttemptID)
130 | val dFiles = tPaths.map {
131 | case TaskPaths(taskPath, finalPath) =>
132 | DataFileUtils.buildDataFileFormPath(
133 | taskPath, iceSchema, partitionSchema,
134 | icePartSpec, iceFmt,
135 | taskContext.getConfiguration,
136 | finalPath.toString
137 | )
138 | }
139 |
140 | val dsgMsg = designate.commitTask(taskContext)
141 | new FileCommitProtocol.TaskCommitMessage(
142 | IcebergTaskCommitMessage(dsgMsg.obj, dFiles)
143 | )
144 | }
145 |
146 | override def abortTask(taskContext: TaskAttemptContext): Unit =
147 | designate.abortTask(taskContext)
148 |
149 | override def deleteWithJob(fs: FileSystem, path: Path, recursive: Boolean): Boolean = {
150 | false
151 | }
152 | }
153 |
154 | object IcebergFileCommitProtocol {
155 | val REAL_FILE_COMMIT_PROTOCOL_CLASS = "spark.sql.sources.real.commitProtocolClass"
156 |
157 | case class TaskPaths(taskPath: String, finalPath: Path)
158 |
159 | case class IcebergTaskCommitMessage(
160 | designatePayload: Any,
161 | dataFiles: Seq[DataFile]
162 | )
163 |
164 | }
165 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/InsertIntoIcebergTable.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import com.netflix.iceberg.{DataFile, PendingUpdate, Snapshot}
21 | import org.apache.hadoop.fs.{FileSystem, Path}
22 | import org.apache.spark.internal.io.FileCommitProtocol
23 | import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
24 | import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable}
25 | import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
26 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
27 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
28 | import org.apache.spark.sql.execution.SparkPlan
29 | import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CommandUtils, DataWritingCommand}
30 | import org.apache.spark.sql.execution.datasources._
31 | import org.apache.spark.sql.execution.datasources.iceberg.{PartitioningUtils => IcePartitioningUtils}
32 | import org.apache.spark.sql.iceberg.table.SparkTables
33 | import org.apache.spark.sql.iceberg.utils.{DataFileUtils, ExpressionConversions, IceTable}
34 | import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
35 | import org.apache.spark.sql.util.SchemaUtils
36 | import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
37 |
38 | /**
39 | * A drop-in replacement for [[InsertIntoHadoopFsRelationCommand]] setup by the
40 | * [[org.apache.spark.sql.iceberg.planning.IcebergTableWriteRule]]. By and large
41 | * follows the same execution flow as [[InsertIntoHadoopFsRelationCommand]] with the
42 | * following behavior overrides.
43 | *
44 | * - The write must be on a [[CatalogTable]]. So `catalogTable` parameter is not optional.
45 | * - Since this is a `iceberg managed table` we load the [[IceTable]] metadata for this table.
46 | * - `initialMatchingPartitions` is computed from the [[IceTable]] metadata; see method
47 | * `matchIceDataFiles`
48 | * - since data files must be managed by iceberg we require that ''custom partition locations''
49 | * are not configured for this table.
50 | * - we setup an [[IcebergFileCommitProtocol]] that wraps the underlying [[FileCommitProtocol]].
51 | * The [[IcebergFileCommitProtocol]] mostly defers to the underlying commitProtocol instance;
52 | * in the process it ensures iceberg [[DataFile]] instances are created for new files on
53 | * task commit which are then delivered to the Driver [[IcebergFileCommitProtocol]] instance
54 | * via [[TaskCommitMessage]].
55 | * - The underlying [[FileCommitProtocol]] is setup with `dynamicPartitionOverwrite` mode
56 | * set to false. Since [[IceTable]] metadata is used by scan operations to compute what
57 | * files to scan we don't have to do an all-or-nothing replacement of files in a
58 | * partition that is needed for dynamic partition mode using the [[FileCommitProtocol]].
59 | * - in case of `dynamicPartitionOverwrite` mode we don't clear specified source Partitions, because
60 | * we want the current files to be able execute queries against older snapshots.
61 | * - once the job finishes the Catalog is updated with 'new' and 'deleted' partitions just as it is in
62 | * a regular [[InsertIntoHadoopFsRelationCommand]]
63 | * - then based on the 'initial set' of [[DataFile]] and the set of [[DataFile]] created by tasks
64 | * of this job a new iceberg [[Snapshot]] is created.
65 | * - finally cache invalidation and stats update actions happen just like in a regular
66 | * [[InsertIntoHadoopFsRelationCommand]]
67 | *
68 | * @param outputPath
69 | * @param staticPartitions
70 | * @param ifPartitionNotExists
71 | * @param partitionColumns
72 | * @param bucketSpec
73 | * @param fileFormat
74 | * @param options
75 | * @param query
76 | * @param mode
77 | * @param catalogTable
78 | * @param fileIndex
79 | * @param outputColumnNames
80 | */
81 | case class InsertIntoIcebergTable(outputPath: Path,
82 | staticPartitions: TablePartitionSpec,
83 | ifPartitionNotExists: Boolean,
84 | partitionColumns: Seq[Attribute],
85 | bucketSpec: Option[BucketSpec],
86 | fileFormat: FileFormat,
87 | options: Map[String, String],
88 | query: LogicalPlan,
89 | mode: SaveMode,
90 | catalogTable: CatalogTable,
91 | fileIndex: Option[FileIndex],
92 | outputColumnNames: Seq[String]
93 | ) extends DataWritingCommand {
94 |
95 | val partitionSchema = catalogTable.partitionSchema
96 |
97 | /**
98 | * Construct an iceberg [[com.netflix.iceberg.TableScan]], optionally apply a filter based on
99 | * the `staticPartitions` provide on this invocation. Execute the scan to get a [[DataFile]],
100 | * and build a set of [[TablePartitionSpec]] from the datafiles.
101 | *
102 | * @param icebergTable
103 | * @return
104 | */
105 | private def matchIceDataFiles(icebergTable :IceTable) :
106 | (Seq[DataFile], Set[TablePartitionSpec]) = {
107 | val spkPartExpression : Option[Expression] = IcePartitioningUtils.
108 | expressionFromPartitionSpec(partitionSchema,CaseInsensitiveMap(staticPartitions))
109 | val iceExpr = spkPartExpression.flatMap(ExpressionConversions.convertStrict(_))
110 | var iceScan = icebergTable.newScan()
111 |
112 | if (iceExpr.isDefined) {
113 | iceScan = iceScan.filter(iceExpr.get)
114 | }
115 |
116 | import scala.collection.JavaConversions._
117 | val dfs = iceScan.planFiles().map(_.file).toSeq
118 | val matchingPSepc = (dfs.map(df => DataFileUtils.sparkPartitionMap(df))).toSet
119 | (dfs, matchingPSepc)
120 | }
121 |
122 | private def checkNoCustomPartitionLocations(fs: FileSystem,
123 | qualifiedOutputPath: Path
124 | )(implicit sparkSession: SparkSession) : Unit = {
125 |
126 | val customLocPart =
127 | sparkSession.sessionState.catalog.listPartitions(catalogTable.identifier).find { p =>
128 | val defaultLocation = qualifiedOutputPath.suffix(
129 | "/" + PartitioningUtils.getPathFragment(p.spec, partitionSchema)).toString
130 | val catalogLocation = new Path(p.location).makeQualified(
131 | fs.getUri, fs.getWorkingDirectory).toString
132 | catalogLocation != defaultLocation
133 | }
134 |
135 | if (customLocPart.isDefined) {
136 | throw new AnalysisException(
137 | s"Cannot have custom partition locations for a managed table"
138 | )
139 | }
140 | }
141 |
142 | private def createSnapShot(iceTable : IceTable,
143 | delFiles : Seq[DataFile],
144 | addFiles : Seq[DataFile]
145 | ) : Unit = {
146 |
147 | val hasDelFiles = delFiles.nonEmpty
148 | val hasAddFiles = addFiles.nonEmpty
149 |
150 | import scala.collection.JavaConversions._
151 | var updt : PendingUpdate[Snapshot] = null
152 | (hasDelFiles, hasAddFiles) match {
153 | case (true, true) => {
154 | val r = iceTable.newRewrite()
155 | val ds = delFiles.toSet
156 | val as = addFiles.toSet
157 | updt = r.rewriteFiles(ds, as)
158 | }
159 | case (false, true) => {
160 | var r = iceTable.newAppend()
161 | for(f <- addFiles) {
162 | r =r .appendFile(f)
163 | }
164 | updt = r
165 | }
166 | case (true, false) => {
167 | var r = iceTable.newDelete()
168 | for(f <- delFiles) {
169 | r = r.deleteFile(f)
170 | }
171 | updt = r
172 | }
173 | case (false, false) => ()
174 | }
175 |
176 | if (updt != null) {
177 | updt.commit()
178 | }
179 | }
180 |
181 | private def _run(child: SparkPlan,
182 | iceTable : IceTable
183 | )(implicit sparkSession: SparkSession): Unit = {
184 |
185 | // Most formats don't do well with duplicate columns, so lets not allow that
186 | SchemaUtils.checkColumnNameDuplication(
187 | outputColumnNames,
188 | s"when inserting into $outputPath",
189 | sparkSession.sessionState.conf.caseSensitiveAnalysis)
190 |
191 | val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options)
192 | val fs = outputPath.getFileSystem(hadoopConf)
193 | val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
194 |
195 | val partitionsTrackedByCatalog = sparkSession.sessionState.conf.manageFilesourcePartitions &&
196 | catalogTable.partitionColumnNames.nonEmpty &&
197 | catalogTable.tracksPartitionsInCatalog
198 |
199 | val (initialMatchingDataFiles: Seq[DataFile],
200 | initialMatchingPartitions: Set[TablePartitionSpec]) = matchIceDataFiles(iceTable)
201 |
202 | checkNoCustomPartitionLocations(fs, qualifiedOutputPath)
203 |
204 | val pathExists = fs.exists(qualifiedOutputPath)
205 | val parameters = CaseInsensitiveMap(options)
206 | val partitionOverwriteMode = parameters.get("partitionOverwriteMode")
207 | // scalastyle:off caselocale
208 | .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase))
209 | // scalastyle:on caselocale
210 | .getOrElse(sparkSession.sessionState.conf.partitionOverwriteMode)
211 |
212 | val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC
213 | // This config only makes sense when we are overwriting a partitioned dataset with dynamic
214 | // partition columns.
215 | val dynamicPartitionOverwrite = enableDynamicOverwrite && mode == SaveMode.Overwrite &&
216 | staticPartitions.size < partitionColumns.length
217 |
218 | val committer : IcebergFileCommitProtocol = {
219 | val designate = FileCommitProtocol.instantiate(
220 | sparkSession.sessionState.conf.fileCommitProtocolClass,
221 | jobId = java.util.UUID.randomUUID().toString,
222 | outputPath = outputPath.toString,
223 | dynamicPartitionOverwrite = false
224 | )
225 |
226 | new IcebergFileCommitProtocol(designate,
227 | outputPath.toString,
228 | partitionSchema,
229 | iceTable.spec(),
230 | iceTable.schema(),
231 | utils.iceFormat(fileFormat)
232 | )
233 | }
234 |
235 | val doInsertion = (mode, pathExists) match {
236 | case (SaveMode.ErrorIfExists, true) =>
237 | throw new AnalysisException(s"path $qualifiedOutputPath already exists.")
238 | case (SaveMode.Overwrite, true) =>
239 | if (ifPartitionNotExists && initialMatchingDataFiles.nonEmpty) {
240 | false
241 | } else {
242 | true
243 | }
244 | case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) =>
245 | true
246 | case (SaveMode.Ignore, exists) =>
247 | !exists
248 | case (s, exists) =>
249 | throw new IllegalStateException(s"unsupported save mode $s ($exists)")
250 | }
251 |
252 | if (doInsertion) {
253 |
254 | def refreshUpdatedPartitions(updatedPartitionPaths: Set[String]): Unit = {
255 | if (partitionsTrackedByCatalog) {
256 | val updatedPartitions = updatedPartitionPaths.map(PartitioningUtils.parsePathFragment)
257 | val newPartitions = updatedPartitions -- initialMatchingPartitions
258 | if (newPartitions.nonEmpty) {
259 | AlterTableAddPartitionCommand(
260 | catalogTable.identifier, newPartitions.toSeq.map(p => (p, None)),
261 | ifNotExists = true).run(sparkSession)
262 | }
263 | // For dynamic partition overwrite, we never remove partitions but only update existing
264 | // ones.
265 | if (mode == SaveMode.Overwrite && !dynamicPartitionOverwrite) {
266 | val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions
267 | if (deletedPartitions.nonEmpty) {
268 | AlterTableDropPartitionCommand(
269 | catalogTable.identifier, deletedPartitions.toSeq,
270 | ifExists = true, purge = false,
271 | retainData = true /* already deleted */).run(sparkSession)
272 | }
273 | }
274 | }
275 | }
276 |
277 | val updatedPartitionPaths =
278 | FileFormatWriter.write(
279 | sparkSession = sparkSession,
280 | plan = child,
281 | fileFormat = fileFormat,
282 | committer = committer,
283 | outputSpec = FileFormatWriter.OutputSpec(
284 | qualifiedOutputPath.toString, Map.empty, outputColumns),
285 | hadoopConf = hadoopConf,
286 | partitionColumns = partitionColumns,
287 | bucketSpec = bucketSpec,
288 | statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)),
289 | options = options)
290 |
291 | createSnapShot(iceTable,
292 | if (mode == SaveMode.Overwrite) initialMatchingDataFiles else Seq.empty,
293 | committer.addedDataFiles
294 | )
295 |
296 | // update metastore partition metadata
297 | if (updatedPartitionPaths.isEmpty && staticPartitions.nonEmpty
298 | && partitionColumns.length == staticPartitions.size) {
299 | // Avoid empty static partition can't loaded to datasource table.
300 | val staticPathFragment =
301 | PartitioningUtils.getPathFragment(staticPartitions, partitionColumns)
302 | refreshUpdatedPartitions(Set(staticPathFragment))
303 | } else {
304 | refreshUpdatedPartitions(updatedPartitionPaths)
305 | }
306 |
307 | // refresh cached files in FileIndex
308 | fileIndex.foreach(_.refresh())
309 | // refresh data cache if table is cached
310 | sparkSession.catalog.refreshByPath(outputPath.toString)
311 |
312 | CommandUtils.updateTableStats(sparkSession, catalogTable)
313 |
314 | } else {
315 | logInfo("Skipping insertion into a relation that already exists.")
316 | }
317 |
318 | }
319 |
320 | override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = {
321 |
322 | implicit val ss = sparkSession
323 | val iceTables = new SparkTables(ss, catalogTable)
324 | val tbId = utils.qualifiedTableIdentifier(catalogTable.identifier, ss)
325 | val iceTable = iceTables.load(tbId.database.getOrElse(null), tbId.table)
326 |
327 | _run(child, iceTable)
328 | Seq.empty[Row]
329 | }
330 | }
331 |
332 | object InsertIntoIcebergTable {
333 | def apply(ihr: InsertIntoHadoopFsRelationCommand,
334 | catalogTable: CatalogTable
335 | ) : InsertIntoIcebergTable = {
336 | InsertIntoIcebergTable(
337 | ihr.outputPath,
338 | ihr.staticPartitions,
339 | ihr.ifPartitionNotExists,
340 | ihr.partitionColumns,
341 | ihr.bucketSpec,
342 | ihr.fileFormat,
343 | ihr.options,
344 | ihr.query,
345 | ihr.mode,
346 | catalogTable,
347 | ihr.fileIndex,
348 | ihr.outputColumnNames
349 | )
350 | }
351 | }
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/parsing/AbstractSparkSQLParser.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.parsing
19 |
20 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
21 |
22 | import scala.util.parsing.combinator.PackratParsers
23 | import scala.util.parsing.combinator.lexical.StdLexical
24 | import scala.util.parsing.combinator.syntactical.StandardTokenParsers
25 | import scala.util.parsing.input.CharArrayReader.EofCh
26 | import scala.language.implicitConversions
27 |
28 | private[parsing] abstract class AbstractSparkSQLParser
29 | extends StandardTokenParsers with PackratParsers {
30 |
31 | def parse(input: String): LogicalPlan = synchronized {
32 | // Initialize the Keywords.
33 | initLexical
34 | phrase(start)(new lexical.Scanner(input)) match {
35 | case Success(plan, _) => plan
36 | case failureOrError => sys.error(failureOrError.toString)
37 | }
38 | }
39 | /* One time initialization of lexical.This avoid reinitialization of lexical in parse method */
40 | protected lazy val initLexical: Unit = lexical.initialize(reservedWords)
41 |
42 | protected case class Keyword(str: String) {
43 | def normalize: String = lexical.normalizeKeyword(str)
44 | def parser: Parser[String] = normalize
45 | }
46 |
47 | protected implicit def asParser(k: Keyword): Parser[String] = k.parser
48 |
49 | // By default, use Reflection to find the reserved words defined in the sub class.
50 | // NOTICE, Since the Keyword properties defined by sub class, we couldn't call this
51 | // method during the parent class instantiation, because the sub class instance
52 | // isn't created yet.
53 | protected lazy val reservedWords: Seq[String] =
54 | this
55 | .getClass
56 | .getMethods
57 | .filter(_.getReturnType == classOf[Keyword])
58 | .map(_.invoke(this).asInstanceOf[Keyword].normalize)
59 |
60 | // Set the keywords as empty by default, will change that later.
61 | override val lexical = new SqlLexical
62 |
63 | protected def start: Parser[LogicalPlan]
64 |
65 | // Returns the whole input string
66 | protected lazy val wholeInput: Parser[String] = new Parser[String] {
67 | def apply(in: Input): ParseResult[String] =
68 | Success(in.source.toString, in.drop(in.source.length()))
69 | }
70 |
71 | // Returns the rest of the input string that are not parsed yet
72 | protected lazy val restInput: Parser[String] = new Parser[String] {
73 | def apply(in: Input): ParseResult[String] =
74 | Success(
75 | in.source.subSequence(in.offset, in.source.length()).toString,
76 | in.drop(in.source.length()))
77 | }
78 | }
79 |
80 | class SqlLexical extends StdLexical {
81 | case class DecimalLit(chars: String) extends Token {
82 | override def toString: String = chars
83 | }
84 |
85 | /* This is a work around to support the lazy setting */
86 | def initialize(keywords: Seq[String]): Unit = {
87 | reserved.clear()
88 | reserved ++= keywords
89 | }
90 |
91 | /* Normal the keyword string */
92 | def normalizeKeyword(str: String): String = str.toLowerCase
93 |
94 | delimiters += (
95 | "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
96 | ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~", "<=>"
97 | )
98 |
99 | protected override def processIdent(name: String) = {
100 | val token = normalizeKeyword(name)
101 | if (reserved contains token) Keyword(token) else Identifier(name)
102 | }
103 |
104 | override lazy val token: Parser[Token] =
105 | ( rep1(digit) ~ scientificNotation ^^ { case i ~ s => DecimalLit(i.mkString + s) }
106 | | '.' ~> (rep1(digit) ~ scientificNotation) ^^
107 | { case i ~ s => DecimalLit("0." + i.mkString + s) }
108 | | rep1(digit) ~ ('.' ~> digit.*) ~ scientificNotation ^^
109 | { case i1 ~ i2 ~ s => DecimalLit(i1.mkString + "." + i2.mkString + s) }
110 | | digit.* ~ identChar ~ (identChar | digit).* ^^
111 | { case first ~ middle ~ rest => processIdent((first ++ (middle :: rest)).mkString) }
112 | | rep1(digit) ~ ('.' ~> digit.*).? ^^ {
113 | case i ~ None => NumericLit(i.mkString)
114 | case i ~ Some(d) => DecimalLit(i.mkString + "." + d.mkString)
115 | }
116 | | '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^
117 | { case chars => StringLit(chars mkString "") }
118 | | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^
119 | { case chars => StringLit(chars mkString "") }
120 | | '`' ~> chrExcept('`', '\n', EofCh).* <~ '`' ^^
121 | { case chars => Identifier(chars mkString "") }
122 | | EofCh ^^^ EOF
123 | | '\'' ~> failure("unclosed string literal")
124 | | '"' ~> failure("unclosed string literal")
125 | | delim
126 | | failure("illegal character")
127 | )
128 |
129 | override def identChar: Parser[Elem] = letter | elem('_')
130 |
131 | private lazy val scientificNotation: Parser[String] =
132 | (elem('e') | elem('E')) ~> (elem('+') | elem('-')).? ~ rep1(digit) ^^ {
133 | case s ~ rest => "e" + s.mkString + rest.mkString
134 | }
135 |
136 | override def whitespace: Parser[Any] =
137 | ( whitespaceChar
138 | | '/' ~ '*' ~ comment
139 | | '/' ~ '/' ~ chrExcept(EofCh, '\n').*
140 | | '#' ~ chrExcept(EofCh, '\n').*
141 | | '-' ~ '-' ~ chrExcept(EofCh, '\n').*
142 | | '/' ~ '*' ~ failure("unclosed comment")
143 | ).*
144 | }
145 |
146 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/parsing/IceParser.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.parsing
19 |
20 | import org.apache.spark.sql.SparkSession
21 | import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
22 | import org.apache.spark.sql.catalyst.expressions.Expression
23 | import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
24 | import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
25 | import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
26 | import org.apache.spark.sql.iceberg.utils
27 | import org.apache.spark.sql.iceberg.utils.TableUtils
28 | import org.apache.spark.sql.iceberg.utils.TableUtils.SNAPSHOTSVIEW_SUFFIX
29 | import org.apache.spark.sql.types.{DataType, StructType}
30 |
31 | import scala.util.Try
32 |
33 | class SparkIceParser(baseParser : ParserInterface) extends ParserInterface {
34 |
35 | val iceParser = new IceParser(baseParser)
36 |
37 | override def parsePlan(sqlText: String): LogicalPlan = {
38 |
39 | val splParsedPlan = Try {
40 | iceParser.parse2(sqlText)
41 | }.getOrElse(iceParser.Failure("Not valid ice extension", null))
42 |
43 | if (splParsedPlan.successful ) {
44 | splParsedPlan.get
45 | } else {
46 | try {
47 | baseParser.parsePlan(sqlText)
48 | } catch {
49 | case pe : ParseException => {
50 | val splFailureDetails = splParsedPlan.asInstanceOf[IceParser#NoSuccess].msg
51 | throw new ParseException(pe.command,
52 | pe.message + s"\nIce parse attempt message: $splFailureDetails",
53 | pe.start,
54 | pe.stop
55 | )
56 | }
57 | }
58 | }
59 | }
60 |
61 | def parseExpression(sqlText: String): Expression =
62 | baseParser.parseExpression(sqlText)
63 |
64 | def parseTableIdentifier(sqlText: String): TableIdentifier =
65 | baseParser.parseTableIdentifier(sqlText)
66 |
67 | override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier =
68 | baseParser.parseFunctionIdentifier(sqlText)
69 |
70 | override def parseTableSchema(sqlText: String): StructType =
71 | baseParser.parseTableSchema(sqlText)
72 |
73 | override def parseDataType(sqlText: String): DataType =
74 | baseParser.parseDataType(sqlText)
75 | }
76 |
77 |
78 | class IceParser(val baseParser : ParserInterface) extends AbstractSparkSQLParser {
79 |
80 | def sparkSession = SparkSession.getActiveSession.get
81 |
82 | def parse2(input: String): ParseResult[LogicalPlan] = synchronized {
83 | // Initialize the Keywords.
84 | initLexical
85 | phrase(start)(new lexical.Scanner(input))
86 | }
87 |
88 | protected override lazy val start: Parser[LogicalPlan] =
89 | selectSnapshotViewStar | selectSnapshotViewProject | asOfSelect
90 |
91 | private lazy val selectSnapshotViewStar : Parser[LogicalPlan] =
92 | (SELECT ~ STAR ~ FROM) ~> qualifiedId ^^ {
93 | case tblNm if tblNm.endsWith(TableUtils.SNAPSHOTSVIEW_SUFFIX) =>
94 | TableUtils.snapShotsLocalRelation(
95 | tblNm.substring(0, tblNm.length - SNAPSHOTSVIEW_SUFFIX.length)
96 | )(sparkSession)
97 | }
98 |
99 | private lazy val selectSnapshotViewProject : Parser[LogicalPlan] =
100 | (SELECT ~> qualifiedCols) ~ (FROM ~> qualifiedId) ^^ {
101 | case qCols ~ tblNm if tblNm.endsWith(TableUtils.SNAPSHOTSVIEW_SUFFIX) =>
102 | val sRel = TableUtils.snapShotsLocalRelation(
103 | tblNm.substring(0, tblNm.length - SNAPSHOTSVIEW_SUFFIX.length)
104 | )(sparkSession)
105 | Project(qCols.map(UnresolvedAttribute(_)), sRel)
106 | }
107 |
108 | private lazy val asOfSelect : Parser[LogicalPlan] =
109 | (AS ~ OF) ~> stringLit ~ restInput ^^ {
110 | case asOfTime ~ query => {
111 | TableUtils.setThreadSnapShotMillis(utils.convertToEpoch(asOfTime))
112 | baseParser.parsePlan(query)
113 | }
114 | }
115 |
116 | private lazy val qualifiedCols : Parser[Seq[String]] =
117 | repsep(qualifiedCol, ",")
118 |
119 | private lazy val qualifiedCol : Parser[String] =
120 | (ident ~ opt("." ~> ident) ~ opt("." ~> ident)) ^^ {
121 | case c ~ None ~ None => c
122 | case t ~ Some(c) ~ None => s"$t.$c"
123 | case d ~ Some(t) ~ Some(c) => s"$d.$t.$c"
124 | }
125 |
126 | private lazy val qualifiedId : Parser[String] =
127 | (ident ~ ("." ~> ident).?) ^^ {
128 | case ~(n, None) => n
129 | case ~(q, Some(n)) => s"$q.$n"
130 | }
131 |
132 | protected val SELECT = Keyword("SELECT")
133 | protected val STAR = Keyword("*")
134 | protected val FROM = Keyword("FROM")
135 | protected val AS = Keyword("AS")
136 | protected val OF = Keyword("OF")
137 |
138 | }
139 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/planning/CreateTableRules.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.planning
19 |
20 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
21 | import org.apache.spark.sql.catalyst.rules.Rule
22 | import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, CreateDataSourceTableCommand}
23 | import org.apache.spark.sql.iceberg.Config.provideManagement
24 | import org.apache.spark.sql.iceberg.{CreateIcebergTable, CreateIcebergTableAsSelect}
25 | import org.apache.spark.sql.{SaveMode, SparkSession}
26 |
27 | import scala.collection.mutable.ArrayBuffer
28 |
29 | object CreateTableCheck extends Rule[LogicalPlan] {
30 |
31 | import IcebergTableValidationChecks.validate
32 |
33 | private def checkPlan(plan : LogicalPlan) : Unit = plan match {
34 | case CreateDataSourceTableCommand(table, ignoreIfExists) if provideManagement(table) => {
35 | val errs = ArrayBuffer[String]()
36 | if (ignoreIfExists) {
37 | errs + "managed table supported only when 'ignoreIfExists' option is false"
38 | }
39 | validate("create", table, errs)
40 |
41 | }
42 | case CreateDataSourceTableAsSelectCommand(table, mode, _, _) if provideManagement(table) => {
43 | val errs = ArrayBuffer[String]()
44 | if (mode != SaveMode.ErrorIfExists) {
45 | errs + "managed table supported only for SaveMode = 'ErrorIfExists'"
46 | }
47 | validate("create as", table, errs)
48 | }
49 | case _ => ()
50 | }
51 |
52 | override def apply(plan: LogicalPlan): LogicalPlan = {
53 | checkPlan(plan)
54 | plan
55 | }
56 |
57 | }
58 |
59 | case class CreateIcebergTableRule(ss: SparkSession) extends Rule[LogicalPlan] {
60 |
61 | override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
62 | case ct@CreateDataSourceTableCommand(table, ignoreIfExists) if provideManagement(table) => {
63 | CreateIcebergTable(ct)
64 | }
65 | case ct@CreateDataSourceTableAsSelectCommand(table, mode, _, _) if provideManagement(table) => {
66 | CreateIcebergTableAsSelect(ct)
67 | }
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/planning/IcebergScanStrategy.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.planning
19 |
20 | import org.apache.spark.internal.Logging
21 | import org.apache.spark.sql.{SparkSession, Strategy}
22 | import org.apache.spark.sql.catalyst.planning.PhysicalOperation
23 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
24 | import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan}
25 | import org.apache.spark.sql.execution.datasources.{FileSourceStrategy, HadoopFsRelation, LogicalRelation}
26 | import org.apache.spark.sql.iceberg.Config.isManaged
27 | import org.apache.spark.sql.iceberg.IceTableScanExec
28 |
29 | case class IcebergScanStrategy(ss: SparkSession) extends Strategy with Logging {
30 |
31 | def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
32 | case PhysicalOperation(projects, filters,
33 | l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _))
34 | if table.isDefined && isManaged(table.get) => {
35 | FileSourceStrategy(plan).map { plan =>
36 | plan.transformUp {
37 | case fsScan: FileSourceScanExec => {
38 | IceTableScanExec(fsScan, table.get)
39 | }
40 | }
41 | }
42 | }
43 | case _ => Nil
44 | }
45 |
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/planning/IcebergTableValidationChecks.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.planning
19 |
20 | import org.apache.spark.sql.AnalysisException
21 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
22 | import org.apache.spark.sql.iceberg.Config.columnDependencies
23 | import org.apache.spark.sql.iceberg.utils.{Transforms, TypeConversions, currentSparkSession}
24 |
25 | import scala.collection.mutable.ArrayBuffer
26 |
27 | object IcebergTableValidationChecks {
28 |
29 | private def validationRules(table : CatalogTable,
30 | errs : ArrayBuffer[String]) : Unit = {
31 | val typMapping = TypeConversions.convertStructType(table.schema)
32 |
33 | if (table.bucketSpec.isDefined) {
34 | errs += "not support for bucketed table"
35 | }
36 |
37 | if (table.partitionColumnNames.isEmpty) {
38 | errs += "no support for non partitioned table"
39 | }
40 |
41 | for (partMappingStr <- columnDependencies(table)) {
42 | Transforms.fromOption(partMappingStr, typMapping)(currentSparkSession) match {
43 | case Left(tErrs) => errs ++= tErrs
44 | case _ => ()
45 | }
46 | }
47 | }
48 |
49 | def validate(action : String,
50 | table : CatalogTable,
51 | errs : ArrayBuffer[String] = ArrayBuffer[String]()) : Unit = {
52 | validationRules(table, errs)
53 |
54 | if (errs.nonEmpty) {
55 | throw new AnalysisException(
56 | s"""Cannot ${action} ${table.qualifiedName} as a managed table:
57 | |${errs.mkString("\n\t", "\n\t", "\n")})""".stripMargin
58 | )
59 | }
60 | }
61 |
62 |
63 | }
64 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/planning/IcebergTableWriteRule.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.planning
19 |
20 | import org.apache.spark.sql.SparkSession
21 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
22 | import org.apache.spark.sql.catalyst.rules.Rule
23 | import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
24 | import org.apache.spark.sql.iceberg.{Config, InsertIntoIcebergTable}
25 |
26 | case class IcebergTableWriteRule(ss: SparkSession) extends Rule[LogicalPlan] {
27 |
28 | import Config._
29 |
30 | override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
31 | case ihr: InsertIntoHadoopFsRelationCommand
32 | if ihr.catalogTable.isDefined && isManaged(ihr.catalogTable.get)
33 | => {
34 | val catalogTable = {
35 | val ct = ihr.catalogTable.get
36 | IcebergTableValidationChecks.validate("insert into", ct)
37 | ct
38 | }
39 | InsertIntoIcebergTable(ihr, catalogTable)
40 | }
41 | }
42 | }
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/planning/SparkSessionExtensions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.planning
19 |
20 | import org.apache.spark.sql.iceberg.parsing.SparkIceParser
21 | import org.apache.spark.sql.{SparkSessionExtensions => ApacheSparkSessionExtensions}
22 |
23 | /**
24 | * Include this class in ''spark.sql.extensions'' for these extensions to take effect.
25 | */
26 | class SparkSessionExtensions extends Function1[ApacheSparkSessionExtensions, Unit] {
27 |
28 | override def apply(spkExtensions: ApacheSparkSessionExtensions): Unit = {
29 |
30 | spkExtensions.injectPostHocResolutionRule {spkSession =>
31 | CreateTableCheck
32 | }
33 |
34 | spkExtensions.injectOptimizerRule(CreateIcebergTableRule)
35 | spkExtensions.injectOptimizerRule(IcebergTableWriteRule)
36 | spkExtensions.injectPlannerStrategy(IcebergScanStrategy)
37 | spkExtensions.injectParser((_, pI) => new SparkIceParser(pI))
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/table/SparkTableOperations.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.table
19 |
20 | import java.lang.String.format
21 | import java.util.{Locale, Objects}
22 |
23 | import com.netflix.iceberg.exceptions.CommitFailedException
24 | import com.netflix.iceberg.{BaseMetastoreTableOperations, TableMetadata}
25 | import org.apache.spark.internal.Logging
26 | import org.apache.spark.sql.SparkSession
27 | import org.apache.spark.sql.catalyst.QualifiedTableName
28 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
29 | import org.apache.spark.sql.execution.command.AlterTableSetPropertiesCommand
30 |
31 | /**
32 | * Very similar to `HiveTableOperations` from `iceberg-hive`.
33 | * The `refresh` step checks that the `catalogTable` is setup as
34 | * a Iceberg Table.
35 | * The ''metadata_location'' property points to the latest iceberg metadata file.
36 | *
37 | * As part of a metadata commit an [[AlterTableSetPropertiesCommand]] is issued
38 | * that updates the ''metadata_location'' property.
39 | *
40 | * @param sparkSession
41 | * @param catalogTable
42 | */
43 | class SparkTableOperations private[iceberg] (val sparkSession: SparkSession,
44 | val catalogTable: CatalogTable)
45 | extends BaseMetastoreTableOperations(sparkSession.sparkContext.hadoopConfiguration) with Logging {
46 |
47 | val tableIdentifier = catalogTable.identifier
48 |
49 | val qualTblNm = {
50 | QualifiedTableName(
51 | catalogTable.identifier.database.getOrElse(
52 | sparkSession.sessionState.catalog.getCurrentDatabase
53 | ),
54 | catalogTable.identifier.table
55 | )
56 | }
57 |
58 | private var metadataLocation : Option[String] = None
59 |
60 | private def setMetadataLocation(l : String) : Unit = {
61 | metadataLocation = Some(l)
62 | }
63 |
64 | override def refresh(): TableMetadata = {
65 |
66 | import BaseMetastoreTableOperations.{ICEBERG_TABLE_TYPE_VALUE, METADATA_LOCATION_PROP, TABLE_TYPE_PROP}
67 |
68 | if (!metadataLocation.isDefined) {
69 | val catTable = catalogTable
70 | val tableType = catTable.properties.get(TABLE_TYPE_PROP)
71 |
72 | if (!tableType.isDefined || !tableType.get.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE)) {
73 | throw new IllegalArgumentException(
74 | format("Invalid tableName, not Iceberg: %s", tableIdentifier.unquotedString)
75 | )
76 | }
77 | metadataLocation = catTable.properties.get(METADATA_LOCATION_PROP)
78 | }
79 |
80 | if (!metadataLocation.isDefined) {
81 | throw new IllegalArgumentException(
82 | format(format("%s is missing %s property",
83 | tableIdentifier.unquotedString, METADATA_LOCATION_PROP)
84 | )
85 | )
86 | }
87 | refreshFromMetadataLocation(metadataLocation.get)
88 | current
89 | }
90 |
91 | override def commit(base: TableMetadata, metadata: TableMetadata): Unit = {
92 |
93 | import BaseMetastoreTableOperations.{ICEBERG_TABLE_TYPE_VALUE, METADATA_LOCATION_PROP, PREVIOUS_METADATA_LOCATION_PROP, TABLE_TYPE_PROP}
94 |
95 | // if the metadata is already out of date, reject it
96 | if (base != current) {
97 | throw new CommitFailedException(format("stale table metadata for %s",
98 | tableIdentifier.unquotedString))
99 | }
100 |
101 | // if the metadata is not changed, return early
102 | if (base == metadata) {
103 | logInfo("Nothing to commit.")
104 | return
105 | }
106 |
107 | val newMetadataLocation: String = writeNewMetadata(metadata, currentVersion + 1)
108 |
109 | var threw: Boolean = true
110 | try {
111 | val catTable = catalogTable
112 |
113 | val metadataLocation = catTable.properties.getOrElse(METADATA_LOCATION_PROP, null)
114 |
115 | if (!Objects.equals(currentMetadataLocation, metadataLocation)) {
116 | throw new CommitFailedException(
117 | format("metadataLocation = %s is not same as table metadataLocation %s for %s",
118 | currentMetadataLocation, metadataLocation, tableIdentifier.unquotedString))
119 | }
120 |
121 | val currentTimeMillis = System.currentTimeMillis
122 | var properties: List[(String, String)] = List(
123 | (TABLE_TYPE_PROP, ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH)),
124 | (METADATA_LOCATION_PROP, newMetadataLocation)
125 | )
126 |
127 | if (currentMetadataLocation != null && !currentMetadataLocation.isEmpty) {
128 | properties = properties :+ (PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation)
129 | }
130 |
131 | val alterTableCmd = new AlterTableSetPropertiesCommand(
132 | tableIdentifier,
133 | properties.toMap,
134 | false)
135 | alterTableCmd.run(sparkSession)
136 | log.info(s"Updated table $tableIdentifier with new metadata location: ${newMetadataLocation}")
137 |
138 | sparkSession.sessionState.catalog.invalidateCachedTable(qualTblNm)
139 | setMetadataLocation(newMetadataLocation)
140 |
141 | threw = false
142 | } finally {
143 | if (threw) { // if anything went wrong, clean up the uncommitted metadata file
144 | io.deleteFile(newMetadataLocation)
145 | }
146 | }
147 | requestRefresh()
148 |
149 | }
150 | }
151 |
152 | class SparkTableOperationsForCreate private[iceberg] (sparkSession: SparkSession,
153 | catalogTable: CatalogTable)
154 | extends SparkTableOperations(sparkSession, catalogTable) {
155 |
156 | override def refresh(): TableMetadata = {
157 | refreshFromMetadataLocation(null)
158 | current()
159 | }
160 | }
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/table/SparkTables.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.table
19 |
20 | import java.util
21 |
22 | import com.netflix.iceberg.TableMetadata.newTableMetadata
23 | import com.netflix.iceberg.{BaseMetastoreTableOperations, BaseMetastoreTables, BaseTable, PartitionSpec, Schema, Table}
24 | import org.apache.hadoop.conf.Configuration
25 | import org.apache.spark.sql.SparkSession
26 | import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
27 | import org.apache.spark.sql.iceberg.utils._
28 |
29 | /**
30 | * Very similar to `HiveTables` from `iceberg-hive`.
31 | * Setups a [[SparkTableOperations]], `tableIdentifier` is interpreted using the session parser.
32 | *
33 | * Uses of this should pass the Spark table's [[CatalogTable]] on which Table operations will be
34 | * applied and the current [[SparkSession]].
35 | *
36 | * @param sparkSession
37 | * @param catalogTable
38 | */
39 | private[iceberg] class SparkTables(val sparkSession: SparkSession,
40 | val catalogTable: CatalogTable)
41 | extends BaseMetastoreTables(sparkSession.sparkContext.hadoopConfiguration) {
42 |
43 | override def newTableOps(configuration: Configuration,
44 | database: String,
45 | table: String
46 | ): BaseMetastoreTableOperations = {
47 | new SparkTableOperations(sparkSession, catalogTable)
48 | }
49 |
50 | override def create(schema: Schema,
51 | partitionSpec: PartitionSpec,
52 | map: util.Map[String, String],
53 | tableIdentifier: String
54 | ): Table = {
55 | val tId = parseIdentifier(tableIdentifier, sparkSession)
56 | create(schema, partitionSpec, map, tId.database.getOrElse(null), tId.table)
57 | }
58 |
59 | override def create(schema: Schema, spec: PartitionSpec,
60 | properties: util.Map[String, String],
61 | database: String,
62 | table: String
63 | ): Table = {
64 | val conf = sparkSession.sparkContext.hadoopConfiguration
65 | val ops = newTableOps(conf, database, table)
66 | val location = catalogTable.storage.locationUri.map(CatalogUtils.URIToString(_)).get
67 | val metadata = newTableMetadata(ops, schema, spec, location, properties)
68 | ops.commit(null, metadata)
69 | new BaseTable(ops, database + "." + table)
70 | }
71 |
72 | override def load(tableIdentifier: String): Table = {
73 | val tId = parseIdentifier(tableIdentifier, sparkSession)
74 | val qtid = qualifiedTableIdentifier(tId, sparkSession)
75 | load(qtid.database.getOrElse(null), qtid.table)
76 | }
77 | }
78 |
79 | private[iceberg] class SparkTablesForCreate(sparkSession: SparkSession,
80 | catalogTable: CatalogTable)
81 | extends SparkTables(sparkSession, catalogTable) {
82 |
83 | override def newTableOps(configuration: Configuration,
84 | database: String,
85 | table: String
86 | ): BaseMetastoreTableOperations = {
87 | new SparkTableOperationsForCreate(sparkSession, catalogTable)
88 | }
89 |
90 | }
91 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/ColumnDependenciesExprVisitor.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.expressions.{BoundPredicate, Expression, Expressions, UnboundPredicate}
21 | import com.netflix.iceberg.expressions.ExpressionVisitors.ExpressionVisitor
22 | import com.netflix.iceberg.transforms.{Transform => IceTransform}
23 | import com.netflix.iceberg.types.Types
24 | import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
25 |
26 | class ColumnDependenciesExprVisitor(iceType : Types.StructType,
27 | columnDep : Map[String, Map[String, IceTransform[_,_]]]
28 | ) extends ExpressionVisitor[Expression] {
29 |
30 | val caseInsensitiveMap : Map[String, Map[String, IceTransform[_,_]]]
31 | = CaseInsensitiveMap(columnDep)
32 |
33 | override def alwaysTrue(): Expression = Expressions.alwaysTrue
34 |
35 | override def alwaysFalse(): Expression = Expressions.alwaysFalse()
36 |
37 | override def not(result: Expression): Expression = Expressions.not(result)
38 |
39 | override def and(leftResult: Expression, rightResult: Expression): Expression =
40 | Expressions.and(leftResult, rightResult)
41 |
42 | override def or(leftResult: Expression, rightResult: Expression): Expression =
43 | Expressions.or(leftResult, rightResult)
44 |
45 | override def predicate[T](pred: BoundPredicate[T]): Expression = pred
46 |
47 | override def predicate[T](pred: UnboundPredicate[T]): Expression = {
48 |
49 | val boundExpr = pred.bind(iceType.asStructType(), false)
50 | val srcCol = pred.ref().name
51 |
52 | if (boundExpr.isInstanceOf[BoundPredicate[_]]) {
53 | val bndPred = boundExpr.asInstanceOf[BoundPredicate[T]]
54 | val transformPreds : Seq[Expression] = (for (
55 | transformsMap <- caseInsensitiveMap.get(srcCol).toSeq;
56 | (destCol, iceTrans) <- transformsMap.iterator
57 | ) yield {
58 | iceTrans.asInstanceOf[IceTransform[T,_]].project(destCol, bndPred)
59 | }).filter(_ != null)
60 |
61 | transformPreds.foldLeft(pred.asInstanceOf[Expression])(Expressions.and(_,_))
62 |
63 | } else {
64 | pred
65 | }
66 | }
67 |
68 | }
69 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/DataFileUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.hadoop.HadoopInputFile
21 | import com.netflix.iceberg.{DataFile, DataFiles, Metrics, PartitionSpec}
22 | import org.apache.hadoop.conf.Configuration
23 | import org.apache.hadoop.fs.Path
24 | import org.apache.spark.sql.execution.datasources.iceberg.PartitioningUtils
25 | import org.apache.spark.sql.types.StructType
26 |
27 | /**
28 | * Utilities to construct and extract information from [[DataFile]]
29 | */
30 | object DataFileUtils {
31 |
32 | def buildDataFileFormPath(pathString : String,
33 | iceSchema: IceSchema,
34 | partSchema: StructType,
35 | partSpec : PartitionSpec,
36 | iceFmt : IceFormat,
37 | conf : Configuration,
38 | finalPath : String
39 | ) : DataFile = {
40 |
41 | val path = new Path(pathString)
42 | var bldr = DataFiles.builder(partSpec)
43 | val inFile = HadoopInputFile.fromPath(path, conf)
44 | bldr.withInputFile(inFile)
45 |
46 | val metrics : Option[Metrics] = iceMetrics(inFile, iceSchema, iceFmt)
47 | val pSpec = icePartStruct(pathString, partSchema)
48 |
49 | bldr.withPartition(pSpec)
50 | bldr.withFormat(iceFmt)
51 | if (metrics.isDefined) {
52 | bldr.withMetrics(metrics.get)
53 | }
54 | bldr.withPath(finalPath)
55 | bldr.build()
56 | }
57 |
58 | def sparkPartitionMap(df : DataFile) : Map[String, String] =
59 | PartitioningUtils.partitionValuesFromFullPath(
60 | PartitioningUtils.removeFileName(df.path().toString)
61 | )
62 |
63 |
64 | }
65 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/DelegationUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import scala.reflect.ClassTag
21 |
22 | object DelegationUtils {
23 |
24 | /*
25 | * FT : field type
26 | * IT : instance type
27 | * ST : super type
28 | */
29 |
30 |
31 | class DelegatedField[FT, ST : ClassTag, IT <: ST](val instance : IT, fieldName : String) {
32 | val field = {
33 | val clz = implicitly[ClassTag[ST]].runtimeClass
34 | val f = clz.getDeclaredField(fieldName)
35 | f.setAccessible(true)
36 | f
37 | }
38 |
39 | def value : FT = field.get(instance).asInstanceOf[FT]
40 |
41 | def `value_=`(value : FT) : Unit = {
42 | field.set(instance, value)
43 | }
44 |
45 | }
46 |
47 | abstract class DelegatedMethod[ST : ClassTag, IT <: ST, RT]
48 | (instance : IT, methodName : String) {
49 | val method = {
50 | val clz = implicitly[ClassTag[ST]].runtimeClass
51 | val m = clz.getDeclaredMethod(methodName)
52 | m.setAccessible(true)
53 | m
54 | }
55 | }
56 |
57 | class DelegatedMethod0[ST : ClassTag, IT <: ST, RT](val instance : IT, methodName : String) {
58 |
59 | val method = {
60 | val clz = implicitly[ClassTag[ST]].runtimeClass
61 | val m = clz.getDeclaredMethod(methodName)
62 | m.setAccessible(true)
63 | m
64 | }
65 |
66 | def apply : RT = {
67 | method.invoke(instance).asInstanceOf[RT]
68 | }
69 | }
70 |
71 | class DelegatedMethod1[ST : ClassTag, IT <: ST, RT, AT1 : ClassTag]
72 | (val instance : IT, methodName : String) {
73 |
74 | val method = {
75 | val clz = implicitly[ClassTag[ST]].runtimeClass
76 | val m = clz.getDeclaredMethod(methodName, implicitly[ClassTag[AT1]].runtimeClass)
77 | m.setAccessible(true)
78 | m
79 | }
80 |
81 | def apply(arg1 : AT1) : RT = {
82 | method.invoke(instance, arg1.asInstanceOf[AnyRef]).asInstanceOf[RT]
83 | }
84 | }
85 |
86 | class DelegatedMethod2[ST : ClassTag, IT <: ST, RT, AT1 : ClassTag, AT2 : ClassTag]
87 | (val instance : IT, methodName : String) {
88 |
89 | val method = {
90 | val clz = implicitly[ClassTag[ST]].runtimeClass
91 | val m = clz.getDeclaredMethod(methodName,
92 | implicitly[ClassTag[AT1]].runtimeClass,
93 | implicitly[ClassTag[AT2]].runtimeClass)
94 | m.setAccessible(true)
95 | m
96 | }
97 |
98 | def apply(arg1 : AT1, arg2 : AT2) : RT = {
99 | method.invoke(instance, arg1.asInstanceOf[AnyRef], arg2.asInstanceOf[AnyRef]).asInstanceOf[RT]
100 | }
101 | }
102 |
103 | // scalastyle:off line.size.limit
104 | class DelegatedMethod3[ST : ClassTag, IT <: ST, RT, AT1 : ClassTag, AT2 : ClassTag, AT3 : ClassTag]
105 | (val instance : IT, methodName : String) {
106 |
107 | val method = {
108 | val clz = implicitly[ClassTag[ST]].runtimeClass
109 | val m = clz.getDeclaredMethod(methodName,
110 | implicitly[ClassTag[AT1]].runtimeClass,
111 | implicitly[ClassTag[AT2]].runtimeClass,
112 | implicitly[ClassTag[AT3]].runtimeClass)
113 | m.setAccessible(true)
114 | m
115 | }
116 |
117 | def apply(arg1 : AT1, arg2 : AT2, arg3 : AT3) : RT = {
118 | method.invoke(instance,
119 | arg1.asInstanceOf[AnyRef],
120 | arg2.asInstanceOf[AnyRef],
121 | arg3.asInstanceOf[AnyRef]).asInstanceOf[RT]
122 | }
123 | }
124 | // scalastyle:on line.size.limit
125 |
126 | }
127 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/ExpressionConversions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.expressions.Expression.Operation
21 | import com.netflix.iceberg.expressions.Projections.ProjectionEvaluator
22 | import com.netflix.iceberg.expressions.{BoundPredicate, ExpressionVisitors, Expressions, NamedReference, Predicate, UnboundPredicate, Expression => IceExpression}
23 | import com.netflix.iceberg.transforms.{Transform => IceTransform}
24 | import com.netflix.iceberg.types.Types
25 | import org.apache.spark.sql.catalyst.expressions.{And, Attribute, BinaryComparison, Expression, In, InSet, IsNotNull, IsNull, Not, Or}
26 |
27 | object ExpressionConversions {
28 |
29 | import org.apache.spark.sql.iceberg.utils.LiteralUtils._
30 |
31 | private[iceberg] object Internals {
32 |
33 | val sparkToIcebergCompOp = Map[String, Operation](
34 | "<" -> Operation.LT,
35 | "<=" -> Operation.LT_EQ,
36 | "=" -> Operation.EQ,
37 | ">=" -> Operation.GT_EQ,
38 | ">" -> Operation.GT,
39 | "<=>" -> Operation.EQ
40 | )
41 |
42 | def predicate(at: Attribute, op: Operation, il: IceLiteral[_]):
43 | Predicate[_, NamedReference] = {
44 | Expressions.predicate(op, at.name, il)
45 | }
46 |
47 | object IcebergAttrLiteralCompare {
48 | def unapply(e: Expression): Option[IceExpression] = e match {
49 | case bc: BinaryComparison => (bc.left, bc.right) match {
50 | case (at: Attribute, IcebergLiteral(iL)) =>
51 | Some(predicate(at, sparkToIcebergCompOp(bc.symbol), iL))
52 | case (IcebergLiteral(iL), at: Attribute) =>
53 | Some(predicate(at, sparkToIcebergCompOp(bc.symbol).flipLR(), iL))
54 | case _ => None
55 | }
56 | case _ => None
57 | }
58 | }
59 |
60 | object IcebergIn {
61 |
62 | def inIceExpr(at: Attribute, vals: Seq[IceLiteral[_]]) = {
63 | val aF: IceExpression = Expressions.alwaysFalse()
64 | Some(
65 | vals.foldLeft(aF) {
66 | case (c, l) => Expressions.or(c, predicate(at, Operation.EQ, l))
67 | }
68 | )
69 | }
70 |
71 | def unapply(e: Expression): Option[IceExpression] = e match {
72 | case In(at: Attribute, IcebergLiteralList(vals)) => inIceExpr(at, vals)
73 | case InSet(at: Attribute, IcebergLiteralList(vals)) => inIceExpr(at, vals)
74 | case _ => None
75 | }
76 | }
77 |
78 | object StrictIcebergExpression {
79 | self =>
80 |
81 | def unapply(expr: Expression): Option[IceExpression] = expr match {
82 | case IsNull(a: Attribute) => Some(Expressions.isNull(a.name))
83 | case IsNotNull(a: Attribute) => Some(Expressions.notNull(a.name))
84 | case IcebergAttrLiteralCompare(iceExpr) => Some(iceExpr)
85 | case IcebergIn(iceExpr) => Some(iceExpr)
86 | case Not(self(iceExpr)) => Some(Expressions.not(iceExpr))
87 | case And(self(lIceExpr), l@self(rICeExpr)) => Some(Expressions.and(lIceExpr, rICeExpr))
88 | case Or(self(lIceExpr), l@self(rICeExpr)) => Some(Expressions.or(lIceExpr, rICeExpr))
89 | case _ => None
90 | }
91 |
92 | }
93 |
94 | private object CopyIceRewriteNotTransform
95 | extends ExpressionVisitors.ExpressionVisitor[IceExpression] {
96 | override def alwaysTrue(): IceExpression = Expressions.alwaysTrue()
97 |
98 | override def alwaysFalse(): IceExpression = Expressions.alwaysFalse()
99 |
100 | override def not(result: IceExpression): IceExpression = result.negate()
101 |
102 | override def and(leftResult: IceExpression, rightResult: IceExpression): IceExpression =
103 | Expressions.and(leftResult, rightResult)
104 |
105 | override def or(leftResult: IceExpression, rightResult: IceExpression): IceExpression =
106 | Expressions.or(leftResult, rightResult)
107 |
108 | override def predicate[T](pred: BoundPredicate[T]): IceExpression = pred
109 |
110 | override def predicate[T](pred: UnboundPredicate[T]): IceExpression = pred
111 | }
112 |
113 | class ApplyPartitionTransforms(val iceSchema: Types.StructType,
114 | val transformMap: Map[String, Map[String, IceTransform[_, _]]],
115 | val strict: Boolean
116 | ) extends ProjectionEvaluator {
117 | override def project(expr: IceExpression): IceExpression =
118 | ExpressionVisitors.visit(ExpressionVisitors.visit(expr, CopyIceRewriteNotTransform), this)
119 |
120 | override def alwaysTrue(): IceExpression = Expressions.alwaysTrue()
121 |
122 | override def alwaysFalse(): IceExpression = Expressions.alwaysFalse()
123 |
124 | override def not(result: IceExpression): IceExpression = {
125 | throw new UnsupportedOperationException("[BUG] project called on expression with a not")
126 | }
127 |
128 | override def and(leftResult: IceExpression, rightResult: IceExpression): IceExpression = {
129 | Expressions.and(leftResult, rightResult)
130 | }
131 |
132 | override def or(leftResult: IceExpression, rightResult: IceExpression): IceExpression = {
133 | Expressions.or(leftResult, rightResult)
134 | }
135 |
136 | override def predicate[T](pred: BoundPredicate[T]): IceExpression = {
137 | val sNm = iceSchema.field(pred.ref().fieldId()).name()
138 | val transformPreds = transformMap.get(sNm).map { transforms =>
139 | for ((pNm, pT) <- transforms) yield {
140 | if (strict) {
141 | (pT.asInstanceOf[IceTransform[T, _]]).projectStrict(pNm, pred)
142 | } else {
143 | (pT.asInstanceOf[IceTransform[T, _]]).project(pNm, pred)
144 | }
145 | }
146 | }
147 |
148 | transformPreds.map { tPreds =>
149 | tPreds.foldLeft(Expressions.alwaysTrue().asInstanceOf[IceExpression]) {
150 | case (c, tp) => Expressions.and(c, tp)
151 | }
152 | }.getOrElse(pred)
153 |
154 | }
155 |
156 | override def predicate[T](pred: UnboundPredicate[T]): IceExpression = {
157 | val bPred = pred.bind(iceSchema, true)
158 |
159 | bPred match {
160 | case bp: BoundPredicate[_] => predicate(bp)
161 | case _ => pred
162 | }
163 | }
164 | }
165 |
166 | }
167 |
168 | import Internals._
169 |
170 | def convertStrict(e: Expression): Option[IceExpression] =
171 | StrictIcebergExpression.unapply(e)
172 |
173 | def convert(e: Expression): IceExpression = e match {
174 | case StrictIcebergExpression(ice) => ice
175 | case And(l, r) => Expressions.and(convert(l), convert(r))
176 | case _ => Expressions.alwaysTrue()
177 | }
178 |
179 | def pushFiltersForScan(e : Expression,
180 | iceSchema: Types.StructType,
181 | transformMap: Map[String, Map[String, IceTransform[_, _]]]
182 | ) : IceExpression = {
183 | val convExpr = convert(e)
184 | val partTransform =
185 | new ApplyPartitionTransforms(iceSchema, transformMap, false)
186 | partTransform.project(convExpr)
187 | }
188 |
189 | }
190 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/LiteralUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import java.lang.{Double => JavaDouble, Float => JavaFloat, Long => JavaLong}
21 | import java.math.{BigDecimal => JavaBigDecimal, BigInteger => JavaBigInteger}
22 |
23 | import com.netflix.iceberg.expressions.Literal.{of => ofIceLiteral}
24 | import com.netflix.iceberg.types.Types
25 | import org.apache.commons.io.Charsets
26 | import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
27 | import org.apache.spark.sql.types._
28 | import org.apache.parquet.io.api.Binary
29 |
30 | /**
31 | * Utilities to convert from and to [[IceLiteral]]
32 | */
33 | object LiteralUtils {
34 |
35 | object IcebergLiteral {
36 | def unapply(e: Any): Option[IceLiteral[_]] = e match {
37 | case l: Literal => (l.value, l.dataType) match {
38 | case (_, StringType) => Some(ofIceLiteral(l.value.toString))
39 | case (_, dt: DecimalType) =>
40 | Some(ofIceLiteral(l.value.asInstanceOf[Decimal].toJavaBigDecimal))
41 | case (v: Double, DoubleType) => Some(ofIceLiteral(v))
42 | case (v: Float, FloatType) => Some(ofIceLiteral(v))
43 | case (v: Long, LongType) => Some(ofIceLiteral(v))
44 | case (v: Byte, ByteType) => Some(ofIceLiteral(v))
45 | case (v: Short, ShortType) => Some(ofIceLiteral(v))
46 | case (v: Int, IntegerType) => Some(ofIceLiteral(v))
47 | case (v: Boolean, BooleanType) => Some(ofIceLiteral(v))
48 | case (v: Array[Byte], dt: ArrayType) if dt.elementType == ByteType =>
49 | Some(ofIceLiteral(v))
50 | case _ => None
51 | }
52 | case cst@Cast(lt : Literal, dt, _) => {
53 | val cVal = cst.eval(null)
54 | IcebergLiteral.unapply(Literal(cVal, dt))
55 | }
56 | case _ => None
57 | }
58 | }
59 |
60 | def fromParquetPrimitive[T](v: T,
61 | iceType: IceType
62 | ): IceLiteral[T] = (v, iceType) match {
63 | case (v : Boolean, _) => ofIceLiteral(v).to(iceType)
64 | case (v : Integer, _) => ofIceLiteral(v).to(iceType)
65 | case (v : JavaLong, _) => ofIceLiteral(v).to(iceType)
66 | case (v : JavaFloat, _) => ofIceLiteral(v).to(iceType)
67 | case (v : JavaDouble, _) => ofIceLiteral(v).to(iceType)
68 | case (v : Binary, t : Types.StringType) =>
69 | ofIceLiteral(Charsets.UTF_8.decode(v.toByteBuffer)).to(iceType)
70 | case (v : Binary, iT : Types.DecimalType) =>
71 | ofIceLiteral(new JavaBigDecimal(new JavaBigInteger(v.getBytes), iT.scale())).to(iceType)
72 | case (v : Binary, t : Types.BinaryType) => ofIceLiteral(v.toByteBuffer).to(iceType)
73 | case _ => throw new IllegalArgumentException("Unsupported primitive type: " + iceType)
74 | }
75 |
76 | object IcebergLiteralList {
77 | def unapply(eL: Seq[Any]): Option[Seq[IceLiteral[_]]] = {
78 | val litOps: Seq[Option[IceLiteral[_]]] = eL.map(IcebergLiteral.unapply(_))
79 | val zero: Option[Seq[IceLiteral[_]]] = Some(Seq.empty[IceLiteral[_]])
80 | litOps.foldLeft(zero) {
81 | case (None, _) => None
82 | case (Some(l), None) => None
83 | case (Some(l), Some(iL)) => Some(iL +: l)
84 | }
85 | }
86 | }
87 | }
88 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/ParquetMetrics.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import java.io.IOException
21 | import java.lang.{Integer => JInt, Long => JLong}
22 | import java.nio.ByteBuffer
23 | import java.util
24 |
25 | import com.netflix.iceberg.Metrics
26 | import com.netflix.iceberg.exceptions.RuntimeIOException
27 | import com.netflix.iceberg.expressions.Literal
28 | import com.netflix.iceberg.hadoop.HadoopInputFile
29 | import com.netflix.iceberg.types.Conversions
30 | import org.apache.parquet.hadoop.ParquetFileReader
31 | import org.apache.parquet.hadoop.metadata.ParquetMetadata
32 |
33 | import scala.collection.mutable.{Map => MMap, Set => MSet}
34 | import scala.language.existentials
35 |
36 | object ParquetMetrics {
37 |
38 | def fromInputFile(file: HadoopInputFile,
39 | iceSchema: IceSchema): Metrics = {
40 | val parqFile =
41 | org.apache.parquet.hadoop.util.HadoopInputFile.fromStatus(file.getStat, file.getConf)
42 | val reader = ParquetFileReader.open(parqFile)
43 | try {
44 | fromMetadata(reader.getFooter, iceSchema)
45 | } catch {
46 | case e: IOException =>
47 | throw new RuntimeIOException(e, "Failed to read footer of file: %s", file)
48 | } finally if (reader != null) reader.close()
49 | }
50 |
51 | def fromMetadata(metadata: ParquetMetadata,
52 | fileSchema: IceSchema
53 | ): Metrics = {
54 |
55 | var rowCount: Long = 0L
56 | val columnSizes = MMap[JInt, JLong]()
57 | val valueCounts = MMap[JInt, JLong]()
58 | val nullValueCounts = MMap[JInt, JLong]()
59 | val lowerBounds = MMap[JInt, IceLiteral[_]]()
60 | val upperBounds = MMap[JInt, IceLiteral[_]]()
61 |
62 | val missingStats = MSet[JInt]()
63 |
64 | val parquetType = metadata.getFileMetaData.getSchema
65 | val blocks = metadata.getBlocks
66 | import scala.collection.JavaConversions._
67 | for (block <- blocks) {
68 | rowCount += block.getRowCount
69 | import scala.collection.JavaConversions._
70 | for (column <- block.getColumns) {
71 |
72 | val field = fileSchema.findField(column.getPath.toDotString)
73 |
74 | if (field != null) {
75 | val fieldId = field.fieldId()
76 | increment(columnSizes, fieldId, column.getTotalSize)
77 | increment(valueCounts, fieldId, column.getValueCount)
78 | val stats = column.getStatistics
79 |
80 | if (stats == null) {
81 | missingStats.add(fieldId)
82 | }
83 | else if (!stats.isEmpty) {
84 | increment(nullValueCounts, fieldId, stats.getNumNulls)
85 | val field = fileSchema.asStruct.field(fieldId)
86 | if (field != null && stats.hasNonNullValue) {
87 | updateMin(
88 | lowerBounds,
89 | fieldId,
90 | LiteralUtils.fromParquetPrimitive(stats.genericGetMin, field.`type`)
91 | )
92 | updateMax(
93 | upperBounds,
94 | fieldId,
95 | LiteralUtils.fromParquetPrimitive(stats.genericGetMax, field.`type`)
96 | )
97 | }
98 | }
99 | }
100 | }
101 | }
102 | // discard accumulated values if any stats were missing
103 | for (fieldId <- missingStats) {
104 | nullValueCounts.remove(fieldId)
105 | lowerBounds.remove(fieldId)
106 | upperBounds.remove(fieldId)
107 | }
108 |
109 | new Metrics(
110 | rowCount,
111 | javaMap(columnSizes.toMap),
112 | javaMap(valueCounts.toMap),
113 | javaMap(nullValueCounts.toMap),
114 | javaMap(toBufferMap(fileSchema, lowerBounds)),
115 | javaMap(toBufferMap(fileSchema, upperBounds))
116 | )
117 | }
118 |
119 | private def increment(columns: MMap[JInt, JLong],
120 | fieldId: Int,
121 | amount: JLong): Unit = {
122 | val v: JLong = columns.getOrElse(fieldId, 0L)
123 | columns(fieldId) = (v + amount)
124 | }
125 |
126 | private def updateMin[T](lowerBounds: MMap[JInt, Literal[_]],
127 | id: Int,
128 | min: Literal[T]
129 | ) = {
130 | val currentMinO: Option[Literal[_]] = lowerBounds.get(id)
131 |
132 | def currentMin: Literal[T] = currentMinO.get.asInstanceOf[Literal[T]]
133 |
134 | if (!currentMinO.isDefined ||
135 | min.comparator().compare(min.value(), currentMin.value()) < 0
136 | ) {
137 | lowerBounds(id) = min
138 | }
139 | }
140 |
141 | private def updateMax[T](upperBounds: MMap[JInt, Literal[_]],
142 | id: Int,
143 | max: Literal[T]
144 | ) = {
145 | val currentMaxO: Option[Literal[_]] = upperBounds.get(id)
146 |
147 | def currentMax: Literal[T] = currentMaxO.get.asInstanceOf[Literal[T]]
148 |
149 | if (!currentMaxO.isDefined ||
150 | max.comparator().compare(max.value(), currentMax.value()) > 0
151 | ) {
152 | upperBounds(id) = max
153 | }
154 | }
155 |
156 | private def toBufferMap(schema: IceSchema,
157 | map: MMap[JInt, IceLiteral[_]]
158 | ): Map[JInt, ByteBuffer] = {
159 |
160 | val mb = MMap[JInt, ByteBuffer]()
161 |
162 | for ((i, l) <- map.iterator) {
163 | val t = schema.findType(i)
164 | mb(i) = Conversions.toByteBuffer(t, l.value())
165 | }
166 |
167 | mb.toMap
168 | }
169 |
170 | private def javaMap[K,V](m : Map[K,V]) : java.util.Map[K,V] = {
171 | val jm = new util.HashMap[K,V](m.size)
172 | for(
173 | (k,v) <- m
174 | ) {
175 | jm.put(k,v)
176 | }
177 | jm
178 | }
179 |
180 | }
181 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/SchemaConversions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.{PartitionSpec, Schema}
21 | import org.apache.spark.sql.SparkSession
22 | import org.apache.spark.sql.catalyst.catalog.CatalogTable
23 | import org.apache.spark.sql.iceberg.utils.TypeConversions.TypeMapping
24 |
25 | object SchemaConversions {
26 |
27 | private def toIcebergSchema(typeMapping: TypeMapping) : Schema = {
28 | new Schema(typeMapping.iceType.fields())
29 | }
30 |
31 | private def toIcebergPartitionSpec(catalogTable: CatalogTable,
32 | typeMapping: TypeMapping,
33 | schema : Schema
34 | )(implicit sparkSession : SparkSession) : PartitionSpec = {
35 |
36 | if (catalogTable.partitionColumnNames.nonEmpty) {
37 | var bldr = PartitionSpec.builderFor(schema)
38 | for(
39 | pField <- catalogTable.partitionSchema.fields;
40 | iceF = schema.findField(pField.name)
41 | ) {
42 | bldr = bldr.identity(iceF.name())
43 | }
44 | bldr.build()
45 | } else {
46 | null
47 | }
48 | }
49 |
50 | def toIcebergSpec(catalogTable: CatalogTable)(implicit sparkSession : SparkSession) :
51 | (Schema, PartitionSpec) = {
52 |
53 | val typMap = TypeConversions.convertStructType(catalogTable.schema)
54 | val schema = toIcebergSchema(typMap)
55 | val pSpec = toIcebergPartitionSpec(catalogTable, typMap, schema)
56 |
57 | (schema, pSpec)
58 |
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/TableUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import java.util.Locale
21 |
22 | import com.netflix.iceberg.Snapshot
23 | import org.apache.spark.sql.catalyst.InternalRow
24 | import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericRow}
25 | import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
26 | import org.apache.spark.sql.iceberg.table.SparkTables
27 | import org.apache.spark.sql.types._
28 | import org.apache.spark.sql.{DataFrame, Row, SparkSession}
29 | import org.apache.spark.unsafe.types.UTF8String
30 |
31 | object TableUtils {
32 |
33 | protected[this] def formatName(name: String)(implicit ss : SparkSession) : String = {
34 | if (ss.sqlContext.conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
35 | }
36 |
37 | def iceTable(tblNm : String)(implicit ss : SparkSession) : IceTable = {
38 | val name = parseIdentifier(tblNm, ss)
39 |
40 | val db = formatName(name.database.getOrElse(ss.sessionState.catalog.getCurrentDatabase))
41 | val table = formatName(name.table)
42 | val catalogTable = ss.sessionState.catalog.externalCatalog.getTable(db, table)
43 | val iceTables = new SparkTables(ss, catalogTable)
44 | val tbId = qualifiedTableIdentifier(catalogTable.identifier, ss)
45 | iceTables.load(tbId.database.getOrElse(null), tbId.table)
46 | }
47 |
48 | val SNAPSHOT_SCHEMA : StructType = {
49 | StructType(
50 | Seq(
51 | StructField("id", LongType),
52 | StructField("parentId", LongType),
53 | StructField("timeMillis", LongType),
54 | StructField("numAddedFiles", IntegerType),
55 | StructField("numdDeletedFiles", IntegerType),
56 | StructField("manifestListLocation", StringType)
57 | )
58 | )
59 | }
60 |
61 | def toRow(pId : Long,
62 | sShot : Snapshot) : Row = {
63 | import scala.collection.JavaConversions._
64 | new GenericRow(
65 | Array[Any](
66 | sShot.snapshotId(),
67 | pId,
68 | sShot.timestampMillis(),
69 | sShot.addedFiles().size,
70 | sShot.deletedFiles().size,
71 | sShot.manifestListLocation()
72 | )
73 | )
74 | }
75 |
76 | def toIRow(pId : Long,
77 | sShot : Snapshot) : InternalRow = {
78 | import scala.collection.JavaConversions._
79 | new GenericInternalRow(
80 | Array[Any](
81 | sShot.snapshotId(),
82 | pId,
83 | sShot.timestampMillis(),
84 | sShot.addedFiles().size,
85 | sShot.deletedFiles().size,
86 | UTF8String.fromString(sShot.manifestListLocation())
87 | )
88 | )
89 | }
90 |
91 | def snapShots[T](iceTable : IceTable,
92 | toRow : (Long, Snapshot) => T)(implicit ss : SparkSession) : Seq[T] = {
93 | import scala.collection.JavaConversions._
94 | val rows = for (sShot <- iceTable.snapshots()) yield {
95 |
96 | val pId : Long = {
97 | val l = sShot.parentId()
98 | if (l == null) -1 else l
99 | }
100 | toRow(pId, sShot)
101 | }
102 | rows.toSeq
103 | }
104 |
105 | private val snapshot_millis = new ThreadLocal[Long] {
106 | override def initialValue = -1L
107 | }
108 |
109 | def setThreadSnapShotMillis(sMillis : Long) = {
110 | snapshot_millis.set(sMillis)
111 | }
112 |
113 | def getThreadSnapShotId(iceTable: IceTable) : Long = {
114 | val sMillis = snapshot_millis.get
115 | setThreadSnapShotMillis(-1L)
116 | if (sMillis == -1) {
117 | iceTable.currentSnapshot().snapshotId()
118 | } else {
119 | snapShotId(iceTable, sMillis)
120 | }
121 | }
122 |
123 | def snapShotId(iceTable : IceTable, timeMs : Long) : Long = {
124 | import scala.collection.JavaConversions._
125 | var spId = iceTable.currentSnapshot().snapshotId()
126 | for (sShot <- iceTable.snapshots()) {
127 | if (sShot.timestampMillis() <= timeMs) {
128 | spId = sShot.snapshotId()
129 | }
130 | }
131 | spId
132 | }
133 |
134 | def snapShotsDF(iceTable : IceTable)(implicit ss : SparkSession) : DataFrame =
135 | dataframe(snapShots(iceTable, toRow), SNAPSHOT_SCHEMA)
136 |
137 | def snapShotsDF(tblNm : String)(implicit ss : SparkSession) : DataFrame = {
138 | val iTbl = iceTable(tblNm)
139 | dataframe(snapShots(iTbl, toRow), SNAPSHOT_SCHEMA)
140 | }
141 |
142 | val SNAPSHOTSVIEW_SUFFIX = "$snapshots"
143 |
144 | def snapShotsLocalRelation(tblNm : String)(implicit ss : SparkSession) : LocalRelation = {
145 | val iTbl = iceTable(tblNm)
146 | LocalRelation(SNAPSHOT_SCHEMA.toAttributes, snapShots(iTbl, toIRow))
147 | }
148 | }
149 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/Transforms.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.transforms.{Transform => IceTransform, Transforms => IcebergTransforms}
21 | import com.netflix.iceberg.types.{Type, Types}
22 | import org.apache.spark.sql.SparkSession
23 | import org.apache.spark.sql.iceberg.Config
24 | import org.apache.spark.sql.iceberg.utils.TypeConversions.TypeMapping
25 | import org.apache.spark.sql.types.StructField
26 |
27 | import scala.collection.mutable.{ArrayBuffer, Map => MMap}
28 | import scala.util.Try
29 |
30 | object Transforms {
31 |
32 | val TRANSFORM_REGX = """(([^\s,=:]+=[^\s,=:]+:[^\s,=:]+)(?:,\s*)?)+""".r
33 |
34 | /**
35 | * Validate [[Config.COLUMN_DEPENDENCIES]] specified for a ''Managed Table''.
36 | * - the option must be in the form of a comma-separated list of ''srCol=destCol:tnExpr''
37 | * - each ''srCol'' must resolve to a attribute in the given [[TypeMapping.sparkType]]
38 | * - each ''destCol'' must resolve to a attribute in the given [[TypeMapping.sparkType]]
39 | * - each ''tnExpr'' must resolve to a valid [[Transforms]] for the column's [[Type]]
40 | *
41 | * @param valueMappings
42 | * @param typeMapping
43 | * @param sparkSession
44 | * @return
45 | */
46 | def fromOption(valueMappings : String,
47 | typeMapping : TypeMapping
48 | )(implicit sparkSession : SparkSession) :
49 | Either[Array[String],Map[String, Map[String, IceTransform[_,_]]]] = {
50 |
51 | val errs = ArrayBuffer[String]()
52 | val transforms = MMap[String, Map[String, IceTransform[_,_]]]()
53 |
54 | def parseTransforms : Array[String] = {
55 | valueMappings.split(",").map(_.trim)
56 | }
57 |
58 | def parseEntry(s : String) : Option[(String,String,String)] = {
59 | val kv = s.split("=")
60 | if (kv.size != 2) {
61 | errs +=
62 | s"Cannot parse transform ${s} in partition transformations: ${valueMappings}"
63 | None
64 | } else {
65 | val vSplit = kv(1).split(":")
66 | if (vSplit.size != 2) {
67 | errs +=
68 | s"Cannot parse transform ${s} in partition transformations: ${valueMappings}"
69 | None
70 | } else {
71 | Some((kv(0).trim, vSplit(0).trim, vSplit(1).trim))
72 | }
73 | }
74 | }
75 |
76 | def validateAttr(attr : String) : Option[(StructField, Types.NestedField)] = {
77 | typeMapping.findField(attr)
78 | }
79 |
80 | def validateTransform(iceF : Types.NestedField,
81 | t: String) : Option[IceTransform[_,_]] = {
82 |
83 | Try {
84 | Some(IcebergTransforms.fromString(iceF.`type`(), t))
85 | }.recover {
86 | case throwable: Throwable =>
87 | errs += s"Failed to parse transform ${t} in partition transformations: " +
88 | s"${valueMappings}; (${throwable.getMessage})"
89 | None
90 | }.get
91 | }
92 |
93 | for(
94 | transform <- parseTransforms;
95 | (sc, pc,v) <- parseEntry(transform) ;
96 | (_, sIceF) <- validateAttr(sc);
97 | t <- validateTransform(sIceF, v);
98 | (_, pIceF) <- validateAttr(pc)
99 | ) {
100 | val tm = transforms.getOrElse(sIceF.name, Map[String, IceTransform[_,_]]())
101 | transforms(sIceF.name) = (tm + ((pIceF.name(), t)))
102 | }
103 |
104 | if (errs.nonEmpty) {
105 | Left(errs.toArray)
106 | } else {
107 | Right(transforms.toMap)
108 | }
109 |
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/TypeConversions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg.utils
19 |
20 | import com.netflix.iceberg.types.{Type, Types}
21 | import org.apache.spark.sql.SparkSession
22 | import org.apache.spark.sql.types._
23 |
24 | object TypeConversions {
25 |
26 | def convertStructType(st : StructType) : TypeMapping = {
27 | implicit val nextId = new Function0[Int] {
28 | var cnt = -1
29 | def apply : Int = {
30 | cnt += 1
31 | cnt
32 | }
33 | }
34 | TypeMapping(st, structType(st).asInstanceOf[Types.StructType])
35 | }
36 |
37 | def toIcebergType(dt : DataType)(implicit nextId : () => Int) : Type = dt match {
38 | case st : StructType => structType(st)
39 | case mt : MapType => mapType(mt)
40 | case at : ArrayType => arrType(at)
41 | case at : AtomicType => atomicType(at)
42 | case _ =>
43 | throw new UnsupportedOperationException(s"Spark datatype ${dt.toString} is not supported")
44 | }
45 |
46 | def structType(st : StructType)(implicit nextId : () => Int) : Type = {
47 | import scala.collection.JavaConversions._
48 |
49 | val iceFields : java.util.List[Types.NestedField] = (
50 | for( (field, i) <- st.fields.zipWithIndex) yield {
51 |
52 | val (id, nm, typ) = (nextId(), field.name, toIcebergType(field.dataType))
53 | if (field.nullable) {
54 | Types.NestedField.optional(id, nm, typ)
55 | } else Types.NestedField.required(id, nm, typ)
56 |
57 | }).toList
58 |
59 | Types.StructType.of(iceFields)
60 | }
61 |
62 | def mapType(mT : MapType)(implicit nextId : () => Int) : Type = {
63 |
64 | val (kId,vId,kT, vT) =
65 | (nextId(), nextId(), toIcebergType(mT.keyType), toIcebergType(mT.valueType))
66 |
67 | if (mT.valueContainsNull) {
68 | Types.MapType.ofOptional(kId, vId, kT, vT)
69 | }
70 | else Types.MapType.ofRequired(kId,vId,kT, vT)
71 | }
72 |
73 | def arrType(aT : ArrayType)(implicit nextId : () => Int) : Type = {
74 |
75 | val (eId,eT) = (nextId(), toIcebergType(aT.elementType))
76 |
77 | if (aT.containsNull) {
78 | Types.ListType.ofOptional(eId, eT)
79 | }
80 | else Types.ListType.ofRequired(eId, eT)
81 | }
82 |
83 |
84 | def atomicType(aT: AtomicType): Type = aT match {
85 | case b : BooleanType => Types.BooleanType.get
86 | case i : IntegerType => Types.IntegerType.get
87 | case s : ShortType => Types.IntegerType.get
88 | case b : ByteType => Types.IntegerType.get
89 | case l : LongType => Types.LongType.get
90 | case f : FloatType => Types.FloatType.get
91 | case d : DoubleType => Types.DoubleType.get
92 | case s : StringType => Types.StringType.get
93 | case c : CharType => Types.StringType.get
94 | case c : VarcharType => Types.StringType.get
95 | case d : DateType => Types.DateType.get
96 | case t : TimestampType => Types.TimestampType.withZone
97 | case d : DecimalType => Types.DecimalType.of(d.precision, d.scale)
98 | case b : BinaryType => Types.BinaryType.get
99 | case _ => throw new UnsupportedOperationException("Not a supported type: " + aT.catalogString)
100 | }
101 |
102 | case class TypeMapping(sparkType : StructType,
103 | iceType : Types.StructType) {
104 |
105 | def findField(nm : String)(implicit sparkSession : SparkSession) :
106 | Option[(StructField, Types.NestedField)] = {
107 | (sparkType.fields.find {
108 | case f if sparkSession.sqlContext.conf.resolver(f.name, nm) => true
109 | case _ => false
110 | }).map(t => (t, iceType.field(t.name) ))
111 | }
112 | }
113 |
114 | }
115 |
--------------------------------------------------------------------------------
/src/main/scala/org/apache/spark/sql/iceberg/utils/utils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.iceberg
19 |
20 | import com.netflix.iceberg.hadoop.HadoopInputFile
21 | import com.netflix.iceberg.{DataFile, FileFormat, StructLike, Table}
22 | import org.apache.spark.sql.catalyst.catalog.SessionCatalog
23 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
24 | import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
25 | import org.apache.spark.sql.execution.datasources.iceberg.PartitioningUtils
26 | import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
27 | import org.apache.spark.sql.execution.datasources.{FileFormat => SparkFileFormat}
28 | import org.apache.spark.sql.types._
29 | import org.apache.spark.sql._
30 | import org.apache.spark.sql.catalyst.util.DateTimeUtils
31 | import org.apache.spark.unsafe.types.UTF8String
32 |
33 | import scala.util.Try
34 |
35 | trait PlanUtils {
36 |
37 | def dataFrame(lP: LogicalPlan)(
38 | implicit sqlContext: SQLContext): DataFrame = {
39 | Dataset.ofRows(sqlContext.sparkSession, lP)
40 | }
41 |
42 | // for testing
43 | def iceTableScanExec(df : DataFrame,
44 | strict : Boolean = true
45 | ) : Option[IceTableScanExec] = {
46 | val icScanO = (df.queryExecution.executedPlan find {
47 | case is : IceTableScanExec => true
48 | case _ => false
49 | }).map(_.asInstanceOf[IceTableScanExec])
50 |
51 | if (!icScanO.isDefined & strict) {
52 | throw new AnalysisException(s"Failed to find a IceTableScan in given dataframe")
53 | }
54 | icScanO
55 | }
56 |
57 | // for testing
58 | def filesScanned(df : DataFrame,
59 | strict : Boolean = true
60 | ) : Seq[String] = {
61 | for(
62 | iScan <- iceTableScanExec(df,strict).toSeq;
63 | pdirs = iScan.showPartitionsScanned;
64 | pd <- pdirs;
65 | fs <- pd.files
66 | ) yield {
67 | fs.getPath.toString
68 | }
69 | }
70 | }
71 |
72 | trait ExprUtils {
73 |
74 | def isApproximateNumeric(dt: DataType) = dt.isInstanceOf[FractionalType]
75 | def isNumeric(dt: DataType) = NumericType.acceptsType(dt)
76 |
77 | }
78 |
79 | trait SessionUtils {
80 |
81 | def currentSparkSession: SparkSession = {
82 | var spkSessionO = SparkSession.getActiveSession
83 | if (!spkSessionO.isDefined) {
84 | spkSessionO = SparkSession.getDefaultSession
85 | }
86 | spkSessionO.getOrElse(???)
87 | }
88 |
89 | def parseIdentifier(name: String,
90 | sparkSession: SparkSession): TableIdentifier = {
91 | sparkSession.sessionState.sqlParser.parseTableIdentifier(name)
92 | }
93 |
94 | def qualifiedTableIdentifier(tabId: TableIdentifier,
95 | sparkSession: SparkSession): TableIdentifier = {
96 | val catalog = sparkSession.sessionState.catalog
97 |
98 | val db = tabId.database.getOrElse(catalog.getCurrentDatabase)
99 | db match {
100 | case SessionCatalog.DEFAULT_DATABASE => TableIdentifier(tabId.table)
101 | case _ => TableIdentifier(tabId.table, Some(db))
102 | }
103 | }
104 |
105 | def dataframe(rows: Seq[Row],
106 | schema: StructType
107 | )(implicit ss: SparkSession): DataFrame = {
108 | ss.createDataFrame(
109 | ss.sparkContext.parallelize(rows, 1),
110 | schema
111 | )
112 | }
113 |
114 | def convertToEpoch(s : String) : Long = {
115 | Try {
116 | DateTimeUtils.stringToTimestamp(UTF8String.fromString(s))
117 | .map(ts => DateTimeUtils.toMillis(ts))
118 | .getOrElse(s.toLong)
119 | } getOrElse(
120 | throw new AnalysisException(s"Cannot intepret value '${s}' as a timestamp")
121 | )
122 | }
123 |
124 | def convertToTimestampString(millis : Long) : String = {
125 | DateTimeUtils.timestampToString(millis * 1000)
126 | }
127 | }
128 |
129 | package object utils extends PlanUtils with ExprUtils with SessionUtils {
130 |
131 | type IcePSpec = com.netflix.iceberg.PartitionSpec
132 | type IceStruct = com.netflix.iceberg.StructLike
133 | type IceTable = com.netflix.iceberg.Table
134 | type IceType = com.netflix.iceberg.types.Type
135 | type IceSchema = com.netflix.iceberg.Schema
136 | type IceFormat = com.netflix.iceberg.FileFormat
137 | type IceMetrics = com.netflix.iceberg.Metrics
138 | type IceLiteral[T] = com.netflix.iceberg.expressions.Literal[T]
139 |
140 | class StructLikeInternalRow(val struct: StructType) extends StructLike {
141 | val fields : Array[StructField] = struct.fields
142 | val types : Array[DataType] = fields.map(_.dataType)
143 |
144 | private var row : InternalRow = null
145 |
146 | def setRow(row: InternalRow) : StructLikeInternalRow = {
147 | this.row = row
148 | this
149 | }
150 |
151 | override def size: Int = types.length
152 |
153 | @SuppressWarnings(Array("unchecked"))
154 | override def get[T](pos: Int, javaClass: Class[T]): T = types(pos) match {
155 | case StringType => row.getUTF8String(pos).toString.asInstanceOf[T]
156 | case _ => javaClass.cast(row.get(pos, types(pos)))
157 | }
158 |
159 | override def set[T](pos: Int, value: T): Unit = ???
160 | }
161 |
162 | def toIceRow(sr : InternalRow, ss : StructType) : StructLike = {
163 | val r = new StructLikeInternalRow(ss)
164 | r.setRow(sr)
165 | r
166 | }
167 |
168 | def iceFormat(ffName : String) : IceFormat = {
169 | Try {
170 | com.netflix.iceberg.FileFormat.valueOf(ffName)
171 | }.getOrElse(com.netflix.iceberg.FileFormat.AVRO)
172 | }
173 |
174 | def partitionSpec(iceTable : Table) = iceTable.spec()
175 |
176 | def iceFormat(ff : SparkFileFormat) : FileFormat = {
177 | if (ff.isInstanceOf[ParquetFileFormat]) {
178 | FileFormat.PARQUET
179 | } else {
180 | FileFormat.AVRO
181 | }
182 | }
183 |
184 | def iceMetrics(inFile: HadoopInputFile,
185 | iceSchema: IceSchema,
186 | iceFmt : IceFormat
187 | ) : Option[IceMetrics] = iceFmt match {
188 | case com.netflix.iceberg.FileFormat.PARQUET =>
189 | Some(ParquetMetrics.fromInputFile(inFile, iceSchema))
190 | case _ => None
191 | }
192 |
193 | def icePartStruct(pathString : String,
194 | partSchema: StructType
195 | ) : IceStruct = {
196 | val iRow = PartitioningUtils.partitionRowFromFullPath(pathString, partSchema)
197 | toIceRow(iRow, partSchema)
198 | }
199 |
200 | }
201 |
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00004-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00004-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00010-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00010-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00017-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00017-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00022-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00022-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00026-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0905245/part-00026-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00016-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00016-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00018-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00018-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00020-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00020-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00023-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00023-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00029-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0906245/part-00029-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00000-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00000-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00003-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00003-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00008-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00008-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00012-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00012-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00019-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0907245/part-00019-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00005-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00005-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00006-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00006-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00014-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00014-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00024-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00024-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00025-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0908245/part-00025-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00001-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00001-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00002-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00002-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00009-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00009-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00013-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00013-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00021-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0909245/part-00021-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00007-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00007-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00011-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00011-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00015-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00015-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00027-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00027-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00028-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/hbutani/icebergSQL/63c26ae00bb03bb7d05bfbeac4fe6787b019d505/src/test/resources/store_sales/ss_sold_date_sk=0910245/part-00028-eeb13bdd-e31e-4992-aa61-60c2c1729751.c000.snappy.parquet
--------------------------------------------------------------------------------
/src/test/scala/org/apache/spark/sql/hive/test/iceberg/AbstractTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.hive.test.iceberg
19 |
20 | import java.io.File
21 | import java.util.TimeZone
22 |
23 | import org.apache.spark.internal.Logging
24 | import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
25 | import org.apache.spark.sql.iceberg.planning.SparkSessionExtensions
26 | import org.apache.spark.sql.types.DataType
27 | import org.apache.spark.sql.{DataFrame, Row}
28 | import org.joda.time.DateTimeZone
29 | import org.scalatest.{BeforeAndAfterAll, fixture}
30 | import org.apache.spark.sql.iceberg.utils
31 |
32 | abstract class AbstractTest extends fixture.FunSuite
33 | with fixture.TestDataFixture with BeforeAndAfterAll with TestTables with Logging {
34 |
35 | override def beforeAll() = {
36 | println("*** Starting TestCase " ++ this.toString() )
37 | // System.setProperty("user.timezone", "UTC")
38 | // TimeZone.setDefault(TimeZone.getTimeZone("UTC"))
39 | // DateTimeZone.setDefault(DateTimeZone.forID("UTC"))
40 |
41 | new SparkSessionExtensions()(TestIcebergHive.sparkSession.extensions)
42 |
43 | TestIcebergHive.sparkContext.setLogLevel("ERROR")
44 | TestIcebergHive.setConf("spark.sql.files.openCostInBytes", (128 * 1024 * 1024).toString)
45 | TestIcebergHive.setConf("spark.sql.files.maxPartitionBytes", (16 * 1024 * 1024).toString)
46 |
47 | setupStoreSalesTables
48 |
49 | }
50 |
51 | def result(df: DataFrame): Array[Row] = {
52 | df.collect()
53 | }
54 |
55 | def test(nm: String, sql: String,
56 | showPlan: Boolean = false,
57 | showResults: Boolean = false,
58 | setupSQL: Option[(String, String)] = None): Unit = {
59 | test(nm) { td =>
60 | println("*** *** Running Test " ++ nm)
61 |
62 | try {
63 |
64 | for((s,e) <- setupSQL) {
65 | TestIcebergHive.sql(s)
66 | }
67 |
68 | try {
69 | val df = sqlAndLog(nm, sql)
70 | if (showPlan) {
71 | logPlan(nm, df)
72 | }
73 | if (showResults) {
74 | df.show(20, false)
75 | }
76 | } finally {
77 | }
78 | } finally {
79 | for((s,e) <- setupSQL) {
80 | TestIcebergHive.sql(e)
81 | }
82 | }
83 | }
84 | }
85 |
86 | def cTest(nm: String,
87 | dsql: String,
88 | bsql: String,
89 | devAllowedInAproxNumeric: Double = 0.0): Unit = {
90 | test(nm) { td =>
91 | println("*** *** Running Correctness Test " ++ nm)
92 |
93 | try {
94 | val df1 = sqlAndLog(nm, dsql)
95 | val df2 = sqlAndLog(nm, bsql)
96 | assert(isTwoDataFrameEqual(df1, df2, devAllowedInAproxNumeric))
97 | } finally {
98 | }
99 | }
100 | }
101 |
102 | def sqlAndLog(nm: String, sqlStr: String): DataFrame = {
103 | logDebug(s"\n$nm SQL:\n" + sqlStr)
104 | TestIcebergHive.sql(sqlStr)
105 | }
106 |
107 | def logPlan(nm: String, df: DataFrame): Unit = {
108 | logInfo(s"\n$nm Plan:")
109 | logInfo(s"\nLogical Plan:\n" + df.queryExecution.logical.toString)
110 | logInfo(s"\nOptimized Plan:\n" + df.queryExecution.optimizedPlan.toString)
111 | logInfo(s"\nPhysical Plan:\n" + df.queryExecution.sparkPlan.toString)
112 | }
113 |
114 | def roundValue(chooseRounding: Boolean, v: Any, dt: DataType): Any = {
115 | if (chooseRounding && v != null &&
116 | utils.isNumeric(dt) &&
117 | !Set[Any](Double.PositiveInfinity, Double.NegativeInfinity,
118 | Float.PositiveInfinity, Float.NegativeInfinity).contains(v)
119 | ) {
120 | BigDecimal(v.toString).setScale(1, BigDecimal.RoundingMode.HALF_UP).toDouble
121 | } else if (v == Float.PositiveInfinity){
122 | Double.PositiveInfinity
123 | } else if (v == Float.NegativeInfinity){
124 | Double.NegativeInfinity
125 | } else {
126 | v
127 | }
128 | }
129 |
130 | def isTwoDataFrameEqual(df1: DataFrame,
131 | df2: DataFrame,
132 | devAllowedInAproxNumeric: Double,
133 | Sorted: Boolean = false,
134 | chooseRounding: Boolean = true): Boolean = {
135 |
136 |
137 | if (df1.schema != df2.schema) {
138 | logWarning(
139 | s"""
140 | |different schemas issue:
141 | | df1 schema = ${df1.schema}
142 | | df2.schema = ${df2.schema}
143 | """.stripMargin)
144 | // return false
145 | }
146 |
147 | import collection.JavaConversions._
148 |
149 | var df11 = df1
150 | var df21 = df2
151 |
152 | var df1_ilist = df11.queryExecution.executedPlan.executeCollect()
153 | var df2_ilist = df21.queryExecution.executedPlan.executeCollect()
154 |
155 |
156 | if (!Sorted && df1_ilist.size > 1) {
157 |
158 | val sortCols = df11.columns
159 |
160 | df1_ilist = {
161 | df11 = utils.dataFrame(
162 | LocalRelation(df11.queryExecution.optimizedPlan.output, df1_ilist)
163 | )(TestIcebergHive.sparkSession.sqlContext)
164 | df11 = df11.sort(sortCols.head, sortCols.tail:_*).
165 | select(sortCols.head, sortCols.tail:_*)
166 | df11.queryExecution.executedPlan.executeCollect()
167 | }
168 |
169 | df2_ilist = {
170 | df21 = utils.dataFrame(
171 | LocalRelation(df21.queryExecution.optimizedPlan.output, df2_ilist)
172 | )(TestIcebergHive.sparkSession.sqlContext)
173 | df21 = df21.sort(sortCols.head, sortCols.tail:_*).
174 | select(sortCols.head, sortCols.tail:_*)
175 | df21.queryExecution.executedPlan.executeCollect()
176 | }
177 | }
178 |
179 | val df1_count = df1_ilist.size
180 | val df2_count = df2_ilist.size
181 | if (df1_count != df2_count) {
182 | println(df1_count + "\t" + df2_count)
183 | println("The row count is not equal")
184 | println(s"""df1=\n${df1_ilist.mkString("\n")}\ndf2=\n ${df2_ilist.mkString("\n")}""")
185 | return false
186 | }
187 |
188 | for (i <- 0 to df1_count.toInt - 1) {
189 | for (j <- 0 to df1.columns.size - 1) {
190 | val res1 = roundValue(chooseRounding,
191 | df1_ilist(i).get(j, df11.schema(j).dataType),
192 | df11.schema(j).dataType)
193 | val res2 = roundValue(chooseRounding,
194 | df2_ilist(i).get(j, df21.schema(j).dataType),
195 | df21.schema(j).dataType)
196 | // account for difference in null aggregation of javascript
197 | if (res2 == null && res1 != null) {
198 | if (!Set(Int.MaxValue, Int.MinValue,
199 | Long.MaxValue, Long.MinValue,
200 | Double.PositiveInfinity, Double.NegativeInfinity,
201 | Float.PositiveInfinity, Float.NegativeInfinity,
202 | 0).contains(res1)) {
203 | println(s"values in row $i, column $j don't match: ${res1} != ${res2}")
204 | println(s"""df1=\n${df1_ilist.mkString("\n")}\ndf2=\n ${df2_ilist.mkString("\n")}""")
205 | return false
206 | }
207 | } else if ((utils.isApproximateNumeric(df1.schema(j).dataType) &&
208 | (Math.abs(res1.asInstanceOf[Double] - res2.asInstanceOf[Double]) >
209 | devAllowedInAproxNumeric)) ||
210 | (!utils.isApproximateNumeric(df1.schema(j).dataType) && res1 != res2)) {
211 | println(s"values in row $i, column $j don't match: ${res1} != ${res2}")
212 | println(s"""df1=\n${df1_ilist.mkString("\n")}\ndf2=\n ${df2_ilist.mkString("\n")}""")
213 | return false
214 | }
215 | }
216 | }
217 | logDebug("The two dataframe is equal " + df1_count)
218 | // println(df1_list.mkString("", "\n", ""))
219 | return true
220 | }
221 |
222 | def delete(f : File) : Unit = {
223 | if (f.exists()) {
224 | if (f.isDirectory) {
225 | f.listFiles().foreach(delete(_))
226 | f.delete()
227 | } else {
228 | f.delete()
229 | }
230 | }
231 | }
232 |
233 |
234 | }
235 |
--------------------------------------------------------------------------------
/src/test/scala/org/apache/spark/sql/hive/test/iceberg/BasicCreateAndInsertTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.hive.test.iceberg
19 |
20 | import java.util.Date
21 |
22 | import org.apache.spark.sql.DataFrame
23 | import org.apache.spark.sql.iceberg.utils
24 | import org.apache.spark.sql.iceberg.utils.TableUtils
25 |
26 | class BasicCreateAndInsertTest extends AbstractTest {
27 |
28 | def showSnapShots(table: String): Unit = {
29 | val sqlText = "select * from `" +
30 | table + TableUtils.SNAPSHOTSVIEW_SUFFIX +
31 | "`"
32 | println(sqlText)
33 | TestIcebergHive.sql(sqlText).
34 | show(10000, false)
35 | }
36 |
37 | val allColumns: String = " * "
38 | val excludeSoldDate: String = "ss_sold_time_sk,ss_item_sk,ss_customer_sk,ss_cdemo_sk," +
39 | "ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_quantity,ss_wholesale_cost," +
40 | "ss_list_price,ss_sales_price,ss_ext_sales_price,ss_sold_month,ss_sold_day"
41 |
42 | def _insertStatement(insClause: String,
43 | toTable: String,
44 | fromTable: String,
45 | partitionPredicate: String,
46 | sourcePredciate: String,
47 | showSnapShot: Boolean
48 | ): Unit = {
49 |
50 | val partSpec = if (partitionPredicate != null) {
51 | s"partition ( ${partitionPredicate} )"
52 | } else ""
53 |
54 | val whereClause = (partitionPredicate, sourcePredciate) match {
55 | case (null, null) => ""
56 | case (p, null) => s"where ${p} "
57 | case (null, s) => s"where ${s} "
58 | case (p, s) => s"where ${p} and ${s} "
59 | }
60 |
61 | val selList = if (partitionPredicate != null) {
62 | excludeSoldDate
63 | } else allColumns
64 |
65 | val insStat =
66 | s"""
67 | |${insClause} ${toTable} ${partSpec}
68 | |select ${selList} from ${fromTable}
69 | |${whereClause}
70 | """.stripMargin
71 |
72 | println(insStat)
73 |
74 | TestIcebergHive.sql(insStat)
75 | if (showSnapShot) {
76 | showSnapShots(toTable)
77 | }
78 |
79 | }
80 |
81 | def insert(toTable: String,
82 | fromTable: String,
83 | partitionPredicate: String = null,
84 | sourcePredciate: String = null,
85 | showSnapShot: Boolean = true
86 | ): Unit = {
87 | _insertStatement(
88 | "insert into ",
89 | toTable,
90 | fromTable,
91 | partitionPredicate,
92 | sourcePredciate,
93 | showSnapShot
94 | )
95 | }
96 |
97 | def insertOverwrite(toTable: String,
98 | fromTable: String,
99 | partitionPredicate: String = null,
100 | sourcePredciate: String = null,
101 | showSnapShot: Boolean = true
102 | ): Unit = {
103 |
104 | _insertStatement(
105 | "insert overwrite table ",
106 | toTable,
107 | fromTable,
108 | partitionPredicate,
109 | sourcePredciate,
110 | showSnapShot
111 | )
112 |
113 | }
114 |
115 | def asOf(tStr : String,
116 | tblNm : String
117 | ) : Unit = {
118 |
119 | val sqlText =
120 | s"""
121 | |as of '${tStr}'
122 | |select count(*) from ${tblNm}
123 | |""".stripMargin
124 |
125 | println(sqlText)
126 | TestIcebergHive.sql(sqlText).
127 | show(10000, false)
128 |
129 | }
130 |
131 | test("test1") { td =>
132 |
133 | var df: DataFrame = null
134 |
135 | println("Initially no snapShots:")
136 | showSnapShots("store_sales_out")
137 |
138 | println("SnapShot on an insert: 30 files added")
139 | insert("store_sales_out", "store_sales")
140 | TestIcebergHive.sql("select count(*) from store_sales_out").
141 | show(10000, false)
142 |
143 | val timeAtFirstInsert = utils.convertToTimestampString(System.currentTimeMillis())
144 | // wait 10 mSecs, so we can demo as Of queries of first insert
145 | Thread.sleep(10)
146 |
147 | // a query with predicate ss_sold_date_sk='0906245'
148 | // has predicate ss_sold_month='09' added
149 | println("Query with ss_sold_date_sk='0906245' has predciate ss_sold_month='09' added")
150 | df = TestIcebergHive.sql(
151 | "select count(*) from store_sales_out where ss_sold_date_sk='0906245'")
152 | assert(
153 | utils.iceTableScanExec(df).get.getIceFilter.toString.
154 | contains("""ref(name="ss_sold_month") == "09"""")
155 | )
156 | df.show(1000, false)
157 |
158 | println("SnapShot on another insert: 30 files added again")
159 | insert("store_sales_out", "store_sales")
160 |
161 | TestIcebergHive.sql("select count(*) from store_sales_out").
162 | show(10000, false)
163 |
164 | asOf(timeAtFirstInsert, "store_sales_out")
165 |
166 | println("SnapShot on an insert overwrite: 30 files added, 60 files deleted")
167 | insertOverwrite("store_sales_out", "store_sales")
168 | TestIcebergHive.sql("select count(*) from store_sales_out").
169 | show(10000, false)
170 |
171 | println("SnapShot on an insert overwrite of 1 partition: 5 files added, 5 files deleted")
172 | insertOverwrite("store_sales_out", "store_sales", "ss_sold_date_sk='0906245'")
173 | TestIcebergHive.sql("select count(*) from store_sales_out").
174 | show(10000, false)
175 |
176 |
177 | println("SnapShot on an insert overwrite of 1 partition, with source predicate:" +
178 | " 5 files added, 5 files deleted")
179 | insertOverwrite("store_sales_out", "store_sales",
180 | "ss_sold_date_sk='0905245'", "ss_item_sk < 5000")
181 | TestIcebergHive.sql("select count(*) from store_sales_out").
182 | show(10000, false)
183 | asOf(timeAtFirstInsert, "store_sales_out")
184 |
185 | // now a query with ss_item_sk > 5000 on ss_sold_date_sk=0905245 should be a null scan
186 | println("now a query with ss_item_sk > 5000 on ss_sold_date_sk=0905245 should be a null scan")
187 | df = TestIcebergHive.sql(
188 | "select * from store_sales_out where ss_item_sk > 5000 and ss_sold_date_sk='0905245'")
189 | assert(utils.filesScanned(df).size == 0)
190 | df.show(10000, false)
191 |
192 | println("Test for fix Issue 5: DelegatedMethod0 is not serializable")
193 | df = TestIcebergHive.sql(
194 | """
195 | |select *
196 | |from store_sales_out a
197 | | join store_sales b on a.ss_item_sk=b.ss_item_sk
198 | |where a.ss_item_sk=17908
199 | |""".stripMargin
200 | )
201 | df.show(10000, false)
202 | }
203 |
204 | }
205 |
--------------------------------------------------------------------------------
/src/test/scala/org/apache/spark/sql/hive/test/iceberg/TestIcebergHive.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.hive.test.iceberg
19 |
20 | import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation
21 | import org.apache.spark.{SparkConf, SparkContext}
22 | import org.apache.spark.sql.hive.test.TestHiveContext
23 | import org.apache.spark.sql.internal.SQLConf
24 |
25 | object TestIcebergHive extends TestHiveContext(
26 | new SparkContext(
27 | System.getProperty("spark.sql.test.master", "local[1]"),
28 | "TestSQLContext",
29 | new SparkConf()
30 | .set("spark.sql.test", "")
31 | .set(SQLConf.CODEGEN_FALLBACK.key, "false")
32 | .set("spark.sql.hive.metastore.barrierPrefixes",
33 | "org.apache.spark.sql.hive.execution.PairSerDe")
34 | .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath)
35 | // SPARK-8910
36 | .set("spark.ui.enabled", "false")
37 | .set("spark.unsafe.exceptionOnMemoryLeak", "true")
38 | // Disable ConvertToLocalRelation for better test coverage. Test cases built on
39 | // LocalRelation will exercise the optimization rules better by disabling it as
40 | // this rule may potentially block testing of other optimization rules such as
41 | // ConstantPropagation etc.
42 | .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName)
43 | .set("spark.sql.extensions", "org.apache.spark.sql.iceberg.planning.SparkSessionExtensions")
44 | ), false
45 | )
--------------------------------------------------------------------------------
/src/test/scala/org/apache/spark/sql/hive/test/iceberg/TestTables.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one or more
3 | * contributor license agreements. See the NOTICE file distributed with
4 | * this work for additional information regarding copyright ownership.
5 | * The ASF licenses this file to You under the Apache License, Version 2.0
6 | * (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package org.apache.spark.sql.hive.test.iceberg
19 |
20 | import java.io.File
21 | import java.nio.file.Files
22 |
23 | trait TestTables {
24 | self : AbstractTest =>
25 |
26 | val data_folder = new File("src/test/resources").getAbsoluteFile
27 | val data_path = data_folder.getAbsolutePath
28 |
29 | val store_sales_out_folder = new File(data_folder, "store_sales_out")
30 |
31 |
32 | def createStoreSalesTable(tName : String,
33 | tFolder : String,
34 | tableManagement : Boolean = false,
35 | tableTransforms : Seq[String] = Seq.empty
36 | ) : Unit = {
37 |
38 | val transformOption = if (tableManagement && tableTransforms.nonEmpty) {
39 | s""",
40 | |columnDependencies "${tableTransforms.mkString(",")}"
41 | |""".stripMargin
42 | } else ""
43 |
44 | TestIcebergHive.sql(
45 | s"""create table if not exists ${tName}
46 | (
47 | ss_sold_time_sk int,
48 | | ss_item_sk int,
49 | | ss_customer_sk int,
50 | | ss_cdemo_sk int,
51 | | ss_hdemo_sk int,
52 | | ss_addr_sk int,
53 | | ss_store_sk int,
54 | | ss_promo_sk int,
55 | | ss_quantity int,
56 | | ss_wholesale_cost decimal(7,2),
57 | | ss_list_price decimal(7,2),
58 | | ss_sales_price decimal(7,2),
59 | | ss_ext_sales_price decimal(7,2),
60 | | ss_sold_month string,
61 | | ss_sold_day string,
62 | | ss_sold_date_sk string
63 | |
64 | )
65 | USING parquet
66 | OPTIONS (
67 | |path "$data_path/$tFolder",
68 | |addTableManagement "${tableManagement}" ${transformOption}
69 | |)
70 | partitioned by (ss_sold_date_sk)""".stripMargin
71 | )
72 | TestIcebergHive.sql(s"msck repair table ${tName}")
73 | }
74 |
75 | def setupStoreSalesTables : Unit = {
76 |
77 | delete(store_sales_out_folder)
78 | Files.createDirectories(store_sales_out_folder.toPath)
79 | createStoreSalesTable("store_sales_out",
80 | "store_sales_out",
81 | true,
82 | Seq("ss_sold_date_sk=ss_sold_month:truncate[2]", "ss_sold_date_sk=ss_sold_day:truncate[4]")
83 | )
84 |
85 | createStoreSalesTable("store_sales", "store_sales")
86 | }
87 |
88 |
89 | }
90 |
--------------------------------------------------------------------------------